From a9b3336318f7798a3dd064b67f9e8d6c161c9c68 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 24 Jun 2024 16:10:23 -0400 Subject: [PATCH 01/66] Hack `asyncio` to not abandon a guest-mode run? Took me a while to figure out what the heck was going on but, turns out `asyncio` changed their SIGINT handling in 3.11 as per: https://docs.python.org/3/library/asyncio-runner.html#handling-keyboard-interruption I'm not entirely sure if it's the 3.11 changes or possibly wtv further updates were made in 3.12 but more or less due to the way our current main task was written the `trio` guest-run was getting abandoned on SIGINTs sent from the OS to the infected child proc.. Note that much of the bug and soln cases are layed out in very detailed comment-notes both in the new test and `run_as_asyncio_guest()`, right above the final "fix" lines. Add new `test_infected_aio.test_sigint_closes_lifetime_stack()` test suite which reliably triggers all abandonment issues with multiple cases of different parent behaviour post-sending-SIGINT-to-child: 1. briefly sleep then raise a KBI in the parent which was originally demonstrating the file leak not being cleaned up by `Actor.lifetime_stack.close()` and simulates a ctl-c from the console (relayed in tandem by the OS to the parent and child processes). 2. do `Context.wait_for_result()` on the child context which would hang and timeout since the actor runtime would never complete and thus never relay a `ContextCancelled`. 3. both with and without running a `asyncio` task in the `manage_file` child actor; originally it seemed that with an aio task scheduled in the child actor the guest-run abandonment always was the "loud" case where there seemed to be some actor teardown but with tbs from python failing to gracefully exit the `trio` runtime.. The (seemingly working) "fix" required 2 lines of code to be run inside a `asyncio.CancelledError` handler around the call to `await trio_done_fut`: - `Actor.cancel_soon()` which schedules the actor runtime to cancel on the next `trio` runner cycle and results in a "self cancellation" of the actor. - "pumping the `asyncio` event loop" with a non-0 `.sleep(0.1)` XD |_ seems that a "shielded" pump with some actual `delay: float >= 0` did the trick to get `asyncio` to allow the `trio` runner/loop to fully complete its guest-run without abandonment. Other supporting changes: - move `._exceptions.AsyncioCancelled`, our renamed `asyncio.CancelledError` error-sub-type-wrapper, to `.to_asyncio` and make it derive from `CancelledError` so as to be sure when raised by our `asyncio` x-> `trio` exception relay machinery that `asyncio` is getting the specific type it expects during cancellation. - do "summary status" style logging in `run_as_asyncio_guest()` wherein we compile the eventual `startup_msg: str` emitted just before waiting on the `trio_done_fut`. - shield-wait with `out: Outcome = await asyncio.shield(trio_done_fut)` even though it seems to do nothing in the SIGINT handling case..(I presume it might help avoid abandonment in a `asyncio.Task.cancel()` case maybe?) --- tests/test_infected_asyncio.py | 225 +++++++++++++++++++++++++++++++-- tractor/_exceptions.py | 13 +- tractor/to_asyncio.py | 153 ++++++++++++++++++---- 3 files changed, 344 insertions(+), 47 deletions(-) diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 45722a63..8d4697fd 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -2,16 +2,25 @@ The hipster way to force SC onto the stdlib's "async": 'infection mode'. ''' -from typing import Optional, Iterable, Union import asyncio import builtins +from contextlib import ExitStack import itertools import importlib +import os +from pathlib import Path +import signal +from typing import ( + Callable, + Iterable, + Union, +) import pytest import trio import tractor from tractor import ( + current_actor, to_asyncio, RemoteActorError, ContextCancelled, @@ -25,8 +34,8 @@ async def sleep_and_err( # just signature placeholders for compat with # ``to_asyncio.open_channel_from()`` - to_trio: Optional[trio.MemorySendChannel] = None, - from_trio: Optional[asyncio.Queue] = None, + to_trio: trio.MemorySendChannel|None = None, + from_trio: asyncio.Queue|None = None, ): if to_trio: @@ -36,7 +45,7 @@ async def sleep_and_err( assert 0 -async def sleep_forever(): +async def aio_sleep_forever(): await asyncio.sleep(float('inf')) @@ -44,7 +53,7 @@ async def trio_cancels_single_aio_task(): # spawn an ``asyncio`` task to run a func and return result with trio.move_on_after(.2): - await tractor.to_asyncio.run_task(sleep_forever) + await tractor.to_asyncio.run_task(aio_sleep_forever) def test_trio_cancels_aio_on_actor_side(reg_addr): @@ -66,14 +75,13 @@ def test_trio_cancels_aio_on_actor_side(reg_addr): async def asyncio_actor( - target: str, expect_err: Exception|None = None ) -> None: assert tractor.current_actor().is_infected_aio() - target = globals()[target] + target: Callable = globals()[target] if '.' in expect_err: modpath, _, name = expect_err.rpartition('.') @@ -140,7 +148,7 @@ def test_tractor_cancels_aio(reg_addr): async with tractor.open_nursery() as n: portal = await n.run_in_actor( asyncio_actor, - target='sleep_forever', + target='aio_sleep_forever', expect_err='trio.Cancelled', infect_asyncio=True, ) @@ -164,7 +172,7 @@ def test_trio_cancels_aio(reg_addr): async with tractor.open_nursery() as n: await n.run_in_actor( asyncio_actor, - target='sleep_forever', + target='aio_sleep_forever', expect_err='trio.Cancelled', infect_asyncio=True, ) @@ -195,7 +203,7 @@ async def trio_ctx( # spawn another asyncio task for the cuck of it. n.start_soon( tractor.to_asyncio.run_task, - sleep_forever, + aio_sleep_forever, ) await trio.sleep_forever() @@ -285,7 +293,7 @@ async def aio_cancel(): # cancel and enter sleep task.cancel() - await sleep_forever() + await aio_sleep_forever() def test_aio_cancelled_from_aio_causes_trio_cancelled(reg_addr): @@ -355,7 +363,6 @@ async def push_from_aio_task( async def stream_from_aio( - exit_early: bool = False, raise_err: bool = False, aio_raise_err: bool = False, @@ -618,6 +625,200 @@ def test_echoserver_detailed_mechanics( trio.run(main) + +@tractor.context +async def manage_file( + ctx: tractor.Context, + tmp_path_str: str, + bg_aio_task: bool = False, +): + ''' + Start an `asyncio` task that just sleeps after registering a context + with `Actor.lifetime_stack`. Trigger a SIGINT to kill the actor tree + and ensure the stack is closed in the infected mode child. + + To verify the teardown state just write a tmpfile to the `testdir` + and delete it on actor close. + + ''' + + tmp_path: Path = Path(tmp_path_str) + tmp_file: Path = tmp_path / f'{" ".join(ctx._actor.uid)}.file' + + # create a the tmp file and tell the parent where it's at + assert not tmp_file.is_file() + tmp_file.touch() + + stack: ExitStack = current_actor().lifetime_stack + stack.callback(tmp_file.unlink) + + await ctx.started(( + str(tmp_file), + os.getpid(), + )) + + # expect to be cancelled from here! + try: + + # NOTE: turns out you don't even need to sched an aio task + # since the original issue, even though seemingly was due to + # the guest-run being abandoned + a `._debug.pause()` inside + # `._runtime._async_main()` (which was originally trying to + # debug the `.lifetime_stack` not closing), IS NOT actually + # the core issue? + # + # further notes: + # + # - `trio` only issues the " RuntimeWarning: Trio guest run + # got abandoned without properly finishing... weird stuff + # might happen" IFF you DO run a asyncio task here, BUT + # - the original issue of the `.lifetime_stack` not closing + # will still happen even if you don't run an `asyncio` task + # here even though the "abandon" messgage won't be shown.. + # + # => ????? honestly i'm lost but it seems to be some issue + # with `asyncio` and SIGINT.. + # + # XXX NOTE XXX SO, if this LINE IS UNCOMMENTED and + # `run_as_asyncio_guest()` is written WITHOUT THE + # `.cancel_soon()` soln, both of these tests will pass ?? + # so maybe it has something to do with `asyncio` loop init + # state?!? + # honestly, this REALLY reminds me why i haven't used + # `asyncio` by choice in years.. XD + # + # await tractor.to_asyncio.run_task(aio_sleep_forever) + if bg_aio_task: + async with trio.open_nursery() as tn: + tn.start_soon( + tractor.to_asyncio.run_task, + aio_sleep_forever, + ) + + await trio.sleep_forever() + + # signalled manually at the OS level (aka KBI) by the parent actor. + except KeyboardInterrupt: + print('child raised KBI..') + assert tmp_file.exists() + raise + else: + raise RuntimeError('shoulda received a KBI?') + + +@pytest.mark.parametrize( + 'bg_aio_task', + [ + False, + + # NOTE: (and see notes in `manage_file()` above as well) if + # we FOR SURE SPAWN AN AIO TASK in the child it seems the + # "silent-abandon" case (as is described in detail in + # `to_asyncio.run_as_asyncio_guest()`) does not happen and + # `asyncio`'s loop will at least abandon the `trio` side + # loudly? .. prolly the state-spot to start looking for + # a soln that results in NO ABANDONMENT.. XD + True, + ], + ids=[ + 'bg_aio_task', + 'just_trio_slee', + ], +) +@pytest.mark.parametrize( + 'wait_for_ctx', + [ + False, + True, + ], + ids=[ + 'raise_KBI_in_rent', + 'wait_for_ctx', + ], +) +def test_sigint_closes_lifetime_stack( + tmp_path: Path, + wait_for_ctx: bool, + bg_aio_task: bool, +): + ''' + Ensure that an infected child can use the `Actor.lifetime_stack` + to make a file on boot and it's automatically cleaned up by the + actor-lifetime-linked exit stack closure. + + ''' + async def main(): + try: + async with tractor.open_nursery() as n: + p = await n.start_actor( + 'file_mngr', + enable_modules=[__name__], + infect_asyncio=True, + ) + async with p.open_context( + manage_file, + tmp_path_str=str(tmp_path), + bg_aio_task=bg_aio_task, + ) as (ctx, first): + + path_str, cpid = first + tmp_file: Path = Path(path_str) + assert tmp_file.exists() + + # XXX originally to simulate what (hopefully) + # the below now triggers.. had to manually + # trigger a SIGINT from a ctl-c in the root. + # await trio.sleep_forever() + + # XXX NOTE XXX signal infected-`asyncio` child to + # OS-cancel with SIGINT; this should trigger the + # bad `asyncio` cancel behaviour that can cause + # a guest-run abandon as was seen causing + # shm-buffer leaks in `piker`'s live quote stream + # susbys! + # + # await trio.sleep(.5) + await trio.sleep(.2) + os.kill( + cpid, + signal.SIGINT, + ) + + # XXX CASE 1: without the bug fixed, in + # the non-KBI-raised-in-parent case, this + # timeout should trigger! + if wait_for_ctx: + print('waiting for ctx outcome in parent..') + try: + with trio.fail_after(.7): + await ctx.wait_for_result() + except tractor.ContextCancelled as ctxc: + assert ctxc.canceller == ctx.chan.uid + raise + + # XXX CASE 2: this seems to be the source of the + # original issue which exhibited BEFORE we put + # a `Actor.cancel_soon()` inside + # `run_as_asyncio_guest()`.. + else: + raise KeyboardInterrupt + + pytest.fail('should have raised some kinda error?!?') + + except ( + KeyboardInterrupt, + ContextCancelled, + ): + # XXX CASE 2: without the bug fixed, in the + # KBI-raised-in-parent case, the actor teardown should + # never get run (silently abaondoned by `asyncio`..) and + # thus the file should leak! + assert not tmp_file.exists() + assert ctx.maybe_error + + trio.run(main) + + # TODO: debug_mode tests once we get support for `asyncio`! # # -[ ] need tests to wrap both scripts: diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 108134ca..b85a1b83 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -973,15 +973,6 @@ class NoRuntime(RuntimeError): "The root actor has not been initialized yet" - -class AsyncioCancelled(Exception): - ''' - Asyncio cancelled translation (non-base) error - for use with the ``to_asyncio`` module - to be raised in the ``trio`` side task - - ''' - class MessagingError(Exception): ''' IPC related msg (typing), transaction (ordering) or dialog @@ -1375,7 +1366,9 @@ def _mk_recv_mte( any_pld: Any = msgpack.decode(msg.pld) message: str = ( f'invalid `{msg_type.__qualname__}` msg payload\n\n' - f'value: `{any_pld!r}` does not match type-spec: ' + f'{any_pld!r}\n\n' + f'has type {type(any_pld)!r}\n\n' + f'and does not match type-spec ' f'`{type(msg).__qualname__}.pld: {codec.pld_spec_str}`' ) bad_msg = msg diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index d1451b4c..e041721f 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -18,11 +18,13 @@ Infection apis for ``asyncio`` loops running ``trio`` using guest mode. ''' +from __future__ import annotations import asyncio from asyncio.exceptions import CancelledError from contextlib import asynccontextmanager as acm from dataclasses import dataclass import inspect +import traceback from typing import ( Any, Callable, @@ -30,20 +32,21 @@ from typing import ( Awaitable, ) -import trio -from outcome import Error - -from tractor.log import get_logger +import tractor from tractor._state import ( - current_actor, debug_mode, ) +from tractor.log import get_logger from tractor.devx import _debug -from tractor._exceptions import AsyncioCancelled from tractor.trionics._broadcast import ( broadcast_receiver, BroadcastReceiver, ) +import trio +from outcome import ( + Error, + Outcome, +) log = get_logger(__name__) @@ -161,7 +164,7 @@ def _run_asyncio_task( ''' __tracebackhide__ = True - if not current_actor().is_infected_aio(): + if not tractor.current_actor().is_infected_aio(): raise RuntimeError( "`infect_asyncio` mode is not enabled!?" ) @@ -172,7 +175,6 @@ def _run_asyncio_task( to_trio, from_aio = trio.open_memory_channel(qsize) # type: ignore args = tuple(inspect.getfullargspec(func).args) - if getattr(func, '_tractor_steam_function', None): # the assumption is that the target async routine accepts the # send channel then it intends to yield more then one return @@ -346,13 +348,22 @@ def _run_asyncio_task( # on a checkpoint. cancel_scope.cancel() - # raise any ``asyncio`` side error. + # raise any `asyncio` side error. raise aio_err task.add_done_callback(cancel_trio) return chan +class AsyncioCancelled(CancelledError): + ''' + Asyncio cancelled translation (non-base) error + for use with the ``to_asyncio`` module + to be raised in the ``trio`` side task + + ''' + + @acm async def translate_aio_errors( @@ -516,7 +527,6 @@ async def open_channel_from( def run_as_asyncio_guest( - trio_main: Callable, ) -> None: @@ -548,6 +558,11 @@ def run_as_asyncio_guest( loop = asyncio.get_running_loop() trio_done_fut = asyncio.Future() + startup_msg: str = ( + 'Starting `asyncio` guest-loop-run\n' + '-> got running loop\n' + '-> built a `trio`-done future\n' + ) if debug_mode(): # XXX make it obvi we know this isn't supported yet! @@ -562,34 +577,120 @@ def run_as_asyncio_guest( def trio_done_callback(main_outcome): if isinstance(main_outcome, Error): - error = main_outcome.error + error: BaseException = main_outcome.error + + # show an dedicated `asyncio`-side tb from the error + tb_str: str = ''.join(traceback.format_exception(error)) + log.exception( + 'Guest-run errored!?\n\n' + f'{main_outcome}\n' + f'{error}\n\n' + f'{tb_str}\n' + ) trio_done_fut.set_exception(error) - # TODO: explicit asyncio tb? - # traceback.print_exception(error) - - # XXX: do we need this? - # actor.cancel_soon() - + # raise inline main_outcome.unwrap() + else: trio_done_fut.set_result(main_outcome) - log.runtime(f"trio_main finished: {main_outcome!r}") + log.runtime(f'trio_main finished: {main_outcome!r}') + + startup_msg += ( + f'-> created {trio_done_callback!r}\n' + f'-> scheduling `trio_main`: {trio_main!r}\n' + ) # start the infection: run trio on the asyncio loop in "guest mode" log.runtime( - 'Infecting `asyncio`-process with a `trio` guest-run of\n\n' - f'{trio_main!r}\n\n' - - f'{trio_done_callback}\n' + f'{startup_msg}\n\n' + + + 'Infecting `asyncio`-process with a `trio` guest-run!\n' ) + trio.lowlevel.start_guest_run( trio_main, run_sync_soon_threadsafe=loop.call_soon_threadsafe, done_callback=trio_done_callback, ) - # NOTE `.unwrap()` will raise on error - return (await trio_done_fut).unwrap() + try: + # TODO: better SIGINT handling since shielding seems to + # make NO DIFFERENCE XD + # -[ ] maybe this is due to 3.11's recent SIGINT handling + # changes and we can better work with/around it? + # https://docs.python.org/3/library/asyncio-runner.html#handling-keyboard-interruption + out: Outcome = await asyncio.shield(trio_done_fut) + # NOTE `Error.unwrap()` will raise + return out.unwrap() + + except asyncio.CancelledError: + actor: tractor.Actor = tractor.current_actor() + log.exception( + '`asyncio`-side main task was cancelled!\n' + 'Cancelling actor-runtime..\n' + f'c)>\n' + f' |_{actor}.cancel_soon()\n' + + ) + + # XXX NOTE XXX the next LOC is super important!!! + # => without it, we can get a guest-run abandonment case + # where asyncio will not trigger `trio` in a final event + # loop cycle! + # + # our test, + # `test_infected_asyncio.test_sigint_closes_lifetime_stack()` + # demonstrates how if when we raise a SIGINT-signal in an infected + # child we get a variable race condition outcome where + # either of the following can indeterminately happen, + # + # - "silent-abandon": `asyncio` abandons the `trio` + # guest-run task silently and no `trio`-guest-run or + # `tractor`-actor-runtime teardown happens whatsoever.. + # this is the WORST (race) case outcome. + # + # - OR, "loud-abandon": the guest run get's abaondoned "loudly" with + # `trio` reporting a console traceback and further tbs of all + # the failed shutdown routines also show on console.. + # + # our test can thus fail and (has been parametrized for) + # the 2 cases: + # + # - when the parent raises a KBI just after + # signalling the child, + # |_silent-abandon => the `Actor.lifetime_stack` will + # never be closed thus leaking a resource! + # -> FAIL! + # |_loud-abandon => despite the abandonment at least the + # stack will be closed out.. + # -> PASS + # + # - when the parent instead simply waits on `ctx.wait_for_result()` + # (i.e. DOES not raise a KBI itself), + # |_silent-abandon => test will just hang and thus the ctx + # and actor will never be closed/cancelled/shutdown + # resulting in leaking a (file) resource since the + # `trio`/`tractor` runtime never relays a ctxc back to + # the parent; the test's timeout will trigger.. + # -> FAIL! + # |_loud-abandon => this case seems to never happen?? + # + # XXX FIRST PART XXX, SO, this is a fix to the + # "silent-abandon" case, NOT the `trio`-guest-run + # abandonment issue in general, for which the NEXT LOC + # is apparently a working fix! + actor.cancel_soon() + + # XXX NOTE XXX PUMP the asyncio event loop to allow `trio`-side to + # `trio`-guest-run to complete and teardown !! + # + # XXX WITHOUT THIS the guest-run gets race-conditionally + # abandoned by `asyncio`!! + # XD XD XD + await asyncio.shield( + asyncio.sleep(.1) # NOPE! it can't be 0 either XD + ) + raise # might as well if it's installed. try: @@ -599,4 +700,6 @@ def run_as_asyncio_guest( except ImportError: pass - return asyncio.run(aio_main(trio_main)) + return asyncio.run( + aio_main(trio_main), + ) From 7f00921be1df64e12fbd3131c39de1c98a1fb7fc Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 25 Jun 2024 19:36:31 -0400 Subject: [PATCH 02/66] Lel, revert `AsyncioCancelled` inherit, module.. Turns out it somehow breaks our `to_asyncio` error relay since obvi `asyncio`'s runtime seems to specially handle it (prolly via `isinstance()` ?) and it caused our `test_aio_cancelled_from_aio_causes_trio_cancelled()` to hang.. Further, obvi `unpack_error()` won't be able to find the type def if not kept inside `._exceptions`.. So given all that, revert the change/move as well as: - tweak the aio-from-aio cancel test to timeout. - do `trio.sleep()` conc with any bg aio task by moving out nursery block. - add a `send_sigint_to: str` parameter to `test_sigint_closes_lifetime_stack()` such that we test the SIGINT being relayed to just the parent or the child. --- tests/test_infected_asyncio.py | 69 ++++++++++++++++++++++++++-------- tractor/_exceptions.py | 11 ++++++ 2 files changed, 64 insertions(+), 16 deletions(-) diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 8d4697fd..645dc4b6 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -289,23 +289,35 @@ async def aio_cancel(): ''' await asyncio.sleep(0.5) - task = asyncio.current_task() # cancel and enter sleep + task = asyncio.current_task() task.cancel() await aio_sleep_forever() def test_aio_cancelled_from_aio_causes_trio_cancelled(reg_addr): + ''' + When the `asyncio.Task` cancels itself the `trio` side cshould + also cancel and teardown and relay the cancellation cross-process + to the caller (parent). + ''' async def main(): - async with tractor.open_nursery() as n: - await n.run_in_actor( + + an: tractor.ActorNursery + async with tractor.open_nursery() as an: + p: tractor.Portal = await an.run_in_actor( asyncio_actor, target='aio_cancel', expect_err='tractor.to_asyncio.AsyncioCancelled', infect_asyncio=True, ) + # NOTE: normally the `an.__aexit__()` waits on the + # portal's result but we do it explicitly here + # to avoid indent levels. + with trio.fail_after(1): + await p.wait_for_result() with pytest.raises( expected_exception=(RemoteActorError, ExceptionGroup), @@ -313,7 +325,7 @@ def test_aio_cancelled_from_aio_causes_trio_cancelled(reg_addr): trio.run(main) # might get multiple `trio.Cancelled`s as well inside an inception - err = excinfo.value + err: RemoteActorError|ExceptionGroup = excinfo.value if isinstance(err, ExceptionGroup): err = next(itertools.dropwhile( lambda exc: not isinstance(exc, tractor.RemoteActorError), @@ -321,7 +333,8 @@ def test_aio_cancelled_from_aio_causes_trio_cancelled(reg_addr): )) assert err - # ensure boxed error is correct + # relayed boxed error should be our `trio`-task's + # cancel-signal-proxy-equivalent of `asyncio.CancelledError`. assert err.boxed_type == to_asyncio.AsyncioCancelled @@ -630,6 +643,7 @@ def test_echoserver_detailed_mechanics( async def manage_file( ctx: tractor.Context, tmp_path_str: str, + send_sigint_to: str, bg_aio_task: bool = False, ): ''' @@ -687,25 +701,39 @@ async def manage_file( # honestly, this REALLY reminds me why i haven't used # `asyncio` by choice in years.. XD # - # await tractor.to_asyncio.run_task(aio_sleep_forever) - if bg_aio_task: - async with trio.open_nursery() as tn: + async with trio.open_nursery() as tn: + if bg_aio_task: tn.start_soon( tractor.to_asyncio.run_task, aio_sleep_forever, ) - await trio.sleep_forever() + # XXX don't-need/doesn't-make-a-diff right + # since we're already doing it from parent? + # if send_sigint_to == 'child': + # os.kill( + # os.getpid(), + # signal.SIGINT, + # ) + await trio.sleep_forever() # signalled manually at the OS level (aka KBI) by the parent actor. except KeyboardInterrupt: print('child raised KBI..') assert tmp_file.exists() raise - else: - raise RuntimeError('shoulda received a KBI?') + + raise RuntimeError('shoulda received a KBI?') +@pytest.mark.parametrize( + 'send_sigint_to', + [ + 'child', + 'parent', + ], + ids='send_SIGINT_to={}'.format, +) @pytest.mark.parametrize( 'bg_aio_task', [ @@ -740,6 +768,8 @@ def test_sigint_closes_lifetime_stack( tmp_path: Path, wait_for_ctx: bool, bg_aio_task: bool, + debug_mode: bool, + send_sigint_to: str, ): ''' Ensure that an infected child can use the `Actor.lifetime_stack` @@ -749,8 +779,11 @@ def test_sigint_closes_lifetime_stack( ''' async def main(): try: - async with tractor.open_nursery() as n: - p = await n.start_actor( + an: tractor.ActorNursery + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + p: tractor.Portal = await an.start_actor( 'file_mngr', enable_modules=[__name__], infect_asyncio=True, @@ -758,6 +791,7 @@ def test_sigint_closes_lifetime_stack( async with p.open_context( manage_file, tmp_path_str=str(tmp_path), + send_sigint_to=send_sigint_to, bg_aio_task=bg_aio_task, ) as (ctx, first): @@ -777,10 +811,13 @@ def test_sigint_closes_lifetime_stack( # shm-buffer leaks in `piker`'s live quote stream # susbys! # - # await trio.sleep(.5) await trio.sleep(.2) + pid: int = ( + cpid if send_sigint_to == 'child' + else os.getpid() + ) os.kill( - cpid, + pid, signal.SIGINT, ) @@ -790,7 +827,7 @@ def test_sigint_closes_lifetime_stack( if wait_for_ctx: print('waiting for ctx outcome in parent..') try: - with trio.fail_after(.7): + with trio.fail_after(1): await ctx.wait_for_result() except tractor.ContextCancelled as ctxc: assert ctxc.canceller == ctx.chan.uid diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index b85a1b83..a0b6ff3f 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -980,6 +980,17 @@ class MessagingError(Exception): ''' +class AsyncioCancelled(Exception): + ''' + Asyncio cancelled translation (non-base) error + for use with the ``to_asyncio`` module + to be raised in the ``trio`` side task + + NOTE: this should NOT inherit from `asyncio.CancelledError` or + tests should break! + + ''' + def pack_error( exc: BaseException|RemoteActorError, From 53409f2942f195f15bc80c556b4f7140db26e7ba Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 26 Jun 2024 11:44:31 -0400 Subject: [PATCH 03/66] Demo-abandonment on shielded `trio`-side work Finally this reproduces the issue as it (originally?) exhibited inside `piker` where the `Actor.lifetime_stack` wasn't closed in cases where during `infected_aio`-actor cancellation/shutdown `trio` side tasks which are doing shielded (teardown) work are NOT being watched/waited on from the `aio_main()` task-closure inside `run_as_asyncio_guest()`! This is then the root cause of the guest-run being abandoned since if our `aio_main()` task-closure doesn't know it should allow the run to finish, it's going to call `loop.close()` eventually resulting in the `GeneratorExit` thrown into `trio._core._run.unrolled_run()`.. So, this extends the `test_sigint_closes_lifetime_stack()` suite to include cases for such shielded `trio`-task ops: - add a new `trio_side_is_shielded: bool` which will toggle whether to add a shielded 0.5s `trio.sleep()` loop to `manage_file()` which should outlive the `asyncio` event-loop shutdown sequence and result in an abandoned guest-run and thus a leaked file. - parametrize the existing suite with this case resulting in a total 16 test set B) This patch demonstrates the problem with our `aio_main()` task-closure impl via the now 4 failing tests, a fix is coming in a follow up commit! --- tests/test_infected_asyncio.py | 28 +++++++++++++++++++++++----- 1 file changed, 23 insertions(+), 5 deletions(-) diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 645dc4b6..42eb35b7 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -644,6 +644,7 @@ async def manage_file( ctx: tractor.Context, tmp_path_str: str, send_sigint_to: str, + trio_side_is_shielded: bool = True, bg_aio_task: bool = False, ): ''' @@ -693,11 +694,6 @@ async def manage_file( # => ????? honestly i'm lost but it seems to be some issue # with `asyncio` and SIGINT.. # - # XXX NOTE XXX SO, if this LINE IS UNCOMMENTED and - # `run_as_asyncio_guest()` is written WITHOUT THE - # `.cancel_soon()` soln, both of these tests will pass ?? - # so maybe it has something to do with `asyncio` loop init - # state?!? # honestly, this REALLY reminds me why i haven't used # `asyncio` by choice in years.. XD # @@ -715,6 +711,15 @@ async def manage_file( # os.getpid(), # signal.SIGINT, # ) + + # XXX spend a half sec doing shielded checkpointing to + # ensure that despite the `trio`-side task ignoring the + # SIGINT, the `asyncio` side won't abandon the guest-run! + if trio_side_is_shielded: + with trio.CancelScope(shield=True): + for i in range(5): + await trio.sleep(0.1) + await trio.sleep_forever() # signalled manually at the OS level (aka KBI) by the parent actor. @@ -726,6 +731,17 @@ async def manage_file( raise RuntimeError('shoulda received a KBI?') +@pytest.mark.parametrize( + 'trio_side_is_shielded', + [ + False, + True, + ], + ids=[ + 'trio_side_no_shielding', + 'trio_side_does_shielded_work', + ], +) @pytest.mark.parametrize( 'send_sigint_to', [ @@ -768,6 +784,7 @@ def test_sigint_closes_lifetime_stack( tmp_path: Path, wait_for_ctx: bool, bg_aio_task: bool, + trio_side_is_shielded: bool, debug_mode: bool, send_sigint_to: str, ): @@ -793,6 +810,7 @@ def test_sigint_closes_lifetime_stack( tmp_path_str=str(tmp_path), send_sigint_to=send_sigint_to, bg_aio_task=bg_aio_task, + trio_side_is_shielded=trio_side_is_shielded, ) as (ctx, first): path_str, cpid = first From 4afab9ca47d284809841f4192e9b9478dbf735e5 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 26 Jun 2024 13:48:36 -0400 Subject: [PATCH 04/66] Solve our abandonment issues.. To make the recent set of tests pass this (hopefully) finally solves all `asyncio` embedded `trio` guest-run abandonment by ensuring we "pump the event loop" until the guest-run future is fully complete. Accomplished via simple poll loop of the form `while not trio_done_fut.done(): await asyncio.sleep(.1)` in the `aio_main()` task's exception teardown sequence. The loop does a naive 10ms "pump-via-sleep & poll" for the `trio` side to complete before finally exiting (and presumably raising) from the SIGINT cancellation. Other related cleanups and refinements: - use `asyncio.Task.result()` inside `cancel_trio()` since it also inline-raises any exception outcome and we can also log-report the result in non-error cases. - comment out buncha not-sure-we-need-it stuff in `cancel_trio()`. - remove the botched `AsyncioCancelled(CancelledError):` idea obvi XD - comment `greenback` init for now in `aio_main()` since (pretty sure) we don't ever want to actually REPL in that specific func-as-task? - always capture any `fute_err: BaseException` from the `main_outcome: Outcome` delivered by the `trio` side guest-run task. - add and raise a new super noisy `AsyncioRuntimeTranslationError` whenever we detect that the guest-run `trio_done_fut` has not completed before task exit; should avoid abandonment issues ever happening again without knowing! --- tractor/to_asyncio.py | 246 ++++++++++++++++++++++++++++++------------ 1 file changed, 176 insertions(+), 70 deletions(-) diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index e041721f..fb18ba88 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -33,11 +33,12 @@ from typing import ( ) import tractor +from tractor._exceptions import AsyncioCancelled from tractor._state import ( debug_mode, ) -from tractor.log import get_logger from tractor.devx import _debug +from tractor.log import get_logger from tractor.trionics._broadcast import ( broadcast_receiver, BroadcastReceiver, @@ -51,7 +52,10 @@ from outcome import ( log = get_logger(__name__) -__all__ = ['run_task', 'run_as_asyncio_guest'] +__all__ = [ + 'run_task', + 'run_as_asyncio_guest', +] @dataclass @@ -155,15 +159,16 @@ def _run_asyncio_task( *, qsize: int = 1, provide_channels: bool = False, + hide_tb: bool = False, **kwargs, ) -> LinkedTaskChannel: ''' Run an ``asyncio`` async function or generator in a task, return - or stream the result back to ``trio``. + or stream the result back to the caller `trio.lowleve.Task`. ''' - __tracebackhide__ = True + __tracebackhide__: bool = hide_tb if not tractor.current_actor().is_infected_aio(): raise RuntimeError( "`infect_asyncio` mode is not enabled!?" @@ -224,6 +229,7 @@ def _run_asyncio_task( try: result = await coro except BaseException as aio_err: + chan._aio_err = aio_err if isinstance(aio_err, CancelledError): log.runtime( '`asyncio` task was cancelled..\n' @@ -232,7 +238,6 @@ def _run_asyncio_task( log.exception( '`asyncio` task errored\n' ) - chan._aio_err = aio_err raise else: @@ -268,7 +273,7 @@ def _run_asyncio_task( aio_task_complete ) ) - chan._aio_task = task + chan._aio_task: asyncio.Task = task # XXX TODO XXX get this actually workin.. XD # maybe setup `greenback` for `asyncio`-side task REPLing @@ -284,19 +289,19 @@ def _run_asyncio_task( def cancel_trio(task: asyncio.Task) -> None: ''' - Cancel the calling ``trio`` task on error. + Cancel the calling `trio` task on error. ''' nonlocal chan - aio_err = chan._aio_err + aio_err: BaseException|None = chan._aio_err task_err: BaseException|None = None - # only to avoid ``asyncio`` complaining about uncaptured + # only to avoid `asyncio` complaining about uncaptured # task exceptions try: - task.exception() + res: Any = task.result() except BaseException as terr: - task_err = terr + task_err: BaseException = terr msg: str = ( 'Infected `asyncio` task {etype_str}\n' @@ -328,42 +333,49 @@ def _run_asyncio_task( if task_err is None: assert aio_err - aio_err.with_traceback(aio_err.__traceback__) - # log.error( - # 'infected task errorred' - # ) + # wait, wut? + # aio_err.with_traceback(aio_err.__traceback__) - # TODO: show that the cancellation originated - # from the ``trio`` side? right? - # elif type(aio_err) is CancelledError: + # TODO: show when cancellation originated + # from each side more pedantically? + # elif ( + # type(aio_err) is CancelledError + # and # trio was the cause? + # cancel_scope.cancel_called + # ): # log.cancel( - # 'infected task was cancelled' + # 'infected task was cancelled by `trio`-side' # ) + # raise aio_err from task_err - # if cancel_scope.cancelled: - # raise aio_err from err - - # XXX: alway cancel the scope on error - # in case the trio task is blocking - # on a checkpoint. + # XXX: if not already, alway cancel the scope + # on a task error in case the trio task is blocking on + # a checkpoint. cancel_scope.cancel() + if ( + task_err + and + aio_err is not task_err + ): + raise aio_err from task_err + # raise any `asyncio` side error. raise aio_err + log.info( + '`trio` received final result from {task}\n' + f'|_{res}\n' + ) + # TODO: do we need this? + # if task_err: + # cancel_scope.cancel() + # raise task_err + task.add_done_callback(cancel_trio) return chan -class AsyncioCancelled(CancelledError): - ''' - Asyncio cancelled translation (non-base) error - for use with the ``to_asyncio`` module - to be raised in the ``trio`` side task - - ''' - - @acm async def translate_aio_errors( @@ -386,7 +398,9 @@ async def translate_aio_errors( ) -> None: aio_err = chan._aio_err if ( - aio_err is not None and + aio_err is not None + and + # not isinstance(aio_err, CancelledError) type(aio_err) != CancelledError ): # always raise from any captured asyncio error @@ -418,13 +432,17 @@ async def translate_aio_errors( ): aio_err = chan._aio_err if ( - task.cancelled() and + task.cancelled() + and type(aio_err) is CancelledError ): - # if an underlying ``asyncio.CancelledError`` triggered this + # if an underlying `asyncio.CancelledError` triggered this # channel close, raise our (non-``BaseException``) wrapper # error: ``AsyncioCancelled`` from that source error. - raise AsyncioCancelled from aio_err + raise AsyncioCancelled( + f'Task cancelled\n' + f'|_{task}\n' + ) from aio_err else: raise @@ -467,8 +485,8 @@ async def run_task( ) -> Any: ''' - Run an ``asyncio`` async function or generator in a task, return - or stream the result back to ``trio``. + Run an `asyncio` async function or generator in a task, return + or stream the result back to `trio`. ''' # simple async func @@ -526,10 +544,27 @@ async def open_channel_from( chan._to_trio.close() +class AsyncioRuntimeTranslationError(RuntimeError): + ''' + We failed to correctly relay runtime semantics and/or maintain SC + supervision rules cross-event-loop. + + ''' + + def run_as_asyncio_guest( trio_main: Callable, + # ^-NOTE-^ when spawned with `infected_aio=True` this func is + # normally `Actor._async_main()` as is passed by some boostrap + # entrypoint like `._entry._trio_main()`. ) -> None: +# ^-TODO-^ technically whatever `trio_main` returns.. we should +# try to use func-typevar-params at leaast by 3.13! +# -[ ] https://typing.readthedocs.io/en/latest/spec/callables.html#callback-protocols +# -[ ] https://peps.python.org/pep-0646/#using-type-variable-tuples-in-functions +# -[ ] https://typing.readthedocs.io/en/latest/spec/callables.html#unpack-for-keyword-arguments +# -[ ] https://peps.python.org/pep-0718/ ''' Entry for an "infected ``asyncio`` actor". @@ -555,7 +590,13 @@ def run_as_asyncio_guest( # :) async def aio_main(trio_main): + ''' + Main `asyncio.Task` which calls + `trio.lowlevel.start_guest_run()` to "infect" the `asyncio` + event-loop by embedding the `trio` scheduler allowing us to + boot the `tractor` runtime and connect back to our parent. + ''' loop = asyncio.get_running_loop() trio_done_fut = asyncio.Future() startup_msg: str = ( @@ -564,17 +605,22 @@ def run_as_asyncio_guest( '-> built a `trio`-done future\n' ) - if debug_mode(): - # XXX make it obvi we know this isn't supported yet! - log.error( - 'Attempting to enter unsupported `greenback` init ' - 'from `asyncio` task..' - ) - await _debug.maybe_init_greenback( - force_reload=True, - ) + # TODO: shoudn't this be done in the guest-run trio task? + # if debug_mode(): + # # XXX make it obvi we know this isn't supported yet! + # log.error( + # 'Attempting to enter unsupported `greenback` init ' + # 'from `asyncio` task..' + # ) + # await _debug.maybe_init_greenback( + # force_reload=True, + # ) def trio_done_callback(main_outcome): + log.info( + f'trio_main finished with\n' + f'|_{main_outcome!r}' + ) if isinstance(main_outcome, Error): error: BaseException = main_outcome.error @@ -594,7 +640,6 @@ def run_as_asyncio_guest( else: trio_done_fut.set_result(main_outcome) - log.runtime(f'trio_main finished: {main_outcome!r}') startup_msg += ( f'-> created {trio_done_callback!r}\n' @@ -613,26 +658,48 @@ def run_as_asyncio_guest( run_sync_soon_threadsafe=loop.call_soon_threadsafe, done_callback=trio_done_callback, ) + fute_err: BaseException|None = None try: - # TODO: better SIGINT handling since shielding seems to - # make NO DIFFERENCE XD - # -[ ] maybe this is due to 3.11's recent SIGINT handling - # changes and we can better work with/around it? - # https://docs.python.org/3/library/asyncio-runner.html#handling-keyboard-interruption out: Outcome = await asyncio.shield(trio_done_fut) - # NOTE `Error.unwrap()` will raise + + # NOTE will raise (via `Error.unwrap()`) from any + # exception packed into the guest-run's `main_outcome`. return out.unwrap() - except asyncio.CancelledError: + except ( + # XXX special SIGINT-handling is required since + # `asyncio.shield()`-ing seems to NOT handle that case as + # per recent changes in 3.11: + # https://docs.python.org/3/library/asyncio-runner.html#handling-keyboard-interruption + # + # NOTE: further, apparently ONLY need to handle this + # special SIGINT case since all other `asyncio`-side + # errors can be processed via our `chan._aio_err` + # relaying (right?); SIGINT seems to be totally diff + # error path in `asyncio`'s runtime..? + asyncio.CancelledError, + + ) as fute_err: + err_message: str = ( + 'main `asyncio` task ' + ) + if isinstance(fute_err, asyncio.CancelledError): + err_message += 'was cancelled!\n' + else: + err_message += f'errored with {out.error!r}\n' + actor: tractor.Actor = tractor.current_actor() log.exception( - '`asyncio`-side main task was cancelled!\n' - 'Cancelling actor-runtime..\n' + err_message + + + 'Cancelling `trio`-side `tractor`-runtime..\n' f'c)>\n' f' |_{actor}.cancel_soon()\n' - ) + # TODO: reduce this comment bloc since abandon issues are + # now solved? + # # XXX NOTE XXX the next LOC is super important!!! # => without it, we can get a guest-run abandonment case # where asyncio will not trigger `trio` in a final event @@ -681,16 +748,55 @@ def run_as_asyncio_guest( # is apparently a working fix! actor.cancel_soon() - # XXX NOTE XXX PUMP the asyncio event loop to allow `trio`-side to - # `trio`-guest-run to complete and teardown !! + # XXX NOTE XXX pump the `asyncio` event-loop to allow + # `trio`-side to `trio`-guest-run to complete and + # teardown !! # - # XXX WITHOUT THIS the guest-run gets race-conditionally - # abandoned by `asyncio`!! - # XD XD XD - await asyncio.shield( - asyncio.sleep(.1) # NOPE! it can't be 0 either XD - ) - raise + # *WITHOUT THIS* the guest-run can get race-conditionally abandoned!! + # XD + # + await asyncio.sleep(.1) # `delay` can't be 0 either XD + while not trio_done_fut.done(): + log.runtime( + 'Waiting on main guest-run `asyncio` task to complete..\n' + f'|_trio_done_fut: {trio_done_fut}\n' + ) + await asyncio.sleep(.1) + + # XXX: don't actually need the shield.. seems to + # make no difference (??) and we know it spawns an + # internal task.. + # await asyncio.shield(asyncio.sleep(.1)) + + # XXX alt approach but can block indefinitely.. + # so don't use? + # loop._run_once() + + try: + return trio_done_fut.result() + except asyncio.exceptions.InvalidStateError as state_err: + + # XXX be super dupere noisy about abandonment issues! + aio_task: asyncio.Task = asyncio.current_task() + message: str = ( + 'The `asyncio`-side task likely exited before the ' + '`trio`-side guest-run completed!\n\n' + ) + if fute_err: + message += ( + f'The main {aio_task}\n' + f'STOPPED due to {type(fute_err)}\n\n' + ) + + message += ( + f'Likely something inside our guest-run-as-task impl is ' + f'not effectively waiting on the `trio`-side to complete ?!\n' + f'This code -> {aio_main!r}\n\n' + + 'Below you will likely see a ' + '"RuntimeWarning: Trio guest run got abandoned.." !!\n' + ) + raise AsyncioRuntimeTranslationError(message) from state_err # might as well if it's installed. try: @@ -698,7 +804,7 @@ def run_as_asyncio_guest( loop = uvloop.new_event_loop() asyncio.set_event_loop(loop) except ImportError: - pass + log.runtime('`uvloop` not available..') return asyncio.run( aio_main(trio_main), From 82bee3c55b0335ff7ee9e13e4f4dc377c6608506 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 27 Jun 2024 19:27:59 -0400 Subject: [PATCH 05/66] Use `delay=0` in pump loop.. Turns out it does work XD Prior presumption was from before I had the fute poll-loop so makes sense we needed more then one sched-tick's worth of context switch vs. now we can just keep looping-n-pumping as fast possible until the guest-run's main task completes. Also, - minimize the preface commentary (as per todo) now that we have tests codifying all the edge cases :finger_crossed: - parameter-ize the pump-loop-cycle delay and default it to 0. --- tractor/to_asyncio.py | 116 ++++++++++++++++++++---------------------- 1 file changed, 55 insertions(+), 61 deletions(-) diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index fb18ba88..5564d0e1 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -558,6 +558,8 @@ def run_as_asyncio_guest( # normally `Actor._async_main()` as is passed by some boostrap # entrypoint like `._entry._trio_main()`. + _sigint_loop_pump_delay: float = 0, + ) -> None: # ^-TODO-^ technically whatever `trio_main` returns.. we should # try to use func-typevar-params at leaast by 3.13! @@ -598,7 +600,7 @@ def run_as_asyncio_guest( ''' loop = asyncio.get_running_loop() - trio_done_fut = asyncio.Future() + trio_done_fute = asyncio.Future() startup_msg: str = ( 'Starting `asyncio` guest-loop-run\n' '-> got running loop\n' @@ -633,13 +635,13 @@ def run_as_asyncio_guest( f'{error}\n\n' f'{tb_str}\n' ) - trio_done_fut.set_exception(error) + trio_done_fute.set_exception(error) # raise inline main_outcome.unwrap() else: - trio_done_fut.set_result(main_outcome) + trio_done_fute.set_result(main_outcome) startup_msg += ( f'-> created {trio_done_callback!r}\n' @@ -660,7 +662,7 @@ def run_as_asyncio_guest( ) fute_err: BaseException|None = None try: - out: Outcome = await asyncio.shield(trio_done_fut) + out: Outcome = await asyncio.shield(trio_done_fute) # NOTE will raise (via `Error.unwrap()`) from any # exception packed into the guest-run's `main_outcome`. @@ -697,83 +699,75 @@ def run_as_asyncio_guest( f' |_{actor}.cancel_soon()\n' ) - # TODO: reduce this comment bloc since abandon issues are - # now solved? + # XXX WARNING XXX the next LOCs are super important, since + # without them, we can get guest-run abandonment cases + # where `asyncio` will not schedule or wait on the `trio` + # guest-run task before final shutdown! This is + # particularly true if the `trio` side has tasks doing + # shielded work when a SIGINT condition occurs. # - # XXX NOTE XXX the next LOC is super important!!! - # => without it, we can get a guest-run abandonment case - # where asyncio will not trigger `trio` in a final event - # loop cycle! + # We now have the + # `test_infected_asyncio.test_sigint_closes_lifetime_stack()` + # suite to ensure we do not suffer this issues + # (hopefully) ever again. # - # our test, - # `test_infected_asyncio.test_sigint_closes_lifetime_stack()` - # demonstrates how if when we raise a SIGINT-signal in an infected - # child we get a variable race condition outcome where - # either of the following can indeterminately happen, + # The original abandonment issue surfaced as 2 different + # race-condition dependent types scenarios all to do with + # `asyncio` handling SIGINT from the system: # - # - "silent-abandon": `asyncio` abandons the `trio` - # guest-run task silently and no `trio`-guest-run or - # `tractor`-actor-runtime teardown happens whatsoever.. - # this is the WORST (race) case outcome. + # - "silent-abandon" (WORST CASE): + # `asyncio` abandons the `trio` guest-run task silently + # and no `trio`-guest-run or `tractor`-actor-runtime + # teardown happens whatsoever.. # - # - OR, "loud-abandon": the guest run get's abaondoned "loudly" with - # `trio` reporting a console traceback and further tbs of all - # the failed shutdown routines also show on console.. + # - "loud-abandon" (BEST-ish CASE): + # the guest run get's abaondoned "loudly" with `trio` + # reporting a console traceback and further tbs of all + # the (failed) GC-triggered shutdown routines which + # thankfully does get dumped to console.. # - # our test can thus fail and (has been parametrized for) - # the 2 cases: + # The abandonment is most easily reproduced if the `trio` + # side has tasks doing shielded work where those tasks + # ignore the normal `Cancelled` condition and continue to + # run, but obviously `asyncio` isn't aware of this and at + # some point bails on the guest-run unless we take manual + # intervention.. # - # - when the parent raises a KBI just after - # signalling the child, - # |_silent-abandon => the `Actor.lifetime_stack` will - # never be closed thus leaking a resource! - # -> FAIL! - # |_loud-abandon => despite the abandonment at least the - # stack will be closed out.. - # -> PASS + # To repeat, *WITHOUT THIS* stuff below the guest-run can + # get race-conditionally abandoned!! # - # - when the parent instead simply waits on `ctx.wait_for_result()` - # (i.e. DOES not raise a KBI itself), - # |_silent-abandon => test will just hang and thus the ctx - # and actor will never be closed/cancelled/shutdown - # resulting in leaking a (file) resource since the - # `trio`/`tractor` runtime never relays a ctxc back to - # the parent; the test's timeout will trigger.. - # -> FAIL! - # |_loud-abandon => this case seems to never happen?? + # XXX SOLUTION XXX + # ------ - ------ + # XXX FIRST PART: + # ------ - ------ + # the obvious fix to the "silent-abandon" case is to + # explicitly cancel the actor runtime such that no + # runtime tasks are even left unaware that the guest-run + # should be terminated due to OS cancellation. # - # XXX FIRST PART XXX, SO, this is a fix to the - # "silent-abandon" case, NOT the `trio`-guest-run - # abandonment issue in general, for which the NEXT LOC - # is apparently a working fix! actor.cancel_soon() - # XXX NOTE XXX pump the `asyncio` event-loop to allow + # ------ - ------ + # XXX SECOND PART: + # ------ - ------ + # Pump the `asyncio` event-loop to allow # `trio`-side to `trio`-guest-run to complete and # teardown !! # - # *WITHOUT THIS* the guest-run can get race-conditionally abandoned!! - # XD - # - await asyncio.sleep(.1) # `delay` can't be 0 either XD - while not trio_done_fut.done(): + # oh `asyncio`, how i don't miss you at all XD + while not trio_done_fute.done(): log.runtime( 'Waiting on main guest-run `asyncio` task to complete..\n' - f'|_trio_done_fut: {trio_done_fut}\n' + f'|_trio_done_fut: {trio_done_fute}\n' ) - await asyncio.sleep(.1) + await asyncio.sleep(_sigint_loop_pump_delay) - # XXX: don't actually need the shield.. seems to - # make no difference (??) and we know it spawns an - # internal task.. - # await asyncio.shield(asyncio.sleep(.1)) - - # XXX alt approach but can block indefinitely.. - # so don't use? + # XXX is there any alt API/approach like the internal + # call below but that doesn't block indefinitely..? # loop._run_once() try: - return trio_done_fut.result() + return trio_done_fute.result() except asyncio.exceptions.InvalidStateError as state_err: # XXX be super dupere noisy about abandonment issues! From b057a1681c6c0286977e613dccae9d03c887c4d4 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 28 Jun 2024 19:26:35 -0400 Subject: [PATCH 06/66] Todo a test for sync-pausing from non-main-root-tasks --- tests/test_debugger.py | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/tests/test_debugger.py b/tests/test_debugger.py index e4f28548..a32eb2cf 100644 --- a/tests/test_debugger.py +++ b/tests/test_debugger.py @@ -1336,6 +1336,23 @@ def test_shield_pause( child.expect(pexpect.EOF) +# TODO: better error for "non-ideal" usage from the root actor. +# -[ ] if called from an async scope emit a message that suggests +# using `await tractor.pause()` instead since it's less overhead +# (in terms of `greenback` and/or extra threads) and if it's from +# a sync scope suggest that usage must first call +# `ensure_portal()` in the (eventual parent) async calling scope? +def test_sync_pause_from_bg_task_in_root_actor_(): + ''' + When used from the root actor, normally we can only implicitly + support `.pause_from_sync()` from the main-parent-task (that + opens the runtime via `open_root_actor()`) since `greenback` + requires a `.ensure_portal()` call per `trio.Task` where it is + used. + + ''' + ... + # TODO: needs ANSI code stripping tho, see `assert_before()` # above! def test_correct_frames_below_hidden(): ''' From 607e1dcf45e2b54218434238d3306a1817b3c52f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 2 Jul 2024 17:06:50 -0400 Subject: [PATCH 07/66] More failed REPL-lock-request refinements In `lock_stdio_for_peer()` better internal-error handling/reporting: - only `Lock._blocked.remove(ctx.cid)` if that same cid was added on entry to avoid needless key-errors. - drop all `Lock.release(force: bool)` usage remnants. - if `req_ctx.cancel()` fails mention it with `ctx_err.add_note()`. - add more explicit internal-failed-request log messaging via a new `fail_reason: str`. - use and use new `x)<=\n|_` annots in any failure logging. Other cleanups/niceties: - drop `force: bool` flag entirely from the `Lock.release()`. - use more supervisor-op-annots in `.pdb()` logging with both `_pause/crash_msg: str` instead of double '|' lines when `.pdb()`-reported from `._set_trace()`/`._post_mortem()`. --- tractor/devx/_debug.py | 96 +++++++++++++++++++++++------------------- 1 file changed, 53 insertions(+), 43 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 1135932c..113371d8 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -299,7 +299,6 @@ class Lock: @pdbp.hideframe def release( cls, - force: bool = False, raise_on_thread: bool = True, ) -> bool: @@ -347,12 +346,9 @@ class Lock: lock: trio.StrictFIFOLock = cls._debug_lock owner: Task = lock.statistics().owner if ( - (lock.locked() or force) - # ^-TODO-NOTE-^ should we just remove this, since the - # RTE case above will always happen when you force - # from the wrong task? - - and (owner is task) + lock.locked() + and + (owner is task) # ^-NOTE-^ if we do NOT ensure this, `trio` will # raise a RTE when a non-owner tries to releasee the # lock. @@ -553,6 +549,7 @@ async def lock_stdio_for_peer( # can try to avoid clobbering any connection from a child # that's currently relying on it. we_finished = Lock.req_handler_finished = trio.Event() + lock_blocked: bool = False try: if ctx.cid in Lock._blocked: raise RuntimeError( @@ -565,7 +562,8 @@ async def lock_stdio_for_peer( 'Consider that an internal bug exists given the TTY ' '`Lock`ing IPC dialog..\n' ) - + Lock._blocked.add(ctx.cid) + lock_blocked = True root_task_name: str = current_task().name if tuple(subactor_uid) in Lock._blocked: log.warning( @@ -575,7 +573,11 @@ async def lock_stdio_for_peer( ) ctx._enter_debugger_on_cancel: bool = False message: str = ( - f'Debug lock blocked for {subactor_uid}\n' + f'Debug lock blocked for subactor\n\n' + f'x)<= {subactor_uid}\n\n' + + f'Likely because the root actor already started shutdown and is ' + 'closing IPC connections for this child!\n\n' 'Cancelling debug request!\n' ) log.cancel(message) @@ -589,7 +591,6 @@ async def lock_stdio_for_peer( f'remote task: {subactor_task_uid}\n' ) DebugStatus.shield_sigint() - Lock._blocked.add(ctx.cid) # NOTE: we use the IPC ctx's cancel scope directly in order to # ensure that on any transport failure, or cancellation request @@ -648,31 +649,34 @@ async def lock_stdio_for_peer( ) except BaseException as req_err: - message: str = ( - f'On behalf of remote peer {subactor_task_uid!r}@{ctx.chan.uid!r}\n\n' - 'Forcing `Lock.release()` for req-ctx since likely an ' - 'internal error!\n\n' - f'{ctx}' + fail_reason: str = ( + f'on behalf of peer\n\n' + f'x)<=\n' + f' |_{subactor_task_uid!r}@{ctx.chan.uid!r}\n\n' + + 'Forcing `Lock.release()` due to acquire failure!\n\n' + f'x)=> {ctx}\n' ) if isinstance(req_err, trio.Cancelled): - message = ( - 'Cancelled during root TTY-lock dialog\n' + fail_reason = ( + 'Cancelled during stdio-mutex request ' + - message + fail_reason ) else: - message = ( - 'Errored during root TTY-lock dialog\n' + fail_reason = ( + 'Failed to deliver stdio-mutex request ' + - message + fail_reason ) - log.exception(message) - Lock.release() #force=True) + log.exception(fail_reason) + Lock.release() raise finally: - Lock._blocked.remove(ctx.cid) + if lock_blocked: + Lock._blocked.remove(ctx.cid) # wakeup any waiters since the lock was (presumably) # released, possibly only temporarily. @@ -1167,7 +1171,7 @@ async def request_root_stdio_lock( ): log.cancel( 'Debug lock request was CANCELLED?\n\n' - f'{req_ctx}\n' + f'<=c) {req_ctx}\n' # f'{pformat_cs(req_cs, var_name="req_cs")}\n\n' # f'{pformat_cs(req_ctx._scope, var_name="req_ctx._scope")}\n\n' ) @@ -1179,22 +1183,26 @@ async def request_root_stdio_lock( message: str = ( 'Failed during debug request dialog with root actor?\n\n' ) - - if req_ctx: + if (req_ctx := DebugStatus.req_ctx): message += ( - f'{req_ctx}\n' + f'<=x) {req_ctx}\n\n' f'Cancelling IPC ctx!\n' ) - await req_ctx.cancel() + try: + await req_ctx.cancel() + except trio.ClosedResourceError as terr: + ctx_err.add_note( + # f'Failed with {type(terr)!r} x)> `req_ctx.cancel()` ' + f'Failed with `req_ctx.cancel()` (\n' + f' |_ {task} @ {actor.uid}\n' + # ^-TODO-^ more compact pformating? # -[ ] make an `Actor.__repr()__` # -[ ] should we use `log.pformat_task_uid()`? - f'|_ {task} @ {actor.uid}\n' ) # presuming the caller passed in the "api frame" # (the last frame before user code - like `.pause()`) @@ -2541,9 +2551,9 @@ def _post_mortem( # here! Bo log.pdb( f'{_crash_msg}\n' - '|\n' - # f'|_ {current_task()}\n' - f'|_ {current_task()} @ {actor.uid}\n' + # '|\n' + f'x>(\n' + f' |_ {current_task()} @ {actor.uid}\n' # f'|_ @{actor.uid}\n' # TODO: make an `Actor.__repr()__` From 4a08d586cdcaaa9274666baa7bc9983c17c03f96 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 5 Jul 2024 13:32:03 -0400 Subject: [PATCH 08/66] Another tweak to REPL entry `.pdb()` headers --- tractor/devx/_debug.py | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 113371d8..ae1c46db 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -1552,7 +1552,7 @@ def shield_sigint_handler( log.devx('exiting SIGINT') -_pause_msg: str = 'Attaching to pdb REPL in actor' +_pause_msg: str = 'Opening a pdb REPL in paused actor' class DebugRequestError(RuntimeError): @@ -2050,9 +2050,8 @@ def _set_trace( # root here? Bo log.pdb( f'{_pause_msg}\n' - # '|\n' f'>(\n' - f' |_ {task} @ {actor.uid}\n' + f'|_ {task} @ {actor.uid}\n' # ^-TODO-^ more compact pformating? # -[ ] make an `Actor.__repr()__` # -[ ] should we use `log.pformat_task_uid()`? @@ -2523,7 +2522,7 @@ async def breakpoint( _crash_msg: str = ( - 'Attaching to pdb REPL in crashed actor' + 'Opening a pdb REPL in crashed actor' ) @@ -2551,11 +2550,9 @@ def _post_mortem( # here! Bo log.pdb( f'{_crash_msg}\n' - # '|\n' f'x>(\n' - f' |_ {current_task()} @ {actor.uid}\n' + f' |_ {current_task()} @ {actor.uid}\n' - # f'|_ @{actor.uid}\n' # TODO: make an `Actor.__repr()__` # f'|_ {current_task()} @ {actor.name}\n' ) From 92713af63eb4d795467cd062b650b39966011e58 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 8 Jul 2024 20:57:41 -0400 Subject: [PATCH 09/66] Get multi-threaded sync-pausing fully workin! The final issue was making sure we do the same thing on ctl-c/SIGINT from the user. That is, if there's already a bg-thread in REPL, we `log.pdb()` about SIGINT shielding and re-draw the prompt; the same UX as normal actor-runtime-task behaviour. Reasons this wasn't workin.. and the fix: - `.pause_from_sync()` was overriding the local `repl` var with `None` delivered by (transitive) calls to `_pause(debug_func=None)`.. so remove all that and only assign it OAOO prior to thread-type case branching. - always call `DebugStatus.shield_sigint()` as needed from all requesting threads/tasks: - in `_pause_from_bg_root_thread()` BEFORE calling `._pause()` AND BEFORE yielding back to the bg-thread via `.started(out)` to ensure we're definitely overriding the handler in the `trio`-main-thread task before unblocking the requesting bg-thread. - from any requesting bg-thread in the root actor such that both its main-`trio`-thread scheduled task (as per above bullet) AND it are SIGINT shielded. - always call `.shield_sigint()` BEFORE any `greenback._await()` case don't entirely grok why yet, but it works)? - for `greenback._await()` case always set `bg_task` to the current one.. - tweaks to the `SIGINT` handler, now renamed `sigint_shield()` so as not to name-collide with the methods when editor-searching: - always try to `repr()` the REPL thread/task "owner" as well as the active `PdbREPL` instance. - add `.devx()` notes around the prompt flushing deats and comments for any root-actor-bg-thread edge cases. Related/supporting refinements: - add `get_lock()`/`get_debug_req()` factory funcs since the plan is to eventually implement both as `@singleton` instances per actor. - fix `acquire_debug_lock()`'s call-sig-bug for scheduling `request_root_stdio_lock()`.. - in `._pause()` only call `mk_pdb()` when `debug_func != None`. - add some todo/warning notes around the `cls.repl = None` in `DebugStatus.release()` `test_pause_from_sync()` tweaks: - don't use a `attach_patts.copy()`, since we always `break` on match. - do `pytest.fail()` on that ^ loop's fallthrough.. - pass `do_ctlc(child, patt=attach_key)` such that we always match the the current thread's name with the ctl-c triggered `.pdb()` emission. - oh yeah, return the last `before: str` from `do_ctlc()`. - in the script, flip `abandon_on_cancel=True` since when `False` it seems to cause `trio.run()` to hang on exit from the last bg-thread case?!? --- examples/debugging/sync_bp.py | 24 ++++- tests/test_debugger.py | 57 +++++++++-- tractor/devx/__init__.py | 2 +- tractor/devx/_debug.py | 184 ++++++++++++++++++++++++++-------- 4 files changed, 211 insertions(+), 56 deletions(-) diff --git a/examples/debugging/sync_bp.py b/examples/debugging/sync_bp.py index 137710fc..95472c93 100644 --- a/examples/debugging/sync_bp.py +++ b/examples/debugging/sync_bp.py @@ -4,6 +4,13 @@ import time import trio import tractor +# TODO: only import these when not running from test harness? +# can we detect `pexpect` usage maybe? +# from tractor.devx._debug import ( +# get_lock, +# get_debug_req, +# ) + def sync_pause( use_builtin: bool = False, @@ -18,7 +25,13 @@ def sync_pause( breakpoint(hide_tb=hide_tb) else: + # TODO: maybe for testing some kind of cm style interface + # where the `._set_trace()` call doesn't happen until block + # exit? + # assert get_lock().ctx_in_debug is None + # assert get_debug_req().repl is None tractor.pause_from_sync() + # assert get_debug_req().repl is None if error: raise RuntimeError('yoyo sync code error') @@ -41,10 +54,11 @@ async def start_n_sync_pause( async def main() -> None: async with ( tractor.open_nursery( - # NOTE: required for pausing from sync funcs - maybe_enable_greenback=True, debug_mode=True, - # loglevel='cancel', + maybe_enable_greenback=True, + enable_stack_on_sig=True, + # loglevel='warning', + # loglevel='devx', ) as an, trio.open_nursery() as tn, ): @@ -138,7 +152,9 @@ async def main() -> None: # the case 2. from above still exists! use_builtin=True, ), - abandon_on_cancel=False, + # TODO: with this `False` we can hang!??! + # abandon_on_cancel=False, + abandon_on_cancel=True, thread_name='inline_root_bg_thread', ) diff --git a/tests/test_debugger.py b/tests/test_debugger.py index a32eb2cf..347a6849 100644 --- a/tests/test_debugger.py +++ b/tests/test_debugger.py @@ -299,7 +299,9 @@ def do_ctlc( # needs some further investigation potentially... expect_prompt: bool = not _ci_env, -) -> None: +) -> str|None: + + before: str|None = None # make sure ctl-c sends don't do anything but repeat output for _ in range(count): @@ -309,15 +311,18 @@ def do_ctlc( # TODO: figure out why this makes CI fail.. # if you run this test manually it works just fine.. if expect_prompt: - before = str(child.before.decode()) time.sleep(delay) child.expect(PROMPT) + before = str(child.before.decode()) time.sleep(delay) if patt: # should see the last line on console assert patt in before + # return the console content up to the final prompt + return before + def test_root_actor_bp_forever( spawn, @@ -1085,10 +1090,10 @@ def test_pause_from_sync( ) if ctlc: do_ctlc(child) + # ^NOTE^ subactor not spawned yet; don't need extra delay. child.sendline('c') - # first `await tractor.pause()` inside `p.open_context()` body child.expect(PROMPT) @@ -1109,7 +1114,27 @@ def test_pause_from_sync( ) if ctlc: - do_ctlc(child) + do_ctlc( + child, + # NOTE: setting this to 0 (or some other sufficient + # small val) can cause the test to fail since the + # `subactor` suffers a race where the root/parent + # sends an actor-cancel prior to it hitting its pause + # point; by def the value is 0.1 + delay=0.3, + ) + + # XXX, fwiw without a brief sleep here the SIGINT might actually + # trigger "subactor" cancellation by its parent before the + # shield-handler is engaged. + # + # => similar to the `delay` input to `do_ctlc()` below, setting + # this too low can cause the test to fail since the `subactor` + # suffers a race where the root/parent sends an actor-cancel + # prior to the context task hitting its pause point (and thus + # engaging the `sigint_shield()` handler in time); this value + # seems be good enuf? + time.sleep(0.6) # one of the bg thread or subactor should have # `Lock.acquire()`-ed @@ -1128,29 +1153,45 @@ def test_pause_from_sync( "('root'", ], } + conts: int = 0 # for debugging below matching logic on failure while attach_patts: child.sendline('c') + conts += 1 child.expect(PROMPT) before = str(child.before.decode()) - for key in attach_patts.copy(): + for key in attach_patts: if key in before: + attach_key: str = key expected_patts: str = attach_patts.pop(key) assert_before( child, - [_pause_msg] + expected_patts + [_pause_msg] + + + expected_patts ) break + else: + pytest.fail( + f'No keys found?\n\n' + f'{attach_patts.keys()}\n\n' + f'{before}\n' + ) # ensure no other task/threads engaged a REPL # at the same time as the one that was detected above. - for key, other_patts in attach_patts.items(): + for key, other_patts in attach_patts.copy().items(): assert not in_prompt_msg( before, other_patts, ) if ctlc: - do_ctlc(child) + do_ctlc( + child, + patt=attach_key, + # NOTE same as comment above + delay=0.3, + ) child.sendline('c') child.expect(pexpect.EOF) diff --git a/tractor/devx/__init__.py b/tractor/devx/__init__.py index cfcff931..7047dbdb 100644 --- a/tractor/devx/__init__.py +++ b/tractor/devx/__init__.py @@ -26,7 +26,7 @@ from ._debug import ( breakpoint as breakpoint, pause as pause, pause_from_sync as pause_from_sync, - shield_sigint_handler as shield_sigint_handler, + sigint_shield as sigint_shield, open_crash_handler as open_crash_handler, maybe_open_crash_handler as maybe_open_crash_handler, maybe_init_greenback as maybe_init_greenback, diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index ae1c46db..02551fa1 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -409,9 +409,9 @@ class Lock: repl_task ) message += ( - f'\nA non-caller task still owns this lock on behalf of ' - f'{behalf_of_task}\n' - f'|_{lock_stats.owner}\n' + f'A non-caller task still owns this lock on behalf of ' + f'`{behalf_of_task}`\n' + f'lock owner task: {lock_stats.owner}\n' ) if ( @@ -523,6 +523,10 @@ class Lock: ) +def get_lock() -> Lock: + return Lock + + @tractor.context( # enable the locking msgspec pld_spec=__pld_spec__, @@ -788,13 +792,13 @@ class DebugStatus: cls._orig_sigint_handler: Callable = trio.from_thread.run_sync( signal.signal, signal.SIGINT, - shield_sigint_handler, + sigint_shield, ) else: cls._orig_sigint_handler = signal.signal( signal.SIGINT, - shield_sigint_handler, + sigint_shield, ) @classmethod @@ -900,12 +904,30 @@ class DebugStatus: # actor-local state, irrelevant for non-root. cls.repl_task = None + + # XXX WARNING needs very special caughtion, and we should + # prolly make a more explicit `@property` API? + # + # - if unset in root multi-threaded case can cause + # issues with detecting that some root thread is + # using a REPL, + # + # - what benefit is there to unsetting, it's always + # set again for the next task in some actor.. + # only thing would be to avoid in the sigint-handler + # logging when we don't need to? cls.repl = None # restore original sigint handler cls.unshield_sigint() + +# TODO: use the new `@lowlevel.singleton` for this! +def get_debug_req() -> DebugStatus|None: + return DebugStatus + + class TractorConfig(pdbp.DefaultConfig): ''' Custom `pdbp` config which tries to use the best tradeoff @@ -1311,7 +1333,7 @@ def any_connected_locker_child() -> bool: return False -def shield_sigint_handler( +def sigint_shield( signum: int, frame: 'frame', # type: ignore # noqa *args, @@ -1351,13 +1373,17 @@ def shield_sigint_handler( # root actor branch that reports whether or not a child # has locked debugger. if is_root_process(): + # log.warning( + log.devx( + 'Handling SIGINT in root actor\n' + f'{Lock.repr()}' + f'{DebugStatus.repr()}\n' + ) # try to see if the supposed (sub)actor in debug still # has an active connection to *this* actor, and if not # it's likely they aren't using the TTY lock / debugger # and we should propagate SIGINT normally. any_connected: bool = any_connected_locker_child() - # if not any_connected: - # return do_cancel() problem = ( f'root {actor.uid} handling SIGINT\n' @@ -1406,19 +1432,25 @@ def shield_sigint_handler( # an actor using the `Lock` (a bug state) ?? # => so immediately cancel any stale lock cs and revert # the handler! - if not repl: + if not DebugStatus.repl: # TODO: WHEN should we revert back to ``trio`` # handler if this one is stale? # -[ ] maybe after a counts work of ctl-c mashes? # -[ ] use a state var like `stale_handler: bool`? problem += ( - '\n' 'No subactor is using a `pdb` REPL according `Lock.ctx_in_debug`?\n' - 'BUT, the root should be using it, WHY this handler ??\n' + 'BUT, the root should be using it, WHY this handler ??\n\n' + 'So either..\n' + '- some root-thread is using it but has no `.repl` set?, OR\n' + '- something else weird is going on outside the runtime!?\n' ) else: + # NOTE: since we emit this msg on ctl-c, we should + # also always re-print the prompt the tail block! log.pdb( 'Ignoring SIGINT while pdb REPL in use by root actor..\n' + f'{DebugStatus.repl_task}\n' + f' |_{repl}\n' ) problem = None @@ -1468,7 +1500,6 @@ def shield_sigint_handler( 'Allowing SIGINT propagation..' ) DebugStatus.unshield_sigint() - # do_cancel() repl_task: str|None = DebugStatus.repl_task req_task: str|None = DebugStatus.req_task @@ -1483,10 +1514,15 @@ def shield_sigint_handler( f' |_{repl}\n' ) elif req_task: - log.pdb( - f'Ignoring SIGINT while debug request task is open\n' + log.debug( + 'Ignoring SIGINT while debug request task is open but either,\n' + '- someone else is already REPL-in and has the `Lock`, or\n' + '- some other local task already is replin?\n' f'|_{req_task}\n' ) + + # TODO can we remove this now? + # -[ ] does this path ever get hit any more? else: msg: str = ( 'SIGINT shield handler still active BUT, \n\n' @@ -1522,31 +1558,47 @@ def shield_sigint_handler( # https://github.com/goodboy/tractor/issues/320 # elif debug_mode(): - # NOTE: currently (at least on ``fancycompleter`` 0.9.2) - # it looks to be that the last command that was run (eg. ll) - # will be repeated by default. - # maybe redraw/print last REPL output to console since # we want to alert the user that more input is expect since # nothing has been done dur to ignoring sigint. if ( - repl # only when current actor has a REPL engaged + DebugStatus.repl # only when current actor has a REPL engaged ): + flush_status: str = ( + 'Flushing stdout to ensure new prompt line!\n' + ) + # XXX: yah, mega hack, but how else do we catch this madness XD - if repl.shname == 'xonsh': + if ( + repl.shname == 'xonsh' + ): + flush_status += ( + '-> ALSO re-flushing due to `xonsh`..\n' + ) repl.stdout.write(repl.prompt) + # log.warning( + log.devx( + flush_status + ) repl.stdout.flush() - # TODO: make this work like sticky mode where if there is output - # detected as written to the tty we redraw this part underneath - # and erase the past draw of this same bit above? + # TODO: better console UX to match the current "mode": + # -[ ] for example if in sticky mode where if there is output + # detected as written to the tty we redraw this part underneath + # and erase the past draw of this same bit above? # repl.sticky = True # repl._print_if_sticky() - # also see these links for an approach from ``ptk``: + # also see these links for an approach from `ptk`: # https://github.com/goodboy/tractor/issues/130#issuecomment-663752040 # https://github.com/prompt-toolkit/python-prompt-toolkit/blob/c2c6af8a0308f9e5d7c0e28cb8a02963fe0ce07a/prompt_toolkit/patch_stdout.py + else: + log.devx( + # log.warning( + 'Not flushing stdout since not needed?\n' + f'|_{repl}\n' + ) # XXX only for tracing this handler log.devx('exiting SIGINT') @@ -1617,7 +1669,7 @@ async def _pause( # 'directly (infected) `asyncio` tasks!' # ) from rte - raise + raise rte if debug_func is not None: debug_func = partial(debug_func) @@ -1625,9 +1677,13 @@ async def _pause( # XXX NOTE XXX set it here to avoid ctl-c from cancelling a debug # request from a subactor BEFORE the REPL is entered by that # process. - if not repl: + if ( + not repl + and + debug_func + ): + repl: PdbREPL = mk_pdb() DebugStatus.shield_sigint() - repl: PdbREPL = repl or mk_pdb() # TODO: move this into a `open_debug_request()` @acm? # -[ ] prolly makes the most sense to do the request @@ -1662,7 +1718,13 @@ async def _pause( # recurrent entries/requests from the same # actor-local task. DebugStatus.repl_task = task - DebugStatus.repl = repl + if repl: + DebugStatus.repl = repl + else: + log.error( + 'No REPl instance set before entering `debug_func`?\n' + f'{debug_func}\n' + ) # invoke the low-level REPL activation routine which itself # should call into a `Pdb.set_trace()` of some sort. @@ -2001,7 +2063,7 @@ async def _pause( DebugStatus.release(cancel_req_task=True) # sanity checks for ^ on request/status teardown - assert DebugStatus.repl is None + # assert DebugStatus.repl is None # XXX no more bc bg thread cases? assert DebugStatus.repl_task is None # sanity, for when hackin on all this? @@ -2240,7 +2302,12 @@ async def _pause_from_bg_root_thread( 'Trying to acquire `Lock` on behalf of bg thread\n' f'|_{behalf_of_thread}\n' ) - # DebugStatus.repl_task = behalf_of_thread + + # NOTE: this is already a task inside the main-`trio`-thread, so + # we don't need to worry about calling it another time from the + # bg thread on which who's behalf this task is operating. + DebugStatus.shield_sigint() + out = await _pause( debug_func=None, repl=repl, @@ -2249,6 +2316,8 @@ async def _pause_from_bg_root_thread( called_from_bg_thread=True, **_pause_kwargs ) + DebugStatus.repl_task = behalf_of_thread + lock: trio.FIFOLock = Lock._debug_lock stats: trio.LockStatistics= lock.statistics() assert stats.owner is task @@ -2282,7 +2351,6 @@ async def _pause_from_bg_root_thread( f'|_{behalf_of_thread}\n' ) task_status.started(out) - DebugStatus.shield_sigint() # wait for bg thread to exit REPL sesh. try: @@ -2323,7 +2391,7 @@ def pause_from_sync( err_on_no_runtime=False, ) message: str = ( - f'{actor.uid} task called `tractor.pause_from_sync()`\n\n' + f'{actor.uid} task called `tractor.pause_from_sync()`\n' ) if not actor: raise RuntimeError( @@ -2347,7 +2415,6 @@ def pause_from_sync( 'for infected `asyncio` mode!' ) - DebugStatus.shield_sigint() repl: PdbREPL = mk_pdb() # message += f'-> created local REPL {repl}\n' @@ -2365,6 +2432,10 @@ def pause_from_sync( # thread which will call `._pause()` manually with special # handling for root-actor caller usage. if not DebugStatus.is_main_trio_thread(): + + # TODO: `threading.Lock()` this so we don't get races in + # multi-thr cases where they're acquiring/releasing the + # REPL and setting request/`Lock` state, etc.. thread: threading.Thread = threading.current_thread() repl_owner = thread @@ -2372,9 +2443,16 @@ def pause_from_sync( if is_root: message += ( f'-> called from a root-actor bg {thread}\n' - f'-> scheduling `._pause_from_sync_thread()`..\n' + f'-> scheduling `._pause_from_bg_root_thread()`..\n' ) - bg_task, repl = trio.from_thread.run( + # XXX SUBTLE BADNESS XXX that should really change! + # don't over-write the `repl` here since when + # this behalf-of-bg_thread-task calls pause it will + # pass `debug_func=None` which will result in it + # returing a `repl==None` output and that get's also + # `.started(out)` back here! So instead just ignore + # that output and assign the `repl` created above! + bg_task, _ = trio.from_thread.run( afn=partial( actor._service_n.start, partial( @@ -2386,8 +2464,9 @@ def pause_from_sync( ), ) ) + DebugStatus.shield_sigint() message += ( - f'-> `._pause_from_sync_thread()` started bg task {bg_task}\n' + f'-> `._pause_from_bg_root_thread()` started bg task {bg_task}\n' ) else: message += f'-> called from a bg {thread}\n' @@ -2396,7 +2475,7 @@ def pause_from_sync( # `request_root_stdio_lock()` and we don't need to # worry about all the special considerations as with # the root-actor per above. - bg_task, repl = trio.from_thread.run( + bg_task, _ = trio.from_thread.run( afn=partial( _pause, debug_func=None, @@ -2411,6 +2490,9 @@ def pause_from_sync( **_pause_kwargs ), ) + # ?TODO? XXX where do we NEED to call this in the + # subactor-bg-thread case? + DebugStatus.shield_sigint() assert bg_task is not DebugStatus.repl_task else: # we are presumably the `trio.run()` + main thread @@ -2423,6 +2505,11 @@ def pause_from_sync( # greenback: ModuleType = await maybe_init_greenback() message += f'-> imported {greenback}\n' + + # NOTE XXX seems to need to be set BEFORE the `_pause()` + # invoke using gb below? + DebugStatus.shield_sigint() + repl_owner: Task = current_task() message += '-> calling `greenback.await_(_pause(debug_func=None))` from sync caller..\n' try: @@ -2448,9 +2535,12 @@ def pause_from_sync( raise if out: - bg_task, repl = out - assert repl is repl - assert bg_task is repl_owner + bg_task, _ = out + else: + bg_task: Task = current_task() + + # assert repl is repl + assert bg_task is repl_owner # NOTE: normally set inside `_enter_repl_sync()` DebugStatus.repl_task: str = repl_owner @@ -2464,7 +2554,10 @@ def pause_from_sync( ) log.devx(message) + # NOTE set as late as possible to avoid state clobbering + # in the multi-threaded case! DebugStatus.repl = repl + _set_trace( api_frame=api_frame or inspect.currentframe(), repl=repl, @@ -2665,7 +2758,8 @@ async def acquire_debug_lock( tuple, ]: ''' - Request to acquire the TTY `Lock` in the root actor, release on exit. + Request to acquire the TTY `Lock` in the root actor, release on + exit. This helper is for actor's who don't actually need to acquired the debugger but want to wait until the lock is free in the @@ -2677,10 +2771,14 @@ async def acquire_debug_lock( yield None return + task: Task = current_task() async with trio.open_nursery() as n: ctx: Context = await n.start( - request_root_stdio_lock, - subactor_uid, + partial( + request_root_stdio_lock, + actor_uid=subactor_uid, + task_uid=(task.name, id(task)), + ) ) yield ctx ctx.cancel() From f195c5ec47f389ae0441ff3feab956b61a2361e7 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 10 Jul 2024 15:40:44 -0400 Subject: [PATCH 10/66] Move `mk_cmd()` to `._testing` Since we're going to need it more generally for `.devx` sub-sys tooling tests. Also, up the sync-pause ctl-c delay another 10ms.. --- tests/test_debugger.py | 36 +++++++++++++++++++----------------- tractor/_testing/__init__.py | 19 +++++++++++++++++++ 2 files changed, 38 insertions(+), 17 deletions(-) diff --git a/tests/test_debugger.py b/tests/test_debugger.py index 347a6849..03b3ae4f 100644 --- a/tests/test_debugger.py +++ b/tests/test_debugger.py @@ -13,7 +13,6 @@ TODO: from functools import partial import itertools import platform -import pathlib import time import pytest @@ -24,7 +23,7 @@ from pexpect.exceptions import ( ) from tractor._testing import ( - examples_dir, + mk_cmd, ) from tractor.devx._debug import ( _pause_msg, @@ -52,15 +51,6 @@ if platform.system() == 'Windows': ) -def mk_cmd(ex_name: str) -> str: - ''' - Generate a command suitable to pass to ``pexpect.spawn()``. - - ''' - script_path: pathlib.Path = examples_dir() / 'debugging' / f'{ex_name}.py' - return ' '.join(['python', str(script_path)]) - - # TODO: was trying to this xfail style but some weird bug i see in CI # that's happening at collect time.. pretty soon gonna dump actions i'm # thinkin... @@ -84,19 +74,31 @@ def spawn( start_method, testdir, reg_addr, -) -> 'pexpect.spawn': +) -> 'pexpect.spawn': + ''' + Use the `pexpect` module shipped via `testdir.spawn()` to + run an `./examples/..` script by name. + + ''' if start_method != 'trio': pytest.skip( - "Debugger tests are only supported on the trio backend" + '`pexpect` based tests only supported on `trio` backend' ) - def _spawn(cmd): + def _spawn( + cmd: str, + **mkcmd_kwargs, + ): return testdir.spawn( - cmd=mk_cmd(cmd), + cmd=mk_cmd( + cmd, + **mkcmd_kwargs, + ), expect_timeout=3, ) + # such that test-dep can pass input script name. return _spawn @@ -1121,7 +1123,7 @@ def test_pause_from_sync( # `subactor` suffers a race where the root/parent # sends an actor-cancel prior to it hitting its pause # point; by def the value is 0.1 - delay=0.3, + delay=0.4, ) # XXX, fwiw without a brief sleep here the SIGINT might actually @@ -1190,7 +1192,7 @@ def test_pause_from_sync( child, patt=attach_key, # NOTE same as comment above - delay=0.3, + delay=0.4, ) child.sendline('c') diff --git a/tractor/_testing/__init__.py b/tractor/_testing/__init__.py index fd79fe20..1f6624e9 100644 --- a/tractor/_testing/__init__.py +++ b/tractor/_testing/__init__.py @@ -54,6 +54,25 @@ def examples_dir() -> pathlib.Path: return repodir() / 'examples' +def mk_cmd( + ex_name: str, + exs_subpath: str = 'debugging', +) -> str: + ''' + Generate a shell command suitable to pass to ``pexpect.spawn()``. + + ''' + script_path: pathlib.Path = ( + examples_dir() + / exs_subpath + / f'{ex_name}.py' + ) + return ' '.join([ + 'python', + str(script_path) + ]) + + @acm async def expect_ctxc( yay: bool, From de7b1143037e7aaa83fc1daa67bdf3a5be41bf96 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 10 Jul 2024 15:52:38 -0400 Subject: [PATCH 11/66] Start a new `tests/devx/` tooling-subsuite-pkg --- tests/{ => devx}/test_debugger.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/{ => devx}/test_debugger.py (100%) diff --git a/tests/test_debugger.py b/tests/devx/test_debugger.py similarity index 100% rename from tests/test_debugger.py rename to tests/devx/test_debugger.py From 64d506970adc0b19b46a0223dd2d63dc5846bbe2 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 10 Jul 2024 18:17:42 -0400 Subject: [PATCH 12/66] Officially test proto-ed `stackscope` integration By re-purposing our `pexpect`-based console matching with a new `debugging/shield_hang_in_sub.py` example, this tests a few "hanging actor" conditions more formally: - that despite a hanging actor's task we can dump a `stackscope.extract()` tree on relay of `SIGUSR1`. - the actor tree will terminate despite a shielded forever-sleep by our "T-800" zombie reaper machinery activating and hard killing the underlying subprocess. Some test deats: - simulates the expect actions of a real user by manually using `os.kill()` to send both signals to the actor-tree program. - `pexpect`-matches against `log.devx()` emissions under normal `debug_mode == True` usage. - ensure we get the actual "T-800 deployed" `log.error()` msg and that the actor tree eventually terminates! Surrounding (re-org/impl/test-suite) changes: - allow disabling usage via a `maybe_enable_greenback: bool` to `open_root_actor()` but enable by def. - pretty up the actual `.devx()` content from `.devx._stackscope` including be extra pedantic about the conc-primitives for each signal event. - try to avoid double handles of `SIGUSR1` even though it seems the original (what i thought was a) problem was actually just double logging in the handler.. |_ avoid double applying the handler func via `signal.signal()`, |_ use a global to avoid double handle func calls and, |_ a `threading.RLock` around handling. - move common fixtures and helper routines from `test_debugger` to `tests/devx/conftest.py` and import them for use in both test mods. --- examples/debugging/shield_hang_in_sub.py | 81 +++++++++++ tests/devx/__init__.py | 0 tests/devx/conftest.py | 167 +++++++++++++++++++++ tests/devx/test_debugger.py | 178 ++--------------------- tests/devx/test_tooling.py | 120 +++++++++++++++ tractor/_root.py | 12 +- tractor/devx/_stackscope.py | 166 +++++++++++++++++---- 7 files changed, 521 insertions(+), 203 deletions(-) create mode 100644 examples/debugging/shield_hang_in_sub.py create mode 100644 tests/devx/__init__.py create mode 100644 tests/devx/conftest.py create mode 100644 tests/devx/test_tooling.py diff --git a/examples/debugging/shield_hang_in_sub.py b/examples/debugging/shield_hang_in_sub.py new file mode 100644 index 00000000..3cc084d5 --- /dev/null +++ b/examples/debugging/shield_hang_in_sub.py @@ -0,0 +1,81 @@ +''' +Verify we can dump a `stackscope` tree on a hang. + +''' +import os +import signal + +import trio +import tractor + +@tractor.context +async def start_n_shield_hang( + ctx: tractor.Context, +): + # actor: tractor.Actor = tractor.current_actor() + + # sync to parent-side task + await ctx.started(os.getpid()) + + print('Entering shield sleep..') + with trio.CancelScope(shield=True): + await trio.sleep_forever() # in subactor + + # XXX NOTE ^^^ since this shields, we expect + # the zombie reaper (aka T800) to engage on + # SIGINT from the user and eventually hard-kill + # this subprocess! + + +async def main( + from_test: bool = False, +) -> None: + + async with ( + tractor.open_nursery( + debug_mode=True, + enable_stack_on_sig=True, + # maybe_enable_greenback=False, + loglevel='devx', + ) as an, + ): + + ptl: tractor.Portal = await an.start_actor( + 'hanger', + enable_modules=[__name__], + debug_mode=True, + ) + async with ptl.open_context( + start_n_shield_hang, + ) as (ctx, cpid): + + _, proc, _ = an._children[ptl.chan.uid] + assert cpid == proc.pid + + print( + 'Yo my child hanging..?\n' + 'Sending SIGUSR1 to see a tree-trace!\n' + ) + + # XXX simulate the wrapping test's "user actions" + # (i.e. if a human didn't run this manually but wants to + # know what they should do to reproduce test behaviour) + if from_test: + os.kill( + cpid, + signal.SIGUSR1, + ) + + # simulate user cancelling program + await trio.sleep(0.5) + os.kill( + os.getpid(), + signal.SIGINT, + ) + else: + # actually let user send the ctl-c + await trio.sleep_forever() # in root + + +if __name__ == '__main__': + trio.run(main) diff --git a/tests/devx/__init__.py b/tests/devx/__init__.py new file mode 100644 index 00000000..e69de29b diff --git a/tests/devx/conftest.py b/tests/devx/conftest.py new file mode 100644 index 00000000..b739569a --- /dev/null +++ b/tests/devx/conftest.py @@ -0,0 +1,167 @@ +''' +`tractor.devx.*` tooling sub-pkg test space. + +''' +from typing import ( + Callable, +) + +import pytest +from pexpect.exceptions import ( + TIMEOUT, +) +from tractor._testing import ( + mk_cmd, +) + + +@pytest.fixture +def spawn( + start_method, + testdir: pytest.Testdir, + reg_addr: tuple[str, int], + +) -> Callable[[str], None]: + ''' + Use the `pexpect` module shipped via `testdir.spawn()` to + run an `./examples/..` script by name. + + ''' + if start_method != 'trio': + pytest.skip( + '`pexpect` based tests only supported on `trio` backend' + ) + + def _spawn( + cmd: str, + **mkcmd_kwargs, + ): + return testdir.spawn( + cmd=mk_cmd( + cmd, + **mkcmd_kwargs, + ), + expect_timeout=3, + ) + + # such that test-dep can pass input script name. + return _spawn + + +@pytest.fixture( + params=[False, True], + ids='ctl-c={}'.format, +) +def ctlc( + request, + ci_env: bool, + +) -> bool: + + use_ctlc = request.param + + node = request.node + markers = node.own_markers + for mark in markers: + if mark.name == 'has_nested_actors': + pytest.skip( + f'Test {node} has nested actors and fails with Ctrl-C.\n' + f'The test can sometimes run fine locally but until' + ' we solve' 'this issue this CI test will be xfail:\n' + 'https://github.com/goodboy/tractor/issues/320' + ) + + if use_ctlc: + # XXX: disable pygments highlighting for auto-tests + # since some envs (like actions CI) will struggle + # the the added color-char encoding.. + from tractor.devx._debug import TractorConfig + TractorConfig.use_pygements = False + + yield use_ctlc + + +def expect( + child, + + # normally a `pdb` prompt by default + patt: str, + + **kwargs, + +) -> None: + ''' + Expect wrapper that prints last seen console + data before failing. + + ''' + try: + child.expect( + patt, + **kwargs, + ) + except TIMEOUT: + before = str(child.before.decode()) + print(before) + raise + + +def in_prompt_msg( + prompt: str, + parts: list[str], + + pause_on_false: bool = False, + err_on_false: bool = False, + print_prompt_on_false: bool = True, + +) -> bool: + ''' + Predicate check if (the prompt's) std-streams output has all + `str`-parts in it. + + Can be used in test asserts for bulk matching expected + log/REPL output for a given `pdb` interact point. + + ''' + __tracebackhide__: bool = False + + for part in parts: + if part not in prompt: + if pause_on_false: + import pdbp + pdbp.set_trace() + + if print_prompt_on_false: + print(prompt) + + if err_on_false: + raise ValueError( + f'Could not find pattern: {part!r} in `before` output?' + ) + return False + + return True + + +# TODO: todo support terminal color-chars stripping so we can match +# against call stack frame output from the the 'll' command the like! +# -[ ] SO answer for stipping ANSI codes: https://stackoverflow.com/a/14693789 +def assert_before( + child, + patts: list[str], + + **kwargs, + +) -> None: + __tracebackhide__: bool = False + + # as in before the prompt end + before: str = str(child.before.decode()) + assert in_prompt_msg( + prompt=before, + parts=patts, + + # since this is an "assert" helper ;) + err_on_false=True, + **kwargs + ) diff --git a/tests/devx/test_debugger.py b/tests/devx/test_debugger.py index 03b3ae4f..5ef039f3 100644 --- a/tests/devx/test_debugger.py +++ b/tests/devx/test_debugger.py @@ -16,15 +16,11 @@ import platform import time import pytest -import pexpect from pexpect.exceptions import ( TIMEOUT, EOF, ) -from tractor._testing import ( - mk_cmd, -) from tractor.devx._debug import ( _pause_msg, _crash_msg, @@ -32,6 +28,9 @@ from tractor.devx._debug import ( ) from .conftest import ( _ci_env, + expect, + in_prompt_msg, + assert_before, ) # TODO: The next great debugger audit could be done by you! @@ -69,154 +68,9 @@ has_nested_actors = pytest.mark.has_nested_actors # ) -@pytest.fixture -def spawn( - start_method, - testdir, - reg_addr, - -) -> 'pexpect.spawn': - ''' - Use the `pexpect` module shipped via `testdir.spawn()` to - run an `./examples/..` script by name. - - ''' - if start_method != 'trio': - pytest.skip( - '`pexpect` based tests only supported on `trio` backend' - ) - - def _spawn( - cmd: str, - **mkcmd_kwargs, - ): - return testdir.spawn( - cmd=mk_cmd( - cmd, - **mkcmd_kwargs, - ), - expect_timeout=3, - ) - - # such that test-dep can pass input script name. - return _spawn - - PROMPT = r"\(Pdb\+\)" -def expect( - child, - - # prompt by default - patt: str = PROMPT, - - **kwargs, - -) -> None: - ''' - Expect wrapper that prints last seen console - data before failing. - - ''' - try: - child.expect( - patt, - **kwargs, - ) - except TIMEOUT: - before = str(child.before.decode()) - print(before) - raise - - -def in_prompt_msg( - prompt: str, - parts: list[str], - - pause_on_false: bool = False, - print_prompt_on_false: bool = True, - -) -> bool: - ''' - Predicate check if (the prompt's) std-streams output has all - `str`-parts in it. - - Can be used in test asserts for bulk matching expected - log/REPL output for a given `pdb` interact point. - - ''' - __tracebackhide__: bool = False - - for part in parts: - if part not in prompt: - if pause_on_false: - import pdbp - pdbp.set_trace() - - if print_prompt_on_false: - print(prompt) - - return False - - return True - - -# TODO: todo support terminal color-chars stripping so we can match -# against call stack frame output from the the 'll' command the like! -# -[ ] SO answer for stipping ANSI codes: https://stackoverflow.com/a/14693789 -def assert_before( - child, - patts: list[str], - - **kwargs, - -) -> None: - __tracebackhide__: bool = False - - # as in before the prompt end - before: str = str(child.before.decode()) - assert in_prompt_msg( - prompt=before, - parts=patts, - - **kwargs - ) - - -@pytest.fixture( - params=[False, True], - ids='ctl-c={}'.format, -) -def ctlc( - request, - ci_env: bool, - -) -> bool: - - use_ctlc = request.param - - node = request.node - markers = node.own_markers - for mark in markers: - if mark.name == 'has_nested_actors': - pytest.skip( - f'Test {node} has nested actors and fails with Ctrl-C.\n' - f'The test can sometimes run fine locally but until' - ' we solve' 'this issue this CI test will be xfail:\n' - 'https://github.com/goodboy/tractor/issues/320' - ) - - if use_ctlc: - # XXX: disable pygments highlighting for auto-tests - # since some envs (like actions CI) will struggle - # the the added color-char encoding.. - from tractor.devx._debug import TractorConfig - TractorConfig.use_pygements = False - - yield use_ctlc - - @pytest.mark.parametrize( 'user_in_out', [ @@ -281,7 +135,7 @@ def test_root_actor_bp(spawn, user_in_out): child.expect('\r\n') # process should exit - child.expect(pexpect.EOF) + child.expect(EOF) if expect_err_str is None: assert 'Error' not in str(child.before) @@ -365,7 +219,7 @@ def test_root_actor_bp_forever( # quit out of the loop child.sendline('q') - child.expect(pexpect.EOF) + child.expect(EOF) @pytest.mark.parametrize( @@ -430,7 +284,7 @@ def test_subactor_error( child.expect('\r\n') # process should exit - child.expect(pexpect.EOF) + child.expect(EOF) def test_subactor_breakpoint( @@ -493,7 +347,7 @@ def test_subactor_breakpoint( child.sendline('c') # process should exit - child.expect(pexpect.EOF) + child.expect(EOF) before = str(child.before.decode()) assert in_prompt_msg( @@ -636,7 +490,7 @@ def test_multi_subactors( # process should exit child.sendline('c') - child.expect(pexpect.EOF) + child.expect(EOF) # repeat of previous multierror for final output assert_before(child, [ @@ -776,7 +630,7 @@ def test_multi_daemon_subactors( ) child.sendline('c') - child.expect(pexpect.EOF) + child.expect(EOF) @has_nested_actors @@ -852,7 +706,7 @@ def test_multi_subactors_root_errors( ]) child.sendline('c') - child.expect(pexpect.EOF) + child.expect(EOF) assert_before(child, [ # "Attaching to pdb in crashed actor: ('root'", @@ -982,7 +836,7 @@ def test_root_nursery_cancels_before_child_releases_tty_lock( for i in range(3): try: - child.expect(pexpect.EOF, timeout=0.5) + child.expect(EOF, timeout=0.5) break except TIMEOUT: child.sendline('c') @@ -1024,7 +878,7 @@ def test_root_cancels_child_context_during_startup( do_ctlc(child) child.sendline('c') - child.expect(pexpect.EOF) + child.expect(EOF) def test_different_debug_mode_per_actor( @@ -1045,7 +899,7 @@ def test_different_debug_mode_per_actor( do_ctlc(child) child.sendline('c') - child.expect(pexpect.EOF) + child.expect(EOF) before = str(child.before.decode()) @@ -1196,7 +1050,7 @@ def test_pause_from_sync( ) child.sendline('c') - child.expect(pexpect.EOF) + child.expect(EOF) def test_post_mortem_api( @@ -1301,7 +1155,7 @@ def test_post_mortem_api( # ) child.sendline('c') - child.expect(pexpect.EOF) + child.expect(EOF) def test_shield_pause( @@ -1376,7 +1230,7 @@ def test_shield_pause( ] ) child.sendline('c') - child.expect(pexpect.EOF) + child.expect(EOF) # TODO: better error for "non-ideal" usage from the root actor. diff --git a/tests/devx/test_tooling.py b/tests/devx/test_tooling.py new file mode 100644 index 00000000..3e48844e --- /dev/null +++ b/tests/devx/test_tooling.py @@ -0,0 +1,120 @@ +''' +That "native" runtime-hackin toolset better be dang useful! + +Verify the funtion of a variety of "developer-experience" tools we +offer from the `.devx` sub-pkg: + +- use of the lovely `stackscope` for dumping actor `trio`-task trees + during operation and hangs. + +TODO: +- demonstration of `CallerInfo` call stack frame filtering such that + for logging and REPL purposes a user sees exactly the layers needed + when debugging a problem inside the stack vs. in their app. + +''' +import os +import signal + +from .conftest import ( + expect, + assert_before, + # in_prompt_msg, +) + + +def test_shield_pause( + spawn, +): + ''' + Verify the `tractor.pause()/.post_mortem()` API works inside an + already cancelled `trio.CancelScope` and that you can step to the + next checkpoint wherein the cancelled will get raised. + + ''' + child = spawn( + 'shield_hang_in_sub' + ) + expect( + child, + 'Yo my child hanging..?', + ) + assert_before( + child, + [ + 'Entering shield sleep..', + 'Enabling trace-trees on `SIGUSR1` since `stackscope` is installed @', + ] + ) + + print( + 'Sending SIGUSR1 to see a tree-trace!', + ) + os.kill( + child.pid, + signal.SIGUSR1, + ) + expect( + child, + # end-of-tree delimiter + "------ \('root', ", + ) + + assert_before( + child, + [ + 'Trying to dump `stackscope` tree..', + 'Dumping `stackscope` tree for actor', + "('root'", # uid line + + # parent block point (non-shielded) + 'await trio.sleep_forever() # in root', + ] + ) + + # expect( + # child, + # # relay to the sub should be reported + # 'Relaying `SIGUSR1`[10] to sub-actor', + # ) + + expect( + child, + # end-of-tree delimiter + "------ \('hanger', ", + ) + assert_before( + child, + [ + # relay to the sub should be reported + 'Relaying `SIGUSR1`[10] to sub-actor', + + "('hanger'", # uid line + + # hanger LOC where it's shield-halted + 'await trio.sleep_forever() # in subactor', + ] + ) + # breakpoint() + + # simulate the user sending a ctl-c to the hanging program. + # this should result in the terminator kicking in since + # the sub is shield blocking and can't respond to SIGINT. + os.kill( + child.pid, + signal.SIGINT, + ) + expect( + child, + 'Shutting down actor runtime', + timeout=6, + ) + assert_before( + child, + [ + 'raise KeyboardInterrupt', + # 'Shutting down actor runtime', + '#T-800 deployed to collect zombie B0', + "'--uid', \"('hanger',", + ] + ) diff --git a/tractor/_root.py b/tractor/_root.py index 882285a5..51dbe5e6 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -80,7 +80,7 @@ async def open_root_actor( # enables the multi-process debugger support debug_mode: bool = False, - maybe_enable_greenback: bool = False, # `.pause_from_sync()/breakpoint()` support + maybe_enable_greenback: bool = True, # `.pause_from_sync()/breakpoint()` support enable_stack_on_sig: bool = False, # internal logging @@ -233,14 +233,8 @@ async def open_root_actor( and enable_stack_on_sig ): - try: - logger.info('Enabling `stackscope` traces on SIGUSR1') - from .devx import enable_stack_on_sig - enable_stack_on_sig() - except ImportError: - logger.warning( - '`stackscope` not installed for use in debug mode!' - ) + from .devx._stackscope import enable_stack_on_sig + enable_stack_on_sig() # closed into below ping task-func ponged_addrs: list[tuple[str, int]] = [] diff --git a/tractor/devx/_stackscope.py b/tractor/devx/_stackscope.py index e8e97d1a..944ae49a 100644 --- a/tractor/devx/_stackscope.py +++ b/tractor/devx/_stackscope.py @@ -24,13 +24,24 @@ disjoint, parallel executing tasks in separate actors. ''' from __future__ import annotations +# from functools import partial +from threading import ( + current_thread, + Thread, + RLock, +) import multiprocessing as mp from signal import ( signal, + getsignal, SIGUSR1, ) -import traceback -from typing import TYPE_CHECKING +# import traceback +from types import ModuleType +from typing import ( + Callable, + TYPE_CHECKING, +) import trio from tractor import ( @@ -51,26 +62,45 @@ if TYPE_CHECKING: @trio.lowlevel.disable_ki_protection def dump_task_tree() -> None: - import stackscope - from tractor.log import get_console_log + ''' + Do a classic `stackscope.extract()` task-tree dump to console at + `.devx()` level. + ''' + import stackscope tree_str: str = str( stackscope.extract( trio.lowlevel.current_root_task(), recurse_child_tasks=True ) ) - log = get_console_log( - name=__name__, - level='cancel', - ) actor: Actor = _state.current_actor() + thr: Thread = current_thread() log.devx( f'Dumping `stackscope` tree for actor\n' - f'{actor.name}: {actor}\n' - f' |_{mp.current_process()}\n\n' + f'{actor.uid}:\n' + f'|_{mp.current_process()}\n' + f' |_{thr}\n' + f' |_{actor}\n\n' + + # start-of-trace-tree delimiter (mostly for testing) + '------ - ------\n' + '\n' + + f'{tree_str}\n' + + + # end-of-trace-tree delimiter (mostly for testing) + f'\n' + f'------ {actor.uid!r} ------\n' ) + # TODO: can remove this right? + # -[ ] was original code from author + # + # print( + # 'DUMPING FROM PRINT\n' + # + + # content + # ) # import logging # try: # with open("/dev/tty", "w") as tty: @@ -80,58 +110,130 @@ def dump_task_tree() -> None: # "task_tree" # ).exception("Error printing task tree") +_handler_lock = RLock() +_tree_dumped: bool = False -def signal_handler( + +def dump_tree_on_sig( sig: int, frame: object, relay_to_subs: bool = True, ) -> None: - try: - trio.lowlevel.current_trio_token( - ).run_sync_soon(dump_task_tree) - except RuntimeError: - # not in async context -- print a normal traceback - traceback.print_stack() + global _tree_dumped, _handler_lock + with _handler_lock: + if _tree_dumped: + log.warning( + 'Already dumped for this actor...??' + ) + return + + _tree_dumped = True + + # actor: Actor = _state.current_actor() + log.devx( + 'Trying to dump `stackscope` tree..\n' + ) + try: + dump_task_tree() + # await actor._service_n.start_soon( + # partial( + # trio.to_thread.run_sync, + # dump_task_tree, + # ) + # ) + # trio.lowlevel.current_trio_token().run_sync_soon( + # dump_task_tree + # ) + + except RuntimeError: + log.exception( + 'Failed to dump `stackscope` tree..\n' + ) + # not in async context -- print a normal traceback + # traceback.print_stack() + raise + + except BaseException: + log.exception( + 'Failed to dump `stackscope` tree..\n' + ) + raise + + log.devx( + 'Supposedly we dumped just fine..?' + ) if not relay_to_subs: return an: ActorNursery for an in _state.current_actor()._actoruid2nursery.values(): - subproc: ProcessType subactor: Actor for subactor, subproc, _ in an._children.values(): - log.devx( + log.warning( f'Relaying `SIGUSR1`[{sig}] to sub-actor\n' f'{subactor}\n' f' |_{subproc}\n' ) - if isinstance(subproc, trio.Process): - subproc.send_signal(sig) + # bc of course stdlib can't have a std API.. XD + match subproc: + case trio.Process(): + subproc.send_signal(sig) - elif isinstance(subproc, mp.Process): - subproc._send_signal(sig) + case mp.Process(): + subproc._send_signal(sig) def enable_stack_on_sig( - sig: int = SIGUSR1 -) -> None: + sig: int = SIGUSR1, +) -> ModuleType: ''' Enable `stackscope` tracing on reception of a signal; by default this is SIGUSR1. + HOT TIP: a task/ctx-tree dump can be triggered from a shell with + fancy cmds. + + For ex. from `bash` using `pgrep` and cmd-sustitution + (https://www.gnu.org/software/bash/manual/bash.html#Command-Substitution) + you could use: + + >> kill -SIGUSR1 $(pgrep -f '') + + Or with with `xonsh` (which has diff capture-from-subproc syntax) + + >> kill -SIGUSR1 @$(pgrep -f '') + ''' + try: + import stackscope + except ImportError: + log.warning( + '`stackscope` not installed for use in debug mode!' + ) + return None + + handler: Callable|int = getsignal(sig) + if handler is dump_tree_on_sig: + log.devx( + 'A `SIGUSR1` handler already exists?\n' + f'|_ {handler!r}\n' + ) + return + signal( sig, - signal_handler, + dump_tree_on_sig, ) - # NOTE: not the above can be triggered from - # a (xonsh) shell using: - # kill -SIGUSR1 @$(pgrep -f '') - # - # for example if you were looking to trace a `pytest` run - # kill -SIGUSR1 @$(pgrep -f 'pytest') + log.devx( + 'Enabling trace-trees on `SIGUSR1` ' + 'since `stackscope` is installed @ \n' + f'{stackscope!r}\n\n' + f'With `SIGUSR1` handler\n' + f'|_{dump_tree_on_sig}\n' + ) + return stackscope From 5cdfee3bcf3d2d563306eaf6659a38050371caca Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 11 Jul 2024 12:11:31 -0400 Subject: [PATCH 13/66] Pass `infect_asyncio` setting via runtime-vars The reason for this "duplication" with the `--asyncio` CLI flag (passed to the child during spawn) is 2-fold: - allows verifying inside `Actor._from_parent()` that the `trio` runtime was started via `.start_guest_run()` as well as if the `Actor._infected_aio` spawn-entrypoint value has been set (by the `._entry._main()` whenever `--asyncio` is passed) such that any mismatch can be signaled via an `InternalError`. - enables checking the `._state._runtime_vars['_is_infected_aio']` value directly (say from a non-actor/`trio`-thread) instead of calling `._state.current_actor(err_on_no_runtime=False)` in certain edge cases. Impl/testing deats: - add `._state._runtime_vars['_is_infected_aio'] = False` default. - raise `InternalError` on any `--asyncio`-flag-passed vs. `_runtime_vars`-value-relayed-from-parent inside `Actor._from_parent()` and include a `Runner.is_guest` assert for good measure B) - set and relay `infect_asyncio: bool` via runtime-vars to child in `ActorNursery.start_actor()`. - verify `actor.is_infected_aio()`, `actor._infected_aio` and `_state._runtime_vars['_is_infected_aio']` are all set in test suite's `asyncio_actor()` endpoint. --- tests/test_infected_asyncio.py | 15 ++++++++++-- tractor/_entry.py | 23 +++++++++++------- tractor/_runtime.py | 44 ++++++++++++++++++++++++++++++---- tractor/_state.py | 5 +++- tractor/_supervise.py | 1 + 5 files changed, 72 insertions(+), 16 deletions(-) diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 42eb35b7..fca971d0 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -21,9 +21,11 @@ import trio import tractor from tractor import ( current_actor, + Actor, to_asyncio, RemoteActorError, ContextCancelled, + _state, ) from tractor.trionics import BroadcastReceiver from tractor._testing import expect_ctxc @@ -80,7 +82,16 @@ async def asyncio_actor( ) -> None: - assert tractor.current_actor().is_infected_aio() + # ensure internal runtime state is consistent + actor: Actor = tractor.current_actor() + assert ( + actor.is_infected_aio() + and + actor._infected_aio + and + _state._runtime_vars['_is_infected_aio'] + ) + target: Callable = globals()[target] if '.' in expect_err: @@ -136,7 +147,7 @@ def test_aio_simple_error(reg_addr): assert err assert isinstance(err, RemoteActorError) - assert err.boxed_type == AssertionError + assert err.boxed_type is AssertionError def test_tractor_cancels_aio(reg_addr): diff --git a/tractor/_entry.py b/tractor/_entry.py index a072706c..19dcb9f6 100644 --- a/tractor/_entry.py +++ b/tractor/_entry.py @@ -20,6 +20,7 @@ Sub-process entry points. """ from __future__ import annotations from functools import partial +import multiprocessing as mp import os import textwrap from typing import ( @@ -64,20 +65,22 @@ def _mp_main( ''' actor._forkserver_info = forkserver_info from ._spawn import try_set_start_method - spawn_ctx = try_set_start_method(start_method) + spawn_ctx: mp.context.BaseContext = try_set_start_method(start_method) + assert spawn_ctx if actor.loglevel is not None: log.info( - f"Setting loglevel for {actor.uid} to {actor.loglevel}") + f'Setting loglevel for {actor.uid} to {actor.loglevel}' + ) get_console_log(actor.loglevel) - assert spawn_ctx + # TODO: use scops headers like for `trio` below! + # (well after we libify it maybe..) log.info( - f"Started new {spawn_ctx.current_process()} for {actor.uid}") - - _state._current_actor = actor - - log.debug(f"parent_addr is {parent_addr}") + f'Started new {spawn_ctx.current_process()} for {actor.uid}' + # f"parent_addr is {parent_addr}" + ) + _state._current_actor: Actor = actor trio_main = partial( async_main, actor=actor, @@ -94,7 +97,9 @@ def _mp_main( pass # handle it the same way trio does? finally: - log.info(f"Subactor {actor.uid} terminated") + log.info( + f'`mp`-subactor {actor.uid} exited' + ) # TODO: move this func to some kinda `.devx._conc_lang.py` eventually diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 662dd67a..7a00d613 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -59,6 +59,7 @@ from types import ModuleType import warnings import trio +from trio._core import _run as trio_runtime from trio import ( CancelScope, Nursery, @@ -80,6 +81,7 @@ from ._context import ( from .log import get_logger from ._exceptions import ( ContextCancelled, + InternalError, ModuleNotExposed, MsgTypeError, unpack_error, @@ -98,6 +100,7 @@ from ._rpc import ( if TYPE_CHECKING: from ._supervise import ActorNursery + from trio._channel import MemoryChannelState log = get_logger('tractor') @@ -896,11 +899,15 @@ class Actor: f'peer: {chan.uid}\n' f'cid:{cid}\n' ) - ctx._allow_overruns = allow_overruns + ctx._allow_overruns: bool = allow_overruns # adjust buffer size if specified - state = ctx._send_chan._state # type: ignore - if msg_buffer_size and state.max_buffer_size != msg_buffer_size: + state: MemoryChannelState = ctx._send_chan._state # type: ignore + if ( + msg_buffer_size + and + state.max_buffer_size != msg_buffer_size + ): state.max_buffer_size = msg_buffer_size except KeyError: @@ -1094,7 +1101,36 @@ class Actor: '`tractor.pause_from_sync()` not available!' ) - rvs['_is_root'] = False + # XXX ensure the "infected `asyncio` mode" setting + # passed down from our spawning parent is consistent + # with `trio`-runtime initialization: + # - during sub-proc boot, the entrypoint func + # (`._entry._main()`) should set + # `._infected_aio = True` before calling + # `run_as_asyncio_guest()`, + # - the value of `infect_asyncio: bool = True` as + # passed to `ActorNursery.start_actor()` must be + # the same as `_runtime_vars['_is_infected_aio']` + if ( + (aio_rtv := rvs['_is_infected_aio']) + != + (aio_attr := self._infected_aio) + ): + raise InternalError( + 'Parent sent runtime-vars that mismatch for the ' + '"infected `asyncio` mode" settings ?!?\n\n' + + f'rvs["_is_infected_aio"] = {aio_rtv}\n' + f'self._infected_aio = {aio_attr}\n' + ) + if aio_rtv: + assert trio_runtime.GLOBAL_RUN_CONTEXT.runner.is_guest + # ^TODO^ possibly add a `sniffio` or + # `trio` pub-API for `is_guest_mode()`? + + rvs['_is_root'] = False # obvi XD + + # update process-wide globals _state._runtime_vars.update(rvs) # XXX: ``msgspec`` doesn't support serializing tuples diff --git a/tractor/_state.py b/tractor/_state.py index 9f896005..a87ad36b 100644 --- a/tractor/_state.py +++ b/tractor/_state.py @@ -44,6 +44,8 @@ _runtime_vars: dict[str, Any] = { '_root_mailbox': (None, None), '_registry_addrs': [], + '_is_infected_aio': False, + # for `tractor.pause_from_sync()` & `breakpoint()` support 'use_greenback': False, } @@ -70,7 +72,8 @@ def current_actor( ''' if ( err_on_no_runtime - and _current_actor is None + and + _current_actor is None ): msg: str = 'No local actor has been initialized yet?\n' from ._exceptions import NoRuntime diff --git a/tractor/_supervise.py b/tractor/_supervise.py index 4a538e9f..de268078 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -158,6 +158,7 @@ class ActorNursery: # configure and pass runtime state _rtv = _state._runtime_vars.copy() _rtv['_is_root'] = False + _rtv['_is_infected_aio'] = infect_asyncio # allow setting debug policy per actor if debug_mode is not None: From 8da7a1ca36f1c9b9c44f1f999d59b03bf3b0b8a6 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 11 Jul 2024 19:22:40 -0400 Subject: [PATCH 14/66] Use "sclang"-style syntax in `to_asyncio` task logging Just like we've started doing throughout the rest of the actor runtime for reporting (and where "sclang" = "structured conc (s)lang", our little supervision-focused operations syntax i've been playing with in log msg content). Further tweaks: - report the `trio_done_fute` alongside the `main_outcome` value. - add a todo list for supporting `greenback` for pause points. --- tractor/to_asyncio.py | 32 +++++++++++++++++++++++++------- 1 file changed, 25 insertions(+), 7 deletions(-) diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index 5564d0e1..c37c0467 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -276,7 +276,10 @@ def _run_asyncio_task( chan._aio_task: asyncio.Task = task # XXX TODO XXX get this actually workin.. XD - # maybe setup `greenback` for `asyncio`-side task REPLing + # -[ ] we need logic to setup `greenback` for `asyncio`-side task + # REPLing.. which should normally be nearly the same as for + # `trio`? + # -[ ] add to a new `.devx._greenback.maybe_init_for_asyncio()`? if ( debug_mode() and @@ -305,15 +308,22 @@ def _run_asyncio_task( msg: str = ( 'Infected `asyncio` task {etype_str}\n' - f'|_{task}\n' ) if isinstance(terr, CancelledError): + msg += ( + f'c)>\n' + f' |_{task}\n' + ) log.cancel( msg.format(etype_str='cancelled') ) else: + msg += ( + f'x)>\n' + f' |_{task}\n' + ) log.exception( - msg.format(etype_str='cancelled') + msg.format(etype_str='errored') ) assert type(terr) is type(aio_err), ( @@ -619,9 +629,10 @@ def run_as_asyncio_guest( # ) def trio_done_callback(main_outcome): - log.info( - f'trio_main finished with\n' - f'|_{main_outcome!r}' + log.runtime( + f'`trio` guest-run finishing with outcome\n' + f'>) {main_outcome}\n' + f'|_{trio_done_fute}\n' ) if isinstance(main_outcome, Error): @@ -643,6 +654,12 @@ def run_as_asyncio_guest( else: trio_done_fute.set_result(main_outcome) + log.info( + f'`trio` guest-run finished with outcome\n' + f')>\n' + f'|_{trio_done_fute}\n' + ) + startup_msg += ( f'-> created {trio_done_callback!r}\n' f'-> scheduling `trio_main`: {trio_main!r}\n' @@ -681,7 +698,8 @@ def run_as_asyncio_guest( # error path in `asyncio`'s runtime..? asyncio.CancelledError, - ) as fute_err: + ) as _fute_err: + fute_err = _fute_err err_message: str = ( 'main `asyncio` task ' ) From 00d1c8ea2962357279d5af7e7080fef009270537 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 12 Jul 2024 15:57:41 -0400 Subject: [PATCH 15/66] Fix multi-daemon debug test `break` signal.. It was expecting `AssertionError` as a proceed-in-test signal (by breaking from a continue loop), but `in_prompt_msg(raise_on_err=True)` was changed to raise `ValueError`; so instead just use as a predicate for the `break`. Also rework `in_prompt_msg()` to accept the `child: BaseSpawn` as input instead of `before: str` remove the casting boilerplate, and adjust all usage to match. --- examples/debugging/multi_daemon_subactors.py | 3 +- tests/devx/conftest.py | 17 +-- tests/devx/test_debugger.py | 118 +++++++++---------- tests/test_infected_asyncio.py | 6 +- 4 files changed, 71 insertions(+), 73 deletions(-) diff --git a/examples/debugging/multi_daemon_subactors.py b/examples/debugging/multi_daemon_subactors.py index 80ef933c..4a462623 100644 --- a/examples/debugging/multi_daemon_subactors.py +++ b/examples/debugging/multi_daemon_subactors.py @@ -25,7 +25,8 @@ async def main(): """ async with tractor.open_nursery( debug_mode=True, - loglevel='cancel', + # loglevel='cancel', + # loglevel='devx', ) as n: p0 = await n.start_actor('bp_forever', enable_modules=[__name__]) diff --git a/tests/devx/conftest.py b/tests/devx/conftest.py index b739569a..28a14cbb 100644 --- a/tests/devx/conftest.py +++ b/tests/devx/conftest.py @@ -10,6 +10,7 @@ import pytest from pexpect.exceptions import ( TIMEOUT, ) +from pexpect.spawnbase import SpawnBase from tractor._testing import ( mk_cmd, ) @@ -107,7 +108,7 @@ def expect( def in_prompt_msg( - prompt: str, + child: SpawnBase, parts: list[str], pause_on_false: bool = False, @@ -125,18 +126,20 @@ def in_prompt_msg( ''' __tracebackhide__: bool = False + before: str = str(child.before.decode()) for part in parts: - if part not in prompt: + if part not in before: if pause_on_false: import pdbp pdbp.set_trace() if print_prompt_on_false: - print(prompt) + print(before) if err_on_false: raise ValueError( - f'Could not find pattern: {part!r} in `before` output?' + f'Could not find pattern in `before` output?\n' + f'part: {part!r}\n' ) return False @@ -147,7 +150,7 @@ def in_prompt_msg( # against call stack frame output from the the 'll' command the like! # -[ ] SO answer for stipping ANSI codes: https://stackoverflow.com/a/14693789 def assert_before( - child, + child: SpawnBase, patts: list[str], **kwargs, @@ -155,10 +158,8 @@ def assert_before( ) -> None: __tracebackhide__: bool = False - # as in before the prompt end - before: str = str(child.before.decode()) assert in_prompt_msg( - prompt=before, + child=child, parts=patts, # since this is an "assert" helper ;) diff --git a/tests/devx/test_debugger.py b/tests/devx/test_debugger.py index 5ef039f3..2a24bf98 100644 --- a/tests/devx/test_debugger.py +++ b/tests/devx/test_debugger.py @@ -94,14 +94,15 @@ def test_root_actor_error( # scan for the prompt expect(child, PROMPT) - before = str(child.before.decode()) - # make sure expected logging and error arrives assert in_prompt_msg( - before, - [_crash_msg, "('root'"] + child, + [ + _crash_msg, + "('root'", + 'AssertionError', + ] ) - assert 'AssertionError' in before # send user command child.sendline(user_input) @@ -241,10 +242,12 @@ def test_subactor_error( # scan for the prompt child.expect(PROMPT) - before = str(child.before.decode()) assert in_prompt_msg( - before, - [_crash_msg, "('name_error'"] + child, + [ + _crash_msg, + "('name_error'", + ] ) if do_next: @@ -263,17 +266,15 @@ def test_subactor_error( child.sendline('continue') child.expect(PROMPT) - before = str(child.before.decode()) - - # root actor gets debugger engaged assert in_prompt_msg( - before, - [_crash_msg, "('root'"] - ) - # error is a remote error propagated from the subactor - assert in_prompt_msg( - before, - [_crash_msg, "('name_error'"] + child, + [ + _crash_msg, + # root actor gets debugger engaged + "('root'", + # error is a remote error propagated from the subactor + "('name_error'", + ] ) # another round @@ -294,14 +295,11 @@ def test_subactor_breakpoint( "Single subactor with an infinite breakpoint loop" child = spawn('subactor_breakpoint') - - # scan for the prompt child.expect(PROMPT) - - before = str(child.before.decode()) assert in_prompt_msg( - before, - [_pause_msg, "('breakpoint_forever'"] + child, + [_pause_msg, + "('breakpoint_forever'",] ) # do some "next" commands to demonstrate recurrent breakpoint @@ -317,9 +315,8 @@ def test_subactor_breakpoint( for _ in range(5): child.sendline('continue') child.expect(PROMPT) - before = str(child.before.decode()) assert in_prompt_msg( - before, + child, [_pause_msg, "('breakpoint_forever'"] ) @@ -332,9 +329,8 @@ def test_subactor_breakpoint( # child process should exit but parent will capture pdb.BdbQuit child.expect(PROMPT) - before = str(child.before.decode()) assert in_prompt_msg( - before, + child, ['RemoteActorError:', "('breakpoint_forever'", 'bdb.BdbQuit',] @@ -349,9 +345,8 @@ def test_subactor_breakpoint( # process should exit child.expect(EOF) - before = str(child.before.decode()) assert in_prompt_msg( - before, + child, ['RemoteActorError:', "('breakpoint_forever'", 'bdb.BdbQuit',] @@ -375,7 +370,7 @@ def test_multi_subactors( before = str(child.before.decode()) assert in_prompt_msg( - before, + child, [_pause_msg, "('breakpoint_forever'"] ) @@ -396,12 +391,14 @@ def test_multi_subactors( # first name_error failure child.expect(PROMPT) - before = str(child.before.decode()) assert in_prompt_msg( - before, - [_crash_msg, "('name_error'"] + child, + [ + _crash_msg, + "('name_error'", + "NameError", + ] ) - assert "NameError" in before if ctlc: do_ctlc(child) @@ -425,9 +422,8 @@ def test_multi_subactors( # breakpoint loop should re-engage child.sendline('c') child.expect(PROMPT) - before = str(child.before.decode()) assert in_prompt_msg( - before, + child, [_pause_msg, "('breakpoint_forever'"] ) @@ -520,25 +516,28 @@ def test_multi_daemon_subactors( # the root's tty lock first so anticipate either crash # message on the first entry. - bp_forev_parts = [_pause_msg, "('bp_forever'"] + bp_forev_parts = [ + _pause_msg, + "('bp_forever'", + ] bp_forev_in_msg = partial( in_prompt_msg, parts=bp_forev_parts, ) - name_error_msg = "NameError: name 'doggypants' is not defined" - name_error_parts = [name_error_msg] + name_error_msg: str = "NameError: name 'doggypants' is not defined" + name_error_parts: list[str] = [name_error_msg] before = str(child.before.decode()) - if bp_forev_in_msg(prompt=before): + if bp_forev_in_msg(child=child): next_parts = name_error_parts elif name_error_msg in before: next_parts = bp_forev_parts else: - raise ValueError("Neither log msg was found !?") + raise ValueError('Neither log msg was found !?') if ctlc: do_ctlc(child) @@ -607,14 +606,12 @@ def test_multi_daemon_subactors( # wait for final error in root # where it crashs with boxed error while True: - try: - child.sendline('c') - child.expect(PROMPT) - assert_before( - child, - bp_forev_parts - ) - except AssertionError: + child.sendline('c') + child.expect(PROMPT) + if not in_prompt_msg( + child, + bp_forev_parts + ): break assert_before( @@ -795,10 +792,13 @@ def test_root_nursery_cancels_before_child_releases_tty_lock( child = spawn('root_cancelled_but_child_is_in_tty_lock') child.expect(PROMPT) - - before = str(child.before.decode()) - assert "NameError: name 'doggypants' is not defined" in before - assert "tractor._exceptions.RemoteActorError: ('name_error'" not in before + assert_before( + child, + [ + "NameError: name 'doggypants' is not defined", + "tractor._exceptions.RemoteActorError: ('name_error'", + ], + ) time.sleep(0.5) if ctlc: @@ -889,9 +889,8 @@ def test_different_debug_mode_per_actor( child.expect(PROMPT) # only one actor should enter the debugger - before = str(child.before.decode()) assert in_prompt_msg( - before, + child, [_crash_msg, "('debugged_boi'", "RuntimeError"], ) @@ -901,8 +900,6 @@ def test_different_debug_mode_per_actor( child.sendline('c') child.expect(EOF) - before = str(child.before.decode()) - # NOTE: this debugged actor error currently WON'T show up since the # root will actually cancel and terminate the nursery before the error # msg reported back from the debug mode actor is processed. @@ -954,9 +951,8 @@ def test_pause_from_sync( child.expect(PROMPT) # XXX shouldn't see gb loaded message with PDB loglevel! - before = str(child.before.decode()) assert not in_prompt_msg( - before, + child, ['`greenback` portal opened!'], ) # should be same root task @@ -1037,7 +1033,7 @@ def test_pause_from_sync( # at the same time as the one that was detected above. for key, other_patts in attach_patts.copy().items(): assert not in_prompt_msg( - before, + child, other_patts, ) diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index fca971d0..f5fa0aab 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -291,7 +291,7 @@ def test_context_spawns_aio_task_that_errors( err = excinfo.value assert isinstance(err, expect) - assert err.boxed_type == AssertionError + assert err.boxed_type is AssertionError async def aio_cancel(): @@ -497,7 +497,7 @@ def test_trio_error_cancels_intertask_chan(reg_addr): trio.run(main) # ensure boxed error type - excinfo.value.boxed_type == Exception + excinfo.value.boxed_type is Exception def test_trio_closes_early_and_channel_exits(reg_addr): @@ -533,7 +533,7 @@ def test_aio_errors_and_channel_propagates_and_closes(reg_addr): ) as excinfo: trio.run(main) - excinfo.value.boxed_type == Exception + excinfo.value.boxed_type is Exception @tractor.context From 6b18fcd4371ab68f40bbc0218e874ce51638f571 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 13 Jul 2024 00:16:28 -0400 Subject: [PATCH 16/66] First draft, `asyncio`-task, sync-pausing Bo Mostly due to magic from @oremanj where we slap in a little bit of `.from_asyncio`-type stuff to run a `trio`-task from `asyncio.Task` code! I'm not gonna go into tooo too much detail but basically the primary thing needed was a way to (blocking-ly) invoke a `trio.lowlevel.Task` from an `asyncio` one (which we now have with a new `run_trio_task_in_future()` thanks to draft code from the aforementioned jefe) which we now invoke from a dedicated aio case-branch inside `.devx._debug.pause_from_sync()`. Further include a case inside `DebugStatus.release()` to handle using the same func to set the `repl_release: trio.Event` from the aio side when releasing the REPL on exit cmds. Prolly more refinements to come ;{o --- examples/debugging/asyncio_bp.py | 21 +-- tractor/devx/_debug.py | 212 +++++++++++++++++++++---------- tractor/to_asyncio.py | 95 ++++++++++++++ 3 files changed, 254 insertions(+), 74 deletions(-) diff --git a/examples/debugging/asyncio_bp.py b/examples/debugging/asyncio_bp.py index baddfe03..f2fabddd 100644 --- a/examples/debugging/asyncio_bp.py +++ b/examples/debugging/asyncio_bp.py @@ -2,7 +2,10 @@ import asyncio import trio import tractor -from tractor import to_asyncio +from tractor import ( + to_asyncio, + Portal, +) async def aio_sleep_forever(): @@ -43,7 +46,7 @@ async def bp_then_error( @tractor.context async def trio_ctx( ctx: tractor.Context, - bp_before_started: bool = False, + bp_before_started: bool = True, ): # this will block until the ``asyncio`` task sends a "first" @@ -57,7 +60,6 @@ async def trio_ctx( trio.open_nursery() as n, ): - assert first == 'start' if bp_before_started: @@ -73,15 +75,18 @@ async def trio_ctx( async def main( - bps_all_over: bool = False, + bps_all_over: bool = True, ) -> None: async with tractor.open_nursery( - # debug_mode=True, + debug_mode=True, + maybe_enable_greenback=True, + # loglevel='devx', + # loglevel='runtime', ) as n: - p = await n.start_actor( + ptl: Portal = await n.start_actor( 'aio_daemon', enable_modules=[__name__], infect_asyncio=True, @@ -89,7 +94,7 @@ async def main( loglevel='cancel', ) - async with p.open_context( + async with ptl.open_context( trio_ctx, bp_before_started=bps_all_over, ) as (ctx, first): @@ -105,7 +110,7 @@ async def main( # TODO: case where we cancel from trio-side while asyncio task # has debugger lock? - # await p.cancel_actor() + # await ptl.cancel_actor() if __name__ == '__main__': diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 02551fa1..0784e594 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -20,6 +20,7 @@ Multi-core debugging for da peeps! """ from __future__ import annotations +import asyncio import bdb from contextlib import ( asynccontextmanager as acm, @@ -67,6 +68,7 @@ from trio import ( TaskStatus, ) import tractor +from tractor.to_asyncio import run_trio_task_in_future from tractor.log import get_logger from tractor._context import Context from tractor import _state @@ -296,7 +298,7 @@ class Lock: ) @classmethod - @pdbp.hideframe + # @pdbp.hideframe def release( cls, raise_on_thread: bool = True, @@ -310,39 +312,40 @@ class Lock: we_released: bool = False ctx_in_debug: Context|None = cls.ctx_in_debug repl_task: Task|Thread|None = DebugStatus.repl_task - if not DebugStatus.is_main_trio_thread(): - thread: threading.Thread = threading.current_thread() - message: str = ( - '`Lock.release()` can not be called from a non-main-`trio` thread!\n' - f'{thread}\n' - ) - if raise_on_thread: - raise RuntimeError(message) - - log.devx(message) - return False - - task: Task = current_task() - - # sanity check that if we're the root actor - # the lock is marked as such. - # note the pre-release value may be diff the the - # post-release task. - if repl_task is task: - assert cls._owned_by_root - message: str = ( - 'TTY lock held by root-actor on behalf of local task\n' - f'|_{repl_task}\n' - ) - else: - assert DebugStatus.repl_task is not task - - message: str = ( - 'TTY lock was NOT released on behalf of caller\n' - f'|_{task}\n' - ) try: + if not DebugStatus.is_main_trio_thread(): + thread: threading.Thread = threading.current_thread() + message: str = ( + '`Lock.release()` can not be called from a non-main-`trio` thread!\n' + f'{thread}\n' + ) + if raise_on_thread: + raise RuntimeError(message) + + log.devx(message) + return False + + task: Task = current_task() + + # sanity check that if we're the root actor + # the lock is marked as such. + # note the pre-release value may be diff the the + # post-release task. + if repl_task is task: + assert cls._owned_by_root + message: str = ( + 'TTY lock held by root-actor on behalf of local task\n' + f'|_{repl_task}\n' + ) + else: + assert DebugStatus.repl_task is not task + + message: str = ( + 'TTY lock was NOT released on behalf of caller\n' + f'|_{task}\n' + ) + lock: trio.StrictFIFOLock = cls._debug_lock owner: Task = lock.statistics().owner if ( @@ -788,7 +791,14 @@ class DebugStatus: # in which case schedule the SIGINT shielding override # to in the main thread. # https://docs.python.org/3/library/signal.html#signals-and-threads - if not cls.is_main_trio_thread(): + if ( + not cls.is_main_trio_thread() + and + not _state._runtime_vars.get( + '_is_infected_aio', + False, + ) + ): cls._orig_sigint_handler: Callable = trio.from_thread.run_sync( signal.signal, signal.SIGINT, @@ -813,7 +823,16 @@ class DebugStatus: # always restore ``trio``'s sigint handler. see notes below in # the pdb factory about the nightmare that is that code swapping # out the handler when the repl activates... - if not cls.is_main_trio_thread(): + # if not cls.is_main_trio_thread(): + if ( + not cls.is_main_trio_thread() + and + # not _state._runtime_vars.get( + # '_is_infected_aio', + # False, + # ) + not current_actor().is_infected_aio() + ): trio.from_thread.run_sync( signal.signal, signal.SIGINT, @@ -871,7 +890,7 @@ class DebugStatus: return False @classmethod - @pdbp.hideframe + # @pdbp.hideframe def release( cls, cancel_req_task: bool = False, @@ -880,11 +899,21 @@ class DebugStatus: try: # sometimes the task might already be terminated in # which case this call will raise an RTE? - if ( - repl_release is not None - ): + if repl_release is not None: if cls.is_main_trio_thread(): repl_release.set() + + elif current_actor().is_infected_aio(): + + async def _set_repl_release(): + repl_release.set() + + fute: asyncio.Future = run_trio_task_in_future( + _set_repl_release + ) + if not fute.done(): + log.warning('REPL release state unknown..?') + else: # XXX NOTE ONLY used for bg root-actor sync # threads, see `.pause_from_sync()`. @@ -1658,18 +1687,24 @@ async def _pause( try: task: Task = current_task() except RuntimeError as rte: + # NOTE, 2 cases we might get here: + # + # - ACTUALLY not a `trio.lowlevel.Task` nor runtime caller, + # |_ error out as normal + # + # - an infected `asycio` actor calls it from an actual + # `asyncio.Task` + # |_ in this case we DO NOT want to RTE! __tracebackhide__: bool = False - log.exception( - 'Failed to get current `trio`-task?' - ) - # if actor.is_infected_aio(): - # mk_pdb().set_trace() - # raise RuntimeError( - # '`tractor.pause[_from_sync]()` not yet supported ' - # 'directly (infected) `asyncio` tasks!' - # ) from rte - - raise rte + if actor.is_infected_aio(): + log.exception( + 'Failed to get current `trio`-task?' + ) + raise RuntimeError( + 'An `asyncio` task should not be calling this!?' + ) from rte + else: + task = asyncio.current_task() if debug_func is not None: debug_func = partial(debug_func) @@ -2060,7 +2095,8 @@ async def _pause( f'on behalf of {repl_task} ??\n' ) - DebugStatus.release(cancel_req_task=True) + if not actor.is_infected_aio(): + DebugStatus.release(cancel_req_task=True) # sanity checks for ^ on request/status teardown # assert DebugStatus.repl is None # XXX no more bc bg thread cases? @@ -2113,7 +2149,9 @@ def _set_trace( log.pdb( f'{_pause_msg}\n' f'>(\n' - f'|_ {task} @ {actor.uid}\n' + f'|_{actor.uid}\n' + f' |_{task}\n' # @ {actor.uid}\n' + # f'|_{task}\n' # ^-TODO-^ more compact pformating? # -[ ] make an `Actor.__repr()__` # -[ ] should we use `log.pformat_task_uid()`? @@ -2390,9 +2428,6 @@ def pause_from_sync( actor: tractor.Actor = current_actor( err_on_no_runtime=False, ) - message: str = ( - f'{actor.uid} task called `tractor.pause_from_sync()`\n' - ) if not actor: raise RuntimeError( 'Not inside the `tractor`-runtime?\n' @@ -2400,6 +2435,9 @@ def pause_from_sync( '- `async with tractor.open_nursery()` or,\n' '- `async with tractor.open_root_actor()`\n' ) + message: str = ( + f'{actor.uid} task called `tractor.pause_from_sync()`\n' + ) # TODO: once supported, remove this AND the one # inside `._pause()`! @@ -2409,16 +2447,17 @@ def pause_from_sync( # injection? # -[ ] should `breakpoint()` work and what does it normally # do in `asyncio` ctxs? - if actor.is_infected_aio(): - raise RuntimeError( - '`tractor.pause[_from_sync]()` not yet supported ' - 'for infected `asyncio` mode!' - ) + # if actor.is_infected_aio(): + # raise RuntimeError( + # '`tractor.pause[_from_sync]()` not yet supported ' + # 'for infected `asyncio` mode!' + # ) repl: PdbREPL = mk_pdb() # message += f'-> created local REPL {repl}\n' is_root: bool = is_root_process() + is_aio: bool = actor.is_infected_aio() # TODO: we could also check for a non-`.to_thread` context # using `trio.from_thread.check_cancelled()` (says @@ -2431,8 +2470,11 @@ def pause_from_sync( # when called from a (bg) thread, run an async task in a new # thread which will call `._pause()` manually with special # handling for root-actor caller usage. - if not DebugStatus.is_main_trio_thread(): - + if ( + not DebugStatus.is_main_trio_thread() + and + not is_aio # see below for this usage + ): # TODO: `threading.Lock()` this so we don't get races in # multi-thr cases where they're acquiring/releasing the # REPL and setting request/`Lock` state, etc.. @@ -2440,10 +2482,21 @@ def pause_from_sync( repl_owner = thread # TODO: make root-actor bg thread usage work! - if is_root: + if ( + is_root + # or + # is_aio + ): + if is_root: + message += ( + f'-> called from a root-actor bg {thread}\n' + ) + elif is_aio: + message += ( + f'-> called from a `asyncio`-task bg {thread}\n' + ) message += ( - f'-> called from a root-actor bg {thread}\n' - f'-> scheduling `._pause_from_bg_root_thread()`..\n' + '-> scheduling `._pause_from_bg_root_thread()`..\n' ) # XXX SUBTLE BADNESS XXX that should really change! # don't over-write the `repl` here since when @@ -2462,7 +2515,7 @@ def pause_from_sync( hide_tb=hide_tb, **_pause_kwargs, ), - ) + ), ) DebugStatus.shield_sigint() message += ( @@ -2495,6 +2548,29 @@ def pause_from_sync( DebugStatus.shield_sigint() assert bg_task is not DebugStatus.repl_task + elif is_aio: + greenback: ModuleType = maybe_import_greenback() + repl_owner: Task = asyncio.current_task() + fute: asyncio.Future = run_trio_task_in_future( + partial( + _pause, + debug_func=None, + repl=repl, + hide_tb=hide_tb, + + # XXX to prevent `._pause()` for setting + # `DebugStatus.repl_task` to the gb task! + called_from_sync=True, + called_from_bg_thread=True, + + **_pause_kwargs + ) + ) + # TODO: for async version -> `.pause_from_aio()`? + # bg_task, _ = await fute + bg_task, _ = greenback.await_(fute) + bg_task: asyncio.Task = asyncio.current_task() + else: # we are presumably the `trio.run()` + main thread # raises on not-found by default greenback: ModuleType = maybe_import_greenback() @@ -2509,8 +2585,8 @@ def pause_from_sync( # NOTE XXX seems to need to be set BEFORE the `_pause()` # invoke using gb below? DebugStatus.shield_sigint() - repl_owner: Task = current_task() + message += '-> calling `greenback.await_(_pause(debug_func=None))` from sync caller..\n' try: out = greenback.await_( @@ -2572,6 +2648,10 @@ def pause_from_sync( # -[ ] tried to use `@pdbp.hideframe` decoration but # still doesn't work except BaseException as err: + log.exception( + 'Failed to sync-pause from\n\n' + f'{repl_owner}\n' + ) __tracebackhide__: bool = False raise err diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index c37c0467..8feaf88f 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -562,6 +562,101 @@ class AsyncioRuntimeTranslationError(RuntimeError): ''' +def run_trio_task_in_future( + async_fn, + *args, +) -> asyncio.Future: + ''' + Run an async-func as a `trio` task from an `asyncio.Task` wrapped + in a `asyncio.Future` which is returned to the caller. + + Another astounding feat by the great @oremanj !! + + Bo + + ''' + result_future = asyncio.Future() + cancel_scope = trio.CancelScope() + finished: bool = False + + # monkey-patch the future's `.cancel()` meth to + # allow cancellation relay to `trio`-task. + cancel_message: str|None = None + orig_cancel = result_future.cancel + + def wrapped_cancel( + msg: str|None = None, + ): + nonlocal cancel_message + if finished: + # We're being called back after the task completed + if msg is not None: + return orig_cancel(msg) + elif cancel_message is not None: + return orig_cancel(cancel_message) + else: + return orig_cancel() + + if result_future.done(): + return False + + # Forward cancellation to the Trio task, don't mark + # future as cancelled until it completes + cancel_message = msg + cancel_scope.cancel() + return True + + result_future.cancel = wrapped_cancel + + async def trio_task() -> None: + nonlocal finished + try: + with cancel_scope: + try: + # TODO: type this with new tech in 3.13 + result: Any = await async_fn(*args) + finally: + finished = True + + # Propagate result or cancellation to the Future + if cancel_scope.cancelled_caught: + result_future.cancel() + + elif not result_future.cancelled(): + result_future.set_result(result) + + except BaseException as exc: + # the result future gets all the non-Cancelled + # exceptions. Any Cancelled need to keep propagating + # out of this stack frame in order to reach the cancel + # scope for which they're intended. + cancelled: BaseException|None + rest: BaseException|None + if isinstance(exc, BaseExceptionGroup): + cancelled, rest = exc.split(trio.Cancelled) + + elif isinstance(exc, trio.Cancelled): + cancelled, rest = exc, None + + else: + cancelled, rest = None, exc + + if not result_future.cancelled(): + if rest: + result_future.set_exception(rest) + else: + result_future.cancel() + + if cancelled: + raise cancelled + + trio.lowlevel.spawn_system_task( + trio_task, + name=async_fn, + ) + return result_future + + def run_as_asyncio_guest( trio_main: Callable, # ^-NOTE-^ when spawned with `infected_aio=True` this func is From 35ebc087dd092830d73a65953935d6e73d083bae Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 29 Jul 2024 17:53:52 -0400 Subject: [PATCH 17/66] Drop `asyncio_bp` loglevel setting by default --- examples/debugging/asyncio_bp.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/examples/debugging/asyncio_bp.py b/examples/debugging/asyncio_bp.py index f2fabddd..161435d1 100644 --- a/examples/debugging/asyncio_bp.py +++ b/examples/debugging/asyncio_bp.py @@ -83,15 +83,13 @@ async def main( debug_mode=True, maybe_enable_greenback=True, # loglevel='devx', - # loglevel='runtime', ) as n: - ptl: Portal = await n.start_actor( 'aio_daemon', enable_modules=[__name__], infect_asyncio=True, debug_mode=True, - loglevel='cancel', + # loglevel='cancel', ) async with ptl.open_context( From 4b9d638be9938804d3150de2c4d348952ff895bc Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 29 Jul 2024 17:56:00 -0400 Subject: [PATCH 18/66] Use our `._post_mortem` from `open_crash_handler()` Since it seems that `pdbp.xpm()` can sometimes lose the up-stack traceback info/frames? Not sure why but ours seems to work just fine from a `asyncio`-handler in `modden`'s use of `i3ipc` B) Also call `DebugStatus.shield_sigint()` from `pause_from_sync()` in the infected-`asyncio` case to get the same shielding behaviour as in all other usage! --- tractor/devx/_debug.py | 22 +++++++++++++++------- 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 0784e594..d38c2239 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -2491,10 +2491,7 @@ def pause_from_sync( message += ( f'-> called from a root-actor bg {thread}\n' ) - elif is_aio: - message += ( - f'-> called from a `asyncio`-task bg {thread}\n' - ) + message += ( '-> scheduling `._pause_from_bg_root_thread()`..\n' ) @@ -2551,6 +2548,7 @@ def pause_from_sync( elif is_aio: greenback: ModuleType = maybe_import_greenback() repl_owner: Task = asyncio.current_task() + DebugStatus.shield_sigint() fute: asyncio.Future = run_trio_task_in_future( partial( _pause, @@ -2566,6 +2564,7 @@ def pause_from_sync( **_pause_kwargs ) ) + # TODO: for async version -> `.pause_from_aio()`? # bg_task, _ = await fute bg_task, _ = greenback.await_(fute) @@ -2700,8 +2699,7 @@ _crash_msg: str = ( def _post_mortem( - # provided and passed by `_pause()` - repl: PdbREPL, + repl: PdbREPL, # normally passed by `_pause()` # XXX all `partial`-ed in by `post_mortem()` below! tb: TracebackType, @@ -3018,7 +3016,17 @@ def open_crash_handler( yield except tuple(catch) as err: if type(err) not in ignore: - pdbp.xpm() + + # use our re-impl-ed version + _post_mortem( + repl=mk_pdb(), + tb=sys.exc_info()[2], + api_frame=inspect.currentframe().f_back, + ) + + # XXX NOTE, `pdbp`'s version seems to lose the up-stack + # tb-info? + # pdbp.xpm() raise From f8222356ce559fc2ac0bbe92367834746f793111 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 31 Jul 2024 21:34:57 -0400 Subject: [PATCH 19/66] Hide `[maybe]_open_crash_handler()` frame by default --- tractor/devx/_debug.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index d38c2239..e0a20c0e 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -2989,6 +2989,7 @@ async def maybe_wait_for_debugger( # - [ ] optional runtime plugging? # - [ ] detection for sync vs. async code? # - [ ] specialized REPL entry when in distributed mode? +# -[x] hide tb by def # - [x] allow ignoring kbi Bo @cm def open_crash_handler( @@ -2999,6 +3000,7 @@ def open_crash_handler( ignore: set[BaseException] = { KeyboardInterrupt, }, + tb_hide: bool = True, ): ''' Generic "post mortem" crash handler using `pdbp` REPL debugger. @@ -3011,6 +3013,8 @@ def open_crash_handler( `trio.run()`. ''' + __tracebackhide__: bool = tb_hide + err: BaseException try: yield @@ -3034,6 +3038,7 @@ def open_crash_handler( @cm def maybe_open_crash_handler( pdb: bool = False, + tb_hide: bool = True, ): ''' Same as `open_crash_handler()` but with bool input flag @@ -3042,6 +3047,8 @@ def maybe_open_crash_handler( Normally this is used with CLI endpoints such that if the --pdb flag is passed the pdb REPL is engaed on any crashes B) ''' + __tracebackhide__: bool = tb_hide + rtctx = nullcontext if pdb: rtctx = open_crash_handler From 6ef06be6d0a8b9d14228e27751ae2c6ba2644a9b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 22 Aug 2024 17:10:01 -0400 Subject: [PATCH 20/66] More `.pause_from_sync()` in bg-threads "polish" Various `try`/`except` blocks around external APIs that raise when not running inside an `tractor` and/or some async framework (mostly to avoid too-late/benign error tbs on certain classes of actor tree teardown): - for the `log.pdb()` prompts emitted before REPL console entry. - inside `DebugStatus.is_main_trio_thread()`'s call to `sniffio`. - in `_post_mortem()` by catching `NoRuntime` when called from a thread still active after the `.open_root_actor()` has already exited. Also, - create a dedicated `DebugStateError` for raising instead of `assert`s when we have actual debug-request inconsistencies (as seem to be most likely with bg thread usage of `breakpoint()`). - show the `open_crash_handler()` frame on `bdb.BdbQuit` (for now?) --- tractor/devx/_debug.py | 90 +++++++++++++++++++++++++++++++++--------- 1 file changed, 71 insertions(+), 19 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index e0a20c0e..5e3b00d7 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -72,6 +72,10 @@ from tractor.to_asyncio import run_trio_task_in_future from tractor.log import get_logger from tractor._context import Context from tractor import _state +from tractor._exceptions import ( + InternalError, + NoRuntime, +) from tractor._state import ( current_actor, is_root_process, @@ -691,6 +695,14 @@ async def lock_stdio_for_peer( DebugStatus.unshield_sigint() +class DebugStateError(InternalError): + ''' + Something inconsistent or unexpected happend with a sub-actor's + debug mutex request to the root actor. + + ''' + + # TODO: rename to ReplState or somethin? # DebugRequest, make it a singleton instance? class DebugStatus: @@ -860,20 +872,37 @@ class DebugStatus: `trio.to_thread.run_sync()`. ''' + try: + async_lib: str = sniffio.current_async_library() + except sniffio.AsyncLibraryNotFoundError: + async_lib = None + + is_main_thread: bool = trio._util.is_main_thread() + # ^TODO, since this is private, @oremanj says + # we should just copy the impl for now..? + if is_main_thread: + thread_name: str = 'main' + else: + thread_name: str = threading.current_thread().name + is_trio_main = ( - # TODO: since this is private, @oremanj says - # we should just copy the impl for now.. - (is_main_thread := trio._util.is_main_thread()) + is_main_thread and - (async_lib := sniffio.current_async_library()) == 'trio' + (async_lib == 'trio') ) - if ( - not is_trio_main - and is_main_thread - ): - log.warning( + + report: str = f'Running thread: {thread_name!r}\n' + if async_lib: + report += ( f'Current async-lib detected by `sniffio`: {async_lib}\n' ) + else: + report += ( + 'No async-lib detected (by `sniffio`) ??\n' + ) + if not is_trio_main: + log.warning(report) + return is_trio_main # XXX apparently unreliable..see ^ # ( @@ -2615,7 +2644,15 @@ def pause_from_sync( bg_task: Task = current_task() # assert repl is repl - assert bg_task is repl_owner + # assert bg_task is repl_owner + if bg_task is not repl_owner: + raise DebugStateError( + f'The registered bg task for this debug request is NOT its owner ??\n' + f'bg_task: {bg_task}\n' + f'repl_owner: {repl_owner}\n\n' + + f'{DebugStatus.repr()}\n' + ) # NOTE: normally set inside `_enter_repl_sync()` DebugStatus.repl_task: str = repl_owner @@ -2715,17 +2752,28 @@ def _post_mortem( ''' __tracebackhide__: bool = hide_tb - actor: tractor.Actor = current_actor() + try: + actor: tractor.Actor = current_actor() + actor_repr: str = str(actor.uid) + # ^TODO, instead a nice runtime-info + maddr + uid? + # -[ ] impl a `Actor.__repr()__`?? + # |_ : @ + + except NoRuntime: + actor_repr: str = '' + + try: + task_repr: Task = current_task() + except RuntimeError: + task_repr: str = '' # TODO: print the actor supervion tree up to the root # here! Bo log.pdb( f'{_crash_msg}\n' f'x>(\n' - f' |_ {current_task()} @ {actor.uid}\n' + f' |_ {task_repr} @ {actor_repr}\n' - # TODO: make an `Actor.__repr()__` - # f'|_ {current_task()} @ {actor.name}\n' ) # NOTE only replacing this from `pdbp.xpm()` to add the @@ -3022,11 +3070,15 @@ def open_crash_handler( if type(err) not in ignore: # use our re-impl-ed version - _post_mortem( - repl=mk_pdb(), - tb=sys.exc_info()[2], - api_frame=inspect.currentframe().f_back, - ) + try: + _post_mortem( + repl=mk_pdb(), + tb=sys.exc_info()[2], + api_frame=inspect.currentframe().f_back, + ) + except bdb.BdbQuit: + __tracebackhide__: bool = False + raise # XXX NOTE, `pdbp`'s version seems to lose the up-stack # tb-info? From bb60a6d623928cdd652104fa849636bc2bb39b4c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 3 Dec 2024 15:26:25 -0500 Subject: [PATCH 21/66] Messy-teardown `DebugStatus` related fixes Mostly fixing edge cases with `asyncio` and/or bg threads where the `.repl_release: trio.Event` needs to be used from the main `trio` thread OW confusing-but-valid teardown tracebacks can show under various races. Also improve, - log reporting for such internal bugs to make them more obvious on console via `log.exception()`. - only restore the SIGINT handler when runtime is (still) active. - reporting when `tractor.pause(shield=True)` should be used and unhiding the internal frames from the tb in that case. - for `pause_from_sync()` some deep fixes.. |_add a `allow_no_runtime: bool = False` flag to allow **not** requiring the actor runtime to be active. |_fix the `greenback` case-branch to only trigger on `not is_trio_thread`. |_add a scope-global `repl_owner: Task|Thread|None = None` to avoid ref errors.. --- tractor/devx/_debug.py | 112 +++++++++++++++++++++++++++++++---------- 1 file changed, 86 insertions(+), 26 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 5e3b00d7..83ca5492 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -730,6 +730,9 @@ class DebugStatus: # -[ ] see if we can get our proto oco task-mngr to work for # this? repl_task: Task|None = None + # repl_thread: Thread|None = None + # ^TODO? + repl_release: trio.Event|None = None req_task: Task|None = None @@ -839,11 +842,12 @@ class DebugStatus: if ( not cls.is_main_trio_thread() and - # not _state._runtime_vars.get( - # '_is_infected_aio', - # False, - # ) - not current_actor().is_infected_aio() + not _state._runtime_vars.get( + '_is_infected_aio', + False, + ) + # not current_actor().is_infected_aio() + # ^XXX, since for bg-thr case will always raise.. ): trio.from_thread.run_sync( signal.signal, @@ -928,12 +932,27 @@ class DebugStatus: try: # sometimes the task might already be terminated in # which case this call will raise an RTE? - if repl_release is not None: + # See below for reporting on that.. + if ( + repl_release is not None + and + not repl_release.is_set() + ): if cls.is_main_trio_thread(): repl_release.set() - elif current_actor().is_infected_aio(): - + elif ( + _state._runtime_vars.get( + '_is_infected_aio', + False, + ) + # ^XXX, again bc we need to not except + # but for bg-thread case it will always raise.. + # + # TODO, is there a better api then using + # `err_on_no_runtime=False` in the below? + # current_actor().is_infected_aio() + ): async def _set_repl_release(): repl_release.set() @@ -949,6 +968,15 @@ class DebugStatus: trio.from_thread.run_sync( repl_release.set ) + + except RuntimeError as rte: + log.exception( + f'Failed to release debug-request ??\n\n' + f'{cls.repr()}\n' + ) + # pdbp.set_trace() + raise rte + finally: # if req_ctx := cls.req_ctx: # req_ctx._scope.cancel() @@ -976,9 +1004,10 @@ class DebugStatus: # logging when we don't need to? cls.repl = None - # restore original sigint handler - cls.unshield_sigint() - + # maybe restore original sigint handler + # XXX requires runtime check to avoid crash! + if current_actor(err_on_no_runtime=False): + cls.unshield_sigint() # TODO: use the new `@lowlevel.singleton` for this! @@ -1066,7 +1095,7 @@ class PdbREPL(pdbp.Pdb): # Lock.release(raise_on_thread=False) Lock.release() - # XXX after `Lock.release()` for root local repl usage + # XXX AFTER `Lock.release()` for root local repl usage DebugStatus.release() def set_quit(self): @@ -1672,7 +1701,7 @@ class DebugRequestError(RuntimeError): ''' -_repl_fail_msg: str = ( +_repl_fail_msg: str|None = ( 'Failed to REPl via `_pause()` ' ) @@ -1712,6 +1741,7 @@ async def _pause( ''' __tracebackhide__: bool = hide_tb + pause_err: BaseException|None = None actor: Actor = current_actor() try: task: Task = current_task() @@ -2094,11 +2124,13 @@ async def _pause( # TODO: prolly factor this plus the similar block from # `_enter_repl_sync()` into a common @cm? - except BaseException as pause_err: + except BaseException as _pause_err: + pause_err: BaseException = _pause_err if isinstance(pause_err, bdb.BdbQuit): log.devx( - 'REPL for pdb was quit!\n' + 'REPL for pdb was explicitly quit!\n' ) + _repl_fail_msg = None # when the actor is mid-runtime cancellation the # `Actor._service_n` might get closed before we can spawn @@ -2117,13 +2149,18 @@ async def _pause( ) return - else: - log.exception( - _repl_fail_msg - + - f'on behalf of {repl_task} ??\n' + elif isinstance(pause_err, trio.Cancelled): + _repl_fail_msg = ( + 'You called `tractor.pause()` from an already cancelled scope!\n\n' + 'Consider `await tractor.pause(shield=True)` to make it work B)\n' ) + else: + _repl_fail_msg += f'on behalf of {repl_task} ??\n' + + if _repl_fail_msg: + log.exception(_repl_fail_msg) + if not actor.is_infected_aio(): DebugStatus.release(cancel_req_task=True) @@ -2152,6 +2189,8 @@ async def _pause( DebugStatus.req_err or repl_err + or + pause_err ): __tracebackhide__: bool = False @@ -2435,6 +2474,8 @@ def pause_from_sync( called_from_builtin: bool = False, api_frame: FrameType|None = None, + allow_no_runtime: bool = False, + # proxy to `._pause()`, for ex: # shield: bool = False, # api_frame: FrameType|None = None, @@ -2453,16 +2494,25 @@ def pause_from_sync( ''' __tracebackhide__: bool = hide_tb + repl_owner: Task|Thread|None = None try: actor: tractor.Actor = current_actor( err_on_no_runtime=False, ) - if not actor: - raise RuntimeError( - 'Not inside the `tractor`-runtime?\n' + if ( + not actor + and + not allow_no_runtime + ): + raise NoRuntime( + 'The actor runtime has not been opened?\n\n' '`tractor.pause_from_sync()` is not functional without a wrapping\n' '- `async with tractor.open_nursery()` or,\n' - '- `async with tractor.open_root_actor()`\n' + '- `async with tractor.open_root_actor()`\n\n' + + 'If you are getting this from a builtin `breakpoint()` call\n' + 'it might mean the runtime was started then ' + 'stopped prematurely?\n' ) message: str = ( f'{actor.uid} task called `tractor.pause_from_sync()`\n' @@ -2485,6 +2535,7 @@ def pause_from_sync( repl: PdbREPL = mk_pdb() # message += f'-> created local REPL {repl}\n' + is_trio_thread: bool = DebugStatus.is_main_trio_thread() is_root: bool = is_root_process() is_aio: bool = actor.is_infected_aio() @@ -2500,7 +2551,7 @@ def pause_from_sync( # thread which will call `._pause()` manually with special # handling for root-actor caller usage. if ( - not DebugStatus.is_main_trio_thread() + not is_trio_thread and not is_aio # see below for this usage ): @@ -2574,7 +2625,11 @@ def pause_from_sync( DebugStatus.shield_sigint() assert bg_task is not DebugStatus.repl_task - elif is_aio: + elif ( + not is_trio_thread + and + is_aio + ): greenback: ModuleType = maybe_import_greenback() repl_owner: Task = asyncio.current_task() DebugStatus.shield_sigint() @@ -2758,9 +2813,11 @@ def _post_mortem( # ^TODO, instead a nice runtime-info + maddr + uid? # -[ ] impl a `Actor.__repr()__`?? # |_ : @ + # no_runtime: bool = False except NoRuntime: actor_repr: str = '' + # no_runtime: bool = True try: task_repr: Task = current_task() @@ -2796,6 +2853,8 @@ def _post_mortem( # Since we presume the post-mortem was enaged to a task-ending # error, we MUST release the local REPL request so that not other # local task nor the root remains blocked! + # if not no_runtime: + # DebugStatus.release() DebugStatus.release() @@ -3033,6 +3092,7 @@ async def maybe_wait_for_debugger( # pass return False + # TODO: better naming and what additionals? # - [ ] optional runtime plugging? # - [ ] detection for sync vs. async code? From c1c93e08a22f3a586957e1488973af2c3e71ec6d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 4 Dec 2024 14:01:38 -0500 Subject: [PATCH 22/66] Rename `n: trio.Nursery` -> `tn` (task nursery) --- examples/debugging/asyncio_bp.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/examples/debugging/asyncio_bp.py b/examples/debugging/asyncio_bp.py index 161435d1..50a0eead 100644 --- a/examples/debugging/asyncio_bp.py +++ b/examples/debugging/asyncio_bp.py @@ -58,7 +58,7 @@ async def trio_ctx( raise_after_bp=not bp_before_started, ) as (first, chan), - trio.open_nursery() as n, + trio.open_nursery() as tn, ): assert first == 'start' @@ -67,7 +67,7 @@ async def trio_ctx( await ctx.started(first) - n.start_soon( + tn.start_soon( to_asyncio.run_task, aio_sleep_forever, ) @@ -83,8 +83,8 @@ async def main( debug_mode=True, maybe_enable_greenback=True, # loglevel='devx', - ) as n: - ptl: Portal = await n.start_actor( + ) as an: + ptl: Portal = await an.start_actor( 'aio_daemon', enable_modules=[__name__], infect_asyncio=True, From ccd60b0c6eb0a6f57d3e1f8250a9cd8156d6a217 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 5 Dec 2024 20:55:12 -0500 Subject: [PATCH 23/66] Add `breakpoint()` hook restoration example + test --- .../debugging/restore_builtin_breakpoint.py | 35 ++++++++++-- tests/devx/test_debugger.py | 53 ++++++++++++++++++- 2 files changed, 82 insertions(+), 6 deletions(-) diff --git a/examples/debugging/restore_builtin_breakpoint.py b/examples/debugging/restore_builtin_breakpoint.py index 6e141dfc..89605075 100644 --- a/examples/debugging/restore_builtin_breakpoint.py +++ b/examples/debugging/restore_builtin_breakpoint.py @@ -6,19 +6,46 @@ import tractor async def main() -> None: - async with tractor.open_nursery(debug_mode=True) as an: - assert os.environ['PYTHONBREAKPOINT'] == 'tractor._debug._set_trace' + # intially unset, no entry. + orig_pybp_var: int = os.environ.get('PYTHONBREAKPOINT') + assert orig_pybp_var in {None, "0"} + + async with tractor.open_nursery( + debug_mode=True, + ) as an: + assert an + assert ( + (pybp_var := os.environ['PYTHONBREAKPOINT']) + == + 'tractor.devx._debug._sync_pause_from_builtin' + ) # TODO: an assert that verifies the hook has indeed been, hooked # XD - assert sys.breakpointhook is not tractor._debug._set_trace + assert ( + (pybp_hook := sys.breakpointhook) + is not tractor.devx._debug._set_trace + ) + print( + f'$PYTHONOBREAKPOINT: {pybp_var!r}\n' + f'`sys.breakpointhook`: {pybp_hook!r}\n' + ) breakpoint() + pass # first bp, tractor hook set. - # TODO: an assert that verifies the hook is unhooked.. + # XXX AFTER EXIT (of actor-runtime) verify the hook is unset.. + # + # YES, this is weird but it's how stdlib docs say to do it.. + # https://docs.python.org/3/library/sys.html#sys.breakpointhook + assert os.environ.get('PYTHONBREAKPOINT') is orig_pybp_var assert sys.breakpointhook + + # now ensure a regular builtin pause still works breakpoint() + pass # last bp, stdlib hook restored + if __name__ == '__main__': trio.run(main) diff --git a/tests/devx/test_debugger.py b/tests/devx/test_debugger.py index 2a24bf98..5327fb0b 100644 --- a/tests/devx/test_debugger.py +++ b/tests/devx/test_debugger.py @@ -1229,6 +1229,53 @@ def test_shield_pause( child.expect(EOF) +def test_breakpoint_hook_restored( + spawn, +): + ''' + Ensures our actor runtime sets a custom `breakpoint()` hook + on open then restores the stdlib's default on close. + + The hook state validation is done via `assert`s inside the + invoked script with only `breakpoint()` (not `tractor.pause()`) + calls used. + + ''' + child = spawn('restore_builtin_breakpoint') + + child.expect(PROMPT) + assert_before( + child, + [ + _pause_msg, + " Date: Mon, 9 Dec 2024 15:38:28 -0500 Subject: [PATCH 24/66] Wrap `asyncio_bp.py` ex into test suite Ensuring we can at least use `breakpoint()` from an infected actor's `asyncio.Task` spawned via a `.to_asyncio` API. Also includes a little `tests/devx/` reorging, - start splitting out non-`tractor.pause()` tests into a new `test_pause_from_non_trio.py` for all the `.pause_from_sync()` use in bg-threaded or `asyncio` applications. - factor harness commonalities to the `devx/conftest` (namely the `do_ctlc()` masher). - mv `test_pause_from_sync` to the new non`-trio` mod. NOTE, the `ctlc=True` is still failing for `test_pause_from_asyncio_task` which is a user-happiness bug but not anything fundamentally broken - just need to handle the `asyncio` case in `.devx._debug.sigint_shield()`! --- examples/debugging/asyncio_bp.py | 42 +++- tests/devx/conftest.py | 51 ++++ tests/devx/test_debugger.py | 235 +----------------- tests/devx/test_pause_from_non_trio.py | 329 +++++++++++++++++++++++++ 4 files changed, 418 insertions(+), 239 deletions(-) create mode 100644 tests/devx/test_pause_from_non_trio.py diff --git a/examples/debugging/asyncio_bp.py b/examples/debugging/asyncio_bp.py index 50a0eead..b55b28fd 100644 --- a/examples/debugging/asyncio_bp.py +++ b/examples/debugging/asyncio_bp.py @@ -1,3 +1,8 @@ +''' +Examples of using the builtin `breakpoint()` from an `asyncio.Task` +running in a subactor spawned with `infect_asyncio=True`. + +''' import asyncio import trio @@ -26,15 +31,16 @@ async def bp_then_error( # NOTE: what happens here inside the hook needs some refinement.. # => seems like it's still `._debug._set_trace()` but # we set `Lock.local_task_in_debug = 'sync'`, we probably want - # some further, at least, meta-data about the task/actoq in debug - # in terms of making it clear it's asyncio mucking about. + # some further, at least, meta-data about the task/actor in debug + # in terms of making it clear it's `asyncio` mucking about. breakpoint() + # short checkpoint / delay - await asyncio.sleep(0.5) + await asyncio.sleep(0.5) # asyncio-side if raise_after_bp: - raise ValueError('blah') + raise ValueError('asyncio side error!') # TODO: test case with this so that it gets cancelled? else: @@ -46,7 +52,7 @@ async def bp_then_error( @tractor.context async def trio_ctx( ctx: tractor.Context, - bp_before_started: bool = True, + bp_before_started: bool = False, ): # this will block until the ``asyncio`` task sends a "first" @@ -55,7 +61,7 @@ async def trio_ctx( to_asyncio.open_channel_from( bp_then_error, - raise_after_bp=not bp_before_started, + # raise_after_bp=not bp_before_started, ) as (first, chan), trio.open_nursery() as tn, @@ -63,9 +69,9 @@ async def trio_ctx( assert first == 'start' if bp_before_started: - await tractor.breakpoint() + await tractor.pause() - await ctx.started(first) + await ctx.started(first) # trio-side tn.start_soon( to_asyncio.run_task, @@ -77,6 +83,10 @@ async def trio_ctx( async def main( bps_all_over: bool = True, + # TODO, WHICH OF THESE HAZ BUGZ? + cancel_from_root: bool = False, + err_from_root: bool = False, + ) -> None: async with tractor.open_nursery( @@ -99,12 +109,18 @@ async def main( assert first == 'start' - if bps_all_over: - await tractor.breakpoint() + # pause in parent to ensure no cross-actor + # locking problems exist! + await tractor.pause() + + if cancel_from_root: + await ctx.cancel() + + if err_from_root: + assert 0 + else: + await trio.sleep_forever() - # await trio.sleep_forever() - await ctx.cancel() - assert 0 # TODO: case where we cancel from trio-side while asyncio task # has debugger lock? diff --git a/tests/devx/conftest.py b/tests/devx/conftest.py index 28a14cbb..e1ad2ea3 100644 --- a/tests/devx/conftest.py +++ b/tests/devx/conftest.py @@ -2,6 +2,7 @@ `tractor.devx.*` tooling sub-pkg test space. ''' +import time from typing import ( Callable, ) @@ -11,9 +12,19 @@ from pexpect.exceptions import ( TIMEOUT, ) from pexpect.spawnbase import SpawnBase + from tractor._testing import ( mk_cmd, ) +from tractor.devx._debug import ( + _pause_msg as _pause_msg, + _crash_msg as _crash_msg, + _repl_fail_msg as _repl_fail_msg, + _ctlc_ignore_header as _ctlc_ignore_header, +) +from conftest import ( + _ci_env, +) @pytest.fixture @@ -107,6 +118,9 @@ def expect( raise +PROMPT = r"\(Pdb\+\)" + + def in_prompt_msg( child: SpawnBase, parts: list[str], @@ -166,3 +180,40 @@ def assert_before( err_on_false=True, **kwargs ) + + +def do_ctlc( + child, + count: int = 3, + delay: float = 0.1, + patt: str|None = None, + + # expect repl UX to reprint the prompt after every + # ctrl-c send. + # XXX: no idea but, in CI this never seems to work even on 3.10 so + # needs some further investigation potentially... + expect_prompt: bool = not _ci_env, + +) -> str|None: + + before: str|None = None + + # make sure ctl-c sends don't do anything but repeat output + for _ in range(count): + time.sleep(delay) + child.sendcontrol('c') + + # TODO: figure out why this makes CI fail.. + # if you run this test manually it works just fine.. + if expect_prompt: + time.sleep(delay) + child.expect(PROMPT) + before = str(child.before.decode()) + time.sleep(delay) + + if patt: + # should see the last line on console + assert patt in before + + # return the console content up to the final prompt + return before diff --git a/tests/devx/test_debugger.py b/tests/devx/test_debugger.py index 5327fb0b..254b92a1 100644 --- a/tests/devx/test_debugger.py +++ b/tests/devx/test_debugger.py @@ -21,7 +21,9 @@ from pexpect.exceptions import ( EOF, ) -from tractor.devx._debug import ( +from .conftest import ( + do_ctlc, + PROMPT, _pause_msg, _crash_msg, _repl_fail_msg, @@ -68,9 +70,6 @@ has_nested_actors = pytest.mark.has_nested_actors # ) -PROMPT = r"\(Pdb\+\)" - - @pytest.mark.parametrize( 'user_in_out', [ @@ -121,8 +120,10 @@ def test_root_actor_error( ids=lambda item: f'{item[0]} -> {item[1]}', ) def test_root_actor_bp(spawn, user_in_out): - """Demonstrate breakpoint from in root actor. - """ + ''' + Demonstrate breakpoint from in root actor. + + ''' user_input, expect_err_str = user_in_out child = spawn('root_actor_breakpoint') @@ -144,43 +145,6 @@ def test_root_actor_bp(spawn, user_in_out): assert expect_err_str in str(child.before) -def do_ctlc( - child, - count: int = 3, - delay: float = 0.1, - patt: str|None = None, - - # expect repl UX to reprint the prompt after every - # ctrl-c send. - # XXX: no idea but, in CI this never seems to work even on 3.10 so - # needs some further investigation potentially... - expect_prompt: bool = not _ci_env, - -) -> str|None: - - before: str|None = None - - # make sure ctl-c sends don't do anything but repeat output - for _ in range(count): - time.sleep(delay) - child.sendcontrol('c') - - # TODO: figure out why this makes CI fail.. - # if you run this test manually it works just fine.. - if expect_prompt: - time.sleep(delay) - child.expect(PROMPT) - before = str(child.before.decode()) - time.sleep(delay) - - if patt: - # should see the last line on console - assert patt in before - - # return the console content up to the final prompt - return before - - def test_root_actor_bp_forever( spawn, ctlc: bool, @@ -917,138 +881,6 @@ def test_different_debug_mode_per_actor( ) -def test_pause_from_sync( - spawn, - ctlc: bool -): - ''' - Verify we can use the `pdbp` REPL from sync functions AND from - any thread spawned with `trio.to_thread.run_sync()`. - - `examples/debugging/sync_bp.py` - - ''' - child = spawn('sync_bp') - - # first `sync_pause()` after nurseries open - child.expect(PROMPT) - assert_before( - child, - [ - # pre-prompt line - _pause_msg, - " similar to the `delay` input to `do_ctlc()` below, setting - # this too low can cause the test to fail since the `subactor` - # suffers a race where the root/parent sends an actor-cancel - # prior to the context task hitting its pause point (and thus - # engaging the `sigint_shield()` handler in time); this value - # seems be good enuf? - time.sleep(0.6) - - # one of the bg thread or subactor should have - # `Lock.acquire()`-ed - # (NOT both, which will result in REPL clobbering!) - attach_patts: dict[str, list[str]] = { - 'subactor': [ - "'start_n_sync_pause'", - "('subactor'", - ], - 'inline_root_bg_thread': [ - " similar to the `delay` input to `do_ctlc()` below, setting + # this too low can cause the test to fail since the `subactor` + # suffers a race where the root/parent sends an actor-cancel + # prior to the context task hitting its pause point (and thus + # engaging the `sigint_shield()` handler in time); this value + # seems be good enuf? + time.sleep(0.6) + + # one of the bg thread or subactor should have + # `Lock.acquire()`-ed + # (NOT both, which will result in REPL clobbering!) + attach_patts: dict[str, list[str]] = { + 'subactor': [ + "'start_n_sync_pause'", + "('subactor'", + ], + 'inline_root_bg_thread': [ + " list[str]: + ''' + Receive any of a `list[str]` of patterns provided in + `attach_patts`. + + Used to test racing prompts from multiple actors and/or + tasks using a common root process' `pdbp` REPL. + + ''' + assert attach_patts + + child.expect(PROMPT) + before = str(child.before.decode()) + + for attach_key in attach_patts: + if attach_key in before: + expected_patts: str = attach_patts.pop(attach_key) + assert_before( + child, + expected_patts + ) + break # from for + else: + pytest.fail( + f'No keys found?\n\n' + f'{attach_patts.keys()}\n\n' + f'{before}\n' + ) + + # ensure no other task/threads engaged a REPL + # at the same time as the one that was detected above. + for key, other_patts in attach_patts.copy().items(): + assert not in_prompt_msg( + child, + other_patts, + ) + + if ctlc: + do_ctlc( + child, + patt=prompt, + # NOTE same as comment above + delay=ctlc_delay, + ) + + return expected_patts + # yield child + + +def test_pause_from_asyncio_task( + spawn, + ctlc: bool + # ^TODO, fix for `asyncio`!! +): + ''' + Verify we can use the `pdbp` REPL from an `asyncio.Task` spawned using + APIs in `.to_asyncio`. + + `examples/debugging/asycio_bp.py` + + ''' + child = spawn('asyncio_bp') + + # RACE on whether trio/asyncio task bps first + attach_patts: dict[str, list[str]] = { + + # first pause in guest-mode (aka "infecting") + # `trio.Task`. + 'trio-side': [ + _pause_msg, + " Date: Mon, 9 Dec 2024 16:08:55 -0500 Subject: [PATCH 25/66] Change `tractor.breakpoint()` to new `.pause()` in test suite --- examples/debugging/fast_error_in_root_after_spawn.py | 2 +- examples/debugging/multi_daemon_subactors.py | 2 +- .../multi_nested_subactors_error_up_through_nurseries.py | 2 +- examples/debugging/multi_subactors.py | 2 +- examples/debugging/root_actor_breakpoint.py | 2 +- examples/debugging/root_actor_breakpoint_forever.py | 2 +- examples/debugging/subactor_bp_in_ctx.py | 6 +++--- tests/test_context_stream_semantics.py | 2 +- tests/test_task_broadcasting.py | 2 +- 9 files changed, 11 insertions(+), 11 deletions(-) diff --git a/examples/debugging/fast_error_in_root_after_spawn.py b/examples/debugging/fast_error_in_root_after_spawn.py index 570cf7ef..86710788 100644 --- a/examples/debugging/fast_error_in_root_after_spawn.py +++ b/examples/debugging/fast_error_in_root_after_spawn.py @@ -1,5 +1,5 @@ ''' -Fast fail test with a context. +Fast fail test with a `Context`. Ensure the partially initialized sub-actor process doesn't cause a hang on error/cancel of the parent diff --git a/examples/debugging/multi_daemon_subactors.py b/examples/debugging/multi_daemon_subactors.py index 4a462623..7844ccef 100644 --- a/examples/debugging/multi_daemon_subactors.py +++ b/examples/debugging/multi_daemon_subactors.py @@ -7,7 +7,7 @@ async def breakpoint_forever(): try: while True: yield 'yo' - await tractor.breakpoint() + await tractor.pause() except BaseException: tractor.log.get_console_log().exception( 'Cancelled while trying to enter pause point!' diff --git a/examples/debugging/multi_nested_subactors_error_up_through_nurseries.py b/examples/debugging/multi_nested_subactors_error_up_through_nurseries.py index 8df52e3b..b63f1945 100644 --- a/examples/debugging/multi_nested_subactors_error_up_through_nurseries.py +++ b/examples/debugging/multi_nested_subactors_error_up_through_nurseries.py @@ -10,7 +10,7 @@ async def name_error(): async def breakpoint_forever(): "Indefinitely re-enter debugger in child actor." while True: - await tractor.breakpoint() + await tractor.pause() # NOTE: if the test never sent 'q'/'quit' commands # on the pdb repl, without this checkpoint line the diff --git a/examples/debugging/multi_subactors.py b/examples/debugging/multi_subactors.py index 22b13ac8..57634cc3 100644 --- a/examples/debugging/multi_subactors.py +++ b/examples/debugging/multi_subactors.py @@ -6,7 +6,7 @@ async def breakpoint_forever(): "Indefinitely re-enter debugger in child actor." while True: await trio.sleep(0.1) - await tractor.breakpoint() + await tractor.pause() async def name_error(): diff --git a/examples/debugging/root_actor_breakpoint.py b/examples/debugging/root_actor_breakpoint.py index 5c858d4c..55b4ca56 100644 --- a/examples/debugging/root_actor_breakpoint.py +++ b/examples/debugging/root_actor_breakpoint.py @@ -10,7 +10,7 @@ async def main(): await trio.sleep(0.1) - await tractor.breakpoint() + await tractor.pause() await trio.sleep(0.1) diff --git a/examples/debugging/root_actor_breakpoint_forever.py b/examples/debugging/root_actor_breakpoint_forever.py index 88a6e0e9..04cd7e7e 100644 --- a/examples/debugging/root_actor_breakpoint_forever.py +++ b/examples/debugging/root_actor_breakpoint_forever.py @@ -11,7 +11,7 @@ async def main( # loglevel='runtime', ): while True: - await tractor.breakpoint() + await tractor.pause() if __name__ == '__main__': diff --git a/examples/debugging/subactor_bp_in_ctx.py b/examples/debugging/subactor_bp_in_ctx.py index a47dbd92..2c5fee8c 100644 --- a/examples/debugging/subactor_bp_in_ctx.py +++ b/examples/debugging/subactor_bp_in_ctx.py @@ -4,9 +4,9 @@ import trio async def gen(): yield 'yo' - await tractor.breakpoint() + await tractor.pause() yield 'yo' - await tractor.breakpoint() + await tractor.pause() @tractor.context @@ -15,7 +15,7 @@ async def just_bp( ) -> None: await ctx.started() - await tractor.breakpoint() + await tractor.pause() # TODO: bps and errors in this call.. async for val in gen(): diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index 8edea510..ade275aa 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -955,7 +955,7 @@ async def echo_back_sequence( ) await ctx.started() - # await tractor.breakpoint() + # await tractor.pause() async with ctx.open_stream( msg_buffer_size=msg_buffer_size, diff --git a/tests/test_task_broadcasting.py b/tests/test_task_broadcasting.py index d7a29134..4a2209eb 100644 --- a/tests/test_task_broadcasting.py +++ b/tests/test_task_broadcasting.py @@ -271,7 +271,7 @@ def test_faster_task_to_recv_is_cancelled_by_slower( # the faster subtask was cancelled break - # await tractor.breakpoint() + # await tractor.pause() # await stream.receive() print(f'final value: {value}') From 5b29dd5d2be8b46b9b396ef843bc88f7aaed2927 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 9 Dec 2024 16:56:30 -0500 Subject: [PATCH 26/66] Expose a `_ctlc_ignore_header: str` for use in `sigint_shield()` --- tractor/devx/_debug.py | 30 +++++++++++++++++++++--------- 1 file changed, 21 insertions(+), 9 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 83ca5492..2f010b15 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -1420,6 +1420,10 @@ def any_connected_locker_child() -> bool: return False +_ctlc_ignore_header: str = ( + 'Ignoring SIGINT while debug REPL in use' +) + def sigint_shield( signum: int, frame: 'frame', # type: ignore # noqa @@ -1501,7 +1505,9 @@ def sigint_shield( # NOTE: don't emit this with `.pdb()` level in # root without a higher level. log.runtime( - f'Ignoring SIGINT while debug REPL in use by child ' + _ctlc_ignore_header + + + f' by child ' f'{uid_in_debug}\n' ) problem = None @@ -1535,7 +1541,9 @@ def sigint_shield( # NOTE: since we emit this msg on ctl-c, we should # also always re-print the prompt the tail block! log.pdb( - 'Ignoring SIGINT while pdb REPL in use by root actor..\n' + _ctlc_ignore_header + + + f' by root actor..\n' f'{DebugStatus.repl_task}\n' f' |_{repl}\n' ) @@ -1596,16 +1604,20 @@ def sigint_shield( repl ): log.pdb( - f'Ignoring SIGINT while local task using debug REPL\n' - f'|_{repl_task}\n' - f' |_{repl}\n' + _ctlc_ignore_header + + + f' by local task\n\n' + f'{repl_task}\n' + f' |_{repl}\n' ) elif req_task: log.debug( - 'Ignoring SIGINT while debug request task is open but either,\n' - '- someone else is already REPL-in and has the `Lock`, or\n' - '- some other local task already is replin?\n' - f'|_{req_task}\n' + _ctlc_ignore_header + + + f' by local request-task and either,\n' + f'- someone else is already REPL-in and has the `Lock`, or\n' + f'- some other local task already is replin?\n\n' + f'{req_task}\n' ) # TODO can we remove this now? From 61c561394322f4a5c1792080aab82cc960159f22 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 9 Dec 2024 16:57:22 -0500 Subject: [PATCH 27/66] Support custom `boxer_header: str` provided by `pformat_boxed_tb()` caller --- tractor/devx/pformat.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tractor/devx/pformat.py b/tractor/devx/pformat.py index 5fe9bc62..d24eaaf4 100644 --- a/tractor/devx/pformat.py +++ b/tractor/devx/pformat.py @@ -53,6 +53,7 @@ def pformat_boxed_tb( tb_box_indent: int|None = None, tb_body_indent: int = 1, + boxer_header: str = '-' ) -> str: ''' @@ -88,9 +89,9 @@ def pformat_boxed_tb( tb_box: str = ( f'|\n' - f' ------ - ------\n' + f' ------ {boxer_header} ------\n' f'{tb_body}' - f' ------ - ------\n' + f' ------ {boxer_header}- ------\n' f'_|\n' ) tb_box_indent: str = ( From 141a842d3d1d85702eda1bfb6be4a32a20fc0947 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 9 Dec 2024 16:57:57 -0500 Subject: [PATCH 28/66] Set `RemoteActorError.pformat(boxer_header=self.relay_uid)` by def --- tractor/_exceptions.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index a0b6ff3f..812664ac 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -609,6 +609,7 @@ class RemoteActorError(Exception): # just after Date: Mon, 9 Dec 2024 17:14:51 -0500 Subject: [PATCH 29/66] Add TODO for a tb frame "filterer" sys.. --- tractor/devx/_frame_stack.py | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/tractor/devx/_frame_stack.py b/tractor/devx/_frame_stack.py index 89a9e849..8e9bf46f 100644 --- a/tractor/devx/_frame_stack.py +++ b/tractor/devx/_frame_stack.py @@ -234,7 +234,7 @@ def find_caller_info( _frame2callerinfo_cache: dict[FrameType, CallerInfo] = {} -# TODO: -[x] move all this into new `.devx._code`! +# TODO: -[x] move all this into new `.devx._frame_stack`! # -[ ] consider rename to _callstack? # -[ ] prolly create a `@runtime_api` dec? # |_ @api_frame seems better? @@ -286,3 +286,18 @@ def api_frame( wrapped._call_infos: dict[FrameType, CallerInfo] = _frame2callerinfo_cache wrapped.__api_func__: bool = True return wrapper(wrapped) + + +# TODO: something like this instead of the adhoc frame-unhiding +# blocks all over the runtime!! XD +# -[ ] ideally we can expect a certain error (set) and if something +# else is raised then all frames below the wrapped one will be +# un-hidden via `__tracebackhide__: bool = False`. +# |_ might need to dynamically mutate the code objs like +# `pdbp.hideframe()` does? +# -[ ] use this as a `@acm` decorator as introed in 3.10? +# @acm +# async def unhide_frame_when_not( +# error_set: set[BaseException], +# ) -> TracebackType: +# ... From 684253ab1178670051abf6b2519ec5e5921bf45c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 9 Dec 2024 17:37:27 -0500 Subject: [PATCH 30/66] Type the inter-loop chans --- tractor/to_asyncio.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index 8feaf88f..f2a8570b 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -245,14 +245,14 @@ def _run_asyncio_task( result != orig and aio_err is None and - # in the ``open_channel_from()`` case we don't + # in the `open_channel_from()` case we don't # relay through the "return value". not provide_channels ): to_trio.send_nowait(result) finally: - # if the task was spawned using ``open_channel_from()`` + # if the task was spawned using `open_channel_from()` # then we close the channels on exit. if provide_channels: # only close the sender side which will relay @@ -500,7 +500,7 @@ async def run_task( ''' # simple async func - chan = _run_asyncio_task( + chan: LinkedTaskChannel = _run_asyncio_task( func, qsize=1, **kwargs, @@ -530,7 +530,7 @@ async def open_channel_from( spawned ``asyncio`` task and ``trio``. ''' - chan = _run_asyncio_task( + chan: LinkedTaskChannel = _run_asyncio_task( target, qsize=2**8, provide_channels=True, From 514fb1a4ac0fc8d158a49cfc8f2dad080985ec05 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 9 Dec 2024 18:04:32 -0500 Subject: [PATCH 31/66] Change masked `.pause()` line --- tractor/_context.py | 21 +++++++++++++++++---- tractor/trionics/_broadcast.py | 2 +- 2 files changed, 18 insertions(+), 5 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index 31db2bad..d4cad88e 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -1703,15 +1703,28 @@ class Context: # TODO: expose as mod func instead! structfmt = pretty_struct.Struct.pformat if self._in_overrun: - log.warning( - f'Queueing OVERRUN msg on caller task:\n\n' - + report: str = ( f'{flow_body}' - f'{structfmt(msg)}\n' ) + over_q: deque = self._overflow_q self._overflow_q.append(msg) + if len(over_q) == over_q.maxlen: + report = ( + 'FAILED to queue OVERRUN msg, OVERAN the OVERRUN QUEUE !!\n\n' + + report + ) + # log.error(report) + log.debug(report) + + else: + report = ( + 'Queueing OVERRUN msg on caller task:\n\n' + + report + ) + log.debug(report) + # XXX NOTE XXX # overrun is the ONLY case where returning early is fine! return False diff --git a/tractor/trionics/_broadcast.py b/tractor/trionics/_broadcast.py index 977b6828..154b037d 100644 --- a/tractor/trionics/_broadcast.py +++ b/tractor/trionics/_broadcast.py @@ -382,7 +382,7 @@ class BroadcastReceiver(ReceiveChannel): # likely it makes sense to unwind back to the # underlying? # import tractor - # await tractor.breakpoint() + # await tractor.pause() log.warning( f'Only one sub left for {self}?\n' 'We can probably unwind from breceiver?' From 8d190bb5055ecd48c20952b8a0d3ff6b990d466c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 9 Dec 2024 18:12:22 -0500 Subject: [PATCH 32/66] Add TODO for a runtime-vars passing mechanism --- tractor/_root.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tractor/_root.py b/tractor/_root.py index 51dbe5e6..bcdee3ef 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -95,6 +95,10 @@ async def open_root_actor( hide_tb: bool = True, + # TODO, a way for actors to augment passing derived + # read-only state to sublayers? + # extra_rt_vars: dict|None = None, + ) -> Actor: ''' Runtime init entry point for ``tractor``. From 0fe6f63012777529c101024c04dc90922684aaa5 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 9 Dec 2024 18:14:11 -0500 Subject: [PATCH 33/66] Add `notes_to_self/howtorelease.md` reminder doc --- notes_to_self/howtorelease.md | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) create mode 100644 notes_to_self/howtorelease.md diff --git a/notes_to_self/howtorelease.md b/notes_to_self/howtorelease.md new file mode 100644 index 00000000..5f31a6d8 --- /dev/null +++ b/notes_to_self/howtorelease.md @@ -0,0 +1,18 @@ +First generate a built disti: + +``` +python -m pip install --upgrade build +python -m build --sdist --outdir dist/alpha5/ +``` + +Then try a test ``pypi`` upload: + +``` +python -m twine upload --repository testpypi dist/alpha5/* +``` + +The push to `pypi` for realz. + +``` +python -m twine upload --repository testpypi dist/alpha5/* +``` From 824801d2bad7c7dee978ed89176e7ed94875e57b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 9 Dec 2024 20:57:56 -0500 Subject: [PATCH 34/66] Spitballing how to expose custom `msgspec` type hooks Such that maybe we can eventually offer a nicer higher-level API which implements much of the boilerplate required by `msgspec` (like type-matched branching to serialization logic) via a type-table interface or something? Not sure if the idea is that useful so leaving it all as TODOs for now obviously. --- tractor/msg/_codec.py | 91 ++++++++++++++++++++++++++++++------------- 1 file changed, 65 insertions(+), 26 deletions(-) diff --git a/tractor/msg/_codec.py b/tractor/msg/_codec.py index c1301bd2..32f690f1 100644 --- a/tractor/msg/_codec.py +++ b/tractor/msg/_codec.py @@ -41,8 +41,10 @@ import textwrap from typing import ( Any, Callable, + Protocol, Type, TYPE_CHECKING, + TypeVar, Union, ) from types import ModuleType @@ -181,7 +183,11 @@ def mk_dec( dec_hook: Callable|None = None, ) -> MsgDec: + ''' + Create an IPC msg decoder, normally used as the + `PayloadMsg.pld: PayloadT` field decoder inside a `PldRx`. + ''' return MsgDec( _dec=msgpack.Decoder( type=spec, # like `MsgType[Any]` @@ -227,6 +233,13 @@ def pformat_msgspec( join_char: str = '\n', ) -> str: + ''' + Pretty `str` format the `msgspec.msgpack.Decoder.type` attribute + for display in (console) log messages as a nice (maybe multiline) + presentation of all supported `Struct`s (subtypes) available for + typed decoding. + + ''' dec: msgpack.Decoder = getattr(codec, 'dec', codec) return join_char.join( mk_msgspec_table( @@ -630,31 +643,57 @@ def limit_msg_spec( # # import pdbp; pdbp.set_trace() # assert ext_codec.pld_spec == extended_spec # yield ext_codec +# +# ^-TODO-^ is it impossible to make something like this orr!? + +# TODO: make an auto-custom hook generator from a set of input custom +# types? +# -[ ] below is a proto design using a `TypeCodec` idea? +# +# type var for the expected interchange-lib's +# IPC-transport type when not available as a built-in +# serialization output. +WireT = TypeVar('WireT') -# TODO: make something similar to this inside `._codec` such that -# user can just pass a type table of some sort? -# -[ ] we would need to decode all msgs to `pretty_struct.Struct` -# and then call `.to_dict()` on them? -# -[x] we're going to need to re-impl all the stuff changed in the -# runtime port such that it can handle dicts or `Msg`s? -# -# def mk_dict_msg_codec_hooks() -> tuple[Callable, Callable]: -# ''' -# Deliver a `enc_hook()`/`dec_hook()` pair which does -# manual convertion from our above native `Msg` set -# to `dict` equivalent (wire msgs) in order to keep legacy compat -# with the original runtime implementation. -# -# Note: this is is/was primarly used while moving the core -# runtime over to using native `Msg`-struct types wherein we -# start with the send side emitting without loading -# a typed-decoder and then later flipping the switch over to -# load to the native struct types once all runtime usage has -# been adjusted appropriately. -# -# ''' -# return ( -# # enc_to_dict, -# dec_from_dict, -# ) +# TODO: some kinda (decorator) API for built-in subtypes +# that builds this implicitly by inspecting the `mro()`? +class TypeCodec(Protocol): + ''' + A per-custom-type wire-transport serialization translator + description type. + + ''' + src_type: Type + wire_type: WireT + + def encode(obj: Type) -> WireT: + ... + + def decode( + obj_type: Type[WireT], + obj: WireT, + ) -> Type: + ... + + +class MsgpackTypeCodec(TypeCodec): + ... + + +def mk_codec_hooks( + type_codecs: list[TypeCodec], + +) -> tuple[Callable, Callable]: + ''' + Deliver a `enc_hook()`/`dec_hook()` pair which handle + manual convertion from an input `Type` set such that whenever + the `TypeCodec.filter()` predicate matches the + `TypeCodec.decode()` is called on the input native object by + the `dec_hook()` and whenever the + `isiinstance(obj, TypeCodec.type)` matches against an + `enc_hook(obj=obj)` the return value is taken from a + `TypeCodec.encode(obj)` callback. + + ''' + ... From 2764d82c1aaa19cfa6aca5a384d02b80a73b2ba5 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 10 Dec 2024 13:50:35 -0500 Subject: [PATCH 35/66] Draft a (pretty)`Struct.fields_diff()` For comparing a `msgspec.Struct` against an input `dict` presumably to be used as input for struct instantiation. The main diff with `.__sub__()` is that non-existing fields on either are reported (loudly). --- tractor/msg/pretty_struct.py | 90 ++++++++++++++++++++++++++++-------- 1 file changed, 70 insertions(+), 20 deletions(-) diff --git a/tractor/msg/pretty_struct.py b/tractor/msg/pretty_struct.py index 15e469e2..91eba8bd 100644 --- a/tractor/msg/pretty_struct.py +++ b/tractor/msg/pretty_struct.py @@ -30,9 +30,9 @@ from msgspec import ( Struct as _Struct, structs, ) -from pprint import ( - saferepr, -) +# from pprint import ( +# saferepr, +# ) from tractor.log import get_logger @@ -75,8 +75,8 @@ class DiffDump(UserList): for k, left, right in self: repstr += ( f'({k},\n' - f'\t{repr(left)},\n' - f'\t{repr(right)},\n' + f' |_{repr(left)},\n' + f' |_{repr(right)},\n' ')\n' ) repstr += ']\n' @@ -144,15 +144,22 @@ def pformat( field_indent=indent + field_indent, ) - else: # the `pprint` recursion-safe format: + else: + val_str: str = repr(v) + + # XXX LOL, below just seems to be f#$%in causing + # recursion errs.. + # + # the `pprint` recursion-safe format: # https://docs.python.org/3.11/library/pprint.html#pprint.saferepr - try: - val_str: str = saferepr(v) - except Exception: - log.exception( - 'Failed to `saferepr({type(struct)})` !?\n' - ) - return _Struct.__repr__(struct) + # try: + # val_str: str = saferepr(v) + # except Exception: + # log.exception( + # 'Failed to `saferepr({type(struct)})` !?\n' + # ) + # raise + # return _Struct.__repr__(struct) # TODO: LOLOL use `textwrap.indent()` instead dawwwwwg! obj_str += (field_ws + f'{k}: {typ_name} = {val_str},\n') @@ -203,12 +210,7 @@ class Struct( return sin_props pformat = pformat - # __repr__ = pformat - # __str__ = __repr__ = pformat - # TODO: use a pprint.PrettyPrinter instance around ONLY rendering - # inside a known tty? - # def __repr__(self) -> str: - # ... + def __repr__(self) -> str: try: return pformat(self) @@ -218,6 +220,13 @@ class Struct( ) return _Struct.__repr__(self) + # __repr__ = pformat + # __str__ = __repr__ = pformat + # TODO: use a pprint.PrettyPrinter instance around ONLY rendering + # inside a known tty? + # def __repr__(self) -> str: + # ... + def copy( self, update: dict | None = None, @@ -267,13 +276,15 @@ class Struct( fi.type(getattr(self, fi.name)), ) + # TODO: make a mod func instead and just point to it here for + # method impl? def __sub__( self, other: Struct, ) -> DiffDump[tuple[str, Any, Any]]: ''' - Compare fields/items key-wise and return a ``DiffDump`` + Compare fields/items key-wise and return a `DiffDump` for easy visual REPL comparison B) ''' @@ -290,3 +301,42 @@ class Struct( )) return diffs + + @classmethod + def fields_diff( + cls, + other: dict|Struct, + + ) -> DiffDump[tuple[str, Any, Any]]: + ''' + Very similar to `PrettyStruct.__sub__()` except accepts an + input `other: dict` (presumably that would normally be called + like `Struct(**other)`) which returns a `DiffDump` of the + fields of the struct and the `dict`'s fields. + + ''' + nullish = object() + consumed: dict = other.copy() + diffs: DiffDump[tuple[str, Any, Any]] = DiffDump() + for fi in structs.fields(cls): + field_name: str = fi.name + # ours: Any = getattr(self, field_name) + theirs: Any = consumed.pop(field_name, nullish) + if theirs is nullish: + diffs.append(( + field_name, + f'{fi.type!r}', + 'NOT-DEFINED in `other: dict`', + )) + + # when there are lingering fields in `other` that this struct + # DOES NOT define we also append those. + if consumed: + for k, v in consumed.items(): + diffs.append(( + k, + f'NOT-DEFINED for `{cls.__name__}`', + f'`other: dict` has value = {v!r}', + )) + + return diffs From c2bbb7e25927a4cee880f27f9f39d191a39e245c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 10 Dec 2024 14:43:39 -0500 Subject: [PATCH 36/66] TOSQUASH: 9002f60 howtorelease.md file --- notes_to_self/howtorelease.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/notes_to_self/howtorelease.md b/notes_to_self/howtorelease.md index 5f31a6d8..a1b52d7a 100644 --- a/notes_to_self/howtorelease.md +++ b/notes_to_self/howtorelease.md @@ -1,7 +1,7 @@ First generate a built disti: ``` -python -m pip install --upgrade build +python -m pip install --upgrade build python -m build --sdist --outdir dist/alpha5/ ``` From a283d8c05aab72e157761c35b8d489b422fa0e2b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 11 Dec 2024 22:23:17 -0500 Subject: [PATCH 37/66] Support and test infected-`asyncio`-mode for root Such that you can use, ```python tractor.to_asyncio.run_as_asyncio_guest( trio_main=_trio_main, ) ``` to boostrap the root actor (and thus main parent process) to embed the actor-rumtime into an `asyncio` loop. Prove it all works with an subactor-free version of the aio echo-server test suite B) --- tests/test_infected_asyncio.py | 112 +++++++++++++++++++++++++-------- tractor/_root.py | 4 ++ tractor/to_asyncio.py | 5 ++ 3 files changed, 96 insertions(+), 25 deletions(-) diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index f5fa0aab..b0a11715 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -5,6 +5,7 @@ The hipster way to force SC onto the stdlib's "async": 'infection mode'. import asyncio import builtins from contextlib import ExitStack +from functools import partial import itertools import importlib import os @@ -536,41 +537,40 @@ def test_aio_errors_and_channel_propagates_and_closes(reg_addr): excinfo.value.boxed_type is Exception +async def aio_echo_server( + to_trio: trio.MemorySendChannel, + from_trio: asyncio.Queue, +) -> None: + + to_trio.send_nowait('start') + + while True: + msg = await from_trio.get() + + # echo the msg back + to_trio.send_nowait(msg) + + # if we get the terminate sentinel + # break the echo loop + if msg is None: + print('breaking aio echo loop') + break + + print('exiting asyncio task') + + @tractor.context async def trio_to_aio_echo_server( - ctx: tractor.Context, + ctx: tractor.Context|None, ): - - async def aio_echo_server( - to_trio: trio.MemorySendChannel, - from_trio: asyncio.Queue, - ) -> None: - - to_trio.send_nowait('start') - - while True: - msg = await from_trio.get() - - # echo the msg back - to_trio.send_nowait(msg) - - # if we get the terminate sentinel - # break the echo loop - if msg is None: - print('breaking aio echo loop') - break - - print('exiting asyncio task') - async with to_asyncio.open_channel_from( aio_echo_server, ) as (first, chan): - assert first == 'start' + await ctx.started(first) async with ctx.open_stream() as stream: - async for msg in stream: print(f'asyncio echoing {msg}') await chan.send(msg) @@ -649,6 +649,68 @@ def test_echoserver_detailed_mechanics( trio.run(main) +@pytest.mark.parametrize( + 'raise_error_mid_stream', + [ + False, + Exception, + KeyboardInterrupt, + ], + ids='raise_error={}'.format, +) +def test_infected_root_actor( + raise_error_mid_stream: bool|Exception, + # conftest wide + loglevel: str, + debug_mode: bool, +): + ''' + Verify you can run the `tractor` runtime with `Actor.is_infected_aio() == True` + in the root actor. + + ''' + async def _trio_main(): + + first: str + chan: to_asyncio.LinkedTaskChannel + async with ( + tractor.open_root_actor( + debug_mode=debug_mode, + loglevel=loglevel, + ), + to_asyncio.open_channel_from( + aio_echo_server, + ) as (first, chan), + ): + assert first == 'start' + + for i in range(1000): + await chan.send(i) + out = await chan.receive() + assert out == i + print(f'asyncio echoing {i}') + + if raise_error_mid_stream and i == 500: + raise raise_error_mid_stream + + if out is None: + try: + out = await chan.receive() + except trio.EndOfChannel: + break + else: + raise RuntimeError('aio channel never stopped?') + + if raise_error_mid_stream: + with pytest.raises(raise_error_mid_stream): + tractor.to_asyncio.run_as_asyncio_guest( + trio_main=_trio_main, + ) + else: + tractor.to_asyncio.run_as_asyncio_guest( + trio_main=_trio_main, + ) + @tractor.context async def manage_file( diff --git a/tractor/_root.py b/tractor/_root.py index bcdee3ef..38ddbe23 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -334,6 +334,10 @@ async def open_root_actor( loglevel=loglevel, enable_modules=enable_modules, ) + # XXX, in case the root actor runtime was actually run from + # `tractor.to_asyncio.run_as_asyncio_guest()` and NOt + # `.trio.run()`. + actor._infected_aio = _state._runtime_vars['_is_infected_aio'] # Start up main task set via core actor-runtime nurseries. try: diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index f2a8570b..24f1ace6 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -36,6 +36,7 @@ import tractor from tractor._exceptions import AsyncioCancelled from tractor._state import ( debug_mode, + _runtime_vars, ) from tractor.devx import _debug from tractor.log import get_logger @@ -767,12 +768,16 @@ def run_as_asyncio_guest( 'Infecting `asyncio`-process with a `trio` guest-run!\n' ) + # TODO, somehow bootstrap this! + _runtime_vars['_is_infected_aio'] = True + trio.lowlevel.start_guest_run( trio_main, run_sync_soon_threadsafe=loop.call_soon_threadsafe, done_callback=trio_done_callback, ) fute_err: BaseException|None = None + try: out: Outcome = await asyncio.shield(trio_done_fute) From 4a8e1f56aecd7d3cc1ad74aaa47ac6c704eec051 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 18 Dec 2024 12:30:17 -0500 Subject: [PATCH 38/66] Support passing pre-conf-ed `Logger` Such that we can hook into 3rd-party-libs more easily to monkey them and use our (prettier/hipper) console logging with something like (an example from the client project `modden`), ```python connection_mod = i3ipc.connection tractor_style_i3ipc_logger: logging.LoggingAdapter = tractor.log.get_console_log( _root_name=connection_mod.__name__, logger=i3ipc.connection_mod.logger, level='info', ) # monkey the instance-ref in 3rd-party module connection_mod.logger = our_logger ``` Impl deats, - expose as `get_console_log(logger: logging.Logger)` and add default failover logic. - toss in more typing, also for mod-global instance. --- tractor/log.py | 53 +++++++++++++++++++++++++++++++++++++------------- 1 file changed, 39 insertions(+), 14 deletions(-) diff --git a/tractor/log.py b/tractor/log.py index 47f1f259..74e0321b 100644 --- a/tractor/log.py +++ b/tractor/log.py @@ -258,20 +258,28 @@ class ActorContextInfo(Mapping): def get_logger( - - name: str | None = None, + name: str|None = None, _root_name: str = _proj_name, + logger: Logger|None = None, + + # TODO, using `.config.dictConfig()` api? + # -[ ] SO answer with docs links + # |_https://stackoverflow.com/questions/7507825/where-is-a-complete-example-of-logging-config-dictconfig + # |_https://docs.python.org/3/library/logging.config.html#configuration-dictionary-schema + subsys_spec: str|None = None, + ) -> StackLevelAdapter: '''Return the package log or a sub-logger for ``name`` if provided. ''' log: Logger - log = rlog = logging.getLogger(_root_name) + log = rlog = logger or logging.getLogger(_root_name) if ( name - and name != _proj_name + and + name != _proj_name ): # NOTE: for handling for modules that use ``get_logger(__name__)`` @@ -283,7 +291,7 @@ def get_logger( # since in python the {filename} is always this same # module-file. - sub_name: None | str = None + sub_name: None|str = None rname, _, sub_name = name.partition('.') pkgpath, _, modfilename = sub_name.rpartition('.') @@ -306,7 +314,10 @@ def get_logger( # add our actor-task aware adapter which will dynamically look up # the actor and task names at each log emit - logger = StackLevelAdapter(log, ActorContextInfo()) + logger = StackLevelAdapter( + log, + ActorContextInfo(), + ) # additional levels for name, val in CUSTOM_LEVELS.items(): @@ -319,15 +330,25 @@ def get_logger( def get_console_log( - level: str | None = None, + level: str|None = None, + logger: Logger|None = None, **kwargs, -) -> LoggerAdapter: - '''Get the package logger and enable a handler which writes to stderr. - Yeah yeah, i know we can use ``DictConfig``. You do it. +) -> LoggerAdapter: ''' - log = get_logger(**kwargs) # our root logger - logger = log.logger + Get a `tractor`-style logging instance: a `Logger` wrapped in + a `StackLevelAdapter` which injects various concurrency-primitive + (process, thread, task) fields and enables a `StreamHandler` that + writes on stderr using `colorlog` formatting. + + Yeah yeah, i know we can use `logging.config.dictConfig()`. You do it. + + ''' + log = get_logger( + logger=logger, + **kwargs + ) # set a root logger + logger: Logger = log.logger if not level: return log @@ -346,9 +367,13 @@ def get_console_log( None, ) ): + fmt = LOG_FORMAT + # if logger: + # fmt = None + handler = StreamHandler() formatter = colorlog.ColoredFormatter( - LOG_FORMAT, + fmt=fmt, datefmt=DATE_FORMAT, log_colors=STD_PALETTE, secondary_log_colors=BOLD_PALETTE, @@ -365,7 +390,7 @@ def get_loglevel() -> str: # global module logger for tractor itself -log = get_logger('tractor') +log: StackLevelAdapter = get_logger('tractor') def at_least_level( From b71d96fdeea970baed7bc870e300d37572552523 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 27 Dec 2024 14:07:50 -0500 Subject: [PATCH 39/66] Accept err-type override in `is_multi_cancelled()` Such that equivalents of `trio.Cancelled` from other runtimes such as `asyncio.CancelledError` and `subprocess.CalledProcessError` (with a `.returncode == -2`) can be gracefully ignored as needed by the caller. For example this is handy if you want to avoid debug-mode REPL entry on an exception-group full of only some subset of exception types since you expect certain tasks to raise such errors after having been cancelled by a request from some parent supervision sys (some "higher up" `trio.CancelScope`, a remote triggered `ContextCancelled` or just from and OS SIGINT). Impl deats, - offer a new `ignore_nested: set[BaseException]` param which by default we add `trio.Cancelled` to when no other types are provided. - use `ExceptionGroup.subgroup(tuple(ignore_nested)` to filter to egs of the "ignored sub-errors set" and return any such match (instead of `True`). - detail a comment on exclusion case. --- tractor/_exceptions.py | 48 +++++++++++++++++++++++++++++++++++------- 1 file changed, 40 insertions(+), 8 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 812664ac..89ea21ad 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -1146,19 +1146,51 @@ def unpack_error( def is_multi_cancelled( - exc: BaseException|BaseExceptionGroup -) -> bool: + exc: BaseException|BaseExceptionGroup, + + ignore_nested: set[BaseException] = set(), + +) -> bool|BaseExceptionGroup: ''' - Predicate to determine if a possible ``BaseExceptionGroup`` contains - only ``trio.Cancelled`` sub-exceptions (and is likely the result of - cancelling a collection of subtasks. + Predicate to determine if an `BaseExceptionGroup` only contains + some (maybe nested) set of sub-grouped exceptions (like only + `trio.Cancelled`s which get swallowed silently by default) and is + thus the result of "gracefully cancelling" a collection of + sub-tasks (or other conc primitives) and receiving a "cancelled + ACK" from each after termination. + + Docs: + ---- + - https://docs.python.org/3/library/exceptions.html#exception-groups + - https://docs.python.org/3/library/exceptions.html#BaseExceptionGroup.subgroup ''' + + if ( + not ignore_nested + or + trio.Cancelled in ignore_nested + # XXX always count-in `trio`'s native signal + ): + ignore_nested |= {trio.Cancelled} + if isinstance(exc, BaseExceptionGroup): - return exc.subgroup( - lambda exc: isinstance(exc, trio.Cancelled) - ) is not None + matched_exc: BaseExceptionGroup|None = exc.subgroup( + tuple(ignore_nested), + # TODO, complain about why not allowed XD + # condition=tuple(ignore_nested), + ) + if matched_exc is not None: + return matched_exc + + # NOTE, IFF no excs types match (throughout the error-tree) + # -> return `False`, OW return the matched sub-eg. + # + # IOW, for the inverse of ^ for the purpose of + # maybe-enter-REPL--logic: "only debug when the err-tree contains + # at least one exc-type NOT in `ignore_nested`" ; i.e. the case where + # we fallthrough and return `False` here. return False From 79f4197d26d632efb101372e82127ac5f73ede96 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 28 Dec 2024 14:07:01 -0500 Subject: [PATCH 40/66] Raise explicitly on missing `greenback` portal When `.pause_from_sync()` is called from an `asyncio.Task` which was never bestowed a portal we want to be mega pedantic about it; indicate that the task was NOT spawned from our `.to_asyncio` API and likely by some out-of-our-control code (normally using `asyncio.ensure_future()/.create_task()`). Though `greenback` already errors on such usage, it's not always clear why no portal exists; explaining the situation of a 3rd-party-bg-spawned-task should avoid dev confusion for most cases. Impl deats, - distinguish between an actor in infected mode versus the actual caller of `.pause_from_sync()` being an `asyncio.Task` with more explicit `asyncio_task` and `is_infected_aio` vars. - ONLY in the case of being both an infected-mode-actor AND detecting that the caller is an `asyncio.Task`, check `greenback.has_portal()` such that when not bestowed we presume the aforementioned 3rd-party-bg-task case above and raise a new explicit RTE with a detailed explanatory message. - add some masked draft code for handling the speical case of a root actor `asyncio.Task` caller which could (in theory) not actually require gb portal use since the `Lock` can be acquired directly without IPC. |_this will likely require factoring of various pause machinery funcs into a `_pause_from_root_task()` to mk the impl sane XD Other, - expose a new `debug_filter: Callable` which can be provided by the caller of `_maybe_enter_pm()` to predicate whether to enter the debugger REPL based on the caught `BaseException|BaseExceptionGroup`; this is handy for customizing the meaning of "graceful cancellations" so as to avoid crash handling on expected egs of more then `trioCancelled`. |_ make the default as it was implemented: `not is_multi_cancelled(err)` - pass-through a new `ignore: set[BaseException]` as `open_crash_handler(ignore_nested=ignore)` to allow for the same silent-cancellation-egs-swallowing as desired from outside the actor runtime. --- tractor/devx/_debug.py | 166 ++++++++++++++++++++++++++++------------- 1 file changed, 113 insertions(+), 53 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 2f010b15..7c178ab1 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -75,6 +75,7 @@ from tractor import _state from tractor._exceptions import ( InternalError, NoRuntime, + is_multi_cancelled, ) from tractor._state import ( current_actor, @@ -1743,7 +1744,7 @@ async def _pause( ] = trio.TASK_STATUS_IGNORED, **debug_func_kwargs, -) -> tuple[PdbREPL, Task]|None: +) -> tuple[Task, PdbREPL]|None: ''' Inner impl for `pause()` to avoid the `trio.CancelScope.__exit__()` stack frame when not shielded (since apparently i can't figure out @@ -1929,7 +1930,7 @@ async def _pause( ) with trio.CancelScope(shield=shield): await trio.lowlevel.checkpoint() - return repl, task + return (repl, task) # elif repl_task: # log.warning( @@ -2530,26 +2531,17 @@ def pause_from_sync( f'{actor.uid} task called `tractor.pause_from_sync()`\n' ) - # TODO: once supported, remove this AND the one - # inside `._pause()`! - # outstanding impl fixes: - # -[ ] need to make `.shield_sigint()` below work here! - # -[ ] how to handle `asyncio`'s new SIGINT-handler - # injection? - # -[ ] should `breakpoint()` work and what does it normally - # do in `asyncio` ctxs? - # if actor.is_infected_aio(): - # raise RuntimeError( - # '`tractor.pause[_from_sync]()` not yet supported ' - # 'for infected `asyncio` mode!' - # ) - repl: PdbREPL = mk_pdb() # message += f'-> created local REPL {repl}\n' is_trio_thread: bool = DebugStatus.is_main_trio_thread() is_root: bool = is_root_process() - is_aio: bool = actor.is_infected_aio() + is_infected_aio: bool = actor.is_infected_aio() + thread: Thread = threading.current_thread() + + asyncio_task: asyncio.Task|None = None + if is_infected_aio: + asyncio_task = asyncio.current_task() # TODO: we could also check for a non-`.to_thread` context # using `trio.from_thread.check_cancelled()` (says @@ -2565,24 +2557,18 @@ def pause_from_sync( if ( not is_trio_thread and - not is_aio # see below for this usage + not asyncio_task ): # TODO: `threading.Lock()` this so we don't get races in # multi-thr cases where they're acquiring/releasing the # REPL and setting request/`Lock` state, etc.. - thread: threading.Thread = threading.current_thread() - repl_owner = thread + repl_owner: Thread = thread # TODO: make root-actor bg thread usage work! - if ( - is_root - # or - # is_aio - ): - if is_root: - message += ( - f'-> called from a root-actor bg {thread}\n' - ) + if is_root: + message += ( + f'-> called from a root-actor bg {thread}\n' + ) message += ( '-> scheduling `._pause_from_bg_root_thread()`..\n' @@ -2637,34 +2623,95 @@ def pause_from_sync( DebugStatus.shield_sigint() assert bg_task is not DebugStatus.repl_task + # TODO: once supported, remove this AND the one + # inside `._pause()`! + # outstanding impl fixes: + # -[ ] need to make `.shield_sigint()` below work here! + # -[ ] how to handle `asyncio`'s new SIGINT-handler + # injection? + # -[ ] should `breakpoint()` work and what does it normally + # do in `asyncio` ctxs? + # if actor.is_infected_aio(): + # raise RuntimeError( + # '`tractor.pause[_from_sync]()` not yet supported ' + # 'for infected `asyncio` mode!' + # ) elif ( not is_trio_thread and - is_aio + is_infected_aio # as in, the special actor-runtime mode + # ^NOTE XXX, that doesn't mean the caller is necessarily + # an `asyncio.Task` just that `trio` has been embedded on + # the `asyncio` event loop! + and + asyncio_task # transitive caller is an actual `asyncio.Task` ): greenback: ModuleType = maybe_import_greenback() - repl_owner: Task = asyncio.current_task() - DebugStatus.shield_sigint() - fute: asyncio.Future = run_trio_task_in_future( - partial( - _pause, - debug_func=None, - repl=repl, - hide_tb=hide_tb, - # XXX to prevent `._pause()` for setting - # `DebugStatus.repl_task` to the gb task! - called_from_sync=True, - called_from_bg_thread=True, + if greenback.has_portal(): + DebugStatus.shield_sigint() + fute: asyncio.Future = run_trio_task_in_future( + partial( + _pause, + debug_func=None, + repl=repl, + hide_tb=hide_tb, - **_pause_kwargs + # XXX to prevent `._pause()` for setting + # `DebugStatus.repl_task` to the gb task! + called_from_sync=True, + called_from_bg_thread=True, + + **_pause_kwargs + ) ) - ) + repl_owner = asyncio_task + bg_task, _ = greenback.await_(fute) + # TODO: ASYNC version -> `.pause_from_aio()`? + # bg_task, _ = await fute - # TODO: for async version -> `.pause_from_aio()`? - # bg_task, _ = await fute - bg_task, _ = greenback.await_(fute) - bg_task: asyncio.Task = asyncio.current_task() + # handle the case where an `asyncio` task has been + # spawned WITHOUT enabling a `greenback` portal.. + # => can often happen in 3rd party libs. + else: + bg_task = repl_owner + + # TODO, ostensibly we can just acquire the + # debug lock directly presuming we're the + # root actor running in infected asyncio + # mode? + # + # TODO, this would be a special case where + # a `_pause_from_root()` would come in very + # handy! + # if is_root: + # import pdbp; pdbp.set_trace() + # log.warning( + # 'Allowing `asyncio` task to acquire debug-lock in root-actor..\n' + # 'This is not fully implemented yet; there may be teardown hangs!\n\n' + # ) + # else: + + # simply unsupported, since there exists no hack (i + # can think of) to workaround this in a subactor + # which needs to lock the root's REPL ow we're sure + # to get prompt stdstreams clobbering.. + cf_repr: str = '' + if api_frame: + caller_frame: FrameType = api_frame.f_back + cf_repr: str = f'caller_frame: {caller_frame!r}\n' + + raise RuntimeError( + f"CAN'T USE `greenback._await()` without a portal !?\n\n" + f'Likely this task was NOT spawned via the `tractor.to_asyncio` API..\n' + f'{asyncio_task}\n' + f'{cf_repr}\n' + + f'Prolly the task was started out-of-band (from some lib?)\n' + f'AND one of the below was never called ??\n' + f'- greenback.ensure_portal()\n' + f'- greenback.bestow_portal()\n' + ) else: # we are presumably the `trio.run()` + main thread # raises on not-found by default @@ -2915,8 +2962,14 @@ async def _maybe_enter_pm( tb: TracebackType|None = None, api_frame: FrameType|None = None, hide_tb: bool = False, + + # only enter debugger REPL when returns `True` + debug_filter: Callable[ + [BaseException|BaseExceptionGroup], + bool, + ] = lambda err: not is_multi_cancelled(err), + ): - from tractor._exceptions import is_multi_cancelled if ( debug_mode() @@ -2933,7 +2986,8 @@ async def _maybe_enter_pm( # Really we just want to mostly avoid catching KBIs here so there # might be a simpler check we can do? - and not is_multi_cancelled(err) + and + debug_filter(err) ): api_frame: FrameType = api_frame or inspect.currentframe() tb: TracebackType = tb or sys.exc_info()[2] @@ -3139,10 +3193,16 @@ def open_crash_handler( try: yield except tuple(catch) as err: - if type(err) not in ignore: - - # use our re-impl-ed version + if ( + type(err) not in ignore + and + not is_multi_cancelled( + err, + ignore_nested=ignore + ) + ): try: + # use our re-impl-ed version _post_mortem( repl=mk_pdb(), tb=sys.exc_info()[2], From 2fd9c0044b5e84e41364f1d32511db2712115c47 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 28 Dec 2024 14:34:24 -0500 Subject: [PATCH 41/66] Drop extra nl from boxed error fmt --- tractor/devx/pformat.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tractor/devx/pformat.py b/tractor/devx/pformat.py index d24eaaf4..1530ef02 100644 --- a/tractor/devx/pformat.py +++ b/tractor/devx/pformat.py @@ -92,7 +92,7 @@ def pformat_boxed_tb( f' ------ {boxer_header} ------\n' f'{tb_body}' f' ------ {boxer_header}- ------\n' - f'_|\n' + f'_|' ) tb_box_indent: str = ( tb_box_indent From 8f0ca44b79ef59c73da68464f0c26faaaee49bcf Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 28 Dec 2024 14:35:05 -0500 Subject: [PATCH 42/66] Expose `debug_filter` from `open_root_actor()` also Such that actor-runtime graceful cancel handling can be used throughout any process tree. --- tractor/_root.py | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/tractor/_root.py b/tractor/_root.py index 38ddbe23..e10b02ef 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -95,6 +95,13 @@ async def open_root_actor( hide_tb: bool = True, + # XXX, proxied directly to `.devx._debug._maybe_enter_pm()` + # for REPL-entry logic. + debug_filter: Callable[ + [BaseException|BaseExceptionGroup], + bool, + ] = lambda err: not is_multi_cancelled(err), + # TODO, a way for actors to augment passing derived # read-only state to sublayers? # extra_rt_vars: dict|None = None, @@ -379,6 +386,7 @@ async def open_root_actor( Exception, BaseExceptionGroup, ) as err: + # XXX NOTE XXX see equiv note inside # `._runtime.Actor._stream_handler()` where in the # non-root or root-that-opened-this-mahually case we @@ -387,11 +395,15 @@ async def open_root_actor( entered: bool = await _debug._maybe_enter_pm( err, api_frame=inspect.currentframe(), + debug_filter=debug_filter, ) + if ( not entered and - not is_multi_cancelled(err) + not is_multi_cancelled( + err, + ) ): logger.exception('Root actor crashed\n') From c5291b7f33c78a5f6853e18e74498d674badf397 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 31 Dec 2024 18:10:09 -0500 Subject: [PATCH 43/66] Much more limited `asyncio.Task.cancel()` use Since it can not only cause the guest-mode run to abandon but also in some edge cases prevent `trio`-errors from propagating (at least on py3.12-13?) as discovered as part of supporting this mode officially in the *root actor*. As such try to avoid that method as much as possible instead opting to pass the `trio`-side error via the iter-task channel ref. Deats, - add a `LinkedTaskChannel._trio_err: BaseException|None` which gets set whenver the `trio.Task` error is caught; ONLY set `AsyncioCancelled` when the `trio` task was for sure the cause, whether itself cancelled or errored. - always check for this error when exiting the `asyncio` side (even when terminated via a call to `asyncio.Task.cancel()` or during any other `CancelledError` handling such that the `asyncio`-task can expect to handle `AsyncioCancelled` due to the above^^ cases. - never `cs.cancel()` the `trio` side unless that cancel scope has not yet been `.cancel_called` whatsoever; it's a noop anyway. - only raise any exc from `asyncio.Task.result()` when `chan._aio_err` does not already match it since the existence of the pre-existing `task_err` means `asyncio` prolly intends (or has already) raised and interrupted the task elsewhere. Various supporting tweaks, - don't bother maybe-init-ing `greenback` from the actor entrypoint since we already need to (and do) bestow the portals to each `asyncio` task spawned using the `run_task()`/`open_channel_from()` API; further the init-ing should be done already by client code that enables infected mode (even in the root actor). |_we should prolly also codify it from any `run_daemon(infected_aio=True, debug_mode=True)` usage we offer. - pass all the `_`s to `Linked TaskChannel` explicitly in named kwarg style. - better sclang-style log reports throughout, particularly on teardowns. - generally more/better comments and docs around (not well understood) edge cases. - prep to just inline `maybe_raise_aio_side_err()` closure.. --- tractor/to_asyncio.py | 471 +++++++++++++++++++++++++++++------------- 1 file changed, 331 insertions(+), 140 deletions(-) diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index 24f1ace6..d5f78ca8 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -33,13 +33,19 @@ from typing import ( ) import tractor -from tractor._exceptions import AsyncioCancelled +from tractor._exceptions import ( + AsyncioCancelled, + is_multi_cancelled, +) from tractor._state import ( debug_mode, _runtime_vars, ) from tractor.devx import _debug -from tractor.log import get_logger +from tractor.log import ( + get_logger, + StackLevelAdapter, +) from tractor.trionics._broadcast import ( broadcast_receiver, BroadcastReceiver, @@ -50,7 +56,7 @@ from outcome import ( Outcome, ) -log = get_logger(__name__) +log: StackLevelAdapter = get_logger(__name__) __all__ = [ @@ -70,9 +76,10 @@ class LinkedTaskChannel(trio.abc.Channel): _to_aio: asyncio.Queue _from_aio: trio.MemoryReceiveChannel _to_trio: trio.MemorySendChannel - _trio_cs: trio.CancelScope _aio_task_complete: trio.Event + + _trio_err: BaseException|None = None _trio_exited: bool = False # set after ``asyncio.create_task()`` @@ -84,28 +91,40 @@ class LinkedTaskChannel(trio.abc.Channel): await self._from_aio.aclose() async def receive(self) -> Any: - async with translate_aio_errors( - self, - - # XXX: obviously this will deadlock if an on-going stream is - # being procesed. - # wait_on_aio_task=False, - ): + ''' + Receive a value from the paired `asyncio.Task` with + exception/cancel handling to teardown both sides on any + unexpected error. + ''' + try: # TODO: do we need this to guarantee asyncio code get's # cancelled in the case where the trio side somehow creates # a state where the asyncio cycle-task isn't getting the # cancel request sent by (in theory) the last checkpoint # cycle on the trio side? # await trio.lowlevel.checkpoint() - return await self._from_aio.receive() + except BaseException as err: + async with translate_aio_errors( + self, + + # XXX: obviously this will deadlock if an on-going stream is + # being procesed. + # wait_on_aio_task=False, + ): + raise err async def wait_asyncio_complete(self) -> None: await self._aio_task_complete.wait() - # def cancel_asyncio_task(self) -> None: - # self._aio_task.cancel() + def cancel_asyncio_task( + self, + msg: str = '', + ) -> None: + self._aio_task.cancel( + msg=msg, + ) async def send(self, item: Any) -> None: ''' @@ -155,7 +174,6 @@ class LinkedTaskChannel(trio.abc.Channel): def _run_asyncio_task( - func: Callable, *, qsize: int = 1, @@ -165,8 +183,9 @@ def _run_asyncio_task( ) -> LinkedTaskChannel: ''' - Run an ``asyncio`` async function or generator in a task, return - or stream the result back to the caller `trio.lowleve.Task`. + Run an `asyncio`-compat async function or generator in a task, + return or stream the result back to the caller + `trio.lowleve.Task`. ''' __tracebackhide__: bool = hide_tb @@ -204,23 +223,23 @@ def _run_asyncio_task( aio_err: BaseException|None = None chan = LinkedTaskChannel( - aio_q, # asyncio.Queue - from_aio, # recv chan - to_trio, # send chan - - cancel_scope, - aio_task_complete, + _to_aio=aio_q, # asyncio.Queue + _from_aio=from_aio, # recv chan + _to_trio=to_trio, # send chan + _trio_cs=cancel_scope, + _aio_task_complete=aio_task_complete, ) async def wait_on_coro_final_result( - to_trio: trio.MemorySendChannel, coro: Awaitable, aio_task_complete: trio.Event, ) -> None: ''' - Await ``coro`` and relay result back to ``trio``. + Await `coro` and relay result back to `trio`. + + This can only be run as an `asyncio.Task`! ''' nonlocal aio_err @@ -243,8 +262,10 @@ def _run_asyncio_task( else: if ( - result != orig and - aio_err is None and + result != orig + and + aio_err is None + and # in the `open_channel_from()` case we don't # relay through the "return value". @@ -260,12 +281,21 @@ def _run_asyncio_task( # a ``trio.EndOfChannel`` to the trio (consumer) side. to_trio.close() + # import pdbp; pdbp.set_trace() aio_task_complete.set() - log.runtime(f'`asyncio` task: {task.get_name()} is complete') + # await asyncio.sleep(0.1) + log.info( + f'`asyncio` task terminated\n' + f'x)>\n' + f' |_{task}\n' + ) # start the asyncio task we submitted from trio if not inspect.isawaitable(coro): - raise TypeError(f"No support for invoking {coro}") + raise TypeError( + f'Pass the async-fn NOT a coroutine\n' + f'{coro!r}' + ) task: asyncio.Task = asyncio.create_task( wait_on_coro_final_result( @@ -289,6 +319,10 @@ def _run_asyncio_task( raise_not_found=False, )) ): + log.info( + f'Bestowing `greenback` portal for `asyncio`-task\n' + f'{task}\n' + ) greenback.bestow_portal(task) def cancel_trio(task: asyncio.Task) -> None: @@ -304,11 +338,22 @@ def _run_asyncio_task( # task exceptions try: res: Any = task.result() + log.info( + '`trio` received final result from {task}\n' + f'|_{res}\n' + ) except BaseException as terr: task_err: BaseException = terr + # read again AFTER the `asyncio` side errors in case + # it was cancelled due to an error from `trio` (or + # some other out of band exc). + aio_err: BaseException|None = chan._aio_err + msg: str = ( - 'Infected `asyncio` task {etype_str}\n' + '`trio`-side reports that the `asyncio`-side ' + '{etype_str}\n' + # ^NOTE filled in below ) if isinstance(terr, CancelledError): msg += ( @@ -327,17 +372,18 @@ def _run_asyncio_task( msg.format(etype_str='errored') ) - assert type(terr) is type(aio_err), ( - '`asyncio` task error mismatch?!?' - ) + assert ( + type(terr) is type(aio_err) + ), '`asyncio` task error mismatch?!?' if aio_err is not None: + # import pdbp; pdbp.set_trace() # XXX: uhh is this true? # assert task_err, f'Asyncio task {task.get_name()} discrepancy!?' # NOTE: currently mem chan closure may act as a form - # of error relay (at least in the ``asyncio.CancelledError`` - # case) since we have no way to directly trigger a ``trio`` + # of error relay (at least in the `asyncio.CancelledError` + # case) since we have no way to directly trigger a `trio` # task error without creating a nursery to throw one. # We might want to change this in the future though. from_aio.close() @@ -359,29 +405,25 @@ def _run_asyncio_task( # ) # raise aio_err from task_err - # XXX: if not already, alway cancel the scope - # on a task error in case the trio task is blocking on + # XXX: if not already, alway cancel the scope on a task + # error in case the trio task is blocking on # a checkpoint. - cancel_scope.cancel() - if ( - task_err - and - aio_err is not task_err + not cancel_scope.cancelled_caught + or + not cancel_scope.cancel_called ): - raise aio_err from task_err + # import pdbp; pdbp.set_trace() + cancel_scope.cancel() - # raise any `asyncio` side error. - raise aio_err - - log.info( - '`trio` received final result from {task}\n' - f'|_{res}\n' - ) - # TODO: do we need this? - # if task_err: - # cancel_scope.cancel() - # raise task_err + if task_err: + # XXX raise any `asyncio` side error IFF it doesn't + # match the one we just caught from the task above! + # (that would indicate something weird/very-wrong + # going on?) + if aio_err is not task_err: + # import pdbp; pdbp.set_trace() + raise aio_err from task_err task.add_done_callback(cancel_trio) return chan @@ -389,13 +431,18 @@ def _run_asyncio_task( @acm async def translate_aio_errors( - chan: LinkedTaskChannel, wait_on_aio_task: bool = False, + cancel_aio_task_on_trio_exit: bool = True, ) -> AsyncIterator[None]: ''' - Error handling context around ``asyncio`` task spawns which + An error handling to cross-loop propagation context around + `asyncio.Task` spawns via one of this module's APIs: + + - `open_channel_from()` + - `run_task()` + appropriately translates errors and cancels into ``trio`` land. ''' @@ -403,88 +450,204 @@ async def translate_aio_errors( aio_err: BaseException|None = None - # TODO: make thisi a channel method? - def maybe_raise_aio_err( - err: Exception|None = None - ) -> None: - aio_err = chan._aio_err - if ( - aio_err is not None - and - # not isinstance(aio_err, CancelledError) - type(aio_err) != CancelledError - ): - # always raise from any captured asyncio error - if err: - raise aio_err from err - else: - raise aio_err - - task = chan._aio_task - assert task + aio_task: asyncio.Task = chan._aio_task + assert aio_task + trio_err: BaseException|None = None try: - yield - + yield # back to one of the cross-loop apis except ( trio.Cancelled, - ): - # relay cancel through to called ``asyncio`` task + ) as _trio_err: + trio_err = _trio_err assert chan._aio_task - chan._aio_task.cancel( - msg=f'the `trio` caller task was cancelled: {trio_task.name}' + + # import pdbp; pdbp.set_trace() # lolevel-debug + + # relay cancel through to called ``asyncio`` task + chan._aio_err = AsyncioCancelled( + f'trio`-side cancelled the `asyncio`-side,\n' + f'c)>\n' + f' |_{trio_task}\n\n' + + + f'{trio_err!r}\n' ) - raise + + # XXX NOTE XXX seems like we can get all sorts of unreliable + # behaviour from `asyncio` under various cancellation + # conditions (like SIGINT/kbi) when this is used.. + # SO FOR NOW, try to avoid it at most costs! + # + # aio_task.cancel( + # msg=f'the `trio` parent task was cancelled: {trio_task.name}' + # ) + # raise except ( - # NOTE: see the note in the ``cancel_trio()`` asyncio task + # NOTE: also see note in the `cancel_trio()` asyncio task # termination callback trio.ClosedResourceError, # trio.BrokenResourceError, - ): + + ) as _trio_err: + trio_err = _trio_err aio_err = chan._aio_err + # import pdbp; pdbp.set_trace() + + # XXX if an underlying `asyncio.CancelledError` triggered + # this channel close, raise our (non-`BaseException`) wrapper + # exception (`AsyncioCancelled`) from that source error. if ( - task.cancelled() + # NOTE, not until it terminates? + aio_task.cancelled() and type(aio_err) is CancelledError ): - # if an underlying `asyncio.CancelledError` triggered this - # channel close, raise our (non-``BaseException``) wrapper - # error: ``AsyncioCancelled`` from that source error. raise AsyncioCancelled( - f'Task cancelled\n' - f'|_{task}\n' + f'asyncio`-side cancelled the `trio`-side,\n' + f'c(>\n' + f' |_{aio_task}\n\n' + + f'{trio_err!r}\n' ) from aio_err else: raise - finally: + except BaseException as _trio_err: + trio_err = _trio_err + log.exception( + '`trio`-side task errored?' + ) + + entered: bool = await _debug._maybe_enter_pm( + trio_err, + api_frame=inspect.currentframe(), + ) if ( - # NOTE: always cancel the ``asyncio`` task if we've made it - # this far and it's not done. - not task.done() and aio_err + not entered + and + not is_multi_cancelled(trio_err) + ): + log.exception('actor crashed\n') + + aio_taskc = AsyncioCancelled( + f'`trio`-side task errored!\n' + f'{trio_err}' + ) #from trio_err + + try: + aio_task.set_exception(aio_taskc) + except ( + asyncio.InvalidStateError, + RuntimeError, + # ^XXX, uhh bc apparently we can't use `.set_exception()` + # any more XD .. ?? + ): + wait_on_aio_task = False + + # import pdbp; pdbp.set_trace() + # raise aio_taskc from trio_err + + finally: + # record wtv `trio`-side error transpired + chan._trio_err = trio_err + + # NOTE! by default always cancel the `asyncio` task if + # we've made it this far and it's not done. + # TODO, how to detect if there's an out-of-band error that + # caused the exit? + if ( + cancel_aio_task_on_trio_exit + and + not aio_task.done() + and + aio_err # or the trio side has exited it's surrounding cancel scope # indicating the lifetime of the ``asyncio``-side task # should also be terminated. - or chan._trio_exited - ): - log.runtime( - f'Cancelling `asyncio`-task: {task.get_name()}' + or ( + chan._trio_exited + and + not chan._trio_err # XXX CRITICAL, `asyncio.Task.cancel()` is cucked man.. ) - # assert not aio_err, 'WTF how did asyncio do this?!' - task.cancel() + ): + # pass + msg: str = ( + f'MANUALLY Cancelling `asyncio`-task: {aio_task.get_name()}!\n\n' + f'**THIS CAN SILENTLY SUPPRESS ERRORS FYI\n\n' - # Required to sync with the far end ``asyncio``-task to ensure + f'trio-side exited silently!' + ) + # TODO XXX, figure out the case where calling this makes the + # `test_infected_asyncio.py::test_trio_closes_early_and_channel_exits` + # hang and then don't call it in that case! + # + aio_task.cancel(msg=msg) + log.warning(msg) + # assert not aio_err, 'WTF how did asyncio do this?!' + # import pdbp; pdbp.set_trace() + + # Required to sync with the far end `asyncio`-task to ensure # any error is captured (via monkeypatching the - # ``channel._aio_err``) before calling ``maybe_raise_aio_err()`` + # `channel._aio_err`) before calling ``maybe_raise_aio_err()`` # below! + # + # XXX NOTE XXX the `task.set_exception(aio_taskc)` call above + # MUST NOT EXCEPT or this WILL HANG!! + # + # so if you get a hang maybe step through and figure out why + # it erroed out up there! + # if wait_on_aio_task: + # await chan.wait_asyncio_complete() await chan._aio_task_complete.wait() + log.info( + 'asyncio-task is done and unblocked trio-side!\n' + ) + + # TODO? + # -[ ] make this a channel method, OR + # -[ ] just put back inline below? + # + def maybe_raise_aio_side_err( + trio_err: Exception, + ) -> None: + ''' + Raise any `trio`-side-caused cancellation or legit task + error normally propagated from the caller of either, + - `open_channel_from()` + - `run_task()` + + ''' + aio_err: BaseException|None = chan._aio_err + + # Check if the asyncio-side is the cause of the trio-side + # error. + if ( + aio_err is not None + and + type(aio_err) is not AsyncioCancelled + + # not isinstance(aio_err, CancelledError) + # type(aio_err) is not CancelledError + ): + # always raise from any captured asyncio error + if trio_err: + raise trio_err from aio_err + + raise aio_err + + if trio_err: + raise trio_err # NOTE: if any ``asyncio`` error was caught, raise it here inline # here in the ``trio`` task - maybe_raise_aio_err() + # if trio_err: + maybe_raise_aio_side_err( + trio_err=trio_err + ) async def run_task( @@ -496,8 +659,8 @@ async def run_task( ) -> Any: ''' - Run an `asyncio` async function or generator in a task, return - or stream the result back to `trio`. + Run an `asyncio`-compat async function or generator in a task, + return or stream the result back to `trio`. ''' # simple async func @@ -537,6 +700,7 @@ async def open_channel_from( provide_channels=True, **kwargs, ) + # TODO, tuple form here? async with chan._from_aio: async with translate_aio_errors( chan, @@ -685,18 +849,21 @@ def run_as_asyncio_guest( # Uh, oh. # # :o - - # It looks like your event loop has caught a case of the ``trio``s. - - # :() - - # Don't worry, we've heard you'll barely notice. You might - # hallucinate a few more propagating errors and feel like your - # digestion has slowed but if anything get's too bad your parents - # will know about it. - + # + # looks like your stdlib event loop has caught a case of "the trios" ! + # + # :O + # + # Don't worry, we've heard you'll barely notice. + # # :) - + # + # You might hallucinate a few more propagating errors and feel + # like your digestion has slowed, but if anything get's too bad + # your parents will know about it. + # + # B) + # async def aio_main(trio_main): ''' Main `asyncio.Task` which calls @@ -713,16 +880,20 @@ def run_as_asyncio_guest( '-> built a `trio`-done future\n' ) - # TODO: shoudn't this be done in the guest-run trio task? - # if debug_mode(): - # # XXX make it obvi we know this isn't supported yet! - # log.error( - # 'Attempting to enter unsupported `greenback` init ' - # 'from `asyncio` task..' - # ) - # await _debug.maybe_init_greenback( - # force_reload=True, - # ) + # TODO: is this evern run or needed? + # -[ ] pretty sure it never gets run for root-infected-aio + # since this main task is always the parent of any + # eventual `open_root_actor()` call? + if debug_mode(): + log.error( + 'Attempting to enter non-required `greenback` init ' + 'from `asyncio` task ???' + ) + # XXX make it obvi we know this isn't supported yet! + assert 0 + # await _debug.maybe_init_greenback( + # force_reload=True, + # ) def trio_done_callback(main_outcome): log.runtime( @@ -732,6 +903,7 @@ def run_as_asyncio_guest( ) if isinstance(main_outcome, Error): + # import pdbp; pdbp.set_trace() error: BaseException = main_outcome.error # show an dedicated `asyncio`-side tb from the error @@ -751,7 +923,7 @@ def run_as_asyncio_guest( trio_done_fute.set_result(main_outcome) log.info( - f'`trio` guest-run finished with outcome\n' + f'`trio` guest-run finished with,\n' f')>\n' f'|_{trio_done_fute}\n' ) @@ -777,9 +949,20 @@ def run_as_asyncio_guest( done_callback=trio_done_callback, ) fute_err: BaseException|None = None - try: out: Outcome = await asyncio.shield(trio_done_fute) + # ^TODO still don't really understand why the `.shield()` + # is required ... ?? + # https://docs.python.org/3/library/asyncio-task.html#asyncio.shield + # ^ seems as though in combo with the try/except here + # we're BOLDLY INGORING cancel of the trio fute? + # + # I guess it makes sense bc we don't want `asyncio` to + # cancel trio just because they can't handle SIGINT + # sanely? XD .. kk + + # XXX, sin-shield causes guest-run abandons on SIGINT.. + # out: Outcome = await trio_done_fute # NOTE will raise (via `Error.unwrap()`) from any # exception packed into the guest-run's `main_outcome`. @@ -802,27 +985,32 @@ def run_as_asyncio_guest( fute_err = _fute_err err_message: str = ( 'main `asyncio` task ' + 'was cancelled!\n' ) - if isinstance(fute_err, asyncio.CancelledError): - err_message += 'was cancelled!\n' - else: - err_message += f'errored with {out.error!r}\n' + # TODO, handle possible edge cases with + # `open_root_actor()` closing before this is run! + # actor: tractor.Actor = tractor.current_actor() + log.exception( err_message + 'Cancelling `trio`-side `tractor`-runtime..\n' - f'c)>\n' + f'c(>\n' f' |_{actor}.cancel_soon()\n' ) - # XXX WARNING XXX the next LOCs are super important, since - # without them, we can get guest-run abandonment cases - # where `asyncio` will not schedule or wait on the `trio` - # guest-run task before final shutdown! This is - # particularly true if the `trio` side has tasks doing - # shielded work when a SIGINT condition occurs. + # XXX WARNING XXX the next LOCs are super important! + # + # SINCE without them, we can get guest-run ABANDONMENT + # cases where `asyncio` will not schedule or wait on the + # guest-run `trio.Task` nor invoke its registered + # `trio_done_callback()` before final shutdown! + # + # This is particularly true if the `trio` side has tasks + # in shielded sections when an OC-cancel (SIGINT) + # condition occurs! # # We now have the # `test_infected_asyncio.test_sigint_closes_lifetime_stack()` @@ -886,7 +1074,10 @@ def run_as_asyncio_guest( try: return trio_done_fute.result() - except asyncio.exceptions.InvalidStateError as state_err: + except ( + asyncio.InvalidStateError, + # asyncio.CancelledError, + )as state_err: # XXX be super dupere noisy about abandonment issues! aio_task: asyncio.Task = asyncio.current_task() From d4f1a02f431103df12f2a608fe85e7a159d34f8d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 2 Jan 2025 15:35:36 -0500 Subject: [PATCH 44/66] Hm, `asyncio.Task._fut_waiter.set_exception()`? Since we can't use it to `Task.set_exception()` (since that task method never seems to work.. XD) and setting the private/internal always seems to do the desired raising in the task? I realize it's an internal `asyncio` runtime field but i'd rather take the risk of it breaking then having to rely on our own equivalent hack.. Also, it seems like the case where the task's associated (and internal) future-waiter field is null, we won't run into the (same?) prior hanging issues (maybe since there's nothing for `asyncio` internals to use to wait XD ??) when `Task.cancel()` is used..?? Main deats, - add and `Future.set_exception()` a new signal-exception `class TrioTaskExited(AsyncioCancelled):` whenever the trio-task exits gracefully and the asyncio-side task is still doing blocking work (of some sort) which *seem to* be predicated by a check that `._fut_waiter is not None`. - always call `asyncio.Queue.shutdown()` for the same^ as well as whenever we decide to call `Task.cancel()`; in that case the shutdown relays correctly? Some further refinements, - only warn about `Task.cancel()` usage when actually used ;) - more local scope vars setting in the exit phase of `translate_aio_errors()`. - also in ^ use explicit caught-exc var names for each error-type. --- tractor/to_asyncio.py | 130 +++++++++++++++++++++++++++++++----------- 1 file changed, 97 insertions(+), 33 deletions(-) diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index d5f78ca8..3f8d20d3 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -342,20 +342,29 @@ def _run_asyncio_task( '`trio` received final result from {task}\n' f'|_{res}\n' ) - except BaseException as terr: - task_err: BaseException = terr + except BaseException as _aio_err: + task_err: BaseException = _aio_err # read again AFTER the `asyncio` side errors in case # it was cancelled due to an error from `trio` (or # some other out of band exc). aio_err: BaseException|None = chan._aio_err + # always true right? + assert ( + type(_aio_err) is type(aio_err) + ), ( + f'`asyncio`-side task errors mismatch?!?\n\n' + f'caught: {_aio_err}\n' + f'chan._aio_err: {aio_err}\n' + ) + msg: str = ( '`trio`-side reports that the `asyncio`-side ' '{etype_str}\n' # ^NOTE filled in below ) - if isinstance(terr, CancelledError): + if isinstance(_aio_err, CancelledError): msg += ( f'c)>\n' f' |_{task}\n' @@ -372,9 +381,6 @@ def _run_asyncio_task( msg.format(etype_str='errored') ) - assert ( - type(terr) is type(aio_err) - ), '`asyncio` task error mismatch?!?' if aio_err is not None: # import pdbp; pdbp.set_trace() @@ -394,7 +400,7 @@ def _run_asyncio_task( # aio_err.with_traceback(aio_err.__traceback__) # TODO: show when cancellation originated - # from each side more pedantically? + # from each side more pedantically in log-msg? # elif ( # type(aio_err) is CancelledError # and # trio was the cause? @@ -429,6 +435,19 @@ def _run_asyncio_task( return chan +class TrioTaskExited(AsyncioCancelled): + ''' + The `trio`-side task exited without explicitly cancelling the + `asyncio.Task` peer. + + This is very similar to how `trio.ClosedResource` acts as + a "clean shutdown" signal to the consumer side of a mem-chan, + + https://trio.readthedocs.io/en/stable/reference-core.html#clean-shutdown-with-channels + + ''' + + @acm async def translate_aio_errors( chan: LinkedTaskChannel, @@ -455,10 +474,11 @@ async def translate_aio_errors( trio_err: BaseException|None = None try: yield # back to one of the cross-loop apis - except ( - trio.Cancelled, - ) as _trio_err: - trio_err = _trio_err + except trio.Cancelled as taskc: + trio_err = taskc + + # should NEVER be the case that `trio` is cancel-handling + # BEFORE the other side's task-ref was set!? assert chan._aio_task # import pdbp; pdbp.set_trace() # lolevel-debug @@ -483,14 +503,13 @@ async def translate_aio_errors( # ) # raise + # NOTE ALSO SEE the matching note in the `cancel_trio()` asyncio + # task-done-callback. except ( - # NOTE: also see note in the `cancel_trio()` asyncio task - # termination callback trio.ClosedResourceError, # trio.BrokenResourceError, - - ) as _trio_err: - trio_err = _trio_err + ) as cre: + trio_err = cre aio_err = chan._aio_err # import pdbp; pdbp.set_trace() @@ -498,10 +517,21 @@ async def translate_aio_errors( # this channel close, raise our (non-`BaseException`) wrapper # exception (`AsyncioCancelled`) from that source error. if ( - # NOTE, not until it terminates? - aio_task.cancelled() + # aio-side is cancelled? + aio_task.cancelled() # not set until it terminates?? and type(aio_err) is CancelledError + + # TODO, if we want suppression of the + # silent-exit-by-`trio` case? + # -[ ] the parent task can also just catch it though? + # -[ ] OR, offer a `signal_aio_side_on_exit=True` ?? + # + # or + # aio_err is None + # and + # chan._trio_exited + ): raise AsyncioCancelled( f'asyncio`-side cancelled the `trio`-side,\n' @@ -511,6 +541,7 @@ async def translate_aio_errors( f'{trio_err!r}\n' ) from aio_err + # maybe the chan-closure is due to something else? else: raise @@ -552,6 +583,7 @@ async def translate_aio_errors( finally: # record wtv `trio`-side error transpired chan._trio_err = trio_err + ya_trio_exited: bool = chan._trio_exited # NOTE! by default always cancel the `asyncio` task if # we've made it this far and it's not done. @@ -568,26 +600,56 @@ async def translate_aio_errors( # indicating the lifetime of the ``asyncio``-side task # should also be terminated. or ( - chan._trio_exited + ya_trio_exited and not chan._trio_err # XXX CRITICAL, `asyncio.Task.cancel()` is cucked man.. ) ): - # pass - msg: str = ( - f'MANUALLY Cancelling `asyncio`-task: {aio_task.get_name()}!\n\n' - f'**THIS CAN SILENTLY SUPPRESS ERRORS FYI\n\n' - - f'trio-side exited silently!' + report: str = ( + 'trio-side exited silently!' ) - # TODO XXX, figure out the case where calling this makes the - # `test_infected_asyncio.py::test_trio_closes_early_and_channel_exits` - # hang and then don't call it in that case! - # - aio_task.cancel(msg=msg) - log.warning(msg) - # assert not aio_err, 'WTF how did asyncio do this?!' - # import pdbp; pdbp.set_trace() + assert not aio_err, 'WTF how did asyncio do this?!' + + # if the `trio.Task` already exited the `open_channel_from()` + # block we ensure the asyncio-side gets signalled via an + # explicit exception and its `Queue` is shutdown. + if ya_trio_exited: + chan._to_aio.shutdown() + + # pump the other side's task? needed? + await trio.lowlevel.checkpoint() + + if ( + not chan._trio_err + and + (fut := aio_task._fut_waiter) + ): + fut.set_exception( + TrioTaskExited( + f'The peer `asyncio` task is still blocking/running?\n' + f'>>\n' + f'|_{aio_task!r}\n' + ) + ) + else: + # from tractor._state import is_root_process + # if is_root_process(): + # breakpoint() + # import pdbp; pdbp.set_trace() + + aio_taskc_warn: str = ( + f'\n' + f'MANUALLY Cancelling `asyncio`-task: {aio_task.get_name()}!\n\n' + f'**THIS CAN SILENTLY SUPPRESS ERRORS FYI\n\n' + ) + report += aio_taskc_warn + # TODO XXX, figure out the case where calling this makes the + # `test_infected_asyncio.py::test_trio_closes_early_and_channel_exits` + # hang and then don't call it in that case! + # + aio_task.cancel(msg=aio_taskc_warn) + + log.warning(report) # Required to sync with the far end `asyncio`-task to ensure # any error is captured (via monkeypatching the @@ -1077,6 +1139,8 @@ def run_as_asyncio_guest( except ( asyncio.InvalidStateError, # asyncio.CancelledError, + # ^^XXX `.shield()` call above prevents this?? + )as state_err: # XXX be super dupere noisy about abandonment issues! From 3c8b1aa8880bb4660700c099b247311e8d62122c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 9 Jan 2025 08:59:30 -0500 Subject: [PATCH 45/66] Add an inter-leaved-task error test Trying to replicate cases where errors are raised in both `trio` and `asyncio` tasks independently (at least in `.to_asyncio` API terms) with a new `test_trio_prestarted_task_bubbles` that generates 3 cases inside a `@acm` calls stack composing a `trio.Nursery` with a `to_asyncio.open_channel_from()` call where a set of `trio` tasks are started in a loop using `.start()` with various exc raising sequences, - the aio task raising *before* the last `trio` task spawns. - the aio task raising just after the last trio task spawns, but before it starts. - after the last trio task `.start()` call returns control to the parent - but (for now) did not error. TODO, still more cases to discover as i'm still fighting a `modden` bug of this sort atm.. Other, - tweak some other tests to have timeouts since some recent hangs were found.. - started mucking with py3.13 and thus adjustments for strict egs in some tests; full patchset to test suite likely coming soon! --- tests/test_infected_asyncio.py | 316 +++++++++++++++++++++++++++------ 1 file changed, 259 insertions(+), 57 deletions(-) diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index b0a11715..5d88920a 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -109,7 +109,9 @@ async def asyncio_actor( except BaseException as err: if expect_err: - assert isinstance(err, error_type) + assert isinstance(err, error_type), ( + f'{type(err)} is not {error_type}?' + ) raise @@ -181,8 +183,8 @@ def test_trio_cancels_aio(reg_addr): with trio.move_on_after(1): # cancel the nursery shortly after boot - async with tractor.open_nursery() as n: - await n.run_in_actor( + async with tractor.open_nursery() as tn: + await tn.run_in_actor( asyncio_actor, target='aio_sleep_forever', expect_err='trio.Cancelled', @@ -202,22 +204,33 @@ async def trio_ctx( # this will block until the ``asyncio`` task sends a "first" # message. with trio.fail_after(2): - async with ( - trio.open_nursery() as n, + try: + async with ( + trio.open_nursery( + # TODO, for new `trio` / py3.13 + # strict_exception_groups=False, + ) as tn, + tractor.to_asyncio.open_channel_from( + sleep_and_err, + ) as (first, chan), + ): - tractor.to_asyncio.open_channel_from( - sleep_and_err, - ) as (first, chan), - ): + assert first == 'start' - assert first == 'start' + # spawn another asyncio task for the cuck of it. + tn.start_soon( + tractor.to_asyncio.run_task, + aio_sleep_forever, + ) + await trio.sleep_forever() - # spawn another asyncio task for the cuck of it. - n.start_soon( - tractor.to_asyncio.run_task, - aio_sleep_forever, - ) - await trio.sleep_forever() + # TODO, factor this into a `trionics.callapse()`? + except* BaseException as beg: + # await tractor.pause(shield=True) + if len(excs := beg.exceptions) == 1: + raise excs[0] + else: + raise @pytest.mark.parametrize( @@ -236,7 +249,6 @@ def test_context_spawns_aio_task_that_errors( ''' async def main(): - with trio.fail_after(2): async with tractor.open_nursery() as n: p = await n.start_actor( @@ -308,7 +320,9 @@ async def aio_cancel(): await aio_sleep_forever() -def test_aio_cancelled_from_aio_causes_trio_cancelled(reg_addr): +def test_aio_cancelled_from_aio_causes_trio_cancelled( + reg_addr: tuple, +): ''' When the `asyncio.Task` cancels itself the `trio` side cshould also cancel and teardown and relay the cancellation cross-process @@ -405,6 +419,7 @@ async def stream_from_aio( sequence=seq, expect_cancel=raise_err or exit_early, fail_early=aio_raise_err, + ) as (first, chan): assert first is True @@ -423,10 +438,15 @@ async def stream_from_aio( if raise_err: raise Exception elif exit_early: + print('`consume()` breaking early!\n') break + print('returning from `consume()`..\n') + + # run 2 tasks each pulling from + # the inter-task-channel with the 2nd + # using a fan-out `BroadcastReceiver`. if fan_out: - # start second task that get's the same stream value set. async with ( # NOTE: this has to come first to avoid @@ -436,11 +456,19 @@ async def stream_from_aio( trio.open_nursery() as n, ): + # start 2nd task that get's broadcast the same + # value set. n.start_soon(consume, br) await consume(chan) else: await consume(chan) + except BaseException as err: + import logging + log = logging.getLogger() + log.exception('aio-subactor errored!\n') + raise err + finally: if ( @@ -461,7 +489,8 @@ async def stream_from_aio( assert not fan_out assert pulled == expect[:51] - print('trio guest mode task completed!') + print('trio guest-mode task completed!') + assert chan._aio_task.done() @pytest.mark.parametrize( @@ -501,19 +530,37 @@ def test_trio_error_cancels_intertask_chan(reg_addr): excinfo.value.boxed_type is Exception -def test_trio_closes_early_and_channel_exits(reg_addr): +def test_trio_closes_early_and_channel_exits( + reg_addr: tuple[str, int], +): + ''' + Check that if the `trio`-task "exits early" on `async for`ing the + inter-task-channel (via a `break`) we exit silently from the + `open_channel_from()` block and get a final `Return[None]` msg. + + ''' async def main(): - async with tractor.open_nursery() as n: - portal = await n.run_in_actor( - stream_from_aio, - exit_early=True, - infect_asyncio=True, - ) - # should raise RAE diectly - await portal.result() + with trio.fail_after(2): + async with tractor.open_nursery( + # debug_mode=True, + # enable_stack_on_sig=True, + ) as n: + portal = await n.run_in_actor( + stream_from_aio, + exit_early=True, + infect_asyncio=True, + ) + # should raise RAE diectly + print('waiting on final infected subactor result..') + res: None = await portal.wait_for_result() + assert res is None + print('infected subactor returned result: {res!r}\n') # should be a quiet exit on a simple channel exit - trio.run(main) + trio.run( + main, + # strict_exception_groups=False, + ) def test_aio_errors_and_channel_propagates_and_closes(reg_addr): @@ -660,6 +707,7 @@ def test_echoserver_detailed_mechanics( ) def test_infected_root_actor( raise_error_mid_stream: bool|Exception, + # conftest wide loglevel: str, debug_mode: bool, @@ -670,36 +718,38 @@ def test_infected_root_actor( ''' async def _trio_main(): + with trio.fail_after(2): + first: str + chan: to_asyncio.LinkedTaskChannel + async with ( + tractor.open_root_actor( + debug_mode=debug_mode, + loglevel=loglevel, + ), + to_asyncio.open_channel_from( + aio_echo_server, + ) as (first, chan), + ): + assert first == 'start' - first: str - chan: to_asyncio.LinkedTaskChannel - async with ( - tractor.open_root_actor( - debug_mode=debug_mode, - loglevel=loglevel, - ), - to_asyncio.open_channel_from( - aio_echo_server, - ) as (first, chan), - ): - assert first == 'start' + for i in range(1000): + await chan.send(i) + out = await chan.receive() + assert out == i + print(f'asyncio echoing {i}') - for i in range(1000): - await chan.send(i) - out = await chan.receive() - assert out == i - print(f'asyncio echoing {i}') + if raise_error_mid_stream and i == 500: + raise raise_error_mid_stream - if raise_error_mid_stream and i == 500: - raise raise_error_mid_stream - - if out is None: - try: - out = await chan.receive() - except trio.EndOfChannel: - break - else: - raise RuntimeError('aio channel never stopped?') + if out is None: + try: + out = await chan.receive() + except trio.EndOfChannel: + break + else: + raise RuntimeError( + 'aio channel never stopped?' + ) if raise_error_mid_stream: with pytest.raises(raise_error_mid_stream): @@ -947,6 +997,158 @@ def test_sigint_closes_lifetime_stack( trio.run(main) +async def sync_and_err( + # just signature placeholders for compat with + # ``to_asyncio.open_channel_from()`` + to_trio: trio.MemorySendChannel, + from_trio: asyncio.Queue, + ev: asyncio.Event, + +): + if to_trio: + to_trio.send_nowait('start') + + await ev.wait() + raise RuntimeError('asyncio-side') + + +@pytest.mark.parametrize( + 'aio_err_trigger', + [ + 'before_start_point', + 'after_trio_task_starts', + 'after_start_point', + ], + ids='aio_err_triggered={}'.format +) +def test_trio_prestarted_task_bubbles( + aio_err_trigger: str, + + # conftest wide + loglevel: str, + debug_mode: bool, +): + + async def pre_started_err( + raise_err: bool = False, + pre_sleep: float|None = None, + aio_trigger: asyncio.Event|None = None, + task_status=trio.TASK_STATUS_IGNORED, + ): + ''' + Maybe pre-started error then sleep. + + ''' + if pre_sleep is not None: + print(f'Sleeping from trio for {pre_sleep!r}s !') + await trio.sleep(pre_sleep) + + # signal aio-task to raise JUST AFTER this task + # starts but has not yet `.started()` + if aio_trigger: + print('Signalling aio-task to raise from `trio`!!') + aio_trigger.set() + + if raise_err: + print('Raising from trio!') + raise TypeError('trio-side') + + task_status.started() + await trio.sleep_forever() + + async def _trio_main(): + # with trio.fail_after(2): + with trio.fail_after(999): + first: str + chan: to_asyncio.LinkedTaskChannel + aio_ev = asyncio.Event() + + async with ( + tractor.open_root_actor( + debug_mode=False, + loglevel=loglevel, + ), + + # where we'll start a sub-task that errors BEFORE + # calling `.started()` such that the error should + # bubble before the guest run terminates! + trio.open_nursery() as tn, + + # THEN start an infect task which should error just + # after the trio-side's task does. + to_asyncio.open_channel_from( + partial( + sync_and_err, + ev=aio_ev, + ) + ) as (first, chan), + ): + + for i in range(5): + pre_sleep: float|None = None + raise_err: bool = False + last_iter: bool = (i == 4) + + if last_iter: + raise_err: bool = True + + # trigger aio task to error on next loop + # tick/checkpoint + if aio_err_trigger == 'before_start_point': + aio_ev.set() + + pre_sleep: float = 0 + + await tn.start( + pre_started_err, + raise_err, + pre_sleep, + (aio_ev if ( + aio_err_trigger == 'after_trio_task_starts' + and + last_iter + ) else None + ), + ) + + if ( + aio_err_trigger == 'after_start_point' + and + last_iter + ): + aio_ev.set() + + with pytest.raises( + expected_exception=ExceptionGroup, + ) as excinfo: + tractor.to_asyncio.run_as_asyncio_guest( + trio_main=_trio_main, + ) + + eg = excinfo.value + rte_eg, rest_eg = eg.split(RuntimeError) + + # ensure the trio-task's error bubbled despite the aio-side + # having (maybe) errored first. + if aio_err_trigger in ( + 'after_trio_task_starts', + 'after_start_point', + ): + assert len(errs := rest_eg.exceptions) == 1 + typerr = errs[0] + assert ( + type(typerr) is TypeError + and + 'trio-side' in typerr.args + ) + + # when aio errors BEFORE (last) trio task is scheduled, we should + # never see anythinb but the aio-side. + else: + assert len(rtes := rte_eg.exceptions) == 1 + assert 'asyncio-side' in rtes[0].args[0] + + # TODO: debug_mode tests once we get support for `asyncio`! # # -[ ] need tests to wrap both scripts: From 3345962253eca16cf53c9491318483bc1e0943a6 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 10 Jan 2025 12:42:23 -0500 Subject: [PATCH 46/66] Yield a boxed-maybe-error from `open_crash_handler()` Along the lines of something like `pytest.raises()` where the handled exception can be inspected from the `pdbp` REPL using its `.value` field B) This is super handy in particular for understanding `BaseException[Group]`s without manually adding surrounding handler code to assign the `except[*] Exception as exc_var:` particularly when trying to understand multi-cancelled eg trees. --- tractor/devx/_debug.py | 24 +++++++++++++++++++----- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 7c178ab1..04df000f 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -317,6 +317,7 @@ class Lock: we_released: bool = False ctx_in_debug: Context|None = cls.ctx_in_debug repl_task: Task|Thread|None = DebugStatus.repl_task + message: str = '' try: if not DebugStatus.is_main_trio_thread(): @@ -444,7 +445,10 @@ class Lock: f'|_{repl_task}\n' ) - log.devx(message) + if message: + log.devx(message) + else: + import pdbp; pdbp.set_trace() return we_released @@ -3168,7 +3172,7 @@ async def maybe_wait_for_debugger( @cm def open_crash_handler( catch: set[BaseException] = { - Exception, + # Exception, BaseException, }, ignore: set[BaseException] = { @@ -3189,10 +3193,20 @@ def open_crash_handler( ''' __tracebackhide__: bool = tb_hide + class BoxedMaybeException(Struct): + value: BaseException|None = None + + # TODO, yield a `outcome.Error`-like boxed type? + # -[~] use `outcome.Value/Error` X-> frozen! + # -[x] write our own..? + # -[ ] consider just wtv is used by `pytest.raises()`? + # + boxed_maybe_exc = BoxedMaybeException() err: BaseException try: - yield + yield boxed_maybe_exc except tuple(catch) as err: + boxed_maybe_exc.value = err if ( type(err) not in ignore and @@ -3210,13 +3224,13 @@ def open_crash_handler( ) except bdb.BdbQuit: __tracebackhide__: bool = False - raise + raise err # XXX NOTE, `pdbp`'s version seems to lose the up-stack # tb-info? # pdbp.xpm() - raise + raise err @cm From bfd186418044cbb945119ddc84acc2abb5f5fa4f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 10 Jan 2025 15:46:00 -0500 Subject: [PATCH 47/66] Add a "raise-from-`finally:`" example test Since i wasted 2 days just to find an example of this inside an `@acm`, figured I better reproduce for the purposes of maybe implementing a warning sys (inside our wip proto `open_taskman()`) when a nursery detects a single `Cancelled` in an eg where the `.__context__` is set to some non-cancel error (which likely means a cancel-causing source exception was suppressed by accident). Left in a buncha commented code using `maybe_open_nursery()` which i thought might be part of the issue but didn't end up being required; will likely remove on a follow up refinement. --- tests/test_trioisms.py | 61 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 61 insertions(+) diff --git a/tests/test_trioisms.py b/tests/test_trioisms.py index 27dc6c34..be29965e 100644 --- a/tests/test_trioisms.py +++ b/tests/test_trioisms.py @@ -3,6 +3,10 @@ Reminders for oddities in `trio` that we need to stay aware of and/or want to see changed. ''' +from contextlib import ( + asynccontextmanager as acm, +) + import pytest import trio from trio import TaskStatus @@ -80,3 +84,60 @@ def test_stashed_child_nursery(use_start_soon): with pytest.raises(NameError): trio.run(main) + + +# @pytest.mark.parametrize( +# 'open_tn_outside_acm', +# [True, False] +# # ids='aio_err_triggered={}'.format +# ) +@pytest.mark.parametrize( + 'canc_from_finally', + [True, False] + # ids='aio_err_triggered={}'.format +) +def test_acm_embedded_nursery_propagates_enter_err( + canc_from_finally: bool, + # open_tn_outside_acm: bool, +): + # from tractor.trionics import maybe_open_nursery + + # async def canc_then_checkpoint(tn): + # tn.cancel_scope.cancel() + # await trio.lowlevel.checkpoint() + + @acm + async def wraps_tn_that_always_cancels( + # maybe_tn: trio.Nursery|None = None + ): + # async with maybe_open_nursery(maybe_tn) as tn: + async with trio.open_nursery() as tn: + try: + yield tn + finally: + if canc_from_finally: + # await canc_then_checkpoint(tn) + tn.cancel_scope.cancel() + await trio.lowlevel.checkpoint() + + async def _main(): + # open_nursery = ( + # trio.open_nursery if open_tn_outside_acm + # else nullcontext + # ) + + async with ( + # open_nursery() as tn, + # wraps_tn_that_always_cancels(maybe_tn=tn) as tn + wraps_tn_that_always_cancels() as tn + ): + assert not tn.cancel_scope.cancel_called + assert 0 + + with pytest.raises(ExceptionGroup) as excinfo: + trio.run(_main) + + eg = excinfo.value + assert_eg, rest_eg = eg.split(AssertionError) + + assert len(assert_eg.exceptions) == 1 From 4edf36a895cef50794c15dba24c03f8816975bdf Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 10 Jan 2025 17:29:11 -0500 Subject: [PATCH 48/66] Impl a proto "unmasker" `@acm` alongside our test Such that the suite verifies the wip `maybe_raise_from_masking_exc()` will raise from a `trio.Cancelled.__context__` since I can't think of any reason a `Cancelled` should ever be raised in-place of a non-`Cancelled` XD Not sure what should be raised instead (or maybe just a `log.warning()` emitted?) but this starts a draft for refinement at the least. Use the new `@pytest.mark.parametrize` explicit tuple-of-params form with an `pytest.param + `.mark.xfail()` for the default behaviour case. --- tests/test_trioisms.py | 115 ++++++++++++++++++++++++++++++----------- 1 file changed, 85 insertions(+), 30 deletions(-) diff --git a/tests/test_trioisms.py b/tests/test_trioisms.py index be29965e..fad99f11 100644 --- a/tests/test_trioisms.py +++ b/tests/test_trioisms.py @@ -86,58 +86,113 @@ def test_stashed_child_nursery(use_start_soon): trio.run(main) -# @pytest.mark.parametrize( -# 'open_tn_outside_acm', -# [True, False] -# # ids='aio_err_triggered={}'.format -# ) @pytest.mark.parametrize( - 'canc_from_finally', - [True, False] - # ids='aio_err_triggered={}'.format + ('unmask_from_canc', 'canc_from_finally'), + [ + (True, False), + (True, True), + pytest.param(False, True, + marks=pytest.mark.xfail(reason="never raises!") + ), + ], + # TODO, ask ronny how to impl this .. XD + # ids='unmask_from_canc={0}, canc_from_finally={1}',#.format, ) def test_acm_embedded_nursery_propagates_enter_err( canc_from_finally: bool, - # open_tn_outside_acm: bool, + unmask_from_canc: bool, ): - # from tractor.trionics import maybe_open_nursery + ''' + Demo how a masking `trio.Cancelled` could be handled by unmasking from the + `.__context__` field when a user (by accident) re-raises from a `finally:`. - # async def canc_then_checkpoint(tn): - # tn.cancel_scope.cancel() - # await trio.lowlevel.checkpoint() + ''' + import tractor @acm - async def wraps_tn_that_always_cancels( - # maybe_tn: trio.Nursery|None = None + async def maybe_raise_from_masking_exc( + tn: trio.Nursery, + unmask_from: BaseException|None = trio.Cancelled + + # TODO, maybe offer a collection? + # unmask_from: set[BaseException] = { + # trio.Cancelled, + # }, ): - # async with maybe_open_nursery(maybe_tn) as tn: - async with trio.open_nursery() as tn: + if not unmask_from: + yield + return + + try: + yield + except* unmask_from as be_eg: + + # TODO, if we offer `unmask_from: set` + # for masker_exc_type in unmask_from: + + matches, rest = be_eg.split(unmask_from) + if not matches: + raise + + for exc_match in be_eg.exceptions: + if ( + (exc_ctx := exc_match.__context__) + and + type(exc_ctx) not in { + # trio.Cancelled, # always by default? + unmask_from, + } + ): + exc_ctx.add_note( + f'\n' + f'WARNING: the above error was masked by a {unmask_from!r} !?!\n' + f'Are you always cancelling? Say from a `finally:` ?\n\n' + + f'{tn!r}' + ) + raise exc_ctx from exc_match + + + @acm + async def wraps_tn_that_always_cancels(): + async with ( + trio.open_nursery() as tn, + maybe_raise_from_masking_exc( + tn=tn, + unmask_from=( + trio.Cancelled + if unmask_from_canc + else None + ), + ) + ): try: yield tn finally: if canc_from_finally: - # await canc_then_checkpoint(tn) tn.cancel_scope.cancel() await trio.lowlevel.checkpoint() async def _main(): - # open_nursery = ( - # trio.open_nursery if open_tn_outside_acm - # else nullcontext - # ) + with tractor.devx.open_crash_handler() as bxerr: + assert not bxerr.value - async with ( - # open_nursery() as tn, - # wraps_tn_that_always_cancels(maybe_tn=tn) as tn - wraps_tn_that_always_cancels() as tn - ): - assert not tn.cancel_scope.cancel_called - assert 0 + async with ( + wraps_tn_that_always_cancels() as tn, + ): + assert not tn.cancel_scope.cancel_called + assert 0 + + assert ( + (err := bxerr.value) + and + type(err) is AssertionError + ) with pytest.raises(ExceptionGroup) as excinfo: trio.run(_main) - eg = excinfo.value + eg: ExceptionGroup = excinfo.value assert_eg, rest_eg = eg.split(AssertionError) assert len(assert_eg.exceptions) == 1 From 9b393338ca3339fcf73a7bb995e9fbc4a0869330 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 10 Jan 2025 17:57:54 -0500 Subject: [PATCH 49/66] Add a `tests/test_root_infect_asyncio` Might as well break apart the specific test set since there are some (minor) subtleties and the orig test mod is already getting pretty big XD Includes both the new "independent"-event-loops test as well as the std usage base case suite. --- tests/test_infected_asyncio.py | 222 +-------------------------- tests/test_root_infect_asyncio.py | 244 ++++++++++++++++++++++++++++++ 2 files changed, 246 insertions(+), 220 deletions(-) create mode 100644 tests/test_root_infect_asyncio.py diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 5d88920a..d462f59d 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -5,7 +5,7 @@ The hipster way to force SC onto the stdlib's "async": 'infection mode'. import asyncio import builtins from contextlib import ExitStack -from functools import partial +# from functools import partial import itertools import importlib import os @@ -224,7 +224,7 @@ async def trio_ctx( ) await trio.sleep_forever() - # TODO, factor this into a `trionics.callapse()`? + # TODO, factor this into a `trionics.collapse()`? except* BaseException as beg: # await tractor.pause(shield=True) if len(excs := beg.exceptions) == 1: @@ -696,72 +696,6 @@ def test_echoserver_detailed_mechanics( trio.run(main) -@pytest.mark.parametrize( - 'raise_error_mid_stream', - [ - False, - Exception, - KeyboardInterrupt, - ], - ids='raise_error={}'.format, -) -def test_infected_root_actor( - raise_error_mid_stream: bool|Exception, - - # conftest wide - loglevel: str, - debug_mode: bool, -): - ''' - Verify you can run the `tractor` runtime with `Actor.is_infected_aio() == True` - in the root actor. - - ''' - async def _trio_main(): - with trio.fail_after(2): - first: str - chan: to_asyncio.LinkedTaskChannel - async with ( - tractor.open_root_actor( - debug_mode=debug_mode, - loglevel=loglevel, - ), - to_asyncio.open_channel_from( - aio_echo_server, - ) as (first, chan), - ): - assert first == 'start' - - for i in range(1000): - await chan.send(i) - out = await chan.receive() - assert out == i - print(f'asyncio echoing {i}') - - if raise_error_mid_stream and i == 500: - raise raise_error_mid_stream - - if out is None: - try: - out = await chan.receive() - except trio.EndOfChannel: - break - else: - raise RuntimeError( - 'aio channel never stopped?' - ) - - if raise_error_mid_stream: - with pytest.raises(raise_error_mid_stream): - tractor.to_asyncio.run_as_asyncio_guest( - trio_main=_trio_main, - ) - else: - tractor.to_asyncio.run_as_asyncio_guest( - trio_main=_trio_main, - ) - - @tractor.context async def manage_file( ctx: tractor.Context, @@ -997,158 +931,6 @@ def test_sigint_closes_lifetime_stack( trio.run(main) -async def sync_and_err( - # just signature placeholders for compat with - # ``to_asyncio.open_channel_from()`` - to_trio: trio.MemorySendChannel, - from_trio: asyncio.Queue, - ev: asyncio.Event, - -): - if to_trio: - to_trio.send_nowait('start') - - await ev.wait() - raise RuntimeError('asyncio-side') - - -@pytest.mark.parametrize( - 'aio_err_trigger', - [ - 'before_start_point', - 'after_trio_task_starts', - 'after_start_point', - ], - ids='aio_err_triggered={}'.format -) -def test_trio_prestarted_task_bubbles( - aio_err_trigger: str, - - # conftest wide - loglevel: str, - debug_mode: bool, -): - - async def pre_started_err( - raise_err: bool = False, - pre_sleep: float|None = None, - aio_trigger: asyncio.Event|None = None, - task_status=trio.TASK_STATUS_IGNORED, - ): - ''' - Maybe pre-started error then sleep. - - ''' - if pre_sleep is not None: - print(f'Sleeping from trio for {pre_sleep!r}s !') - await trio.sleep(pre_sleep) - - # signal aio-task to raise JUST AFTER this task - # starts but has not yet `.started()` - if aio_trigger: - print('Signalling aio-task to raise from `trio`!!') - aio_trigger.set() - - if raise_err: - print('Raising from trio!') - raise TypeError('trio-side') - - task_status.started() - await trio.sleep_forever() - - async def _trio_main(): - # with trio.fail_after(2): - with trio.fail_after(999): - first: str - chan: to_asyncio.LinkedTaskChannel - aio_ev = asyncio.Event() - - async with ( - tractor.open_root_actor( - debug_mode=False, - loglevel=loglevel, - ), - - # where we'll start a sub-task that errors BEFORE - # calling `.started()` such that the error should - # bubble before the guest run terminates! - trio.open_nursery() as tn, - - # THEN start an infect task which should error just - # after the trio-side's task does. - to_asyncio.open_channel_from( - partial( - sync_and_err, - ev=aio_ev, - ) - ) as (first, chan), - ): - - for i in range(5): - pre_sleep: float|None = None - raise_err: bool = False - last_iter: bool = (i == 4) - - if last_iter: - raise_err: bool = True - - # trigger aio task to error on next loop - # tick/checkpoint - if aio_err_trigger == 'before_start_point': - aio_ev.set() - - pre_sleep: float = 0 - - await tn.start( - pre_started_err, - raise_err, - pre_sleep, - (aio_ev if ( - aio_err_trigger == 'after_trio_task_starts' - and - last_iter - ) else None - ), - ) - - if ( - aio_err_trigger == 'after_start_point' - and - last_iter - ): - aio_ev.set() - - with pytest.raises( - expected_exception=ExceptionGroup, - ) as excinfo: - tractor.to_asyncio.run_as_asyncio_guest( - trio_main=_trio_main, - ) - - eg = excinfo.value - rte_eg, rest_eg = eg.split(RuntimeError) - - # ensure the trio-task's error bubbled despite the aio-side - # having (maybe) errored first. - if aio_err_trigger in ( - 'after_trio_task_starts', - 'after_start_point', - ): - assert len(errs := rest_eg.exceptions) == 1 - typerr = errs[0] - assert ( - type(typerr) is TypeError - and - 'trio-side' in typerr.args - ) - - # when aio errors BEFORE (last) trio task is scheduled, we should - # never see anythinb but the aio-side. - else: - assert len(rtes := rte_eg.exceptions) == 1 - assert 'asyncio-side' in rtes[0].args[0] - - # TODO: debug_mode tests once we get support for `asyncio`! # # -[ ] need tests to wrap both scripts: diff --git a/tests/test_root_infect_asyncio.py b/tests/test_root_infect_asyncio.py new file mode 100644 index 00000000..331b6311 --- /dev/null +++ b/tests/test_root_infect_asyncio.py @@ -0,0 +1,244 @@ +''' +Special attention cases for using "infect `asyncio`" mode from a root +actor; i.e. not using a std `trio.run()` bootstrap. + +''' +import asyncio +from functools import partial + +import pytest +import trio +import tractor +from tractor import ( + to_asyncio, +) +from tests.test_infected_asyncio import ( + aio_echo_server, +) + + +@pytest.mark.parametrize( + 'raise_error_mid_stream', + [ + False, + Exception, + KeyboardInterrupt, + ], + ids='raise_error={}'.format, +) +def test_infected_root_actor( + raise_error_mid_stream: bool|Exception, + + # conftest wide + loglevel: str, + debug_mode: bool, +): + ''' + Verify you can run the `tractor` runtime with `Actor.is_infected_aio() == True` + in the root actor. + + ''' + async def _trio_main(): + with trio.fail_after(2): + first: str + chan: to_asyncio.LinkedTaskChannel + async with ( + tractor.open_root_actor( + debug_mode=debug_mode, + loglevel=loglevel, + ), + to_asyncio.open_channel_from( + aio_echo_server, + ) as (first, chan), + ): + assert first == 'start' + + for i in range(1000): + await chan.send(i) + out = await chan.receive() + assert out == i + print(f'asyncio echoing {i}') + + if raise_error_mid_stream and i == 500: + raise raise_error_mid_stream + + if out is None: + try: + out = await chan.receive() + except trio.EndOfChannel: + break + else: + raise RuntimeError( + 'aio channel never stopped?' + ) + + if raise_error_mid_stream: + with pytest.raises(raise_error_mid_stream): + tractor.to_asyncio.run_as_asyncio_guest( + trio_main=_trio_main, + ) + else: + tractor.to_asyncio.run_as_asyncio_guest( + trio_main=_trio_main, + ) + + + +async def sync_and_err( + # just signature placeholders for compat with + # ``to_asyncio.open_channel_from()`` + to_trio: trio.MemorySendChannel, + from_trio: asyncio.Queue, + ev: asyncio.Event, + +): + if to_trio: + to_trio.send_nowait('start') + + await ev.wait() + raise RuntimeError('asyncio-side') + + +@pytest.mark.parametrize( + 'aio_err_trigger', + [ + 'before_start_point', + 'after_trio_task_starts', + 'after_start_point', + ], + ids='aio_err_triggered={}'.format +) +def test_trio_prestarted_task_bubbles( + aio_err_trigger: str, + + # conftest wide + loglevel: str, + debug_mode: bool, +): + async def pre_started_err( + raise_err: bool = False, + pre_sleep: float|None = None, + aio_trigger: asyncio.Event|None = None, + task_status=trio.TASK_STATUS_IGNORED, + ): + ''' + Maybe pre-started error then sleep. + + ''' + if pre_sleep is not None: + print(f'Sleeping from trio for {pre_sleep!r}s !') + await trio.sleep(pre_sleep) + + # signal aio-task to raise JUST AFTER this task + # starts but has not yet `.started()` + if aio_trigger: + print('Signalling aio-task to raise from `trio`!!') + aio_trigger.set() + + if raise_err: + print('Raising from trio!') + raise TypeError('trio-side') + + task_status.started() + await trio.sleep_forever() + + async def _trio_main(): + # with trio.fail_after(2): + with trio.fail_after(999): + first: str + chan: to_asyncio.LinkedTaskChannel + aio_ev = asyncio.Event() + + async with ( + tractor.open_root_actor( + debug_mode=False, + loglevel=loglevel, + ), + ): + # TODO, tests for this with 3.13 egs? + # from tractor.devx import open_crash_handler + # with open_crash_handler(): + async with ( + # where we'll start a sub-task that errors BEFORE + # calling `.started()` such that the error should + # bubble before the guest run terminates! + trio.open_nursery() as tn, + + # THEN start an infect task which should error just + # after the trio-side's task does. + to_asyncio.open_channel_from( + partial( + sync_and_err, + ev=aio_ev, + ) + ) as (first, chan), + ): + + for i in range(5): + pre_sleep: float|None = None + last_iter: bool = (i == 4) + + # TODO, missing cases? + # -[ ] error as well on + # 'after_start_point' case as well for + # another case? + raise_err: bool = False + + if last_iter: + raise_err: bool = True + + # trigger aio task to error on next loop + # tick/checkpoint + if aio_err_trigger == 'before_start_point': + aio_ev.set() + + pre_sleep: float = 0 + + await tn.start( + pre_started_err, + raise_err, + pre_sleep, + (aio_ev if ( + aio_err_trigger == 'after_trio_task_starts' + and + last_iter + ) else None + ), + ) + + if ( + aio_err_trigger == 'after_start_point' + and + last_iter + ): + aio_ev.set() + + with pytest.raises( + expected_exception=ExceptionGroup, + ) as excinfo: + tractor.to_asyncio.run_as_asyncio_guest( + trio_main=_trio_main, + ) + + eg = excinfo.value + rte_eg, rest_eg = eg.split(RuntimeError) + + # ensure the trio-task's error bubbled despite the aio-side + # having (maybe) errored first. + if aio_err_trigger in ( + 'after_trio_task_starts', + 'after_start_point', + ): + assert len(errs := rest_eg.exceptions) == 1 + typerr = errs[0] + assert ( + type(typerr) is TypeError + and + 'trio-side' in typerr.args + ) + + # when aio errors BEFORE (last) trio task is scheduled, we should + # never see anythinb but the aio-side. + else: + assert len(rtes := rte_eg.exceptions) == 1 + assert 'asyncio-side' in rtes[0].args[0] From f199cac5e8a7f9878bfdd634e92daa5ad4ed55db Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 21 Feb 2025 18:39:18 -0500 Subject: [PATCH 50/66] Raise "independent" task errors in an eg The (rare) condition is heavily detailed in new comments in the `cancel_trio()` callback but, more or less the idea here is to be extra pedantic in raising an `Exceptiongroup` of errors from each task (both `asyncio` and `trio`) whenever the 2 tasks raise "independently" - in the sense that it's not obviously one side's task causing an error (or cancellation) in the other. In this case we set the error for each side on the `LinkedTaskChannel` (via new attrs described later). As a synopsis, most of this work was refined out of supporting `infected_aio=True` mode in the **root actor** and in particular as part of getting that to work inside the `modden` daemon which at the time of writing was still using the `i3ipc` lib and thus `asyncio`. Impl deats, - extend the `LinkedTaskChannel` field/API set (and type it), - `._trio_task: trio.Task` for test/user introspection. - also "stage" some ideas for a more refined interface, - `.started()` to deliver the value yielded to the `trio.Task` parent. |_ also includes some todos for how to implement this design underneath. - `._aio_first: Any|None = None` to hold that value ^. - `.wait_aio_complete()` for syncing to the asyncio task. - some detailed logging around "asyncio cancelled trio" case. - Move `AsyncioCancelled` in this module. Styling changes, - generally more explicit var naming. - some todos for getting modern and fancy with typing.. NB, Let it be known this commit msg was written on a friday with the help of various "mr. white" solns. --- tractor/to_asyncio.py | 225 +++++++++++++++++++++++++++++++----------- 1 file changed, 167 insertions(+), 58 deletions(-) diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index 3f8d20d3..75dfb5cb 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -34,7 +34,6 @@ from typing import ( import tractor from tractor._exceptions import ( - AsyncioCancelled, is_multi_cancelled, ) from tractor._state import ( @@ -46,6 +45,11 @@ from tractor.log import ( get_logger, StackLevelAdapter, ) +# TODO, wite the equiv of `trio.abc.Channel` but without attrs.. +# -[ ] `trionics.chan_types.ChanStruct` maybe? +# from tractor.msg import ( +# pretty_struct, +# ) from tractor.trionics._broadcast import ( broadcast_receiver, BroadcastReceiver, @@ -66,7 +70,12 @@ __all__ = [ @dataclass -class LinkedTaskChannel(trio.abc.Channel): +class LinkedTaskChannel( + trio.abc.Channel, + + # XXX LAME! meta-base conflict.. + # pretty_struct.Struct, +): ''' A "linked task channel" which allows for two-way synchronized msg passing between a ``trio``-in-guest-mode task and an ``asyncio`` @@ -77,12 +86,14 @@ class LinkedTaskChannel(trio.abc.Channel): _from_aio: trio.MemoryReceiveChannel _to_trio: trio.MemorySendChannel _trio_cs: trio.CancelScope + _trio_task: trio.Task _aio_task_complete: trio.Event _trio_err: BaseException|None = None _trio_exited: bool = False # set after ``asyncio.create_task()`` + # _aio_first: Any|None = None _aio_task: asyncio.Task|None = None _aio_err: BaseException|None = None _broadcaster: BroadcastReceiver|None = None @@ -90,6 +101,25 @@ class LinkedTaskChannel(trio.abc.Channel): async def aclose(self) -> None: await self._from_aio.aclose() + def started( + self, + val: Any = None, + ) -> None: + self._aio_started_val = val + return self._to_trio.send_nowait(val) + + # TODO, mk this side-agnostic? + # + # -[ ] add private meths for both sides and dynamically + # determine which to use based on task-type read at calltime? + # -[ ] `._recv_trio()`: receive to trio<-asyncio + # -[ ] `._send_trio()`: send from trio->asyncio + # -[ ] `._recv_aio()`: send from asyncio->trio + # -[ ] `._send_aio()`: receive to asyncio<-trio + # + # -[ ] pass the instance to the aio side instead of the separate + # per-side chan types? + # async def receive(self) -> Any: ''' Receive a value from the paired `asyncio.Task` with @@ -115,7 +145,16 @@ class LinkedTaskChannel(trio.abc.Channel): ): raise err - async def wait_asyncio_complete(self) -> None: + async def send(self, item: Any) -> None: + ''' + Send a value through to the asyncio task presuming + it defines a ``from_trio`` argument, if it does not + this method will raise an error. + + ''' + self._to_aio.put_nowait(item) + + async def wait_aio_complete(self) -> None: await self._aio_task_complete.wait() def cancel_asyncio_task( @@ -126,15 +165,6 @@ class LinkedTaskChannel(trio.abc.Channel): msg=msg, ) - async def send(self, item: Any) -> None: - ''' - Send a value through to the asyncio task presuming - it defines a ``from_trio`` argument, if it does not - this method will raise an error. - - ''' - self._to_aio.put_nowait(item) - def closed(self) -> bool: return self._from_aio._closed # type: ignore @@ -218,7 +248,8 @@ def _run_asyncio_task( coro = func(**kwargs) - cancel_scope = trio.CancelScope() + trio_task: trio.Task = trio.lowlevel.current_task() + trio_cs = trio.CancelScope() aio_task_complete = trio.Event() aio_err: BaseException|None = None @@ -226,7 +257,8 @@ def _run_asyncio_task( _to_aio=aio_q, # asyncio.Queue _from_aio=from_aio, # recv chan _to_trio=to_trio, # send chan - _trio_cs=cancel_scope, + _trio_cs=trio_cs, + _trio_task=trio_task, _aio_task_complete=aio_task_complete, ) @@ -274,6 +306,9 @@ def _run_asyncio_task( to_trio.send_nowait(result) finally: + # breakpoint() + # import pdbp; pdbp.set_trace() + # if the task was spawned using `open_channel_from()` # then we close the channels on exit. if provide_channels: @@ -281,7 +316,6 @@ def _run_asyncio_task( # a ``trio.EndOfChannel`` to the trio (consumer) side. to_trio.close() - # import pdbp; pdbp.set_trace() aio_task_complete.set() # await asyncio.sleep(0.1) log.info( @@ -325,14 +359,17 @@ def _run_asyncio_task( ) greenback.bestow_portal(task) - def cancel_trio(task: asyncio.Task) -> None: + def cancel_trio( + task: asyncio.Task, + ) -> None: ''' - Cancel the calling `trio` task on error. + Cancel the parent `trio` task on any error raised by the + `asyncio` side. ''' nonlocal chan - aio_err: BaseException|None = chan._aio_err - task_err: BaseException|None = None + relayed_aio_err: BaseException|None = chan._aio_err + aio_err: BaseException|None = None # only to avoid `asyncio` complaining about uncaptured # task exceptions @@ -343,20 +380,20 @@ def _run_asyncio_task( f'|_{res}\n' ) except BaseException as _aio_err: - task_err: BaseException = _aio_err - + aio_err: BaseException = _aio_err # read again AFTER the `asyncio` side errors in case # it was cancelled due to an error from `trio` (or - # some other out of band exc). - aio_err: BaseException|None = chan._aio_err + # some other out of band exc) and then set to something + # else? + relayed_aio_err: BaseException|None = chan._aio_err # always true right? assert ( - type(_aio_err) is type(aio_err) + type(_aio_err) is type(relayed_aio_err) ), ( f'`asyncio`-side task errors mismatch?!?\n\n' - f'caught: {_aio_err}\n' - f'chan._aio_err: {aio_err}\n' + f'(caught) aio_err: {aio_err}\n' + f'chan._aio_err: {relayed_aio_err}\n' ) msg: str = ( @@ -381,12 +418,13 @@ def _run_asyncio_task( msg.format(etype_str='errored') ) - - if aio_err is not None: + trio_err: BaseException|None = chan._trio_err + if ( + relayed_aio_err + or + trio_err + ): # import pdbp; pdbp.set_trace() - # XXX: uhh is this true? - # assert task_err, f'Asyncio task {task.get_name()} discrepancy!?' - # NOTE: currently mem chan closure may act as a form # of error relay (at least in the `asyncio.CancelledError` # case) since we have no way to directly trigger a `trio` @@ -394,8 +432,6 @@ def _run_asyncio_task( # We might want to change this in the future though. from_aio.close() - if task_err is None: - assert aio_err # wait, wut? # aio_err.with_traceback(aio_err.__traceback__) @@ -404,7 +440,7 @@ def _run_asyncio_task( # elif ( # type(aio_err) is CancelledError # and # trio was the cause? - # cancel_scope.cancel_called + # trio_cs.cancel_called # ): # log.cancel( # 'infected task was cancelled by `trio`-side' @@ -415,26 +451,83 @@ def _run_asyncio_task( # error in case the trio task is blocking on # a checkpoint. if ( - not cancel_scope.cancelled_caught + not trio_cs.cancelled_caught or - not cancel_scope.cancel_called + not trio_cs.cancel_called ): # import pdbp; pdbp.set_trace() - cancel_scope.cancel() + trio_cs.cancel() - if task_err: + # maybe the `trio` task errored independent from the + # `asyncio` one and likely in between + # a guest-run-sched-tick. + # + # The obvious ex. is where one side errors during + # the current tick and then the other side immediately + # errors before its next checkpoint; i.e. the 2 errors + # are "independent". + # + # "Independent" here means in the sense that neither task + # was the explicit cause of the other side's exception + # according to our `tractor.to_asyncio` SC API's error + # relaying mechanism(s); the error pair is *possibly + # due-to* but **not necessarily** inter-related by some + # (subsys) state between the tasks, + # + # NOTE, also see the `test_trio_prestarted_task_bubbles` + # for reproducing detailed edge cases as per the above + # cases. + # + if ( + not trio_cs.cancelled_caught + and + (trio_err := chan._trio_err) + and + type(trio_err) not in { + trio.Cancelled, + } + and ( + aio_err + and + type(aio_err) not in { + asyncio.CancelledError + } + ) + ): + eg = ExceptionGroup( + 'Both the `trio` and `asyncio` tasks errored independently!!\n', + (trio_err, aio_err), + ) + chan._trio_err = eg + chan._aio_err = eg + raise eg + + elif aio_err: # XXX raise any `asyncio` side error IFF it doesn't # match the one we just caught from the task above! # (that would indicate something weird/very-wrong # going on?) - if aio_err is not task_err: - # import pdbp; pdbp.set_trace() - raise aio_err from task_err + if aio_err is not relayed_aio_err: + raise aio_err from relayed_aio_err + + raise aio_err task.add_done_callback(cancel_trio) return chan +class AsyncioCancelled(Exception): + ''' + Asyncio cancelled translation (non-base) error + for use with the ``to_asyncio`` module + to be raised in the ``trio`` side task + + NOTE: this should NOT inherit from `asyncio.CancelledError` or + tests should break! + + ''' + + class TrioTaskExited(AsyncioCancelled): ''' The `trio`-side task exited without explicitly cancelling the @@ -483,13 +576,12 @@ async def translate_aio_errors( # import pdbp; pdbp.set_trace() # lolevel-debug - # relay cancel through to called ``asyncio`` task + # relay cancel through to called `asyncio` task chan._aio_err = AsyncioCancelled( f'trio`-side cancelled the `asyncio`-side,\n' f'c)>\n' f' |_{trio_task}\n\n' - f'{trio_err!r}\n' ) @@ -546,6 +638,7 @@ async def translate_aio_errors( raise except BaseException as _trio_err: + # await tractor.pause(shield=True) trio_err = _trio_err log.exception( '`trio`-side task errored?' @@ -619,11 +712,17 @@ async def translate_aio_errors( # pump the other side's task? needed? await trio.lowlevel.checkpoint() + # from tractor._state import is_root_process + # if is_root_process(): + # breakpoint() + if ( not chan._trio_err and (fut := aio_task._fut_waiter) ): + # await trio.lowlevel.checkpoint() + # import pdbp; pdbp.set_trace() fut.set_exception( TrioTaskExited( f'The peer `asyncio` task is still blocking/running?\n' @@ -632,11 +731,6 @@ async def translate_aio_errors( ) ) else: - # from tractor._state import is_root_process - # if is_root_process(): - # breakpoint() - # import pdbp; pdbp.set_trace() - aio_taskc_warn: str = ( f'\n' f'MANUALLY Cancelling `asyncio`-task: {aio_task.get_name()}!\n\n' @@ -663,7 +757,7 @@ async def translate_aio_errors( # it erroed out up there! # if wait_on_aio_task: - # await chan.wait_asyncio_complete() + # await chan.wait_aio_complete() await chan._aio_task_complete.wait() log.info( 'asyncio-task is done and unblocked trio-side!\n' @@ -771,11 +865,22 @@ async def open_channel_from( # sync to a "started()"-like first delivered value from the # ``asyncio`` task. try: - with chan._trio_cs: + with (cs := chan._trio_cs): first = await chan.receive() # deliver stream handle upward yield first, chan + except trio.Cancelled as taskc: + # await tractor.pause(shield=True) # ya it worx ;) + if cs.cancel_called: + log.cancel( + f'trio-side was manually cancelled by aio side\n' + f'|_c>}}{cs!r}?\n' + ) + # TODO, maybe a special `TrioCancelled`??? + + raise taskc + finally: chan._trio_exited = True chan._to_trio.close() @@ -893,12 +998,12 @@ def run_as_asyncio_guest( _sigint_loop_pump_delay: float = 0, ) -> None: -# ^-TODO-^ technically whatever `trio_main` returns.. we should -# try to use func-typevar-params at leaast by 3.13! -# -[ ] https://typing.readthedocs.io/en/latest/spec/callables.html#callback-protocols -# -[ ] https://peps.python.org/pep-0646/#using-type-variable-tuples-in-functions -# -[ ] https://typing.readthedocs.io/en/latest/spec/callables.html#unpack-for-keyword-arguments -# -[ ] https://peps.python.org/pep-0718/ + # ^-TODO-^ technically whatever `trio_main` returns.. we should + # try to use func-typevar-params at leaast by 3.13! + # -[ ] https://typing.readthedocs.io/en/latest/spec/callables.html#callback-protocols + # -[ ] https://peps.python.org/pep-0646/#using-type-variable-tuples-in-functions + # -[ ] https://typing.readthedocs.io/en/latest/spec/callables.html#unpack-for-keyword-arguments + # -[ ] https://peps.python.org/pep-0718/ ''' Entry for an "infected ``asyncio`` actor". @@ -957,15 +1062,15 @@ def run_as_asyncio_guest( # force_reload=True, # ) - def trio_done_callback(main_outcome): + def trio_done_callback(main_outcome: Outcome): log.runtime( f'`trio` guest-run finishing with outcome\n' f'>) {main_outcome}\n' f'|_{trio_done_fute}\n' ) + # import pdbp; pdbp.set_trace() if isinstance(main_outcome, Error): - # import pdbp; pdbp.set_trace() error: BaseException = main_outcome.error # show an dedicated `asyncio`-side tb from the error @@ -1165,6 +1270,10 @@ def run_as_asyncio_guest( ) raise AsyncioRuntimeTranslationError(message) from state_err + # XXX, should never get here ;) + # else: + # import pdbp; pdbp.set_trace() + # might as well if it's installed. try: import uvloop From 66a7d660f61c4a1ed2be2455d53b17efc28dad38 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 24 Feb 2025 13:08:23 -0500 Subject: [PATCH 51/66] Draft test-doc for "out-of-band" `asyncio.Task`.. Since there's no way to activate `greenback`'s portal in such cases, we should at least have a test verifying our very loud error about the inability to support this usage.. --- tests/conftest.py | 12 ++++++++++++ tests/devx/test_pause_from_non_trio.py | 25 +++++++++++++++++++++++-- 2 files changed, 35 insertions(+), 2 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index 5ce84425..810b642a 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -150,6 +150,18 @@ def pytest_generate_tests(metafunc): metafunc.parametrize("start_method", [spawn_backend], scope='module') +# TODO: a way to let test scripts (like from `examples/`) +# guarantee they won't registry addr collide! +# @pytest.fixture +# def open_test_runtime( +# reg_addr: tuple, +# ) -> AsyncContextManager: +# return partial( +# tractor.open_nursery, +# registry_addrs=[reg_addr], +# ) + + def sig_prog(proc, sig): "Kill the actor-process with ``sig``." proc.send_signal(sig) diff --git a/tests/devx/test_pause_from_non_trio.py b/tests/devx/test_pause_from_non_trio.py index f3fd15ad..3a7140e6 100644 --- a/tests/devx/test_pause_from_non_trio.py +++ b/tests/devx/test_pause_from_non_trio.py @@ -218,10 +218,9 @@ def expect_any_of( ) return expected_patts - # yield child -def test_pause_from_asyncio_task( +def test_sync_pause_from_aio_task( spawn, ctlc: bool # ^TODO, fix for `asyncio`!! @@ -327,3 +326,25 @@ def test_pause_from_asyncio_task( child.sendline('c') child.expect(EOF) + + +def test_sync_pause_from_non_greenbacked_aio_task(): + ''' + Where the `breakpoint()` caller task is NOT spawned by + `tractor.to_asyncio` and thus never activates + a `greenback.ensure_portal()` beforehand, presumably bc the task + was started by some lib/dep as in often seen in the field. + + Ensure sync pausing works when the pause is in, + + - the root actor running in infected-mode? + |_ since we don't need any IPC to acquire the debug lock? + |_ is there some way to handle this like the non-main-thread case? + + All other cases need to error out appropriately right? + + - for any subactor we can't avoid needing the repl lock.. + |_ is there a way to hook into `asyncio.ensure_future(obj)`? + + ''' + pass From 255db4b1275885441743595d4b205e16fa5b3bcf Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 22 Mar 2025 20:28:08 -0400 Subject: [PATCH 52/66] Save an MIA `breakpoint()`-restore test from prior!? It appears that during the reorg commit a356233b47fcd2431f4414e7a310e4d3a100fc58 this was intended to be moved (presumably where i have here) to `test_tooling` but was somehow just never pasted over XD Good thing this was caught while going through the remaining TODO bullets in https://pikers.dev/goodboy/tractor/pulls/2 !! Also includes fixed relative `.conftest` imports! --- tests/devx/conftest.py | 2 +- tests/devx/test_debugger.py | 1 - tests/devx/test_tooling.py | 55 ++++++++++++++++++++++++++++++++++++- 3 files changed, 55 insertions(+), 3 deletions(-) diff --git a/tests/devx/conftest.py b/tests/devx/conftest.py index e1ad2ea3..8bc800ec 100644 --- a/tests/devx/conftest.py +++ b/tests/devx/conftest.py @@ -22,7 +22,7 @@ from tractor.devx._debug import ( _repl_fail_msg as _repl_fail_msg, _ctlc_ignore_header as _ctlc_ignore_header, ) -from conftest import ( +from ..conftest import ( _ci_env, ) diff --git a/tests/devx/test_debugger.py b/tests/devx/test_debugger.py index 254b92a1..8b723c6f 100644 --- a/tests/devx/test_debugger.py +++ b/tests/devx/test_debugger.py @@ -29,7 +29,6 @@ from .conftest import ( _repl_fail_msg, ) from .conftest import ( - _ci_env, expect, in_prompt_msg, assert_before, diff --git a/tests/devx/test_tooling.py b/tests/devx/test_tooling.py index 3e48844e..1ed741e8 100644 --- a/tests/devx/test_tooling.py +++ b/tests/devx/test_tooling.py @@ -19,7 +19,13 @@ import signal from .conftest import ( expect, assert_before, - # in_prompt_msg, + in_prompt_msg, + PROMPT, + _pause_msg, +) +from pexpect.exceptions import ( + # TIMEOUT, + EOF, ) @@ -118,3 +124,50 @@ def test_shield_pause( "'--uid', \"('hanger',", ] ) + + +def test_breakpoint_hook_restored( + spawn, +): + ''' + Ensures our actor runtime sets a custom `breakpoint()` hook + on open then restores the stdlib's default on close. + + The hook state validation is done via `assert`s inside the + invoked script with only `breakpoint()` (not `tractor.pause()`) + calls used. + + ''' + child = spawn('restore_builtin_breakpoint') + + child.expect(PROMPT) + assert_before( + child, + [ + _pause_msg, + " Date: Tue, 25 Feb 2025 11:16:01 -0500 Subject: [PATCH 53/66] Tweak some test asserts to better `is` style --- tests/test_cancellation.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/test_cancellation.py b/tests/test_cancellation.py index 92540ed4..ece4d3c7 100644 --- a/tests/test_cancellation.py +++ b/tests/test_cancellation.py @@ -130,7 +130,7 @@ def test_multierror( try: await portal2.result() except tractor.RemoteActorError as err: - assert err.boxed_type == AssertionError + assert err.boxed_type is AssertionError print("Look Maa that first actor failed hard, hehh") raise @@ -182,7 +182,7 @@ def test_multierror_fast_nursery(reg_addr, start_method, num_subactors, delay): for exc in exceptions: assert isinstance(exc, tractor.RemoteActorError) - assert exc.boxed_type == AssertionError + assert exc.boxed_type is AssertionError async def do_nothing(): @@ -504,7 +504,9 @@ def test_cancel_via_SIGINT_other_task( if is_win(): # smh timeout += 1 - async def spawn_and_sleep_forever(task_status=trio.TASK_STATUS_IGNORED): + async def spawn_and_sleep_forever( + task_status=trio.TASK_STATUS_IGNORED + ): async with tractor.open_nursery() as tn: for i in range(3): await tn.run_in_actor( From 6de17a39498d1fca3f5813393c6876de6b8f3c81 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 25 Feb 2025 20:14:38 -0500 Subject: [PATCH 54/66] Unset `$PYTHON_COLORS` for test debugger suite.. Since obvi all our `pexpect` patterns aren't going to match with a heck-ton of terminal color escape sequences in the output XD --- tests/devx/conftest.py | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/tests/devx/conftest.py b/tests/devx/conftest.py index 8bc800ec..55aef48a 100644 --- a/tests/devx/conftest.py +++ b/tests/devx/conftest.py @@ -30,7 +30,7 @@ from ..conftest import ( @pytest.fixture def spawn( start_method, - testdir: pytest.Testdir, + testdir: pytest.Pytester, reg_addr: tuple[str, int], ) -> Callable[[str], None]: @@ -44,16 +44,32 @@ def spawn( '`pexpect` based tests only supported on `trio` backend' ) + def unset_colors(): + ''' + Python 3.13 introduced colored tracebacks that break patt + matching, + + https://docs.python.org/3/using/cmdline.html#envvar-PYTHON_COLORS + https://docs.python.org/3/using/cmdline.html#using-on-controlling-color + + ''' + import os + os.environ['PYTHON_COLORS'] = '0' + def _spawn( cmd: str, **mkcmd_kwargs, ): + unset_colors() return testdir.spawn( cmd=mk_cmd( cmd, **mkcmd_kwargs, ), expect_timeout=3, + # preexec_fn=unset_colors, + # ^TODO? get `pytest` core to expose underlying + # `pexpect.spawn()` stuff? ) # such that test-dep can pass input script name. From 182218a776e4f1bbcd64fe68684cefddc8d8d34f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 25 Feb 2025 20:15:59 -0500 Subject: [PATCH 55/66] Another `is` fix.. --- tests/test_inter_peer_cancellation.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/test_inter_peer_cancellation.py b/tests/test_inter_peer_cancellation.py index 7bf9a2bd..bac9a791 100644 --- a/tests/test_inter_peer_cancellation.py +++ b/tests/test_inter_peer_cancellation.py @@ -170,7 +170,7 @@ def test_do_not_swallow_error_before_started_by_remote_contextcancelled( trio.run(main) rae = excinfo.value - assert rae.boxed_type == TypeError + assert rae.boxed_type is TypeError @tractor.context From 639186aa379f8ca8f6bdafceabd1d8e686368a69 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 1 Mar 2025 21:25:05 -0500 Subject: [PATCH 56/66] Add per-side graceful-exit/cancel excs-as-signals Such that any combination of task terminations/exits can be explicitly handled and "dual side independent" crash cases re-raised in egs. The main error-or-exit impl changes include, - use of new per-side "signaling exceptions": - TrioTaskExited|TrioCancelled for signalling aio. - AsyncioTaskExited|AsyncioCancelled for signalling trio. - NOT overloading the `LinkedTaskChannel._trio/aio_err` fields for err-as-signal relay and instead add a new pair of `._trio/aio_to_raise` maybe-exc-attrs which allow each side's task to specify what it would want the other side to raise to signal its/a termination outcome: - `._trio_to_raise: AsyncioTaskExited|AsyncioCancelled` to signal, |_ the aio task having returned while the trio side was still reading from the `asyncio.Queue` or is just not `.done()`. |_ the aio task being self or trio-request cancelled where a `asyncio.CancelledError` is raised and caught but NOT relayed as is back to trio; instead signal a "more explicit" exc type. - `._aio_to_raise: TrioTaskExited|TrioCancelled` to signal, |_ the trio task having returned while the aio side was still reading from the mem chan and indicating that the trio side might not care any more about future streamed values (like the `Stop/EndOfChannel` equivs for ipc `Context`s). |_ when the trio task canceld we do a `asyncio.Future.set_exception(TrioTaskExited())` to indicate to the aio side verbosely that it should cancel due to the trio parent. - `_aio/trio_err` are now left to only capturing the **actual** per-side task excs for introspection / other side's handling logic. - supporting "graceful exits" depending on API in use from `translate_aio_errors()` such that if either side exits but the other side isn't expect to consume the final `return`ed value, we just exit silently, which required: - adding a `suppress_graceful_exits: bool` flag. - adjusting the `maybe_raise_aio_side_err()` logic to use that flag and suppress only on certain combos of `._trio_to_raise/._trio_err`. - prefer to raise `._trio_to_raise` when the aio-side is the src and vice versa. - filling out pedantic logging for cancellation cases indicating which side is the cause. - add a `LinkedTaskChannel._aio_result` modelled after our `Context._result` a a similar `.wait_for_result()` interface which allows maybe accessing the aio task's final return value if desired when using the `open_channel_from()` API. - rename `cancel_trio()` done handler -> `signal_trio_when_done()` Also some fairly major test suite updates, - add a `delay: int` producing fixture which delivers a much larger timeout whenever `debug_mode` is set so that the REPL can be used without a surrounding cancel firing. - add a new `test_aio_exits_early_relays_AsyncioTaskExited` including a paired `exit_early: bool` flag to `push_from_aio_task()`. - adjust `test_trio_closes_early_causes_aio_checkpoint_raise` to expect a `to_asyncio.TrioTaskExited`. --- tests/test_infected_asyncio.py | 237 ++++++++--- tractor/_exceptions.py | 52 ++- tractor/to_asyncio.py | 740 ++++++++++++++++++++++++--------- 3 files changed, 775 insertions(+), 254 deletions(-) diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index d462f59d..77877568 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -32,6 +32,17 @@ from tractor.trionics import BroadcastReceiver from tractor._testing import expect_ctxc +@pytest.fixture( + scope='module', + # autouse=True, +) +def delay(debug_mode: bool) -> int: + if debug_mode: + return 999 + else: + return 1 + + async def sleep_and_err( sleep_for: float = 0.1, @@ -59,20 +70,24 @@ async def trio_cancels_single_aio_task(): await tractor.to_asyncio.run_task(aio_sleep_forever) -def test_trio_cancels_aio_on_actor_side(reg_addr): +def test_trio_cancels_aio_on_actor_side( + reg_addr: tuple[str, int], + delay: int, +): ''' Spawn an infected actor that is cancelled by the ``trio`` side task using std cancel scope apis. ''' async def main(): - async with tractor.open_nursery( - registry_addrs=[reg_addr] - ) as n: - await n.run_in_actor( - trio_cancels_single_aio_task, - infect_asyncio=True, - ) + with trio.fail_after(1 + delay): + async with tractor.open_nursery( + registry_addrs=[reg_addr] + ) as n: + await n.run_in_actor( + trio_cancels_single_aio_task, + infect_asyncio=True, + ) trio.run(main) @@ -116,7 +131,9 @@ async def asyncio_actor( raise -def test_aio_simple_error(reg_addr): +def test_aio_simple_error( + reg_addr: tuple[str, int], +): ''' Verify a simple remote asyncio error propagates back through trio to the parent actor. @@ -153,7 +170,9 @@ def test_aio_simple_error(reg_addr): assert err.boxed_type is AssertionError -def test_tractor_cancels_aio(reg_addr): +def test_tractor_cancels_aio( + reg_addr: tuple[str, int], +): ''' Verify we can cancel a spawned asyncio task gracefully. @@ -172,7 +191,9 @@ def test_tractor_cancels_aio(reg_addr): trio.run(main) -def test_trio_cancels_aio(reg_addr): +def test_trio_cancels_aio( + reg_addr: tuple[str, int], +): ''' Much like the above test with ``tractor.Portal.cancel_actor()`` except we just use a standard ``trio`` cancellation api. @@ -203,7 +224,8 @@ async def trio_ctx( # this will block until the ``asyncio`` task sends a "first" # message. - with trio.fail_after(2): + delay: int = 999 if tractor.debug_mode() else 1 + with trio.fail_after(1 + delay): try: async with ( trio.open_nursery( @@ -239,7 +261,8 @@ async def trio_ctx( ids='parent_actor_cancels_child={}'.format ) def test_context_spawns_aio_task_that_errors( - reg_addr, + reg_addr: tuple[str, int], + delay: int, parent_cancels: bool, ): ''' @@ -249,7 +272,7 @@ def test_context_spawns_aio_task_that_errors( ''' async def main(): - with trio.fail_after(2): + with trio.fail_after(1 + delay): async with tractor.open_nursery() as n: p = await n.start_actor( 'aio_daemon', @@ -322,11 +345,12 @@ async def aio_cancel(): def test_aio_cancelled_from_aio_causes_trio_cancelled( reg_addr: tuple, + delay: int, ): ''' - When the `asyncio.Task` cancels itself the `trio` side cshould + When the `asyncio.Task` cancels itself the `trio` side should also cancel and teardown and relay the cancellation cross-process - to the caller (parent). + to the parent caller. ''' async def main(): @@ -342,7 +366,7 @@ def test_aio_cancelled_from_aio_causes_trio_cancelled( # NOTE: normally the `an.__aexit__()` waits on the # portal's result but we do it explicitly here # to avoid indent levels. - with trio.fail_after(1): + with trio.fail_after(1 + delay): await p.wait_for_result() with pytest.raises( @@ -353,11 +377,10 @@ def test_aio_cancelled_from_aio_causes_trio_cancelled( # might get multiple `trio.Cancelled`s as well inside an inception err: RemoteActorError|ExceptionGroup = excinfo.value if isinstance(err, ExceptionGroup): - err = next(itertools.dropwhile( - lambda exc: not isinstance(exc, tractor.RemoteActorError), - err.exceptions - )) - assert err + excs = err.exceptions + assert len(excs) == 1 + final_exc = excs[0] + assert isinstance(final_exc, tractor.RemoteActorError) # relayed boxed error should be our `trio`-task's # cancel-signal-proxy-equivalent of `asyncio.CancelledError`. @@ -370,15 +393,18 @@ async def no_to_trio_in_args(): async def push_from_aio_task( - sequence: Iterable, to_trio: trio.abc.SendChannel, expect_cancel: False, fail_early: bool, + exit_early: bool, ) -> None: try: + # print('trying breakpoint') + # breakpoint() + # sync caller ctx manager to_trio.send_nowait(True) @@ -387,10 +413,27 @@ async def push_from_aio_task( to_trio.send_nowait(i) await asyncio.sleep(0.001) - if i == 50 and fail_early: - raise Exception + if ( + i == 50 + ): + if fail_early: + print('Raising exc from aio side!') + raise Exception - print('asyncio streamer complete!') + if exit_early: + # TODO? really you could enforce the same + # SC-proto we use for actors here with asyncio + # such that a Return[None] msg would be + # implicitly delivered to the trio side? + # + # XXX => this might be the end-all soln for + # converting any-inter-task system (regardless + # of maybe-remote runtime or language) to be + # SC-compat no? + print(f'asyncio breaking early @ {i!r}') + break + + print('asyncio streaming complete!') except asyncio.CancelledError: if not expect_cancel: @@ -402,9 +445,10 @@ async def push_from_aio_task( async def stream_from_aio( - exit_early: bool = False, - raise_err: bool = False, + trio_exit_early: bool = False, + trio_raise_err: bool = False, aio_raise_err: bool = False, + aio_exit_early: bool = False, fan_out: bool = False, ) -> None: @@ -417,8 +461,17 @@ async def stream_from_aio( async with to_asyncio.open_channel_from( push_from_aio_task, sequence=seq, - expect_cancel=raise_err or exit_early, + expect_cancel=trio_raise_err or trio_exit_early, fail_early=aio_raise_err, + exit_early=aio_exit_early, + + # such that we can test exit early cases + # for each side explicitly. + suppress_graceful_exits=(not( + aio_exit_early + or + trio_exit_early + )) ) as (first, chan): @@ -435,9 +488,9 @@ async def stream_from_aio( pulled.append(value) if value == 50: - if raise_err: + if trio_raise_err: raise Exception - elif exit_early: + elif trio_exit_early: print('`consume()` breaking early!\n') break @@ -471,10 +524,14 @@ async def stream_from_aio( finally: - if ( - not raise_err and - not exit_early and - not aio_raise_err + if not ( + trio_raise_err + or + trio_exit_early + or + aio_raise_err + or + aio_exit_early ): if fan_out: # we get double the pulled values in the @@ -484,6 +541,7 @@ async def stream_from_aio( assert list(sorted(pulled)) == expect else: + # await tractor.pause() assert pulled == expect else: assert not fan_out @@ -497,7 +555,10 @@ async def stream_from_aio( 'fan_out', [False, True], ids='fan_out_w_chan_subscribe={}'.format ) -def test_basic_interloop_channel_stream(reg_addr, fan_out): +def test_basic_interloop_channel_stream( + reg_addr: tuple[str, int], + fan_out: bool, +): async def main(): async with tractor.open_nursery() as n: portal = await n.run_in_actor( @@ -517,7 +578,7 @@ def test_trio_error_cancels_intertask_chan(reg_addr): async with tractor.open_nursery() as n: portal = await n.run_in_actor( stream_from_aio, - raise_err=True, + trio_raise_err=True, infect_asyncio=True, ) # should trigger remote actor error @@ -530,42 +591,114 @@ def test_trio_error_cancels_intertask_chan(reg_addr): excinfo.value.boxed_type is Exception -def test_trio_closes_early_and_channel_exits( +def test_trio_closes_early_causes_aio_checkpoint_raise( reg_addr: tuple[str, int], + delay: int, ): ''' - Check that if the `trio`-task "exits early" on `async for`ing the - inter-task-channel (via a `break`) we exit silently from the - `open_channel_from()` block and get a final `Return[None]` msg. + Check that if the `trio`-task "exits early and silently" (in this + case during `async for`-ing the inter-task-channel via + a `break`-from-loop), we raise `TrioTaskExited` on the + `asyncio`-side which also then bubbles up through the + `open_channel_from()` block indicating that the `asyncio.Task` + hit a ran another checkpoint despite the `trio.Task` exit. ''' async def main(): - with trio.fail_after(2): + with trio.fail_after(1 + delay): async with tractor.open_nursery( # debug_mode=True, # enable_stack_on_sig=True, ) as n: portal = await n.run_in_actor( stream_from_aio, - exit_early=True, + trio_exit_early=True, infect_asyncio=True, ) # should raise RAE diectly print('waiting on final infected subactor result..') res: None = await portal.wait_for_result() assert res is None - print('infected subactor returned result: {res!r}\n') + print(f'infected subactor returned result: {res!r}\n') # should be a quiet exit on a simple channel exit - trio.run( - main, - # strict_exception_groups=False, - ) + with pytest.raises(RemoteActorError) as excinfo: + trio.run(main) + + # ensure remote error is an explicit `AsyncioCancelled` sub-type + # which indicates to the aio task that the trio side exited + # silently WITHOUT raising a `trio.Cancelled` (which would + # normally be raised instead as a `AsyncioCancelled`). + excinfo.value.boxed_type is to_asyncio.TrioTaskExited -def test_aio_errors_and_channel_propagates_and_closes(reg_addr): +def test_aio_exits_early_relays_AsyncioTaskExited( + # TODO, parametrize the 3 possible trio side conditions: + # - trio blocking on receive, aio exits early + # - trio cancelled AND aio exits early on its next tick + # - trio errors AND aio exits early on its next tick + reg_addr: tuple[str, int], + debug_mode: bool, + delay: int, +): + ''' + Check that if the `asyncio`-task "exits early and silently" (in this + case during `push_from_aio_task()` pushing to the `InterLoopTaskChannel` + it `break`s from the loop), we raise `AsyncioTaskExited` on the + `trio`-side which then DOES NOT BUBBLE up through the + `open_channel_from()` block UNLESS, + + - the trio.Task also errored/cancelled, in which case we wrap + both errors in an eg + - the trio.Task was blocking on rxing a value from the + `InterLoopTaskChannel`. + + ''' async def main(): - async with tractor.open_nursery() as n: + with trio.fail_after(1 + delay): + async with tractor.open_nursery( + debug_mode=debug_mode, + # enable_stack_on_sig=True, + ) as an: + portal = await an.run_in_actor( + stream_from_aio, + infect_asyncio=True, + trio_exit_early=False, + aio_exit_early=True, + ) + # should raise RAE diectly + print('waiting on final infected subactor result..') + res: None = await portal.wait_for_result() + assert res is None + print(f'infected subactor returned result: {res!r}\n') + + # should be a quiet exit on a simple channel exit + with pytest.raises(RemoteActorError) as excinfo: + trio.run(main) + + exc = excinfo.value + + # TODO, wow bug! + # -[ ] bp handler not replaced!?!? + # breakpoint() + + # import pdbp; pdbp.set_trace() + + # ensure remote error is an explicit `AsyncioCancelled` sub-type + # which indicates to the aio task that the trio side exited + # silently WITHOUT raising a `trio.Cancelled` (which would + # normally be raised instead as a `AsyncioCancelled`). + assert exc.boxed_type is to_asyncio.AsyncioTaskExited + + +def test_aio_errors_and_channel_propagates_and_closes( + reg_addr: tuple[str, int], + debug_mode: bool, +): + async def main(): + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as n: portal = await n.run_in_actor( stream_from_aio, aio_raise_err=True, @@ -852,6 +985,8 @@ def test_sigint_closes_lifetime_stack( ''' async def main(): + + delay = 999 if tractor.debug_mode() else 1 try: an: tractor.ActorNursery async with tractor.open_nursery( @@ -902,7 +1037,7 @@ def test_sigint_closes_lifetime_stack( if wait_for_ctx: print('waiting for ctx outcome in parent..') try: - with trio.fail_after(1): + with trio.fail_after(1 + delay): await ctx.wait_for_result() except tractor.ContextCancelled as ctxc: assert ctxc.canceller == ctx.chan.uid diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 89ea21ad..db863359 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -82,6 +82,39 @@ class InternalError(RuntimeError): ''' +class AsyncioCancelled(Exception): + ''' + Asyncio cancelled translation (non-base) error + for use with the ``to_asyncio`` module + to be raised in the ``trio`` side task + + NOTE: this should NOT inherit from `asyncio.CancelledError` or + tests should break! + + ''' + + +class AsyncioTaskExited(Exception): + ''' + asyncio.Task "exited" translation error for use with the + `to_asyncio` APIs to be raised in the `trio` side task indicating + on `.run_task()`/`.open_channel_from()` exit that the aio side + exited early/silently. + + ''' + +class TrioTaskExited(AsyncioCancelled): + ''' + The `trio`-side task exited without explicitly cancelling the + `asyncio.Task` peer. + + This is very similar to how `trio.ClosedResource` acts as + a "clean shutdown" signal to the consumer side of a mem-chan, + + https://trio.readthedocs.io/en/stable/reference-core.html#clean-shutdown-with-channels + + ''' + # NOTE: more or less should be close to these: # 'boxed_type', @@ -127,8 +160,8 @@ _body_fields: list[str] = list( def get_err_type(type_name: str) -> BaseException|None: ''' - Look up an exception type by name from the set of locally - known namespaces: + Look up an exception type by name from the set of locally known + namespaces: - `builtins` - `tractor._exceptions` @@ -358,6 +391,13 @@ class RemoteActorError(Exception): self._ipc_msg.src_type_str ) + if not self._src_type: + raise TypeError( + f'Failed to lookup src error type with ' + f'`tractor._exceptions.get_err_type()` :\n' + f'{self.src_type_str}' + ) + return self._src_type @property @@ -652,16 +692,10 @@ class RemoteActorError(Exception): failing actor's remote env. ''' - src_type_ref: Type[BaseException] = self.src_type - if not src_type_ref: - raise TypeError( - 'Failed to lookup src error type:\n' - f'{self.src_type_str}' - ) - # TODO: better tb insertion and all the fancier dunder # metadata stuff as per `.__context__` etc. and friends: # https://github.com/python-trio/trio/issues/611 + src_type_ref: Type[BaseException] = self.src_type return src_type_ref(self.tb_str) # TODO: local recontruction of nested inception for a given diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index 75dfb5cb..f65cc7ef 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -20,7 +20,12 @@ Infection apis for ``asyncio`` loops running ``trio`` using guest mode. ''' from __future__ import annotations import asyncio -from asyncio.exceptions import CancelledError +from asyncio.exceptions import ( + CancelledError, +) +from asyncio import ( + QueueShutDown, +) from contextlib import asynccontextmanager as acm from dataclasses import dataclass import inspect @@ -34,12 +39,18 @@ from typing import ( import tractor from tractor._exceptions import ( + InternalError, is_multi_cancelled, + TrioTaskExited, + TrioCancelled, + AsyncioTaskExited, + AsyncioCancelled, ) from tractor._state import ( debug_mode, _runtime_vars, ) +from tractor._context import Unresolved from tractor.devx import _debug from tractor.log import ( get_logger, @@ -69,6 +80,21 @@ __all__ = [ ] +# TODO, generally speaking we can generalize this abstraction, a "SC linked +# parent->child task pair", as the same "supervision scope primitive" +# **that is** our `._context.Context` with the only difference being +# in how the tasks conduct msg-passing comms. +# +# For `LinkedTaskChannel` we are passing the equivalent of (once you +# include all the recently added `._trio/aio_to_raise` +# exd-as-signals) our SC-dialog-proto over each asyncIO framework's +# mem-chan impl, +# +# verus in `Context` +# +# We are doing the same thing but msg-passing comms happens over an +# IPC transport between tasks in different memory domains. + @dataclass class LinkedTaskChannel( trio.abc.Channel, @@ -84,18 +110,85 @@ class LinkedTaskChannel( ''' _to_aio: asyncio.Queue _from_aio: trio.MemoryReceiveChannel + _to_trio: trio.MemorySendChannel _trio_cs: trio.CancelScope _trio_task: trio.Task _aio_task_complete: trio.Event + _suppress_graceful_exits: bool = True + _trio_err: BaseException|None = None + _trio_to_raise: ( + AsyncioTaskExited| # aio task exits while trio ongoing + AsyncioCancelled| # aio task is (self-)cancelled + BaseException| + None + ) = None _trio_exited: bool = False - # set after ``asyncio.create_task()`` - # _aio_first: Any|None = None + # set after `asyncio.create_task()` _aio_task: asyncio.Task|None = None _aio_err: BaseException|None = None + _aio_to_raise: ( + TrioTaskExited| # trio task exits while aio ongoing + BaseException| + None + ) = None + # _aio_first: Any|None = None # TODO? + _aio_result: Any|Unresolved = Unresolved + + def _final_result_is_set(self) -> bool: + return self._aio_result is not Unresolved + + # TODO? equiv from `Context`? + # @property + # def has_outcome(self) -> bool: + # return ( + # bool(self.maybe_error) + # or + # self._final_result_is_set() + # ) + + async def wait_for_result( + self, + hide_tb: bool = True, + + ) -> Any: + ''' + Wait for the `asyncio.Task.result()` from `trio` + + ''' + __tracebackhide__: bool = hide_tb + assert self._portal, ( + '`Context.wait_for_result()` can not be called from callee side!' + ) + if self._final_result_is_set(): + return self._aio_result + + async with translate_aio_errors( + chan=self, + wait_aio_task=False, + ): + await self._aio_task_complete.wait() + + if ( + not self._final_result_is_set() + ): + if (trio_to_raise := self._trio_to_raise): + raise trio_to_raise from self._aio_err + + elif aio_err := self._aio_err: + raise aio_err + + else: + raise InternalError( + f'Asyncio-task has no result or error set !?\n' + f'{self._aio_task}' + ) + + return self._aio_result + _broadcaster: BroadcastReceiver|None = None async def aclose(self) -> None: @@ -137,7 +230,9 @@ class LinkedTaskChannel( return await self._from_aio.receive() except BaseException as err: async with translate_aio_errors( - self, + chan=self, + # NOTE, determined by `open_channel_from()` input arg + suppress_graceful_exits=self._suppress_graceful_exits, # XXX: obviously this will deadlock if an on-going stream is # being procesed. @@ -154,8 +249,9 @@ class LinkedTaskChannel( ''' self._to_aio.put_nowait(item) - async def wait_aio_complete(self) -> None: - await self._aio_task_complete.wait() + # TODO? needed? + # async def wait_aio_complete(self) -> None: + # await self._aio_task_complete.wait() def cancel_asyncio_task( self, @@ -208,6 +304,7 @@ def _run_asyncio_task( *, qsize: int = 1, provide_channels: bool = False, + suppress_graceful_exits: bool = True, hide_tb: bool = False, **kwargs, @@ -260,6 +357,7 @@ def _run_asyncio_task( _trio_cs=trio_cs, _trio_task=trio_task, _aio_task_complete=aio_task_complete, + _suppress_graceful_exits=suppress_graceful_exits, ) async def wait_on_coro_final_result( @@ -269,17 +367,16 @@ def _run_asyncio_task( ) -> None: ''' - Await `coro` and relay result back to `trio`. - - This can only be run as an `asyncio.Task`! + Await input `coro` as/in an `asyncio.Task` and deliver final + `return`-ed result back to `trio`. ''' - nonlocal aio_err nonlocal chan orig = result = id(coro) try: - result = await coro + result: Any = await coro + chan._aio_result = result except BaseException as aio_err: chan._aio_err = aio_err if isinstance(aio_err, CancelledError): @@ -291,7 +388,6 @@ def _run_asyncio_task( '`asyncio` task errored\n' ) raise - else: if ( result != orig @@ -306,22 +402,46 @@ def _run_asyncio_task( to_trio.send_nowait(result) finally: - # breakpoint() - # import pdbp; pdbp.set_trace() - # if the task was spawned using `open_channel_from()` # then we close the channels on exit. if provide_channels: + # breakpoint() # TODO! why no work!? + # import pdbp; pdbp.set_trace() + + # IFF there is a blocked trio waiter, we set the + # aio-side error to be an explicit "exited early" + # (much like a `Return` in our SC IPC proto) for the + # `.open_channel_from()` case where the parent trio + # task might not wait directly for a final returned + # result (i.e. the trio side might be waiting on + # a streamed value) - this is a signal that the + # asyncio.Task has returned early! + # + # TODO, solve other cases where trio side might, + # - raise Cancelled but aio side exits on next tick. + # - raise error but aio side exits on next tick. + # - raise error and aio side errors "independently" + # on next tick (SEE draft HANDLER BELOW). + stats: trio.MemoryChannelStatistics = to_trio.statistics() + if ( + stats.tasks_waiting_receive + and + not chan._aio_err + ): + chan._trio_to_raise = AsyncioTaskExited( + f'Task existed with final result\n' + f'{result!r}\n' + ) + # only close the sender side which will relay - # a ``trio.EndOfChannel`` to the trio (consumer) side. + # a `trio.EndOfChannel` to the trio (consumer) side. to_trio.close() aio_task_complete.set() - # await asyncio.sleep(0.1) - log.info( - f'`asyncio` task terminated\n' - f'x)>\n' - f' |_{task}\n' + log.runtime( + f'`asyncio` task completed\n' + f')>\n' + f' |_{task}\n' ) # start the asyncio task we submitted from trio @@ -331,6 +451,7 @@ def _run_asyncio_task( f'{coro!r}' ) + # schedule the (bg) `asyncio.Task` task: asyncio.Task = asyncio.create_task( wait_on_coro_final_result( to_trio, @@ -359,16 +480,36 @@ def _run_asyncio_task( ) greenback.bestow_portal(task) - def cancel_trio( + def signal_trio_when_done( task: asyncio.Task, ) -> None: ''' - Cancel the parent `trio` task on any error raised by the - `asyncio` side. + Maybe-cancel, relay-and-raise an error to, OR pack a final + `return`-value for the parent (in SC terms) `trio.Task` on + completion of the `asyncio.Task`. + + Note for certain "edge" scheduling-race-conditions we allow + the aio side to dictate dedicated `tractor`-defined excs to + be raised in the `trio` parent task; the intention is to + indicate those races in a VERY pedantic manner! ''' nonlocal chan - relayed_aio_err: BaseException|None = chan._aio_err + trio_err: BaseException|None = chan._trio_err + + # XXX, since the original error we read from the asyncio.Task + # might change between BEFORE and AFTER we here call + # `asyncio.Task.result()` + # + # -> THIS is DUE TO US in `translate_aio_errors()`! + # + # => for example we might set a special exc + # (`AsyncioCancelled|AsyncioTaskExited`) meant to be raised + # in trio (and maybe absorbed depending on the called API) + # BEFORE this done-callback is invoked by `asyncio`'s + # runtime. + trio_to_raise: BaseException|None = chan._trio_to_raise + orig_aio_err: BaseException|None = chan._aio_err aio_err: BaseException|None = None # only to avoid `asyncio` complaining about uncaptured @@ -376,24 +517,45 @@ def _run_asyncio_task( try: res: Any = task.result() log.info( - '`trio` received final result from {task}\n' - f'|_{res}\n' + f'`trio` received final result from `asyncio` task,\n' + f')> {res}\n' + f' |_{task}\n' ) + if not chan._aio_result: + chan._aio_result = res + + # ?TODO, should we also raise `AsyncioTaskExited[res]` + # in any case where trio is NOT blocking on the + # `._to_trio` chan? + # + # -> ?NO RIGHT? since the + # `open_channel_from().__aexit__()` should detect this + # and then set any final `res` from above as a field + # that can optionally be read by the trio-paren-task as + # needed (just like in our + # `Context.wait_for_result()/.result` API yah? + # + # if provide_channels: + except BaseException as _aio_err: aio_err: BaseException = _aio_err - # read again AFTER the `asyncio` side errors in case + + # READ AGAIN, AFTER the `asyncio` side errors, in case # it was cancelled due to an error from `trio` (or # some other out of band exc) and then set to something # else? - relayed_aio_err: BaseException|None = chan._aio_err + curr_aio_err: BaseException|None = chan._aio_err # always true right? assert ( - type(_aio_err) is type(relayed_aio_err) + type(aio_err) + is type(orig_aio_err) + is type(curr_aio_err) ), ( f'`asyncio`-side task errors mismatch?!?\n\n' f'(caught) aio_err: {aio_err}\n' - f'chan._aio_err: {relayed_aio_err}\n' + f'ORIG chan._aio_err: {orig_aio_err}\n' + f'chan._aio_err: {curr_aio_err}\n' ) msg: str = ( @@ -401,7 +563,7 @@ def _run_asyncio_task( '{etype_str}\n' # ^NOTE filled in below ) - if isinstance(_aio_err, CancelledError): + if isinstance(aio_err, CancelledError): msg += ( f'c)>\n' f' |_{task}\n' @@ -409,6 +571,28 @@ def _run_asyncio_task( log.cancel( msg.format(etype_str='cancelled') ) + + # XXX when the asyncio.Task exits early (before the trio + # side) we relay through an exc-as-signal which is + # normally suppressed unless the trio.Task also errors + # + # ?TODO, is this even needed (does it happen) now? + elif isinstance(aio_err, QueueShutDown): + # import pdbp; pdbp.set_trace() + trio_err = AsyncioTaskExited( + 'Task exited before `trio` side' + ) + if not chan._trio_err: + chan._trio_err = trio_err + + msg += ( + f')>\n' + f' |_{task}\n' + ) + log.info( + msg.format(etype_str='exited') + ) + else: msg += ( f'x)>\n' @@ -418,13 +602,20 @@ def _run_asyncio_task( msg.format(etype_str='errored') ) + # is trio the src of the aio task's exc-as-outcome? trio_err: BaseException|None = chan._trio_err + curr_aio_err: BaseException|None = chan._aio_err if ( - relayed_aio_err + curr_aio_err or trio_err + or + trio_to_raise ): - # import pdbp; pdbp.set_trace() + # XXX, if not already, ALWAYs cancel the trio-side on an + # aio-side error or early return. In the case where the trio task is + # blocking on a checkpoint or `asyncio.Queue.get()`. + # NOTE: currently mem chan closure may act as a form # of error relay (at least in the `asyncio.CancelledError` # case) since we have no way to directly trigger a `trio` @@ -432,30 +623,18 @@ def _run_asyncio_task( # We might want to change this in the future though. from_aio.close() - # wait, wut? - # aio_err.with_traceback(aio_err.__traceback__) - - # TODO: show when cancellation originated - # from each side more pedantically in log-msg? - # elif ( - # type(aio_err) is CancelledError - # and # trio was the cause? - # trio_cs.cancel_called - # ): - # log.cancel( - # 'infected task was cancelled by `trio`-side' - # ) - # raise aio_err from task_err - - # XXX: if not already, alway cancel the scope on a task - # error in case the trio task is blocking on - # a checkpoint. if ( not trio_cs.cancelled_caught or not trio_cs.cancel_called ): - # import pdbp; pdbp.set_trace() + log.cancel( + f'Cancelling `trio` side due to aio-side src exc\n' + f'{curr_aio_err}\n' + f'\n' + f'(c>\n' + f' |_{trio_task}\n' + ) trio_cs.cancel() # maybe the `trio` task errored independent from the @@ -478,28 +657,36 @@ def _run_asyncio_task( # for reproducing detailed edge cases as per the above # cases. # + trio_to_raise: AsyncioCancelled|AsyncioTaskExited = chan._trio_to_raise + aio_to_raise: TrioTaskExited|TrioCancelled = chan._aio_to_raise if ( + not chan._aio_result + and not trio_cs.cancelled_caught - and - (trio_err := chan._trio_err) - and - type(trio_err) not in { - trio.Cancelled, - } and ( - aio_err - and - type(aio_err) not in { + (aio_err and type(aio_err) not in { asyncio.CancelledError - } + }) + or + aio_to_raise + ) + and ( + ((trio_err := chan._trio_err) and type(trio_err) not in { + trio.Cancelled, + }) + or + trio_to_raise ) ): eg = ExceptionGroup( 'Both the `trio` and `asyncio` tasks errored independently!!\n', - (trio_err, aio_err), + ( + trio_to_raise or trio_err, + aio_to_raise or aio_err, + ), ) - chan._trio_err = eg - chan._aio_err = eg + # chan._trio_err = eg + # chan._aio_err = eg raise eg elif aio_err: @@ -507,45 +694,34 @@ def _run_asyncio_task( # match the one we just caught from the task above! # (that would indicate something weird/very-wrong # going on?) - if aio_err is not relayed_aio_err: - raise aio_err from relayed_aio_err + if ( + aio_err is not trio_to_raise + and ( + not suppress_graceful_exits + and ( + chan._aio_result is not Unresolved + and + isinstance(trio_to_raise, AsyncioTaskExited) + ) + ) + ): + # raise aio_err from relayed_aio_err + raise trio_to_raise from curr_aio_err raise aio_err - task.add_done_callback(cancel_trio) + task.add_done_callback(signal_trio_when_done) return chan -class AsyncioCancelled(Exception): - ''' - Asyncio cancelled translation (non-base) error - for use with the ``to_asyncio`` module - to be raised in the ``trio`` side task - - NOTE: this should NOT inherit from `asyncio.CancelledError` or - tests should break! - - ''' - - -class TrioTaskExited(AsyncioCancelled): - ''' - The `trio`-side task exited without explicitly cancelling the - `asyncio.Task` peer. - - This is very similar to how `trio.ClosedResource` acts as - a "clean shutdown" signal to the consumer side of a mem-chan, - - https://trio.readthedocs.io/en/stable/reference-core.html#clean-shutdown-with-channels - - ''' - - @acm async def translate_aio_errors( chan: LinkedTaskChannel, wait_on_aio_task: bool = False, cancel_aio_task_on_trio_exit: bool = True, + suppress_graceful_exits: bool = True, + + hide_tb: bool = True, ) -> AsyncIterator[None]: ''' @@ -558,17 +734,20 @@ async def translate_aio_errors( appropriately translates errors and cancels into ``trio`` land. ''' + __tracebackhide__: bool = hide_tb + trio_task = trio.lowlevel.current_task() - - aio_err: BaseException|None = None - + aio_err: BaseException|None = chan._aio_err aio_task: asyncio.Task = chan._aio_task + aio_done_before_trio: bool = aio_task.done() assert aio_task trio_err: BaseException|None = None + to_raise_trio: BaseException|None = None try: yield # back to one of the cross-loop apis except trio.Cancelled as taskc: trio_err = taskc + chan._trio_err = trio_err # should NEVER be the case that `trio` is cancel-handling # BEFORE the other side's task-ref was set!? @@ -577,12 +756,12 @@ async def translate_aio_errors( # import pdbp; pdbp.set_trace() # lolevel-debug # relay cancel through to called `asyncio` task - chan._aio_err = AsyncioCancelled( + chan._aio_to_raise = TrioCancelled( f'trio`-side cancelled the `asyncio`-side,\n' f'c)>\n' - f' |_{trio_task}\n\n' - - f'{trio_err!r}\n' + f' |_{trio_task}\n' + f'\n' + f'trio src exc: {trio_err!r}\n' ) # XXX NOTE XXX seems like we can get all sorts of unreliable @@ -595,22 +774,32 @@ async def translate_aio_errors( # ) # raise + # XXX always passthrough EoC since this translator is often + # called from `LinkedTaskChannel.receive()` which we want + # passthrough and further we have no special meaning for it in + # terms of relaying errors or signals from the aio side! + except trio.EndOfChannel: + raise + # NOTE ALSO SEE the matching note in the `cancel_trio()` asyncio # task-done-callback. + # + # when the aio side is (possibly self-)cancelled it will close + # the `chan._to_trio` and thus trigger the trio side to raise + # a dedicated `AsyncioCancelled` except ( trio.ClosedResourceError, - # trio.BrokenResourceError, ) as cre: - trio_err = cre + chan._trio_err = cre aio_err = chan._aio_err - # import pdbp; pdbp.set_trace() # XXX if an underlying `asyncio.CancelledError` triggered # this channel close, raise our (non-`BaseException`) wrapper # exception (`AsyncioCancelled`) from that source error. if ( # aio-side is cancelled? - aio_task.cancelled() # not set until it terminates?? + # |_ first not set until it terminates?? + aio_task.cancelled() and type(aio_err) is CancelledError @@ -618,32 +807,26 @@ async def translate_aio_errors( # silent-exit-by-`trio` case? # -[ ] the parent task can also just catch it though? # -[ ] OR, offer a `signal_aio_side_on_exit=True` ?? - # - # or - # aio_err is None - # and - # chan._trio_exited - ): - raise AsyncioCancelled( + # await tractor.pause(shield=True) + chan._trio_to_raise = AsyncioCancelled( f'asyncio`-side cancelled the `trio`-side,\n' f'c(>\n' f' |_{aio_task}\n\n' - f'{trio_err!r}\n' - ) from aio_err + f'(triggered on the `trio`-side by a {cre!r})\n' + ) + # TODO?? needed or does this just get reraised in the + # `finally:` block below? + # raise to_raise_trio from aio_err # maybe the chan-closure is due to something else? else: - raise + raise cre except BaseException as _trio_err: - # await tractor.pause(shield=True) - trio_err = _trio_err - log.exception( - '`trio`-side task errored?' - ) - + trio_err = chan._trio_err = trio_err + # await tractor.pause(shield=True) # workx! entered: bool = await _debug._maybe_enter_pm( trio_err, api_frame=inspect.currentframe(), @@ -653,89 +836,177 @@ async def translate_aio_errors( and not is_multi_cancelled(trio_err) ): - log.exception('actor crashed\n') + log.exception( + '`trio`-side task errored?' + ) + # __tracebackhide__: bool = False - aio_taskc = AsyncioCancelled( - f'`trio`-side task errored!\n' - f'{trio_err}' - ) #from trio_err + # TODO, just a log msg here indicating the scope closed + # and that the trio-side expects that and what the final + # result from the aio side was? + # + # if isinstance(chan._aio_err, AsyncioTaskExited): + # await tractor.pause(shield=True) - try: - aio_task.set_exception(aio_taskc) - except ( - asyncio.InvalidStateError, - RuntimeError, - # ^XXX, uhh bc apparently we can't use `.set_exception()` - # any more XD .. ?? + # if aio side is still active cancel it due to the trio-side + # error! + # ?TODO, mk `AsyncioCancelled[typeof(trio_err)]` embed the + # current exc? + if ( + # not aio_task.cancelled() + # and + not aio_task.done() # TODO? only need this one? + + # XXX LOL, so if it's not set it's an error !? + # yet another good jerb by `ascyncio`.. + # and + # not aio_task.exception() ): + aio_taskc = TrioCancelled( + f'The `trio`-side task crashed!\n' + f'{trio_err}' + ) + aio_task.set_exception(aio_taskc) wait_on_aio_task = False - - # import pdbp; pdbp.set_trace() - # raise aio_taskc from trio_err + # try: + # aio_task.set_exception(aio_taskc) + # except ( + # asyncio.InvalidStateError, + # RuntimeError, + # # ^XXX, uhh bc apparently we can't use `.set_exception()` + # # any more XD .. ?? + # ): + # wait_on_aio_task = False finally: # record wtv `trio`-side error transpired - chan._trio_err = trio_err - ya_trio_exited: bool = chan._trio_exited + if trio_err: + if chan._trio_err is not trio_err: + await tractor.pause(shield=True) - # NOTE! by default always cancel the `asyncio` task if + # assert chan._trio_err is trio_err + + ya_trio_exited: bool = chan._trio_exited + graceful_trio_exit: bool = ( + ya_trio_exited + and + not chan._trio_err # XXX CRITICAL, `asyncio.Task.cancel()` is cucked man.. + ) + + # XXX NOTE! XXX by default always cancel the `asyncio` task if # we've made it this far and it's not done. # TODO, how to detect if there's an out-of-band error that # caused the exit? if ( - cancel_aio_task_on_trio_exit - and not aio_task.done() - and - aio_err + and ( + cancel_aio_task_on_trio_exit + # and + # chan._aio_err # TODO, if it's not .done() is this possible? - # or the trio side has exited it's surrounding cancel scope - # indicating the lifetime of the ``asyncio``-side task - # should also be terminated. - or ( - ya_trio_exited - and - not chan._trio_err # XXX CRITICAL, `asyncio.Task.cancel()` is cucked man.. + # did the `.open_channel_from()` parent caller already + # (gracefully) exit scope before this translator was + # invoked? + # => since we couple the lifetime of the `asyncio.Task` + # to the `trio` parent task, it should should also be + # terminated via either, + # + # 1. raising an explicit `TrioTaskExited|TrioCancelled` + # in task via `asyncio.Task._fut_waiter.set_exception()` + # + # 2. or (worst case) by cancelling the aio task using + # the std-but-never-working `asyncio.Task.cancel()` + # (which i can't figure out why that nor + # `Task.set_exception()` seem to never ever do the + # rignt thing! XD). + or + graceful_trio_exit ) ): report: str = ( 'trio-side exited silently!' ) - assert not aio_err, 'WTF how did asyncio do this?!' + assert not chan._aio_err, ( + 'WTF why duz asyncio have err but not dun?!' + ) - # if the `trio.Task` already exited the `open_channel_from()` - # block we ensure the asyncio-side gets signalled via an - # explicit exception and its `Queue` is shutdown. + # if the `trio.Task` terminated without raising + # `trio.Cancelled` (curently handled above) there's + # 2 posibilities, + # + # i. it raised a `trio_err` + # ii. it did a "silent exit" where the + # `open_channel_from().__aexit__()` phase ran without + # any raise or taskc (task cancel) and no final result + # was collected (yet) from the aio side. + # + # SO, ensure the asyncio-side is notified and terminated + # by a dedicated exc-as-signal which distinguishes + # various aio-task-state at termination cases. + # + # Consequently if the aio task doesn't absorb said + # exc-as-signal, the trio side should then see the same exc + # propagate up through the .open_channel_from() call to + # the parent task. + # + # if the `trio.Task` already exited (only can happen for + # the `open_channel_from()` use case) block due to to + # either plain ol' graceful `__aexit__()` or due to taskc + # or an error, we ensure the aio-side gets signalled via + # an explicit exception and its `Queue` is shutdown. if ya_trio_exited: + # raise `QueueShutDown` on next `Queue.get()` call on + # aio side. chan._to_aio.shutdown() - # pump the other side's task? needed? + # pump this event-loop (well `Runner` but ya) + # + # TODO? is this actually needed? + # -[ ] theory is this let's the aio side error on + # next tick and then we sync task states from + # here onward? await trio.lowlevel.checkpoint() - # from tractor._state import is_root_process - # if is_root_process(): - # breakpoint() - + # TODO? factor the next 2 branches into a func like + # `try_terminate_aio_task()` and use it for the taskc + # case above as well? + fut: asyncio.Future|None = aio_task._fut_waiter if ( - not chan._trio_err + fut and - (fut := aio_task._fut_waiter) + not fut.done() ): - # await trio.lowlevel.checkpoint() - # import pdbp; pdbp.set_trace() - fut.set_exception( - TrioTaskExited( - f'The peer `asyncio` task is still blocking/running?\n' - f'>>\n' - f'|_{aio_task!r}\n' + # await tractor.pause() + if graceful_trio_exit: + fut.set_exception( + TrioTaskExited( + f'the `trio.Task` gracefully exited but ' + f'its `asyncio` peer is not done?\n' + f')>\n' + f' |_{trio_task}\n' + f'\n' + f'>>\n' + f' |_{aio_task!r}\n' + ) + ) + + # TODO? should this need to exist given the equiv + # `TrioCancelled` equivalent in the be handler + # above?? + else: + fut.set_exception( + TrioTaskExited( + f'The `trio`-side task crashed!\n' + f'{trio_err}' + ) ) - ) else: aio_taskc_warn: str = ( f'\n' f'MANUALLY Cancelling `asyncio`-task: {aio_task.get_name()}!\n\n' f'**THIS CAN SILENTLY SUPPRESS ERRORS FYI\n\n' ) + # await tractor.pause() report += aio_taskc_warn # TODO XXX, figure out the case where calling this makes the # `test_infected_asyncio.py::test_trio_closes_early_and_channel_exits` @@ -745,19 +1016,17 @@ async def translate_aio_errors( log.warning(report) - # Required to sync with the far end `asyncio`-task to ensure - # any error is captured (via monkeypatching the - # `channel._aio_err`) before calling ``maybe_raise_aio_err()`` - # below! + # sync with the `asyncio.Task`'s completion to ensure any + # error is captured and relayed (via + # `channel._aio_err/._trio_to_raise`) BEFORE calling + # `maybe_raise_aio_side_err()` below! # - # XXX NOTE XXX the `task.set_exception(aio_taskc)` call above - # MUST NOT EXCEPT or this WILL HANG!! - # - # so if you get a hang maybe step through and figure out why - # it erroed out up there! + # XXX WARNING NOTE + # the `task.set_exception(aio_taskc)` call above MUST NOT + # EXCEPT or this WILL HANG!! SO, if you get a hang maybe step + # through and figure out why it erroed out up there! # if wait_on_aio_task: - # await chan.wait_aio_complete() await chan._aio_task_complete.wait() log.info( 'asyncio-task is done and unblocked trio-side!\n' @@ -767,6 +1036,8 @@ async def translate_aio_errors( # -[ ] make this a channel method, OR # -[ ] just put back inline below? # + # await tractor.pause(shield=True) + # TODO, go back to inlining this.. def maybe_raise_aio_side_err( trio_err: Exception, ) -> None: @@ -778,31 +1049,86 @@ async def translate_aio_errors( ''' aio_err: BaseException|None = chan._aio_err + trio_to_raise: ( + AsyncioCancelled| + AsyncioTaskExited| + None + ) = chan._trio_to_raise + + if not suppress_graceful_exits: + raise trio_to_raise from (aio_err or trio_err) + + if trio_to_raise: + # import pdbp; pdbp.set_trace() + match ( + trio_to_raise, + trio_err, + ): + case ( + AsyncioTaskExited(), + trio.Cancelled()|None, + ): + log.info( + 'Ignoring aio exit signal since trio also exited!' + ) + return + + case ( + AsyncioCancelled(), + trio.Cancelled(), + ): + if not aio_done_before_trio: + log.info( + 'Ignoring aio cancelled signal since trio was also cancelled!' + ) + return + case _: + raise trio_to_raise from (aio_err or trio_err) # Check if the asyncio-side is the cause of the trio-side # error. - if ( + elif ( aio_err is not None and type(aio_err) is not AsyncioCancelled + # and ( + # type(aio_err) is not AsyncioTaskExited + # and + # not ya_trio_exited + # and + # not trio_err + # ) - # not isinstance(aio_err, CancelledError) - # type(aio_err) is not CancelledError + # TODO, case where trio_err is not None and + # aio_err is AsyncioTaskExited => raise eg! + # -[ ] maybe use a match bc this get's real + # complex fast XD + # + # or + # type(aio_err) is not AsyncioTaskExited + # and + # trio_err + # ) ): # always raise from any captured asyncio error if trio_err: raise trio_err from aio_err + # XXX NOTE! above in the `trio.ClosedResourceError` + # handler we specifically set the + # `aio_err = AsyncioCancelled` such that it is raised + # as that special exc here! raise aio_err if trio_err: raise trio_err + # await tractor.pause() # NOTE: if any ``asyncio`` error was caught, raise it here inline # here in the ``trio`` task # if trio_err: maybe_raise_aio_side_err( - trio_err=trio_err + trio_err=to_raise_trio or trio_err ) @@ -829,20 +1155,24 @@ async def run_task( async with translate_aio_errors( chan, wait_on_aio_task=True, + suppress_graceful_exits=chan._suppress_graceful_exits, ): # return single value that is the output from the - # ``asyncio`` function-as-task. Expect the mem chan api to - # do the job of handling cross-framework cancellations + # ``asyncio`` function-as-task. Expect the mem chan api + # to do the job of handling cross-framework cancellations # / errors via closure and translation in the - # ``translate_aio_errors()`` in the above ctx mngr. - return await chan.receive() + # `translate_aio_errors()` in the above ctx mngr. + + return await chan._from_aio.receive() + # return await chan.receive() @acm async def open_channel_from( target: Callable[..., Any], - **kwargs, + suppress_graceful_exits: bool = True, + **target_kwargs, ) -> AsyncIterator[Any]: ''' @@ -854,13 +1184,15 @@ async def open_channel_from( target, qsize=2**8, provide_channels=True, - **kwargs, + suppress_graceful_exits=suppress_graceful_exits, + **target_kwargs, ) # TODO, tuple form here? async with chan._from_aio: async with translate_aio_errors( chan, wait_on_aio_task=True, + suppress_graceful_exits=suppress_graceful_exits, ): # sync to a "started()"-like first delivered value from the # ``asyncio`` task. @@ -873,17 +1205,37 @@ async def open_channel_from( except trio.Cancelled as taskc: # await tractor.pause(shield=True) # ya it worx ;) if cs.cancel_called: - log.cancel( - f'trio-side was manually cancelled by aio side\n' - f'|_c>}}{cs!r}?\n' - ) + if isinstance(chan._trio_to_raise, AsyncioCancelled): + log.cancel( + f'trio-side was manually cancelled by aio side\n' + f'|_c>}}{cs!r}?\n' + ) # TODO, maybe a special `TrioCancelled`??? raise taskc finally: chan._trio_exited = True - chan._to_trio.close() + + # when the aio side is still ongoing but trio exits + # early we signal with a special exc (kinda like + # a `Return`-msg for IPC ctxs) + aio_task: asyncio.Task = chan._aio_task + if not aio_task.done(): + fut: asyncio.Future|None = aio_task._fut_waiter + if fut: + fut.set_exception( + TrioTaskExited( + f'but the child `asyncio` task is still running?\n' + f'>>\n' + f' |_{aio_task!r}\n' + ) + ) + else: + # XXX SHOULD NEVER HAPPEN! + await tractor.pause() + else: + chan._to_trio.close() class AsyncioRuntimeTranslationError(RuntimeError): From d5ba9be3a95b7210260305125552b0a0b14aaed8 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 3 Mar 2025 12:11:50 -0500 Subject: [PATCH 57/66] More `debug_mode` test support, better nursery var names --- tests/test_infected_asyncio.py | 64 ++++++++++++++++++++-------------- 1 file changed, 37 insertions(+), 27 deletions(-) diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 77877568..8726ad80 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -34,7 +34,6 @@ from tractor._testing import expect_ctxc @pytest.fixture( scope='module', - # autouse=True, ) def delay(debug_mode: bool) -> int: if debug_mode: @@ -73,6 +72,7 @@ async def trio_cancels_single_aio_task(): def test_trio_cancels_aio_on_actor_side( reg_addr: tuple[str, int], delay: int, + debug_mode: bool, ): ''' Spawn an infected actor that is cancelled by the ``trio`` side @@ -82,9 +82,10 @@ def test_trio_cancels_aio_on_actor_side( async def main(): with trio.fail_after(1 + delay): async with tractor.open_nursery( - registry_addrs=[reg_addr] - ) as n: - await n.run_in_actor( + registry_addrs=[reg_addr], + debug_mode=debug_mode, + ) as an: + await an.run_in_actor( trio_cancels_single_aio_task, infect_asyncio=True, ) @@ -133,6 +134,7 @@ async def asyncio_actor( def test_aio_simple_error( reg_addr: tuple[str, int], + debug_mode: bool, ): ''' Verify a simple remote asyncio error propagates back through trio @@ -142,9 +144,10 @@ def test_aio_simple_error( ''' async def main(): async with tractor.open_nursery( - registry_addrs=[reg_addr] - ) as n: - await n.run_in_actor( + registry_addrs=[reg_addr], + debug_mode=debug_mode, + ) as an: + await an.run_in_actor( asyncio_actor, target='sleep_and_err', expect_err='AssertionError', @@ -172,14 +175,17 @@ def test_aio_simple_error( def test_tractor_cancels_aio( reg_addr: tuple[str, int], + debug_mode: bool, ): ''' Verify we can cancel a spawned asyncio task gracefully. ''' async def main(): - async with tractor.open_nursery() as n: - portal = await n.run_in_actor( + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.run_in_actor( asyncio_actor, target='aio_sleep_forever', expect_err='trio.Cancelled', @@ -264,6 +270,7 @@ def test_context_spawns_aio_task_that_errors( reg_addr: tuple[str, int], delay: int, parent_cancels: bool, + debug_mode: bool, ): ''' Verify that spawning a task via an intertask channel ctx mngr that @@ -273,12 +280,12 @@ def test_context_spawns_aio_task_that_errors( ''' async def main(): with trio.fail_after(1 + delay): - async with tractor.open_nursery() as n: - p = await n.start_actor( + async with tractor.open_nursery() as an: + p = await an.start_actor( 'aio_daemon', enable_modules=[__name__], infect_asyncio=True, - # debug_mode=True, + debug_mode=debug_mode, loglevel='cancel', ) async with ( @@ -507,11 +514,11 @@ async def stream_from_aio( # tasks are joined.. chan.subscribe() as br, - trio.open_nursery() as n, + trio.open_nursery() as tn, ): # start 2nd task that get's broadcast the same # value set. - n.start_soon(consume, br) + tn.start_soon(consume, br) await consume(chan) else: @@ -560,8 +567,8 @@ def test_basic_interloop_channel_stream( fan_out: bool, ): async def main(): - async with tractor.open_nursery() as n: - portal = await n.run_in_actor( + async with tractor.open_nursery() as an: + portal = await an.run_in_actor( stream_from_aio, infect_asyncio=True, fan_out=fan_out, @@ -575,8 +582,8 @@ def test_basic_interloop_channel_stream( # TODO: parametrize the above test and avoid the duplication here? def test_trio_error_cancels_intertask_chan(reg_addr): async def main(): - async with tractor.open_nursery() as n: - portal = await n.run_in_actor( + async with tractor.open_nursery() as an: + portal = await an.run_in_actor( stream_from_aio, trio_raise_err=True, infect_asyncio=True, @@ -594,6 +601,7 @@ def test_trio_error_cancels_intertask_chan(reg_addr): def test_trio_closes_early_causes_aio_checkpoint_raise( reg_addr: tuple[str, int], delay: int, + debug_mode: bool, ): ''' Check that if the `trio`-task "exits early and silently" (in this @@ -607,10 +615,10 @@ def test_trio_closes_early_causes_aio_checkpoint_raise( async def main(): with trio.fail_after(1 + delay): async with tractor.open_nursery( - # debug_mode=True, + debug_mode=debug_mode, # enable_stack_on_sig=True, - ) as n: - portal = await n.run_in_actor( + ) as an: + portal = await an.run_in_actor( stream_from_aio, trio_exit_early=True, infect_asyncio=True, @@ -698,8 +706,8 @@ def test_aio_errors_and_channel_propagates_and_closes( async def main(): async with tractor.open_nursery( debug_mode=debug_mode, - ) as n: - portal = await n.run_in_actor( + ) as an: + portal = await an.run_in_actor( stream_from_aio, aio_raise_err=True, infect_asyncio=True, @@ -774,13 +782,15 @@ async def trio_to_aio_echo_server( ids='raise_error={}'.format, ) def test_echoserver_detailed_mechanics( - reg_addr, + reg_addr: tuple[str, int], + debug_mode: bool, raise_error_mid_stream, ): - async def main(): - async with tractor.open_nursery() as n: - p = await n.start_actor( + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + p = await an.start_actor( 'aio_server', enable_modules=[__name__], infect_asyncio=True, From fc325a621b489d3b2af2b5b53d4e14a1394a73eb Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 3 Mar 2025 21:50:51 -0500 Subject: [PATCH 58/66] Be extra sure to re-raise EoCs from translator That is whenever `trio.EndOfChannel` is raised (presumably from the `._to_trio.receive()` call inside `LinkedTaskChannel.receive()`) we need to be extra certain that we let it bubble upward transparently DESPITE special exc-as-signal handling that is normally suppressed from the aio side; REPEAT we want to ALWAYS bubble any `trio_err == trio.EndOfChannel` in the `finally:` handler of `translate_aio_errors()` despite `chan._trio_to_raise == AsyncioTaskExited` such that the caller's iterable machinery will operate as normal when the inter-task stream is stopped (again, presumably by the aio side task terminating the inter-task stream). Main impl deats for this, - in the EoC handler block ensure we assign both `chan._trio_err` and the local `trio_err` as well as continue to re-raise. - add a case to the match block in the `finally:` handler which FOR SURE re-raises any `type(trio_err) is EndOfChannel`! Additionally fix a bad bug, - a ref bug where we were NOT using the `except BaseException as _trio_err` to assign to `chan._trio_err` (by accident was missing the leading `_`..) Unrelated impl tweak, - move all `maybe_raise_aio_side_err()` content back to inline with its parent func - makes it easier to use `tractor.pause()` mostly Bp - go back to trying to use `aio_task.set_exception(aio_taskc)` for now even though i'm pretty sure we're going to move to a try-fute-first style helper for this in the future. Adjust some tests to match/mk-them-green, - break from `aio_echo_server()` recv loop on `to_asyncio.TrioTaskExited` much like how you'd expect to (implicitly with a `for`) with a `trio.EndOfChannel`. - toss in a masked `value is None` pause point i needed for debugging inf looping caused by not re-raising EoCs per the main patch description. - add a debug-mode sized delay to root-infected test. --- tests/test_infected_asyncio.py | 16 ++- tests/test_root_infect_asyncio.py | 8 +- tractor/to_asyncio.py | 198 ++++++++++++++---------------- 3 files changed, 109 insertions(+), 113 deletions(-) diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 8726ad80..465decca 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -491,7 +491,13 @@ async def stream_from_aio( ], ): async for value in chan: - print(f'trio received {value}') + print(f'trio received: {value!r}') + + # XXX, debugging EoC not being handled correctly + # in `transate_aio_errors()`.. + # if value is None: + # await tractor.pause(shield=True) + pulled.append(value) if value == 50: @@ -733,7 +739,13 @@ async def aio_echo_server( to_trio.send_nowait('start') while True: - msg = await from_trio.get() + try: + msg = await from_trio.get() + except to_asyncio.TrioTaskExited: + print( + 'breaking aio echo loop due to `trio` exit!' + ) + break # echo the msg back to_trio.send_nowait(msg) diff --git a/tests/test_root_infect_asyncio.py b/tests/test_root_infect_asyncio.py index 331b6311..93deba13 100644 --- a/tests/test_root_infect_asyncio.py +++ b/tests/test_root_infect_asyncio.py @@ -39,7 +39,7 @@ def test_infected_root_actor( ''' async def _trio_main(): - with trio.fail_after(2): + with trio.fail_after(2 if not debug_mode else 999): first: str chan: to_asyncio.LinkedTaskChannel async with ( @@ -59,7 +59,11 @@ def test_infected_root_actor( assert out == i print(f'asyncio echoing {i}') - if raise_error_mid_stream and i == 500: + if ( + raise_error_mid_stream + and + i == 500 + ): raise raise_error_mid_stream if out is None: diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index f65cc7ef..fd2edfe7 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -429,8 +429,7 @@ def _run_asyncio_task( not chan._aio_err ): chan._trio_to_raise = AsyncioTaskExited( - f'Task existed with final result\n' - f'{result!r}\n' + f'Task exited with final result: {result!r}\n' ) # only close the sender side which will relay @@ -742,7 +741,6 @@ async def translate_aio_errors( aio_done_before_trio: bool = aio_task.done() assert aio_task trio_err: BaseException|None = None - to_raise_trio: BaseException|None = None try: yield # back to one of the cross-loop apis except trio.Cancelled as taskc: @@ -778,8 +776,9 @@ async def translate_aio_errors( # called from `LinkedTaskChannel.receive()` which we want # passthrough and further we have no special meaning for it in # terms of relaying errors or signals from the aio side! - except trio.EndOfChannel: - raise + except trio.EndOfChannel as eoc: + trio_err = chan._trio_err = eoc + raise eoc # NOTE ALSO SEE the matching note in the `cancel_trio()` asyncio # task-done-callback. @@ -825,7 +824,7 @@ async def translate_aio_errors( raise cre except BaseException as _trio_err: - trio_err = chan._trio_err = trio_err + trio_err = chan._trio_err = _trio_err # await tractor.pause(shield=True) # workx! entered: bool = await _debug._maybe_enter_pm( trio_err, @@ -866,25 +865,27 @@ async def translate_aio_errors( f'The `trio`-side task crashed!\n' f'{trio_err}' ) - aio_task.set_exception(aio_taskc) - wait_on_aio_task = False - # try: - # aio_task.set_exception(aio_taskc) - # except ( - # asyncio.InvalidStateError, - # RuntimeError, - # # ^XXX, uhh bc apparently we can't use `.set_exception()` - # # any more XD .. ?? - # ): - # wait_on_aio_task = False + # ??TODO? move this into the func that tries to use + # `Task._fut_waiter: Future` instead?? + # + # aio_task.set_exception(aio_taskc) + # wait_on_aio_task = False + try: + aio_task.set_exception(aio_taskc) + except ( + asyncio.InvalidStateError, + RuntimeError, + # ^XXX, uhh bc apparently we can't use `.set_exception()` + # any more XD .. ?? + ): + wait_on_aio_task = False finally: # record wtv `trio`-side error transpired if trio_err: - if chan._trio_err is not trio_err: - await tractor.pause(shield=True) - - # assert chan._trio_err is trio_err + assert chan._trio_err is trio_err + # if chan._trio_err is not trio_err: + # await tractor.pause(shield=True) ya_trio_exited: bool = chan._trio_exited graceful_trio_exit: bool = ( @@ -1032,104 +1033,83 @@ async def translate_aio_errors( 'asyncio-task is done and unblocked trio-side!\n' ) - # TODO? - # -[ ] make this a channel method, OR - # -[ ] just put back inline below? - # - # await tractor.pause(shield=True) - # TODO, go back to inlining this.. - def maybe_raise_aio_side_err( - trio_err: Exception, - ) -> None: - ''' - Raise any `trio`-side-caused cancellation or legit task - error normally propagated from the caller of either, - - `open_channel_from()` - - `run_task()` + # NOTE, was a `maybe_raise_aio_side_err()` closure that + # i moved inline BP + ''' + Raise any `trio`-side-caused cancellation or legit task + error normally propagated from the caller of either, + - `open_channel_from()` + - `run_task()` - ''' - aio_err: BaseException|None = chan._aio_err - trio_to_raise: ( - AsyncioCancelled| - AsyncioTaskExited| - None - ) = chan._trio_to_raise + ''' + aio_err: BaseException|None = chan._aio_err + trio_to_raise: ( + AsyncioCancelled| + AsyncioTaskExited| + None + ) = chan._trio_to_raise - if not suppress_graceful_exits: - raise trio_to_raise from (aio_err or trio_err) + if not suppress_graceful_exits: + raise trio_to_raise from (aio_err or trio_err) - if trio_to_raise: - # import pdbp; pdbp.set_trace() - match ( - trio_to_raise, - trio_err, + if trio_to_raise: + match ( + trio_to_raise, + trio_err, + ): + case ( + AsyncioTaskExited(), + trio.Cancelled()| + None, ): - case ( - AsyncioTaskExited(), - trio.Cancelled()|None, - ): + log.info( + 'Ignoring aio exit signal since trio also exited!' + ) + return + + case ( + AsyncioTaskExited(), + trio.EndOfChannel(), + ): + raise trio_err + + case ( + AsyncioCancelled(), + trio.Cancelled(), + ): + if not aio_done_before_trio: log.info( - 'Ignoring aio exit signal since trio also exited!' + 'Ignoring aio cancelled signal since trio was also cancelled!' ) return + case _: + raise trio_to_raise from (aio_err or trio_err) - case ( - AsyncioCancelled(), - trio.Cancelled(), - ): - if not aio_done_before_trio: - log.info( - 'Ignoring aio cancelled signal since trio was also cancelled!' - ) - return - case _: - raise trio_to_raise from (aio_err or trio_err) - - # Check if the asyncio-side is the cause of the trio-side - # error. - elif ( - aio_err is not None - and - type(aio_err) is not AsyncioCancelled - # and ( - # type(aio_err) is not AsyncioTaskExited - # and - # not ya_trio_exited - # and - # not trio_err - # ) - - # TODO, case where trio_err is not None and - # aio_err is AsyncioTaskExited => raise eg! - # -[ ] maybe use a match bc this get's real - # complex fast XD - # - # or - # type(aio_err) is not AsyncioTaskExited - # and - # trio_err - # ) - ): - # always raise from any captured asyncio error - if trio_err: - raise trio_err from aio_err - - # XXX NOTE! above in the `trio.ClosedResourceError` - # handler we specifically set the - # `aio_err = AsyncioCancelled` such that it is raised - # as that special exc here! - raise aio_err - + # Check if the asyncio-side is the cause of the trio-side + # error. + elif ( + aio_err is not None + and + type(aio_err) is not AsyncioCancelled + ): + # always raise from any captured asyncio error if trio_err: - raise trio_err + raise trio_err from aio_err - # await tractor.pause() - # NOTE: if any ``asyncio`` error was caught, raise it here inline - # here in the ``trio`` task - # if trio_err: - maybe_raise_aio_side_err( - trio_err=to_raise_trio or trio_err - ) + # XXX NOTE! above in the `trio.ClosedResourceError` + # handler we specifically set the + # `aio_err = AsyncioCancelled` such that it is raised + # as that special exc here! + raise aio_err + + if trio_err: + raise trio_err + + # ^^TODO?? case where trio_err is not None and + # aio_err is AsyncioTaskExited => raise eg! + # -[x] maybe use a match bc this get's real + # complex fast XD + # => i did this above for silent exit cases ya? async def run_task( From b0acc9ffe8c25b9d12f07c3ffe5aa7abeb9bf22a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 4 Mar 2025 19:53:24 -0500 Subject: [PATCH 59/66] Add a mark to `pytest.xfail()` questionably conc py stuff (ur mam `.xfail()`s bish!) --- tests/devx/conftest.py | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tests/devx/conftest.py b/tests/devx/conftest.py index 55aef48a..c45265dc 100644 --- a/tests/devx/conftest.py +++ b/tests/devx/conftest.py @@ -99,6 +99,14 @@ def ctlc( 'https://github.com/goodboy/tractor/issues/320' ) + if mark.name == 'ctlcs_bish': + pytest.skip( + f'Test {node} prolly uses something from the stdlib (namely `asyncio`..)\n' + f'The test and/or underlying example script can *sometimes* run fine ' + f'locally but more then likely until the cpython peeps get their sh#$ together, ' + f'this test will definitely not behave like `trio` under SIGINT..\n' + ) + if use_ctlc: # XXX: disable pygments highlighting for auto-tests # since some envs (like actions CI) will struggle From 03406e020c11c8028a3bbd5f6a8061000955981b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 5 Mar 2025 11:34:36 -0500 Subject: [PATCH 60/66] Repair/update `stackscope` test Seems that on 3.13 it's not showing our script code in the output now? Gotta get an example for @oremanj to see what's up but really it'd be nice to just custom format stuff above `trio`'s runtime by def.. Anyway, update the `.devx._stackscope`, - log formatting to be a little more "sclangy" lookin. - change the per-actor "delimiter" lines style. - report the `signal.getsignal(SIGINT)` which i needed in the `sync_bp.py` with ctl-c causing a hang.. - mask the `_tree_dumped` duplicator log report as well as the "dumped fine" one. - add an example `pkill --signal SIGUSR1` cmdline. Tweak the test to cope with, - not showing our script lines now.. which i've commented in the `assert_before()` patts.. - to expect the newly formatted delimiter (ascii) lines to separate the root vs. hanger sub-actor sections. --- examples/debugging/shield_hang_in_sub.py | 6 +- tests/devx/test_tooling.py | 33 ++++++----- tractor/devx/_stackscope.py | 73 ++++++++++++++++-------- 3 files changed, 69 insertions(+), 43 deletions(-) diff --git a/examples/debugging/shield_hang_in_sub.py b/examples/debugging/shield_hang_in_sub.py index 3cc084d5..5387353f 100644 --- a/examples/debugging/shield_hang_in_sub.py +++ b/examples/debugging/shield_hang_in_sub.py @@ -39,7 +39,6 @@ async def main( loglevel='devx', ) as an, ): - ptl: tractor.Portal = await an.start_actor( 'hanger', enable_modules=[__name__], @@ -54,13 +53,16 @@ async def main( print( 'Yo my child hanging..?\n' - 'Sending SIGUSR1 to see a tree-trace!\n' + # "i'm a user who wants to see a `stackscope` tree!\n" ) # XXX simulate the wrapping test's "user actions" # (i.e. if a human didn't run this manually but wants to # know what they should do to reproduce test behaviour) if from_test: + print( + f'Sending SIGUSR1 to {cpid!r}!\n' + ) os.kill( cpid, signal.SIGUSR1, diff --git a/tests/devx/test_tooling.py b/tests/devx/test_tooling.py index 1ed741e8..2debe3f7 100644 --- a/tests/devx/test_tooling.py +++ b/tests/devx/test_tooling.py @@ -15,6 +15,7 @@ TODO: ''' import os import signal +import time from .conftest import ( expect, @@ -53,41 +54,39 @@ def test_shield_pause( ] ) + script_pid: int = child.pid print( - 'Sending SIGUSR1 to see a tree-trace!', + f'Sending SIGUSR1 to {script_pid}\n' + f'(kill -s SIGUSR1 {script_pid})\n' ) os.kill( - child.pid, + script_pid, signal.SIGUSR1, ) + time.sleep(0.2) expect( child, # end-of-tree delimiter - "------ \('root', ", + "end-of-\('root'", ) - assert_before( child, [ - 'Trying to dump `stackscope` tree..', - 'Dumping `stackscope` tree for actor', + # 'Srying to dump `stackscope` tree..', + # 'Dumping `stackscope` tree for actor', "('root'", # uid line + # TODO!? this used to show? + # -[ ] mk reproducable for @oremanj? + # # parent block point (non-shielded) - 'await trio.sleep_forever() # in root', + # 'await trio.sleep_forever() # in root', ] ) - - # expect( - # child, - # # relay to the sub should be reported - # 'Relaying `SIGUSR1`[10] to sub-actor', - # ) - expect( child, # end-of-tree delimiter - "------ \('hanger', ", + "end-of-\('hanger'", ) assert_before( child, @@ -97,11 +96,11 @@ def test_shield_pause( "('hanger'", # uid line + # TODO!? SEE ABOVE # hanger LOC where it's shield-halted - 'await trio.sleep_forever() # in subactor', + # 'await trio.sleep_forever() # in subactor', ] ) - # breakpoint() # simulate the user sending a ctl-c to the hanging program. # this should result in the terminator kicking in since diff --git a/tractor/devx/_stackscope.py b/tractor/devx/_stackscope.py index 944ae49a..ccc46534 100644 --- a/tractor/devx/_stackscope.py +++ b/tractor/devx/_stackscope.py @@ -35,6 +35,7 @@ from signal import ( signal, getsignal, SIGUSR1, + SIGINT, ) # import traceback from types import ModuleType @@ -48,6 +49,7 @@ from tractor import ( _state, log as logmod, ) +from tractor.devx import _debug log = logmod.get_logger(__name__) @@ -76,22 +78,45 @@ def dump_task_tree() -> None: ) actor: Actor = _state.current_actor() thr: Thread = current_thread() + current_sigint_handler: Callable = getsignal(SIGINT) + if ( + current_sigint_handler + is not + _debug.DebugStatus._trio_handler + ): + sigint_handler_report: str = ( + 'The default `trio` SIGINT handler was replaced?!' + ) + else: + sigint_handler_report: str = ( + 'The default `trio` SIGINT handler is in use?!' + ) + + # sclang symbology + # |_ + # |_(Task/Thread/Process/Actor + # |_{Supervisor/Scope + # |_[Storage/Memory/IPC-Stream/Data-Struct + log.devx( f'Dumping `stackscope` tree for actor\n' - f'{actor.uid}:\n' - f'|_{mp.current_process()}\n' - f' |_{thr}\n' - f' |_{actor}\n\n' - - # start-of-trace-tree delimiter (mostly for testing) - '------ - ------\n' - '\n' - + - f'{tree_str}\n' - + - # end-of-trace-tree delimiter (mostly for testing) + f'(>: {actor.uid!r}\n' + f' |_{mp.current_process()}\n' + f' |_{thr}\n' + f' |_{actor}\n' f'\n' - f'------ {actor.uid!r} ------\n' + f'{sigint_handler_report}\n' + f'signal.getsignal(SIGINT) -> {current_sigint_handler!r}\n' + # f'\n' + # start-of-trace-tree delimiter (mostly for testing) + # f'------ {actor.uid!r} ------\n' + f'\n' + f'------ start-of-{actor.uid!r} ------\n' + f'|\n' + f'{tree_str}' + # end-of-trace-tree delimiter (mostly for testing) + f'|\n' + f'|_____ end-of-{actor.uid!r} ______\n' ) # TODO: can remove this right? # -[ ] was original code from author @@ -123,11 +148,11 @@ def dump_tree_on_sig( ) -> None: global _tree_dumped, _handler_lock with _handler_lock: - if _tree_dumped: - log.warning( - 'Already dumped for this actor...??' - ) - return + # if _tree_dumped: + # log.warning( + # 'Already dumped for this actor...??' + # ) + # return _tree_dumped = True @@ -161,9 +186,9 @@ def dump_tree_on_sig( ) raise - log.devx( - 'Supposedly we dumped just fine..?' - ) + # log.devx( + # 'Supposedly we dumped just fine..?' + # ) if not relay_to_subs: return @@ -202,11 +227,11 @@ def enable_stack_on_sig( (https://www.gnu.org/software/bash/manual/bash.html#Command-Substitution) you could use: - >> kill -SIGUSR1 $(pgrep -f '') + >> kill -SIGUSR1 $(pgrep -f ) - Or with with `xonsh` (which has diff capture-from-subproc syntax) + OR without a sub-shell, - >> kill -SIGUSR1 @$(pgrep -f '') + >> pkill --signal SIGUSR1 -f ''' try: From b021772a1e7487cc9228f5ee18cffa14a22c0c07 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 5 Mar 2025 11:58:03 -0500 Subject: [PATCH 61/66] Mask ctlc borked REPL tests Namely the `tractor.pause_from_sync()` examples using both bg threads and `asyncio` which seem to go into bad states where SIGINT is ignored.. Deats, - add `maybe_expect_timeout()` cm to ensure the EOF hangs get `.xfail()`ed instead. - @pytest.mark.ctlcs_bish` `test_pause_from_sync` and don't expect the greenback prompt msg. - also mark `test_sync_pause_from_aio_task`. --- tests/devx/test_pause_from_non_trio.py | 45 ++++++++++++++++++++++---- 1 file changed, 38 insertions(+), 7 deletions(-) diff --git a/tests/devx/test_pause_from_non_trio.py b/tests/devx/test_pause_from_non_trio.py index 3a7140e6..4a03a123 100644 --- a/tests/devx/test_pause_from_non_trio.py +++ b/tests/devx/test_pause_from_non_trio.py @@ -6,6 +6,9 @@ All these tests can be understood (somewhat) by running the equivalent `examples/debugging/` scripts manually. ''' +from contextlib import ( + contextmanager as cm, +) # from functools import partial # import itertools import time @@ -15,7 +18,7 @@ import time import pytest from pexpect.exceptions import ( - # TIMEOUT, + TIMEOUT, EOF, ) @@ -32,7 +35,23 @@ from .conftest import ( # _repl_fail_msg, ) +@cm +def maybe_expect_timeout( + ctlc: bool = False, +) -> None: + try: + yield + except TIMEOUT: + # breakpoint() + if ctlc: + pytest.xfail( + 'Some kinda redic threading SIGINT bug i think?\n' + 'See the notes in `examples/debugging/sync_bp.py`..\n' + ) + raise + +@pytest.mark.ctlcs_bish def test_pause_from_sync( spawn, ctlc: bool, @@ -67,10 +86,10 @@ def test_pause_from_sync( child.expect(PROMPT) # XXX shouldn't see gb loaded message with PDB loglevel! - assert not in_prompt_msg( - child, - ['`greenback` portal opened!'], - ) + # assert not in_prompt_msg( + # child, + # ['`greenback` portal opened!'], + # ) # should be same root task assert_before( child, @@ -162,7 +181,14 @@ def test_pause_from_sync( ) child.sendline('c') - child.expect(EOF) + + # XXX TODO, weird threading bug it seems despite the + # `abandon_on_cancel: bool` setting to + # `trio.to_thread.run_sync()`.. + with maybe_expect_timeout( + ctlc=ctlc, + ): + child.expect(EOF) def expect_any_of( @@ -220,8 +246,10 @@ def expect_any_of( return expected_patts +@pytest.mark.ctlcs_bish def test_sync_pause_from_aio_task( spawn, + ctlc: bool # ^TODO, fix for `asyncio`!! ): @@ -270,10 +298,12 @@ def test_sync_pause_from_aio_task( # error raised in `asyncio.Task` "raise ValueError('asyncio side error!')": [ _crash_msg, - 'return await chan.receive()', # `.to_asyncio` impl internals in tb " Date: Mon, 3 Mar 2025 19:45:29 -0500 Subject: [PATCH 62/66] Fix an `aio_err` ref bug --- tractor/to_asyncio.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index fd2edfe7..7b87be0b 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -348,7 +348,6 @@ def _run_asyncio_task( trio_task: trio.Task = trio.lowlevel.current_task() trio_cs = trio.CancelScope() aio_task_complete = trio.Event() - aio_err: BaseException|None = None chan = LinkedTaskChannel( _to_aio=aio_q, # asyncio.Queue @@ -392,7 +391,7 @@ def _run_asyncio_task( if ( result != orig and - aio_err is None + chan._aio_err is None and # in the `open_channel_from()` case we don't From 9e10064bda1e6e1c9b8e38cba98efce02db19e53 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 19 Mar 2025 13:30:05 -0400 Subject: [PATCH 63/66] Continue supporting py3.11+ Apparently the only thing needing a guard was use of `asyncio.Queue.shutdown()` and the paired `QueueShutDown` exception? Cool. --- tractor/to_asyncio.py | 31 ++++++++++++++++++++++++------- 1 file changed, 24 insertions(+), 7 deletions(-) diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index 7b87be0b..08b1ed25 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -23,12 +23,10 @@ import asyncio from asyncio.exceptions import ( CancelledError, ) -from asyncio import ( - QueueShutDown, -) from contextlib import asynccontextmanager as acm from dataclasses import dataclass import inspect +import platform import traceback from typing import ( Any, @@ -79,6 +77,20 @@ __all__ = [ 'run_as_asyncio_guest', ] +if (_py_313 := ( + ('3', '13') + == + platform.python_version_tuple()[:-1] + ) +): + # 3.13+ only.. lel. + # https://docs.python.org/3.13/library/asyncio-queue.html#asyncio.QueueShutDown + from asyncio import ( + QueueShutDown, + ) +else: + QueueShutDown = False + # TODO, generally speaking we can generalize this abstraction, a "SC linked # parent->child task pair", as the same "supervision scope primitive" @@ -575,7 +587,11 @@ def _run_asyncio_task( # normally suppressed unless the trio.Task also errors # # ?TODO, is this even needed (does it happen) now? - elif isinstance(aio_err, QueueShutDown): + elif ( + _py_313 + and + isinstance(aio_err, QueueShutDown) + ): # import pdbp; pdbp.set_trace() trio_err = AsyncioTaskExited( 'Task exited before `trio` side' @@ -955,9 +971,10 @@ async def translate_aio_errors( # or an error, we ensure the aio-side gets signalled via # an explicit exception and its `Queue` is shutdown. if ya_trio_exited: - # raise `QueueShutDown` on next `Queue.get()` call on - # aio side. - chan._to_aio.shutdown() + # XXX py3.13+ ONLY.. + # raise `QueueShutDown` on next `Queue.get/put()` + if _py_313: + chan._to_aio.shutdown() # pump this event-loop (well `Runner` but ya) # From 13adaa110ac9e8beae4c474f859b66badde31853 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 25 Feb 2025 11:20:07 -0500 Subject: [PATCH 64/66] Drop `asyncio`-canc error from `._exceptions` --- tractor/_exceptions.py | 14 +------------- 1 file changed, 1 insertion(+), 13 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index db863359..3382be10 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -1015,18 +1015,6 @@ class MessagingError(Exception): ''' -class AsyncioCancelled(Exception): - ''' - Asyncio cancelled translation (non-base) error - for use with the ``to_asyncio`` module - to be raised in the ``trio`` side task - - NOTE: this should NOT inherit from `asyncio.CancelledError` or - tests should break! - - ''' - - def pack_error( exc: BaseException|RemoteActorError, @@ -1206,7 +1194,7 @@ def is_multi_cancelled( trio.Cancelled in ignore_nested # XXX always count-in `trio`'s native signal ): - ignore_nested |= {trio.Cancelled} + ignore_nested.update({trio.Cancelled}) if isinstance(exc, BaseExceptionGroup): matched_exc: BaseExceptionGroup|None = exc.subgroup( From f1af87007efdfe7fc6f29f6038893e74271108b2 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 3 Mar 2025 12:13:25 -0500 Subject: [PATCH 65/66] Add equiv of `AsyncioCancelled` for aio side Such that a `TrioCancelled` is raised in the aio task via `.set_exception()` to explicitly indicate and allow that task to handle a taskc request from the parent `trio.Task`. --- tractor/_exceptions.py | 21 ++++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 3382be10..f90df5fe 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -103,7 +103,16 @@ class AsyncioTaskExited(Exception): ''' -class TrioTaskExited(AsyncioCancelled): +class TrioCancelled(Exception): + ''' + Trio cancelled translation (non-base) error + for use with the `to_asyncio` module + to be raised in the `asyncio.Task` to indicate + that the `trio` side raised `Cancelled` or an error. + + ''' + +class TrioTaskExited(Exception): ''' The `trio`-side task exited without explicitly cancelling the `asyncio.Task` peer. @@ -406,6 +415,9 @@ class RemoteActorError(Exception): String-name of the (last hop's) boxed error type. ''' + # TODO, maybe support also serializing the + # `ExceptionGroup.exeptions: list[BaseException]` set under + # certain conditions? bt: Type[BaseException] = self.boxed_type if bt: return str(bt.__name__) @@ -821,8 +833,11 @@ class MsgTypeError( ''' if ( (_bad_msg := self.msgdata.get('_bad_msg')) - and - isinstance(_bad_msg, PayloadMsg) + and ( + isinstance(_bad_msg, PayloadMsg) + or + isinstance(_bad_msg, msgtypes.Start) + ) ): return _bad_msg From c91373148a1eae1ae54eb19b8daf90ff7a95384c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 5 Mar 2025 09:54:56 -0500 Subject: [PATCH 66/66] Comment-tag pause points in `asycnio_bp.py` Thought i already did this but, obvi needed these to make the expect matches pass in our test. --- examples/debugging/asyncio_bp.py | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/examples/debugging/asyncio_bp.py b/examples/debugging/asyncio_bp.py index b55b28fd..296dbccb 100644 --- a/examples/debugging/asyncio_bp.py +++ b/examples/debugging/asyncio_bp.py @@ -25,7 +25,7 @@ async def bp_then_error( ) -> None: - # sync with ``trio``-side (caller) task + # sync with `trio`-side (caller) task to_trio.send_nowait('start') # NOTE: what happens here inside the hook needs some refinement.. @@ -33,8 +33,7 @@ async def bp_then_error( # we set `Lock.local_task_in_debug = 'sync'`, we probably want # some further, at least, meta-data about the task/actor in debug # in terms of making it clear it's `asyncio` mucking about. - breakpoint() - + breakpoint() # asyncio-side # short checkpoint / delay await asyncio.sleep(0.5) # asyncio-side @@ -58,7 +57,6 @@ async def trio_ctx( # this will block until the ``asyncio`` task sends a "first" # message, see first line in above func. async with ( - to_asyncio.open_channel_from( bp_then_error, # raise_after_bp=not bp_before_started, @@ -69,7 +67,7 @@ async def trio_ctx( assert first == 'start' if bp_before_started: - await tractor.pause() + await tractor.pause() # trio-side await ctx.started(first) # trio-side @@ -111,7 +109,7 @@ async def main( # pause in parent to ensure no cross-actor # locking problems exist! - await tractor.pause() + await tractor.pause() # trio-root if cancel_from_root: await ctx.cancel()