From a870df68c0c91d510b0ca52c8cc98ca3d3a9b30b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 24 Jun 2024 16:10:23 -0400 Subject: [PATCH 01/42] 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 45722a6..8d4697f 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 7164d6a..46b64c1 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -922,15 +922,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 @@ -1324,7 +1315,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 d1451b4..e041721 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), + ) -- 2.34.1 From 4f1db1ff52e6a5a3a5063e10337d9c5266f1efc0 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 25 Jun 2024 19:36:31 -0400 Subject: [PATCH 02/42] 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 8d4697f..645dc4b 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 46b64c1..265f506 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -929,6 +929,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, -- 2.34.1 From 268bd0d8ecac10c06506ccf3a308360a856b9cbd Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 26 Jun 2024 11:44:31 -0400 Subject: [PATCH 03/42] 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 645dc4b..42eb35b 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 -- 2.34.1 From 9133f42b07e69cbae2d8c5da077397b946cf4525 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 26 Jun 2024 13:48:36 -0400 Subject: [PATCH 04/42] 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 e041721..fb18ba8 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), -- 2.34.1 From 9f9b0b17dc0f2f3e4c4b3e8d78e406d4eab145c5 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 26 Jun 2024 16:00:14 -0400 Subject: [PATCH 05/42] Add a `Context.portal`, more cancel tooing Might as well add a public maybe-getter for use on the "parent" side since it can be handy to check out-of-band cancellation conditions (like from `Portal.cancel_actor()`). Buncha bitty tweaks for more easily debugging cancel conditions: - add a `@.cancel_called.setter` for hooking into `.cancel_called = True` being set in hard to decipher "who cancelled us" scenarios. - use a new `self_ctxc: bool` var in `.cancel()` to capture the output state from `._is_self_cancelled(remote_error)` at call time so it can be compared against the measured value at crash-time (when REPL-ing it can often have already changed due to runtime teardown sequencing vs. the crash handler hook entry). - proxy `hide_tb` to `.drain_to_final_msg()` from `.wait_for_result()`. - use `remote_error.sender` attr directly instead of through `RAE.msgdata: dict` lookup. - change var name `our_uid` -> `peer_uid`; it's not "ours".. Other various docs/comment updates: - extend the main class doc to include some other name ideas. - change over all remaining `.result()` refs to `.wait_for_result()`. - doc more details on how we want `.outcome` to eventually signature. --- tractor/_context.py | 126 +++++++++++++++++++++++++++++++------------- 1 file changed, 88 insertions(+), 38 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index 32acf83..620363c 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -121,10 +121,19 @@ class Unresolved: @dataclass class Context: ''' - An inter-actor, SC transitive, `Task` communication context. + An inter-actor, SC transitive, `trio.Task` (pair) + communication context. - NB: This class should **never be instatiated directly**, it is allocated - by the runtime in 2 ways: + (We've also considered other names and ideas: + - "communicating tasks scope": cts + - "distributed task scope": dts + - "communicating tasks context": ctc + + **Got a better idea for naming? Make an issue dawg!** + ) + + NB: This class should **never be instatiated directly**, it is + allocated by the runtime in 2 ways: - by entering `Portal.open_context()` which is the primary public API for any "parent" task or, - by the RPC machinery's `._rpc._invoke()` as a `ctx` arg @@ -210,6 +219,16 @@ class Context: # more the the `Context` is needed? _portal: Portal | None = None + @property + def portal(self) -> Portal|None: + ''' + Return any wrapping memory-`Portal` if this is + a 'parent'-side task which called `Portal.open_context()`, + otherwise `None`. + + ''' + return self._portal + # NOTE: each side of the context has its own cancel scope # which is exactly the primitive that allows for # cross-actor-task-supervision and thus SC. @@ -299,6 +318,8 @@ class Context: # boxed exception. NOW, it's used for spawning overrun queuing # tasks when `.allow_overruns == True` !!! _scope_nursery: trio.Nursery|None = None + # ^-TODO-^ change name? + # -> `._scope_tn` "scope task nursery" # streaming overrun state tracking _in_overrun: bool = False @@ -408,10 +429,23 @@ class Context: ''' return self._cancel_called + @cancel_called.setter + def cancel_called(self, val: bool) -> None: + ''' + Set the self-cancelled request `bool` value. + + ''' + # to debug who frickin sets it.. + # if val: + # from .devx import pause_from_sync + # pause_from_sync() + + self._cancel_called = val + @property def canceller(self) -> tuple[str, str]|None: ''' - ``Actor.uid: tuple[str, str]`` of the (remote) + `Actor.uid: tuple[str, str]` of the (remote) actor-process who's task was cancelled thus causing this (side of the) context to also be cancelled. @@ -515,7 +549,7 @@ class Context: # the local scope was never cancelled # and instead likely we received a remote side - # # cancellation that was raised inside `.result()` + # # cancellation that was raised inside `.wait_for_result()` # or ( # (se := self._local_error) # and se is re @@ -585,6 +619,10 @@ class Context: self, error: BaseException, + # TODO: manual toggle for cases where we wouldn't normally + # mark ourselves cancelled but want to? + # set_cancel_called: bool = False, + ) -> None: ''' (Maybe) cancel this local scope due to a received remote @@ -603,7 +641,7 @@ class Context: - `Portal.open_context()` - `Portal.result()` - `Context.open_stream()` - - `Context.result()` + - `Context.wait_for_result()` when called/closed by actor local task(s). @@ -729,7 +767,7 @@ class Context: # Cancel the local `._scope`, catch that # `._scope.cancelled_caught` and re-raise any remote error - # once exiting (or manually calling `.result()`) the + # once exiting (or manually calling `.wait_for_result()`) the # `.open_context()` block. cs: trio.CancelScope = self._scope if ( @@ -764,8 +802,9 @@ class Context: # `trio.Cancelled` subtype here ;) # https://github.com/goodboy/tractor/issues/368 message: str = 'Cancelling `Context._scope` !\n\n' + # from .devx import pause_from_sync + # pause_from_sync() self._scope.cancel() - else: message: str = 'NOT cancelling `Context._scope` !\n\n' # from .devx import mk_pdb @@ -845,15 +884,15 @@ class Context: @property def repr_api(self) -> str: + return 'Portal.open_context()' + + # TODO: use `.dev._frame_stack` scanning to find caller! # ci: CallerInfo|None = self._caller_info # if ci: # return ( # f'{ci.api_nsp}()\n' # ) - # TODO: use `.dev._frame_stack` scanning to find caller! - return 'Portal.open_context()' - async def cancel( self, timeout: float = 0.616, @@ -889,7 +928,8 @@ class Context: ''' side: str = self.side - self._cancel_called: bool = True + # XXX for debug via the `@.setter` + self.cancel_called = True header: str = ( f'Cancelling ctx with peer from {side.upper()} side\n\n' @@ -912,7 +952,7 @@ class Context: # `._scope.cancel()` since we expect the eventual # `ContextCancelled` from the other side to trigger this # when the runtime finally receives it during teardown - # (normally in `.result()` called from + # (normally in `.wait_for_result()` called from # `Portal.open_context().__aexit__()`) if side == 'parent': if not self._portal: @@ -1025,10 +1065,10 @@ class Context: ''' __tracebackhide__: bool = hide_tb - our_uid: tuple = self.chan.uid + peer_uid: tuple = self.chan.uid # XXX NOTE XXX: `ContextCancelled`/`StreamOverrun` absorption - # for "graceful cancellation" case: + # for "graceful cancellation" case(s): # # Whenever a "side" of a context (a `Task` running in # an actor) **is** the side which requested ctx @@ -1045,9 +1085,11 @@ class Context: # set to the `Actor.uid` of THIS task (i.e. the # cancellation requesting task's actor is the actor # checking whether it should absorb the ctxc). + self_ctxc: bool = self._is_self_cancelled(remote_error) if ( + self_ctxc + and not raise_ctxc_from_self_call - and self._is_self_cancelled(remote_error) # TODO: ?potentially it is useful to emit certain # warning/cancel logs for the cases where the @@ -1077,8 +1119,8 @@ class Context: and isinstance(remote_error, RemoteActorError) and remote_error.boxed_type is StreamOverrun - # and tuple(remote_error.msgdata['sender']) == our_uid - and tuple(remote_error.sender) == our_uid + # and tuple(remote_error.msgdata['sender']) == peer_uid + and tuple(remote_error.sender) == peer_uid ): # NOTE: we set the local scope error to any "self # cancellation" error-response thus "absorbing" @@ -1140,9 +1182,9 @@ class Context: of the remote cancellation. ''' - __tracebackhide__ = hide_tb + __tracebackhide__: bool = hide_tb assert self._portal, ( - "Context.result() can not be called from callee side!" + '`Context.wait_for_result()` can not be called from callee side!' ) if self._final_result_is_set(): return self._result @@ -1197,10 +1239,11 @@ class Context: # raising something we know might happen # during cancellation ;) (not self._cancel_called) - ) + ), + hide_tb=hide_tb, ) # TODO: eventually make `.outcome: Outcome` and thus return - # `self.outcome.unwrap()` here! + # `self.outcome.unwrap()` here? return self.outcome # TODO: switch this with above! @@ -1284,17 +1327,24 @@ class Context: Any| RemoteActorError| ContextCancelled + # TODO: make this a `outcome.Outcome`! ): ''' - The final "outcome" from an IPC context which can either be - some Value returned from the target `@context`-decorated - remote task-as-func, or an `Error` wrapping an exception - raised from an RPC task fault or cancellation. + Return the "final outcome" (state) of the far end peer task + non-blocking. If the remote task has not completed then this + field always resolves to the module defined `Unresolved` + handle. - Note that if the remote task has not terminated then this - field always resolves to the module defined `Unresolved` handle. + ------ - ------ + TODO->( this is doc-driven-dev content not yet actual ;P ) - TODO: implement this using `outcome.Outcome` types? + The final "outcome" from an IPC context which can be any of: + - some `outcome.Value` which boxes the returned output from the peer task's + `@context`-decorated remote task-as-func, or + - an `outcome.Error` wrapping an exception raised that same RPC task + after a fault or cancellation, or + - an unresolved `outcome.Outcome` when the peer task is still + executing and has not yet completed. ''' return ( @@ -1583,7 +1633,7 @@ class Context: - NEVER `return` early before delivering the msg! bc if the error is a ctxc and there is a task waiting on - `.result()` we need the msg to be + `.wait_for_result()` we need the msg to be `send_chan.send_nowait()`-ed over the `._rx_chan` so that the error is relayed to that waiter task and thus raised in user code! @@ -1828,7 +1878,7 @@ async def open_context_from_portal( When the "callee" (side that is "called"/started by a call to *this* method) returns, the caller side (this) unblocks and any final value delivered from the other end can be - retrieved using the `Contex.result()` api. + retrieved using the `Contex.wait_for_result()` api. The yielded ``Context`` instance further allows for opening bidirectional streams, explicit cancellation and @@ -1965,14 +2015,14 @@ async def open_context_from_portal( yield ctx, first # ??TODO??: do we still want to consider this or is - # the `else:` block handling via a `.result()` + # the `else:` block handling via a `.wait_for_result()` # call below enough?? # - # -[ ] pretty sure `.result()` internals do the + # -[ ] pretty sure `.wait_for_result()` internals do the # same as our ctxc handler below so it ended up # being same (repeated?) behaviour, but ideally we # wouldn't have that duplication either by somehow - # factoring the `.result()` handler impl in a way + # factoring the `.wait_for_result()` handler impl in a way # that we can re-use it around the `yield` ^ here # or vice versa? # @@ -2110,7 +2160,7 @@ async def open_context_from_portal( # AND a group-exc is only raised if there was > 1 # tasks started *here* in the "caller" / opener # block. If any one of those tasks calls - # `.result()` or `MsgStream.receive()` + # `.wait_for_result()` or `MsgStream.receive()` # `._maybe_raise_remote_err()` will be transitively # called and the remote error raised causing all # tasks to be cancelled. @@ -2180,7 +2230,7 @@ async def open_context_from_portal( f'|_{ctx._task}\n' ) # XXX NOTE XXX: the below call to - # `Context.result()` will ALWAYS raise + # `Context.wait_for_result()` will ALWAYS raise # a `ContextCancelled` (via an embedded call to # `Context._maybe_raise_remote_err()`) IFF # a `Context._remote_error` was set by the runtime @@ -2190,10 +2240,10 @@ async def open_context_from_portal( # ALWAYS SET any time "callee" side fails and causes "caller # side" cancellation via a `ContextCancelled` here. try: - result_or_err: Exception|Any = await ctx.result() + result_or_err: Exception|Any = await ctx.wait_for_result() except BaseException as berr: # on normal teardown, if we get some error - # raised in `Context.result()` we still want to + # raised in `Context.wait_for_result()` we still want to # save that error on the ctx's state to # determine things like `.cancelled_caught` for # cases where there was remote cancellation but -- 2.34.1 From 2ac999cc3c5e08f4aacec1ba7c71d03378d2e89a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 27 Jun 2024 16:25:46 -0400 Subject: [PATCH 06/42] Prep for legacy RPC API factor-n-remove This change is adding commentary about the upcoming API removal and simplification of nursery + portal internals; no actual code changes are included. The plan to (re)move the old RPC methods: - `ActorNursery.run_in_actor()` - `Portal.run()` - `Portal.run_from_ns()` and any related impl internals out of each conc-primitive and instead into something like a `.hilevel.rpc` set of APIs which then are all implemented using the newer and more lowlevel `Context`/`MsgStream` primitives instead Bo Further, - formally deprecate the `Portal.result()` meth for `.wait_for_result()`. - only `log.info()` about runtime shutdown in the implicit root case. --- tractor/_portal.py | 42 +++++++++++++++++++++++++++++++++++++---- tractor/_spawn.py | 4 ++-- tractor/_supervise.py | 44 +++++++++++++++++++++++++++++++------------ 3 files changed, 72 insertions(+), 18 deletions(-) diff --git a/tractor/_portal.py b/tractor/_portal.py index 2c676e1..0f69883 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -121,7 +121,8 @@ class Portal: ) return self.chan - # TODO: factor this out into an `ActorNursery` wrapper + # TODO: factor this out into a `.highlevel` API-wrapper that uses + # a single `.open_context()` call underneath. async def _submit_for_result( self, ns: str, @@ -141,13 +142,22 @@ class Portal: portal=self, ) + # TODO: we should deprecate this API right? since if we remove + # `.run_in_actor()` (and instead move it to a `.highlevel` + # wrapper api (around a single `.open_context()` call) we don't + # really have any notion of a "main" remote task any more? + # # @api_frame - async def result(self) -> Any: + async def wait_for_result( + self, + hide_tb: bool = True, + ) -> Any: ''' - Return the result(s) from the remote actor's "main" task. + Return the final result delivered by a `Return`-msg from the + remote peer actor's "main" task's `return` statement. ''' - __tracebackhide__ = True + __tracebackhide__: bool = hide_tb # Check for non-rpc errors slapped on the # channel for which we always raise exc = self.channel._exc @@ -182,6 +192,23 @@ class Portal: return self._final_result_pld + # TODO: factor this out into a `.highlevel` API-wrapper that uses + # a single `.open_context()` call underneath. + async def result( + self, + *args, + **kwargs, + ) -> Any|Exception: + typname: str = type(self).__name__ + log.warning( + f'`{typname}.result()` is DEPRECATED!\n' + f'Use `{typname}.wait_for_result()` instead!\n' + ) + return await self.wait_for_result( + *args, + **kwargs, + ) + async def _cancel_streams(self): # terminate all locally running async generator # IPC calls @@ -240,6 +267,7 @@ class Portal: f'{reminfo}' ) + # XXX the one spot we set it? self.channel._cancel_called: bool = True try: # send cancel cmd - might not get response @@ -279,6 +307,8 @@ class Portal: ) return False + # TODO: do we still need this for low level `Actor`-runtime + # method calls or can we also remove it? async def run_from_ns( self, namespace_path: str, @@ -316,6 +346,8 @@ class Portal: expect_msg=Return, ) + # TODO: factor this out into a `.highlevel` API-wrapper that uses + # a single `.open_context()` call underneath. async def run( self, func: str, @@ -370,6 +402,8 @@ class Portal: expect_msg=Return, ) + # TODO: factor this out into a `.highlevel` API-wrapper that uses + # a single `.open_context()` call underneath. @acm async def open_stream_from( self, diff --git a/tractor/_spawn.py b/tractor/_spawn.py index aeb7a7c..c06fa22 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -149,7 +149,7 @@ async def exhaust_portal( # XXX: streams should never be reaped here since they should # always be established and shutdown using a context manager api - final: Any = await portal.result() + final: Any = await portal.wait_for_result() except ( Exception, @@ -223,8 +223,8 @@ async def cancel_on_completion( async def hard_kill( proc: trio.Process, - terminate_after: int = 1.6, + terminate_after: int = 1.6, # NOTE: for mucking with `.pause()`-ing inside the runtime # whilst also hacking on it XD # terminate_after: int = 99999, diff --git a/tractor/_supervise.py b/tractor/_supervise.py index 8f3574b..fb737c1 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -80,6 +80,7 @@ class ActorNursery: ''' def __init__( self, + # TODO: maybe def these as fields of a struct looking type? actor: Actor, ria_nursery: trio.Nursery, da_nursery: trio.Nursery, @@ -88,8 +89,10 @@ class ActorNursery: ) -> None: # self.supervisor = supervisor # TODO self._actor: Actor = actor - self._ria_nursery = ria_nursery + + # TODO: rename to `._tn` for our conventional "task-nursery" self._da_nursery = da_nursery + self._children: dict[ tuple[str, str], tuple[ @@ -98,15 +101,13 @@ class ActorNursery: Portal | None, ] ] = {} - # portals spawned with ``run_in_actor()`` are - # cancelled when their "main" result arrives - self._cancel_after_result_on_exit: set = set() + self.cancelled: bool = False self._join_procs = trio.Event() self._at_least_one_child_in_debug: bool = False self.errors = errors - self.exited = trio.Event() self._scope_error: BaseException|None = None + self.exited = trio.Event() # NOTE: when no explicit call is made to # `.open_root_actor()` by application code, @@ -116,6 +117,13 @@ class ActorNursery: # and syncing purposes to any actor opened nurseries. self._implicit_runtime_started: bool = False + # TODO: remove the `.run_in_actor()` API and thus this 2ndary + # nursery when that API get's moved outside this primitive! + self._ria_nursery = ria_nursery + # portals spawned with ``run_in_actor()`` are + # cancelled when their "main" result arrives + self._cancel_after_result_on_exit: set = set() + async def start_actor( self, name: str, @@ -126,10 +134,14 @@ class ActorNursery: rpc_module_paths: list[str]|None = None, enable_modules: list[str]|None = None, loglevel: str|None = None, # set log level per subactor - nursery: trio.Nursery|None = None, debug_mode: bool|None = None, infect_asyncio: bool = False, + # TODO: ideally we can rm this once we no longer have + # a `._ria_nursery` since the dependent APIs have been + # removed! + nursery: trio.Nursery|None = None, + ) -> Portal: ''' Start a (daemon) actor: an process that has no designated @@ -200,6 +212,7 @@ class ActorNursery: # |_ dynamic @context decoration on child side # |_ implicit `Portal.open_context() as (ctx, first):` # and `return first` on parent side. + # |_ mention how it's similar to `trio-parallel` API? # -[ ] use @api_frame on the wrapper async def run_in_actor( self, @@ -269,11 +282,14 @@ class ActorNursery: ) -> None: ''' - Cancel this nursery by instructing each subactor to cancel - itself and wait for all subactors to terminate. + Cancel this actor-nursery by instructing each subactor's + runtime to cancel and wait for all underlying sub-processes + to terminate. - If ``hard_killl`` is set to ``True`` then kill the processes - directly without any far end graceful ``trio`` cancellation. + If `hard_kill` is set then kill the processes directly using + the spawning-backend's API/OS-machinery without any attempt + at (graceful) `trio`-style cancellation using our + `Actor.cancel()`. ''' __runtimeframe__: int = 1 # noqa @@ -629,8 +645,12 @@ async def open_nursery( f'|_{an}\n' ) - # shutdown runtime if it was started if implicit_runtime: + # shutdown runtime if it was started and report noisly + # that we're did so. msg += '=> Shutting down actor runtime <=\n' + log.info(msg) - log.info(msg) + else: + # keep noise low during std operation. + log.runtime(msg) -- 2.34.1 From 5739e796457d618175f95ef75a52f88e5549f287 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 27 Jun 2024 19:27:59 -0400 Subject: [PATCH 07/42] 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 fb18ba8..5564d0e 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! -- 2.34.1 From b72a025d0f554c2f6422c19efd7619cbbfae8537 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 28 Jun 2024 12:59:31 -0400 Subject: [PATCH 08/42] Always reset `._state._ctxvar_Context` to prior Not sure how I forgot this but, obviously it's correct context-var semantics to revert the current IPC `Context` (set in the latest `.open_context()` block) such that any prior instance is reset.. This ensures the sanity `assert`s pass inside `.msg._ops.maybe_limit_plds()` and just in general ensures for any task that the last opened `Context` is the one returned from `current_ipc_ctx()`. --- tractor/_context.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tractor/_context.py b/tractor/_context.py index 620363c..6b5f2eb 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -38,6 +38,7 @@ from collections import deque from contextlib import ( asynccontextmanager as acm, ) +from contextvars import Token from dataclasses import ( dataclass, field, @@ -1943,7 +1944,7 @@ async def open_context_from_portal( ) assert ctx._remote_func_type == 'context' assert ctx._caller_info - _ctxvar_Context.set(ctx) + prior_ctx_tok: Token = _ctxvar_Context.set(ctx) # placeholder for any exception raised in the runtime # or by user tasks which cause this context's closure. @@ -2394,6 +2395,9 @@ async def open_context_from_portal( None, ) + # XXX revert to prior IPC-task-ctx scope + _ctxvar_Context.reset(prior_ctx_tok) + def mk_context( chan: Channel, -- 2.34.1 From 5e009a822966dd3731862363cc7df9f74319a5c4 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 28 Jun 2024 14:25:53 -0400 Subject: [PATCH 09/42] Further formalize `greenback` integration Since we more or less require it for `tractor.pause_from_sync()` this refines enable toggles and their relay down the actor tree as well as more explicit logging around init and activation. Tweaks summary: - `.info()` report the module if discovered during root boot. - use a `._state._runtime_vars['use_greenback']: bool` activation flag inside `Actor._from_parent()` to determine if the sub should try to use it and set to `False` if mod-loading fails / not installed. - expose `maybe_init_greenback()` from `.devx` sugpkg. - comment out RTE in `._pause()` for now since we already have it in `.pause_from_sync()`. - always `.exception()` on `maybe_init_greenback()` import errors to clarify the underlying failure deats. - always explicitly report if `._state._runtime_vars['use_greenback']` was NOT set when `.pause_from_sync()` is called. Other `._runtime.async_main()` adjustments: - combine the "internal error call ur parents" message and the failed registry contact status into one new `err_report: str`. - drop the final exception handler's call to `Actor.lifetime_stack.close()` since we're already doing it in the `finally:` block and the earlier call has no currently known benefit. - only report on the `.lifetime_stack()` callbacks if any are detected as registered. --- tractor/_root.py | 30 ++++++--- tractor/_runtime.py | 137 +++++++++++++++++++++++++++------------ tractor/_state.py | 2 +- tractor/devx/__init__.py | 1 + tractor/devx/_debug.py | 87 ++++++++++++++++--------- 5 files changed, 175 insertions(+), 82 deletions(-) diff --git a/tractor/_root.py b/tractor/_root.py index 7cdef60..882285a 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -21,6 +21,7 @@ Root actor runtime ignition(s). from contextlib import asynccontextmanager as acm from functools import partial import importlib +import inspect import logging import os import signal @@ -115,10 +116,16 @@ async def open_root_actor( if ( debug_mode and maybe_enable_greenback - and await _debug.maybe_init_greenback( - raise_not_found=False, + and ( + maybe_mod := await _debug.maybe_init_greenback( + raise_not_found=False, + ) ) ): + logger.info( + f'Found `greenback` installed @ {maybe_mod}\n' + 'Enabling `tractor.pause_from_sync()` support!\n' + ) os.environ['PYTHONBREAKPOINT'] = ( 'tractor.devx._debug._sync_pause_from_builtin' ) @@ -264,7 +271,9 @@ async def open_root_actor( except OSError: # TODO: make this a "discovery" log level? - logger.warning(f'No actor registry found @ {addr}') + logger.info( + f'No actor registry found @ {addr}\n' + ) async with trio.open_nursery() as tn: for addr in registry_addrs: @@ -278,7 +287,6 @@ async def open_root_actor( # Create a new local root-actor instance which IS NOT THE # REGISTRAR if ponged_addrs: - if ensure_registry: raise RuntimeError( f'Failed to open `{name}`@{ponged_addrs}: ' @@ -365,23 +373,25 @@ async def open_root_actor( ) try: yield actor - except ( Exception, BaseExceptionGroup, ) as err: - - import inspect + # XXX NOTE XXX see equiv note inside + # `._runtime.Actor._stream_handler()` where in the + # non-root or root-that-opened-this-mahually case we + # wait for the local actor-nursery to exit before + # exiting the transport channel handler. entered: bool = await _debug._maybe_enter_pm( err, api_frame=inspect.currentframe(), ) - if ( not entered - and not is_multi_cancelled(err) + and + not is_multi_cancelled(err) ): - logger.exception('Root actor crashed:\n') + logger.exception('Root actor crashed\n') # ALWAYS re-raise any error bubbled up from the # runtime! diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 3cf35ff..f472c06 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -1046,6 +1046,10 @@ class Actor: # TODO: another `Struct` for rtvs.. rvs: dict[str, Any] = spawnspec._runtime_vars if rvs['_debug_mode']: + from .devx import ( + enable_stack_on_sig, + maybe_init_greenback, + ) try: # TODO: maybe return some status msgs upward # to that we can emit them in `con_status` @@ -1053,13 +1057,27 @@ class Actor: log.devx( 'Enabling `stackscope` traces on SIGUSR1' ) - from .devx import enable_stack_on_sig enable_stack_on_sig() + except ImportError: log.warning( '`stackscope` not installed for use in debug mode!' ) + if rvs.get('use_greenback', False): + maybe_mod: ModuleType|None = await maybe_init_greenback() + if maybe_mod: + log.devx( + 'Activated `greenback` ' + 'for `tractor.pause_from_sync()` support!' + ) + else: + rvs['use_greenback'] = False + log.warning( + '`greenback` not installed for use in debug mode!\n' + '`tractor.pause_from_sync()` not available!' + ) + rvs['_is_root'] = False _state._runtime_vars.update(rvs) @@ -1717,8 +1735,8 @@ async def async_main( # Register with the arbiter if we're told its addr log.runtime( - f'Registering `{actor.name}` ->\n' - f'{pformat(accept_addrs)}' + f'Registering `{actor.name}` => {pformat(accept_addrs)}\n' + # ^-TODO-^ we should instead show the maddr here^^ ) # TODO: ideally we don't fan out to all registrars @@ -1776,57 +1794,90 @@ async def async_main( # Blocks here as expected until the root nursery is # killed (i.e. this actor is cancelled or signalled by the parent) - except Exception as err: - log.runtime("Closing all actor lifetime contexts") - actor.lifetime_stack.close() - + except Exception as internal_err: if not is_registered: + err_report: str = ( + '\n' + "Actor runtime (internally) failed BEFORE contacting the registry?\n" + f'registrars -> {actor.reg_addrs} ?!?!\n\n' + + '^^^ THIS IS PROBABLY AN INTERNAL `tractor` BUG! ^^^\n\n' + '\t>> CALMLY CANCEL YOUR CHILDREN AND CALL YOUR PARENTS <<\n\n' + + '\tIf this is a sub-actor hopefully its parent will keep running ' + 'and cancel/reap this sub-process..\n' + '(well, presuming this error was propagated upward)\n\n' + + '\t---------------------------------------------\n' + '\tPLEASE REPORT THIS TRACEBACK IN A BUG REPORT @ ' # oneline + 'https://github.com/goodboy/tractor/issues\n' + '\t---------------------------------------------\n' + ) + # TODO: I guess we could try to connect back # to the parent through a channel and engage a debugger # once we have that all working with std streams locking? - log.exception( - f"Actor errored and failed to register with arbiter " - f"@ {actor.reg_addrs[0]}?") - log.error( - "\n\n\t^^^ THIS IS PROBABLY AN INTERNAL `tractor` BUG! ^^^\n\n" - "\t>> CALMLY CALL THE AUTHORITIES AND HIDE YOUR CHILDREN <<\n\n" - "\tIf this is a sub-actor hopefully its parent will keep running " - "correctly presuming this error was safely ignored..\n\n" - "\tPLEASE REPORT THIS TRACEBACK IN A BUG REPORT: " - "https://github.com/goodboy/tractor/issues\n" - ) + log.exception(err_report) if actor._parent_chan: await try_ship_error_to_remote( actor._parent_chan, - err, + internal_err, ) # always! - match err: + match internal_err: case ContextCancelled(): log.cancel( f'Actor: {actor.uid} was task-context-cancelled with,\n' - f'str(err)' + f'str(internal_err)' ) case _: - log.exception("Actor errored:") - raise + log.exception( + 'Main actor-runtime task errored\n' + f' Closing all actor lifetime contexts..' + teardown_report: str = ( + 'Main actor-runtime task completed\n' ) - # tear down all lifetime contexts if not in guest mode - # XXX: should this just be in the entrypoint? - actor.lifetime_stack.close() - # TODO: we can't actually do this bc the debugger - # uses the _service_n to spawn the lock task, BUT, - # in theory if we had the root nursery surround this finally - # block it might be actually possible to debug THIS - # machinery in the same way as user task code? + # ?TODO? should this be in `._entry`/`._root` mods instead? + # + # teardown any actor-lifetime-bound contexts + ls: ExitStack = actor.lifetime_stack + # only report if there are any registered + cbs: list[Callable] = [ + repr(tup[1].__wrapped__) + for tup in ls._exit_callbacks + ] + if cbs: + cbs_str: str = '\n'.join(cbs) + teardown_report += ( + '-> Closing actor-lifetime-bound callbacks\n\n' + f'}}>\n' + f' |_{ls}\n' + f' |_{cbs_str}\n' + ) + # XXX NOTE XXX this will cause an error which + # prevents any `infected_aio` actor from continuing + # and any callbacks in the `ls` here WILL NOT be + # called!! + # await _debug.pause(shield=True) + + ls.close() + + # XXX TODO but hard XXX + # we can't actually do this bc the debugger uses the + # _service_n to spawn the lock task, BUT, in theory if we had + # the root nursery surround this finally block it might be + # actually possible to debug THIS machinery in the same way + # as user task code? + # # if actor.name == 'brokerd.ib': # with CancelScope(shield=True): # await _debug.breakpoint() @@ -1856,9 +1907,9 @@ async def async_main( failed = True if failed: - log.warning( - f'Failed to unregister {actor.name} from ' - f'registar @ {addr}' + teardown_report += ( + f'-> Failed to unregister {actor.name} from ' + f'registar @ {addr}\n' ) # Ensure all peers (actors connected to us as clients) are finished @@ -1866,13 +1917,17 @@ async def async_main( if any( chan.connected() for chan in chain(*actor._peers.values()) ): - log.runtime( - f"Waiting for remaining peers {actor._peers} to clear") + teardown_report += ( + f'-> Waiting for remaining peers {actor._peers} to clear..\n' + ) + log.runtime(teardown_report) with CancelScope(shield=True): await actor._no_more_peers.wait() - log.runtime("All peer channels are complete") - log.runtime("Runtime completed") + teardown_report += ('-> All peer channels are complete\n') + + teardown_report += ('Actor runtime exited') + log.info(teardown_report) # TODO: rename to `Registry` and move to `._discovery`! diff --git a/tractor/_state.py b/tractor/_state.py index 8c5cca1..9f89600 100644 --- a/tractor/_state.py +++ b/tractor/_state.py @@ -44,7 +44,7 @@ _runtime_vars: dict[str, Any] = { '_root_mailbox': (None, None), '_registry_addrs': [], - # for `breakpoint()` support + # for `tractor.pause_from_sync()` & `breakpoint()` support 'use_greenback': False, } diff --git a/tractor/devx/__init__.py b/tractor/devx/__init__.py index ab9d2d1..86dd128 100644 --- a/tractor/devx/__init__.py +++ b/tractor/devx/__init__.py @@ -29,6 +29,7 @@ from ._debug import ( shield_sigint_handler as shield_sigint_handler, open_crash_handler as open_crash_handler, maybe_open_crash_handler as maybe_open_crash_handler, + maybe_init_greenback as maybe_init_greenback, post_mortem as post_mortem, mk_pdb as mk_pdb, ) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index ccf57d6..1135932 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -69,6 +69,7 @@ from trio import ( import tractor from tractor.log import get_logger from tractor._context import Context +from tractor import _state from tractor._state import ( current_actor, is_root_process, @@ -87,9 +88,6 @@ if TYPE_CHECKING: from tractor._runtime import ( Actor, ) - from tractor.msg import ( - _codec, - ) log = get_logger(__name__) @@ -1599,12 +1597,16 @@ async def _pause( try: task: Task = current_task() except RuntimeError as rte: - log.exception('Failed to get current task?') - if actor.is_infected_aio(): - raise RuntimeError( - '`tractor.pause[_from_sync]()` not yet supported ' - 'for infected `asyncio` mode!' - ) from 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 @@ -2163,22 +2165,22 @@ def maybe_import_greenback( return False -async def maybe_init_greenback( - **kwargs, -) -> None|ModuleType: - - if mod := maybe_import_greenback(**kwargs): - await mod.ensure_portal() - log.devx( - '`greenback` portal opened!\n' - 'Sync debug support activated!\n' - ) - return mod +async def maybe_init_greenback(**kwargs) -> None|ModuleType: + try: + if mod := maybe_import_greenback(**kwargs): + await mod.ensure_portal() + log.devx( + '`greenback` portal opened!\n' + 'Sync debug support activated!\n' + ) + return mod + except BaseException: + log.exception('Failed to init `greenback`..') + raise return None - async def _pause_from_bg_root_thread( behalf_of_thread: Thread, repl: PdbREPL, @@ -2324,6 +2326,12 @@ def pause_from_sync( # 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 ' @@ -2399,18 +2407,37 @@ def pause_from_sync( else: # we are presumably the `trio.run()` + main thread # raises on not-found by default greenback: ModuleType = maybe_import_greenback() + + # TODO: how to ensure this is either dynamically (if + # needed) called here (in some bg tn??) or that the + # subactor always already called it? + # greenback: ModuleType = await maybe_init_greenback() + message += f'-> imported {greenback}\n' repl_owner: Task = current_task() message += '-> calling `greenback.await_(_pause(debug_func=None))` from sync caller..\n' - out = greenback.await_( - _pause( - debug_func=None, - repl=repl, - hide_tb=hide_tb, - called_from_sync=True, - **_pause_kwargs, + try: + out = greenback.await_( + _pause( + debug_func=None, + repl=repl, + hide_tb=hide_tb, + called_from_sync=True, + **_pause_kwargs, + ) ) - ) + except RuntimeError as rte: + if not _state._runtime_vars.get( + 'use_greenback', + False, + ): + raise RuntimeError( + '`greenback` was never initialized in this actor!?\n\n' + f'{_state._runtime_vars}\n' + ) from rte + + raise + if out: bg_task, repl = out assert repl is repl @@ -2801,10 +2828,10 @@ def open_crash_handler( `trio.run()`. ''' + err: BaseException try: yield except tuple(catch) as err: - if type(err) not in ignore: pdbp.xpm() -- 2.34.1 From cb90f3e6ba4fe0e241dd58f33b78dd3c1013e419 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 28 Jun 2024 14:46:29 -0400 Subject: [PATCH 10/42] Update `MsgTypeError` content matching to latest --- tests/test_pldrx_limiting.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/test_pldrx_limiting.py b/tests/test_pldrx_limiting.py index e5ce691..bb9a3ef 100644 --- a/tests/test_pldrx_limiting.py +++ b/tests/test_pldrx_limiting.py @@ -285,14 +285,14 @@ def test_basic_payload_spec( if invalid_started: msg_type_str: str = 'Started' - bad_value_str: str = '10' + bad_value: int = 10 elif invalid_return: msg_type_str: str = 'Return' - bad_value_str: str = "'yo'" + bad_value: str = 'yo' else: # XXX but should never be used below then.. msg_type_str: str = '' - bad_value_str: str = '' + bad_value: str = '' maybe_mte: MsgTypeError|None = None should_raise: Exception|None = ( @@ -307,8 +307,10 @@ def test_basic_payload_spec( raises=should_raise, ensure_in_message=[ f"invalid `{msg_type_str}` msg payload", - f"value: `{bad_value_str}` does not " - f"match type-spec: `{msg_type_str}.pld: PldMsg|NoneType`", + f'{bad_value}', + f'has type {type(bad_value)!r}', + 'not match type-spec', + f'`{msg_type_str}.pld: PldMsg|NoneType`', ], # only for debug # post_mortem=True, -- 2.34.1 From 4fbd469c332d50f923859caad2cdd74c98913469 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 28 Jun 2024 18:45:52 -0400 Subject: [PATCH 11/42] Update `._entry` actor status log Log-report the different types of actor exit conditions including cancel via KBI, error or normal return with varying levels depending on case. Also, start proto-ing out this weird ascii-syntax idea for describing conc system states and implement the first bit in a `nest_from_op()` log-message fmter that joins and indents an obj `repr()` with a tree-like `'>)\n|_'` header. --- tractor/_entry.py | 160 ++++++++++++++++++++++++++++++++++++++++------ 1 file changed, 141 insertions(+), 19 deletions(-) diff --git a/tractor/_entry.py b/tractor/_entry.py index e22a4f1..60050ea 100644 --- a/tractor/_entry.py +++ b/tractor/_entry.py @@ -20,7 +20,8 @@ Sub-process entry points. """ from __future__ import annotations from functools import partial -# import textwrap +import os +import textwrap from typing import ( Any, TYPE_CHECKING, @@ -58,7 +59,7 @@ def _mp_main( ) -> None: ''' - The routine called *after fork* which invokes a fresh ``trio.run`` + The routine called *after fork* which invokes a fresh `trio.run()` ''' actor._forkserver_info = forkserver_info @@ -96,6 +97,107 @@ def _mp_main( log.info(f"Subactor {actor.uid} terminated") +# TODO: move this func to some kinda `.devx._conc_lang.py` eventually +# as we work out our multi-domain state-flow-syntax! +def nest_from_op( + input_op: str, + # + # ?TODO? an idea for a syntax to the state of concurrent systems + # as a "3-domain" (execution, scope, storage) model and using + # a minimal ascii/utf-8 operator-set. + # + # try not to take any of this seriously yet XD + # + # > is a "play operator" indicating (CPU bound) + # exec/work/ops required at the "lowest level computing" + # + # execution primititves (tasks, threads, actors..) denote their + # lifetime with '(' and ')' since parentheses normally are used + # in many langs to denote function calls. + # + # starting = ( + # >( opening/starting; beginning of the thread-of-exec (toe?) + # (> opened/started, (finished spawning toe) + # |_ repr of toe, in py these look like + # + # >) closing/exiting/stopping, + # )> closed/exited/stopped, + # |_ + # [OR <), )< ?? ] + # + # ending = ) + # >c) cancelling to close/exit + # c)> cancelled (caused close), OR? + # |_ + # OR maybe "x) erroring to eventuall exit + # x)> errored and terminated + # |_ + # + # scopes: supers/nurseries, IPC-ctxs, sessions, perms, etc. + # >{ opening + # {> opened + # }> closed + # >} closing + # + # storage: like queues, shm-buffers, files, etc.. + # >[ opening + # [> opened + # |_ + # + # >] closing + # ]> closed + + # IPC ops: channels, transports, msging + # => req msg + # <= resp msg + # <=> 2-way streaming (of msgs) + # <- recv 1 msg + # -> send 1 msg + # + # TODO: still not sure on R/L-HS approach..? + # =>( send-req to exec start (task, actor, thread..) + # (<= recv-req to ^ + # + # (<= recv-req ^ + # <=( recv-resp opened remote exec primitive + # <=) recv-resp closed + # + # )<=c req to stop due to cancel + # c=>) req to stop due to cancel + # + # =>{ recv-req to open + # <={ send-status that it closed + + tree_str: str, + + # NOTE: so move back-from-the-left of the `input_op` by + # this amount. + back_from_op: int = 0, +) -> str: + ''' + Depth-increment the input (presumably hierarchy/supervision) + input "tree string" below the provided `input_op` execution + operator, so injecting a `"\n|_{input_op}\n"`and indenting the + `tree_str` to nest content aligned with the ops last char. + + ''' + return ( + f'{input_op}\n' + + + textwrap.indent( + tree_str, + prefix=( + len(input_op) + - + (back_from_op + 1) + ) * ' ', + ) + ) + + def _trio_main( actor: Actor, *, @@ -107,7 +209,6 @@ def _trio_main( Entry point for a `trio_run_in_process` subactor. ''' - # __tracebackhide__: bool = True _debug.hide_runtime_frames() _state._current_actor = actor @@ -119,7 +220,6 @@ def _trio_main( if actor.loglevel is not None: get_console_log(actor.loglevel) - import os actor_info: str = ( f'|_{actor}\n' f' uid: {actor.uid}\n' @@ -128,13 +228,23 @@ def _trio_main( f' loglevel: {actor.loglevel}\n' ) log.info( - 'Started new trio subactor:\n' + 'Starting new `trio` subactor:\n' + - '>\n' # like a "started/play"-icon from super perspective - + - actor_info, + nest_from_op( + input_op='>(', # see syntax ideas above + tree_str=actor_info, + back_from_op=1, + ) ) - + logmeth = log.info + exit_status: str = ( + 'Subactor exited\n' + + + nest_from_op( + input_op=')>', # like a "closed-to-play"-icon from super perspective + tree_str=actor_info, + ) + ) try: if infect_asyncio: actor._infected_aio = True @@ -143,16 +253,28 @@ def _trio_main( trio.run(trio_main) except KeyboardInterrupt: - log.cancel( - 'Actor received KBI\n' + logmeth = log.cancel + exit_status: str = ( + 'Actor received KBI (aka an OS-cancel)\n' + - actor_info + nest_from_op( + input_op='c)>', # closed due to cancel (see above) + tree_str=actor_info, + ) ) + except BaseException as err: + logmeth = log.error + exit_status: str = ( + 'Main actor task crashed during exit?\n' + + + nest_from_op( + input_op='x)>', # closed by error + tree_str=actor_info, + ) + ) + # NOTE since we raise a tb will already be shown on the + # console, thus we do NOT use `.exception()` above. + raise err + finally: - log.info( - 'Subactor terminated\n' - + - 'x\n' # like a "crossed-out/killed" from super perspective - + - actor_info - ) + logmeth(exit_status) -- 2.34.1 From 7e93b81a83996472657fa2c01f87979d455a774f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 28 Jun 2024 19:06:17 -0400 Subject: [PATCH 12/42] Don't use pretty struct stuff in `._invoke` It's too fragile to put in side core RPC machinery since `msgspec.Struct` defs can fail if a field type can't be looked up at creation time (like can easily happen if you conditionally import using `if TYPE_CHECKING:`) Also, - rename `cs` to `rpc_ctx_cs: CancelScope` since it's literally the wrapping RPC `Context._scope`. - report self cancellation via `explain: str` and add tail case for "unknown cause". - put a ?TODO? around what to do about KBIs if a context is opened from an `infected_aio`-actor task. - similar to our nursery and portal add TODO list for moving all `_invoke_non_context()` content out the RPC core and instead implement them as `.hilevel` endpoint helpers (maybe as decorators?)which under neath define `@context`-funcs. --- tractor/_rpc.py | 70 +++++++++++++++++++++++++++++++------------------ 1 file changed, 45 insertions(+), 25 deletions(-) diff --git a/tractor/_rpc.py b/tractor/_rpc.py index 166ee96..fc8687c 100644 --- a/tractor/_rpc.py +++ b/tractor/_rpc.py @@ -68,7 +68,7 @@ from .msg import ( MsgCodec, PayloadT, NamespacePath, - pretty_struct, + # pretty_struct, _ops as msgops, ) from tractor.msg.types import ( @@ -89,6 +89,16 @@ if TYPE_CHECKING: log = get_logger('tractor') +# ?TODO? move to a `tractor.lowlevel._rpc` with the below +# func-type-cases implemented "on top of" `@context` defs: +# -[ ] std async func helper decorated with `@rpc_func`? +# -[ ] `Portal.open_stream_from()` with async-gens? +# |_ possibly a duplex form of this with a +# `sent_from_peer = yield send_to_peer` form, which would require +# syncing the send/recv side with possibly `.receive_nowait()` +# on each `yield`? +# -[ ] some kinda `@rpc_acm` maybe that does a fixture style with +# user only defining a single-`yield` generator-func? async def _invoke_non_context( actor: Actor, cancel_scope: CancelScope, @@ -108,8 +118,9 @@ async def _invoke_non_context( ] = trio.TASK_STATUS_IGNORED, ): __tracebackhide__: bool = True + cs: CancelScope|None = None # ref when activated - # TODO: can we unify this with the `context=True` impl below? + # ?TODO? can we unify this with the `context=True` impl below? if inspect.isasyncgen(coro): await chan.send( StartAck( @@ -160,10 +171,6 @@ async def _invoke_non_context( functype='asyncgen', ) ) - # XXX: the async-func may spawn further tasks which push - # back values like an async-generator would but must - # manualy construct the response dict-packet-responses as - # above with cancel_scope as cs: ctx._scope = cs task_status.started(ctx) @@ -175,15 +182,13 @@ async def _invoke_non_context( await chan.send( Stop(cid=cid) ) + + # simplest function/method request-response pattern + # XXX: in the most minimally used case, just a scheduled internal runtime + # call to `Actor._cancel_task()` from the ctx-peer task since we + # don't (yet) have a dedicated IPC msg. + # ------ - ------ else: - # regular async function/method - # XXX: possibly just a scheduled `Actor._cancel_task()` - # from a remote request to cancel some `Context`. - # ------ - ------ - # TODO: ideally we unify this with the above `context=True` - # block such that for any remote invocation ftype, we - # always invoke the far end RPC task scheduling the same - # way: using the linked IPC context machinery. failed_resp: bool = False try: ack = StartAck( @@ -354,8 +359,15 @@ async def _errors_relayed_via_ipc( # channel. task_status.started(err) - # always reraise KBIs so they propagate at the sys-process level. - if isinstance(err, KeyboardInterrupt): + # always propagate KBIs at the sys-process level. + if ( + isinstance(err, KeyboardInterrupt) + + # ?TODO? except when running in asyncio mode? + # |_ wut if you want to open a `@context` FROM an + # infected_aio task? + # and not actor.is_infected_aio() + ): raise # RPC task bookeeping. @@ -458,7 +470,6 @@ async def _invoke( # tb: TracebackType = None cancel_scope = CancelScope() - cs: CancelScope|None = None # ref when activated ctx = actor.get_context( chan=chan, cid=cid, @@ -607,6 +618,8 @@ async def _invoke( # `@context` marked RPC function. # - `._portal` is never set. try: + tn: trio.Nursery + rpc_ctx_cs: CancelScope async with ( trio.open_nursery() as tn, msgops.maybe_limit_plds( @@ -616,7 +629,7 @@ async def _invoke( ), ): ctx._scope_nursery = tn - ctx._scope = tn.cancel_scope + rpc_ctx_cs = ctx._scope = tn.cancel_scope task_status.started(ctx) # TODO: better `trionics` tooling: @@ -642,7 +655,7 @@ async def _invoke( # itself calls `ctx._maybe_cancel_and_set_remote_error()` # which cancels the scope presuming the input error # is not a `.cancel_acked` pleaser. - if ctx._scope.cancelled_caught: + if rpc_ctx_cs.cancelled_caught: our_uid: tuple = actor.uid # first check for and raise any remote error @@ -652,9 +665,7 @@ async def _invoke( if re := ctx._remote_error: ctx._maybe_raise_remote_err(re) - cs: CancelScope = ctx._scope - - if cs.cancel_called: + if rpc_ctx_cs.cancel_called: canceller: tuple = ctx.canceller explain: str = f'{ctx.side!r}-side task was cancelled by ' @@ -680,9 +691,15 @@ async def _invoke( elif canceller == ctx.chan.uid: explain += f'its {ctx.peer_side!r}-side peer' - else: + elif canceller == our_uid: + explain += 'itself' + + elif canceller: explain += 'a remote peer' + else: + explain += 'an unknown cause?' + explain += ( add_div(message=explain) + @@ -911,7 +928,10 @@ async def process_messages( f'IPC msg from peer\n' f'<= {chan.uid}\n\n' - # TODO: avoid fmting depending on loglevel for perf? + # TODO: use of the pprinting of structs is + # FRAGILE and should prolly not be + # + # avoid fmting depending on loglevel for perf? # -[ ] specifically `pretty_struct.pformat()` sub-call..? # - how to only log-level-aware actually call this? # -[ ] use `.msg.pretty_struct` here now instead! @@ -1238,7 +1258,7 @@ async def process_messages( 'Exiting IPC msg loop with final msg\n\n' f'<= peer: {chan.uid}\n' f' |_{chan}\n\n' - f'{pretty_struct.pformat(msg)}' + # f'{pretty_struct.pformat(msg)}' ) log.runtime(message) -- 2.34.1 From edac717613c3281a70a1e046fb3a2d982f45c6ab Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 28 Jun 2024 19:17:05 -0400 Subject: [PATCH 13/42] Use `msgspec.Struct.__repr__()` failover impl In case the struct doesn't import a field type (which will cause the `.pformat()` to raise) just report the issue and try to fall back to the original `repr()` version. --- tractor/msg/pretty_struct.py | 21 +++++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/tractor/msg/pretty_struct.py b/tractor/msg/pretty_struct.py index f27fb89..15e469e 100644 --- a/tractor/msg/pretty_struct.py +++ b/tractor/msg/pretty_struct.py @@ -34,6 +34,9 @@ from pprint import ( saferepr, ) +from tractor.log import get_logger + +log = get_logger() # TODO: auto-gen type sig for input func both for # type-msgs and logging of RPC tasks? # taken and modified from: @@ -143,7 +146,13 @@ def pformat( else: # the `pprint` recursion-safe format: # https://docs.python.org/3.11/library/pprint.html#pprint.saferepr - val_str: str = saferepr(v) + try: + val_str: str = saferepr(v) + except Exception: + log.exception( + 'Failed to `saferepr({type(struct)})` !?\n' + ) + return _Struct.__repr__(struct) # TODO: LOLOL use `textwrap.indent()` instead dawwwwwg! obj_str += (field_ws + f'{k}: {typ_name} = {val_str},\n') @@ -194,12 +203,20 @@ 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: # ... - __repr__ = pformat + def __repr__(self) -> str: + try: + return pformat(self) + except Exception: + log.exception( + f'Failed to `pformat({type(self)})` !?\n' + ) + return _Struct.__repr__(self) def copy( self, -- 2.34.1 From 18d440c2074a1a2d22f0146afd73fe2aac8083e1 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 28 Jun 2024 19:24:03 -0400 Subject: [PATCH 14/42] (Re)type annot some tests - For the (still not finished) `test_caps_based_msging`, switch to using the new `PayloadMsg`. - add `testdir` fixture type. --- tests/test_caps_based_msging.py | 45 ++++++++++++--------------------- tests/test_docs_examples.py | 2 +- 2 files changed, 17 insertions(+), 30 deletions(-) diff --git a/tests/test_caps_based_msging.py b/tests/test_caps_based_msging.py index 9a73ba8..6064c2c 100644 --- a/tests/test_caps_based_msging.py +++ b/tests/test_caps_based_msging.py @@ -11,9 +11,6 @@ from typing import ( Type, Union, ) -from contextvars import ( - Context, -) from msgspec import ( structs, @@ -27,6 +24,7 @@ import tractor from tractor import ( _state, MsgTypeError, + Context, ) from tractor.msg import ( _codec, @@ -41,7 +39,7 @@ from tractor.msg import ( from tractor.msg.types import ( _payload_msgs, log, - Msg, + PayloadMsg, Started, mk_msg_spec, ) @@ -61,7 +59,7 @@ def mk_custom_codec( uid: tuple[str, str] = tractor.current_actor().uid # XXX NOTE XXX: despite defining `NamespacePath` as a type - # field on our `Msg.pld`, we still need a enc/dec_hook() pair + # field on our `PayloadMsg.pld`, we still need a enc/dec_hook() pair # to cast to/from that type on the wire. See the docs: # https://jcristharif.com/msgspec/extending.html#mapping-to-from-native-types @@ -321,12 +319,12 @@ def dec_type_union( import importlib types: list[Type] = [] for type_name in type_names: - for ns in [ + for mod in [ typing, importlib.import_module(__name__), ]: if type_ref := getattr( - ns, + mod, type_name, False, ): @@ -744,7 +742,7 @@ def chk_pld_type( # 'Error', .pld: ErrorData codec: MsgCodec = mk_codec( - # NOTE: this ONLY accepts `Msg.pld` fields of a specified + # NOTE: this ONLY accepts `PayloadMsg.pld` fields of a specified # type union. ipc_pld_spec=payload_spec, ) @@ -752,7 +750,7 @@ def chk_pld_type( # make a one-off dec to compare with our `MsgCodec` instance # which does the below `mk_msg_spec()` call internally ipc_msg_spec: Union[Type[Struct]] - msg_types: list[Msg[payload_spec]] + msg_types: list[PayloadMsg[payload_spec]] ( ipc_msg_spec, msg_types, @@ -761,7 +759,7 @@ def chk_pld_type( ) _enc = msgpack.Encoder() _dec = msgpack.Decoder( - type=ipc_msg_spec or Any, # like `Msg[Any]` + type=ipc_msg_spec or Any, # like `PayloadMsg[Any]` ) assert ( @@ -806,7 +804,7 @@ def chk_pld_type( 'cid': '666', 'pld': pld, } - enc_msg: Msg = typedef(**kwargs) + enc_msg: PayloadMsg = typedef(**kwargs) _wire_bytes: bytes = _enc.encode(enc_msg) wire_bytes: bytes = codec.enc.encode(enc_msg) @@ -883,25 +881,16 @@ def test_limit_msgspec(): debug_mode=True ): - # ensure we can round-trip a boxing `Msg` + # ensure we can round-trip a boxing `PayloadMsg` assert chk_pld_type( - # Msg, - Any, - None, + payload_spec=Any, + pld=None, expect_roundtrip=True, ) - # TODO: don't need this any more right since - # `msgspec>=0.15` has the nice generics stuff yah?? - # - # manually override the type annot of the payload - # field and ensure it propagates to all msg-subtypes. - # Msg.__annotations__['pld'] = Any - # verify that a mis-typed payload value won't decode assert not chk_pld_type( - # Msg, - int, + payload_spec=int, pld='doggy', ) @@ -913,18 +902,16 @@ def test_limit_msgspec(): value: Any assert not chk_pld_type( - # Msg, - CustomPayload, + payload_spec=CustomPayload, pld='doggy', ) assert chk_pld_type( - # Msg, - CustomPayload, + payload_spec=CustomPayload, pld=CustomPayload(name='doggy', value='urmom') ) - # uhh bc we can `.pause_from_sync()` now! :surfer: + # yah, we can `.pause_from_sync()` now! # breakpoint() trio.run(main) diff --git a/tests/test_docs_examples.py b/tests/test_docs_examples.py index 79a2200..3a1d2f2 100644 --- a/tests/test_docs_examples.py +++ b/tests/test_docs_examples.py @@ -19,7 +19,7 @@ from tractor._testing import ( @pytest.fixture def run_example_in_subproc( loglevel: str, - testdir, + testdir: pytest.Testdir, reg_addr: tuple[str, int], ): -- 2.34.1 From ba83bab776ab6d4ba88c22d5de6be320bdaa6bb1 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 28 Jun 2024 19:26:35 -0400 Subject: [PATCH 15/42] 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 43dadbb..71c691a 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(): ''' -- 2.34.1 From e3d59964afd4e2ea94209126b61e4abd7b945aee Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 28 Jun 2024 19:27:13 -0400 Subject: [PATCH 16/42] Woops, set `.cancel()` level in custom levels table.. --- tractor/log.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tractor/log.py b/tractor/log.py index edb058e..47f1f25 100644 --- a/tractor/log.py +++ b/tractor/log.py @@ -54,11 +54,12 @@ LOG_FORMAT = ( DATE_FORMAT = '%b %d %H:%M:%S' # FYI, ERROR is 40 +# TODO: use a `bidict` to avoid the :155 check? CUSTOM_LEVELS: dict[str, int] = { 'TRANSPORT': 5, 'RUNTIME': 15, 'DEVX': 17, - 'CANCEL': 18, + 'CANCEL': 22, 'PDB': 500, } STD_PALETTE = { @@ -147,6 +148,8 @@ class StackLevelAdapter(LoggerAdapter): Delegate a log call to the underlying logger, after adding contextual information from this adapter instance. + NOTE: all custom level methods (above) delegate to this! + ''' if self.isEnabledFor(level): stacklevel: int = 3 -- 2.34.1 From 3907cba68e881e0c66d32adab74de4f6b56c437b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 28 Jun 2024 19:28:12 -0400 Subject: [PATCH 17/42] Refine some `.trionics` docs and logging - allow passing and report the lib name (`trio` or `tractor`) from `maybe_open_nursery()`. - use `.runtime()` level when reporting `_Cache`-hits in `maybe_open_context()`. - tidy up some doc strings. --- tractor/trionics/_broadcast.py | 9 +++--- tractor/trionics/_mngrs.py | 53 ++++++++++++++++++++++------------ 2 files changed, 39 insertions(+), 23 deletions(-) diff --git a/tractor/trionics/_broadcast.py b/tractor/trionics/_broadcast.py index a5d3187..977b682 100644 --- a/tractor/trionics/_broadcast.py +++ b/tractor/trionics/_broadcast.py @@ -156,11 +156,12 @@ class BroadcastState(Struct): class BroadcastReceiver(ReceiveChannel): ''' - A memory receive channel broadcaster which is non-lossy for the - fastest consumer. + A memory receive channel broadcaster which is non-lossy for + the fastest consumer. - Additional consumer tasks can receive all produced values by registering - with ``.subscribe()`` and receiving from the new instance it delivers. + Additional consumer tasks can receive all produced values by + registering with ``.subscribe()`` and receiving from the new + instance it delivers. ''' def __init__( diff --git a/tractor/trionics/_mngrs.py b/tractor/trionics/_mngrs.py index 08e70ad..fd224d6 100644 --- a/tractor/trionics/_mngrs.py +++ b/tractor/trionics/_mngrs.py @@ -18,8 +18,12 @@ Async context manager primitives with hard ``trio``-aware semantics ''' -from contextlib import asynccontextmanager as acm +from __future__ import annotations +from contextlib import ( + asynccontextmanager as acm, +) import inspect +from types import ModuleType from typing import ( Any, AsyncContextManager, @@ -30,13 +34,16 @@ from typing import ( Optional, Sequence, TypeVar, + TYPE_CHECKING, ) import trio - from tractor._state import current_actor from tractor.log import get_logger +if TYPE_CHECKING: + from tractor import ActorNursery + log = get_logger(__name__) @@ -46,8 +53,10 @@ T = TypeVar("T") @acm async def maybe_open_nursery( - nursery: trio.Nursery | None = None, + nursery: trio.Nursery|ActorNursery|None = None, shield: bool = False, + lib: ModuleType = trio, + ) -> AsyncGenerator[trio.Nursery, Any]: ''' Create a new nursery if None provided. @@ -58,13 +67,12 @@ async def maybe_open_nursery( if nursery is not None: yield nursery else: - async with trio.open_nursery() as nursery: + async with lib.open_nursery() as nursery: nursery.cancel_scope.shield = shield yield nursery async def _enter_and_wait( - mngr: AsyncContextManager[T], unwrapped: dict[int, T], all_entered: trio.Event, @@ -91,7 +99,6 @@ async def _enter_and_wait( @acm async def gather_contexts( - mngrs: Sequence[AsyncContextManager[T]], ) -> AsyncGenerator[ @@ -102,15 +109,17 @@ async def gather_contexts( None, ]: ''' - Concurrently enter a sequence of async context managers, each in - a separate ``trio`` task and deliver the unwrapped values in the - same order once all managers have entered. On exit all contexts are - subsequently and concurrently exited. + Concurrently enter a sequence of async context managers (acms), + each from a separate `trio` task and deliver the unwrapped + `yield`-ed values in the same order once all managers have entered. - This function is somewhat similar to common usage of - ``contextlib.AsyncExitStack.enter_async_context()`` (in a loop) in - combo with ``asyncio.gather()`` except the managers are concurrently - entered and exited, and cancellation just works. + On exit, all acms are subsequently and concurrently exited. + + This function is somewhat similar to a batch of non-blocking + calls to `contextlib.AsyncExitStack.enter_async_context()` + (inside a loop) *in combo with* a `asyncio.gather()` to get the + `.__aenter__()`-ed values, except the managers are both + concurrently entered and exited and *cancellation just works*(R). ''' seed: int = id(mngrs) @@ -210,9 +219,10 @@ async def maybe_open_context( ) -> AsyncIterator[tuple[bool, T]]: ''' - Maybe open a context manager if there is not already a _Cached - version for the provided ``key`` for *this* actor. Return the - _Cached instance on a _Cache hit. + Maybe open an async-context-manager (acm) if there is not already + a `_Cached` version for the provided (input) `key` for *this* actor. + + Return the `_Cached` instance on a _Cache hit. ''' fid = id(acm_func) @@ -273,8 +283,13 @@ async def maybe_open_context( else: _Cache.users += 1 log.runtime( - f'Reusing resource for `_Cache` user {_Cache.users}\n\n' - f'{ctx_key!r} -> {yielded!r}\n' + f'Re-using cached resource for user {_Cache.users}\n\n' + f'{ctx_key!r} -> {type(yielded)}\n' + + # TODO: make this work with values but without + # `msgspec.Struct` causing frickin crashes on field-type + # lookups.. + # f'{ctx_key!r} -> {yielded!r}\n' ) lock.release() yield True, yielded -- 2.34.1 From af3745684c20064de44a50103921f6c504ebb574 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 2 Jul 2024 12:21:26 -0400 Subject: [PATCH 18/42] More formal `TransportClosed` reporting/raising Since it was all ad-hoc defined inside `._ipc.MsgpackTCPStream._iter_pkts()` more or less, this starts formalizing a way for particular transport backends to indicate whether a disconnect condition should be re-raised in the RPC msg loop and if not what log level to report it at (if any). Based on our lone transport currently we try to suppress any logging noise from ephemeral connections expected during normal actor interaction and discovery subsys ops: - any short lived discovery related TCP connects are only logged as `.transport()` level. - both `.error()` and raise on any underlying `trio.ClosedResource` cause since that normally means some task touched transport layer internals that it shouldn't have. - do a `.warning()` on anything else unexpected. Impl deats: - extend the `._exceptions.TransportClosed` to accept an input log level, raise-on-report toggle and custom reporting & raising via a new `.report_n_maybe_raise()` method. - construct the TCs with inputs per case in (the newly named) `._iter_pkts(). - call ^ this method from the `TransportClosed` handler block inside the RPC msg loop thus delegating reporting levels and/or raising to the backend's per-case TC instantiating. Related `._ipc` changes: - mask out all the `MsgpackTCPStream._codec` debug helper stuff and drop any lingering cruft from the initial proto-ing of msg-codecs. - rename some attrs/methods: |_`MsgpackTCPStream._iter_packets()` -> `._iter_pkts()` and `._agen` -> `_aiter_pkts`. |_`Channel._aiter_recv()` -> `._aiter_msgs()` and `._agen` -> `_aiter_msgs`. - add `hide_tb: bool` support to `Channel.send()` and only show the frame on non-MTEs. --- tractor/__init__.py | 1 + tractor/_exceptions.py | 55 +++++++- tractor/_ipc.py | 284 +++++++++++++++++++++++++++-------------- tractor/_rpc.py | 20 ++- 4 files changed, 259 insertions(+), 101 deletions(-) diff --git a/tractor/__init__.py b/tractor/__init__.py index ad3144d..bda4ac2 100644 --- a/tractor/__init__.py +++ b/tractor/__init__.py @@ -49,6 +49,7 @@ from ._exceptions import ( ModuleNotExposed as ModuleNotExposed, MsgTypeError as MsgTypeError, RemoteActorError as RemoteActorError, + TransportClosed as TransportClosed, ) from .devx import ( breakpoint as breakpoint, diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 265f506..a0b6ff3 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -906,8 +906,59 @@ class StreamOverrun( ''' -class TransportClosed(trio.ClosedResourceError): - "Underlying channel transport was closed prior to use" +class TransportClosed(trio.BrokenResourceError): + ''' + IPC transport (protocol) connection was closed or broke and + indicates that the wrapping communication `Channel` can no longer + be used to send/receive msgs from the remote peer. + + ''' + def __init__( + self, + message: str, + loglevel: str = 'transport', + cause: BaseException|None = None, + raise_on_report: bool = False, + + ) -> None: + self.message: str = message + self._loglevel = loglevel + super().__init__(message) + + if cause is not None: + self.__cause__ = cause + + # flag to toggle whether the msg loop should raise + # the exc in its `TransportClosed` handler block. + self._raise_on_report = raise_on_report + + def report_n_maybe_raise( + self, + message: str|None = None, + + ) -> None: + ''' + Using the init-specified log level emit a logging report + for this error. + + ''' + message: str = message or self.message + # when a cause is set, slap it onto the log emission. + if cause := self.__cause__: + cause_tb_str: str = ''.join( + traceback.format_tb(cause.__traceback__) + ) + message += ( + f'{cause_tb_str}\n' # tb + f' {cause}\n' # exc repr + ) + + getattr(log, self._loglevel)(message) + + # some errors we want to blow up from + # inside the RPC msg loop + if self._raise_on_report: + raise self from cause class NoResult(RuntimeError): diff --git a/tractor/_ipc.py b/tractor/_ipc.py index e5e3d10..a1cb035 100644 --- a/tractor/_ipc.py +++ b/tractor/_ipc.py @@ -54,7 +54,7 @@ from tractor._exceptions import ( ) from tractor.msg import ( _ctxvar_MsgCodec, - _codec, + # _codec, XXX see `self._codec` sanity/debug checks MsgCodec, types as msgtypes, pretty_struct, @@ -65,8 +65,18 @@ log = get_logger(__name__) _is_windows = platform.system() == 'Windows' -def get_stream_addrs(stream: trio.SocketStream) -> tuple: - # should both be IP sockets +def get_stream_addrs( + stream: trio.SocketStream +) -> tuple[ + tuple[str, int], # local + tuple[str, int], # remote +]: + ''' + Return the `trio` streaming transport prot's socket-addrs for + both the local and remote sides as a pair. + + ''' + # rn, should both be IP sockets lsockname = stream.socket.getsockname() rsockname = stream.socket.getpeername() return ( @@ -75,17 +85,22 @@ def get_stream_addrs(stream: trio.SocketStream) -> tuple: ) -# TODO: this should be our `Union[*msgtypes.__spec__]` now right? -MsgType = TypeVar("MsgType") - -# TODO: consider using a generic def and indexing with our eventual -# msg definition/types? -# - https://docs.python.org/3/library/typing.html#typing.Protocol -# - https://jcristharif.com/msgspec/usage.html#structs +# from tractor.msg.types import MsgType +# ?TODO? this should be our `Union[*msgtypes.__spec__]` alias now right..? +# => BLEH, except can't bc prots must inherit typevar or param-spec +# vars.. +MsgType = TypeVar('MsgType') +# TODO: break up this mod into a subpkg so we can start adding new +# backends and move this type stuff into a dedicated file.. Bo +# @runtime_checkable class MsgTransport(Protocol[MsgType]): +# +# ^-TODO-^ consider using a generic def and indexing with our +# eventual msg definition/types? +# - https://docs.python.org/3/library/typing.html#typing.Protocol stream: trio.SocketStream drained: list[MsgType] @@ -120,9 +135,9 @@ class MsgTransport(Protocol[MsgType]): ... -# TODO: not sure why we have to inherit here, but it seems to be an -# issue with ``get_msg_transport()`` returning a ``Type[Protocol]``; -# probably should make a `mypy` issue? +# TODO: typing oddity.. not sure why we have to inherit here, but it +# seems to be an issue with `get_msg_transport()` returning +# a `Type[Protocol]`; probably should make a `mypy` issue? class MsgpackTCPStream(MsgTransport): ''' A ``trio.SocketStream`` delivering ``msgpack`` formatted data @@ -145,7 +160,7 @@ class MsgpackTCPStream(MsgTransport): # https://jcristharif.com/msgspec/extending.html#mapping-to-from-native-types # # TODO: define this as a `Codec` struct which can be - # overriden dynamically by the application/runtime. + # overriden dynamically by the application/runtime? codec: tuple[ Callable[[Any], Any]|None, # coder Callable[[type, Any], Any]|None, # decoder @@ -160,7 +175,7 @@ class MsgpackTCPStream(MsgTransport): self._laddr, self._raddr = get_stream_addrs(stream) # create read loop instance - self._agen = self._iter_packets() + self._aiter_pkts = self._iter_packets() self._send_lock = trio.StrictFIFOLock() # public i guess? @@ -174,15 +189,12 @@ class MsgpackTCPStream(MsgTransport): # allow for custom IPC msg interchange format # dynamic override Bo self._task = trio.lowlevel.current_task() - self._codec: MsgCodec = ( - codec - or - _codec._ctxvar_MsgCodec.get() - ) - # TODO: mask out before release? - # log.runtime( - # f'New {self} created with codec\n' - # f'codec: {self._codec}\n' + + # XXX for ctxvar debug only! + # self._codec: MsgCodec = ( + # codec + # or + # _codec._ctxvar_MsgCodec.get() # ) async def _iter_packets(self) -> AsyncGenerator[dict, None]: @@ -190,6 +202,11 @@ class MsgpackTCPStream(MsgTransport): Yield `bytes`-blob decoded packets from the underlying TCP stream using the current task's `MsgCodec`. + This is a streaming routine implemented as an async generator + func (which was the original design, but could be changed?) + and is allocated by a `.__call__()` inside `.__init__()` where + it is assigned to the `._aiter_pkts` attr. + ''' decodes_failed: int = 0 @@ -204,16 +221,82 @@ class MsgpackTCPStream(MsgTransport): # seem to be getting racy failures here on # arbiter/registry name subs.. trio.BrokenResourceError, - ): - raise TransportClosed( - f'transport {self} was already closed prior ro read' - ) + ) as trans_err: + + loglevel = 'transport' + match trans_err: + # case ( + # ConnectionResetError() + # ): + # loglevel = 'transport' + + # peer actor (graceful??) TCP EOF but `tricycle` + # seems to raise a 0-bytes-read? + case ValueError() if ( + 'unclean EOF' in trans_err.args[0] + ): + pass + + # peer actor (task) prolly shutdown quickly due + # to cancellation + case trio.BrokenResourceError() if ( + 'Connection reset by peer' in trans_err.args[0] + ): + pass + + # unless the disconnect condition falls under "a + # normal operation breakage" we usualy console warn + # about it. + case _: + loglevel: str = 'warning' + + + raise TransportClosed( + message=( + f'IPC transport already closed by peer\n' + f'x)> {type(trans_err)}\n' + f' |_{self}\n' + ), + loglevel=loglevel, + ) from trans_err + + # XXX definitely can happen if transport is closed + # manually by another `trio.lowlevel.Task` in the + # same actor; we use this in some simulated fault + # testing for ex, but generally should never happen + # under normal operation! + # + # NOTE: as such we always re-raise this error from the + # RPC msg loop! + except trio.ClosedResourceError as closure_err: + raise TransportClosed( + message=( + f'IPC transport already manually closed locally?\n' + f'x)> {type(closure_err)} \n' + f' |_{self}\n' + ), + loglevel='error', + raise_on_report=( + closure_err.args[0] == 'another task closed this fd' + or + closure_err.args[0] in ['another task closed this fd'] + ), + ) from closure_err + + # graceful TCP EOF disconnect if header == b'': raise TransportClosed( - f'transport {self} was already closed prior ro read' + message=( + f'IPC transport already gracefully closed\n' + f')>\n' + f'|_{self}\n' + ), + loglevel='transport', + # cause=??? # handy or no? ) + size: int size, = struct.unpack(" None: ''' Send a msgpack encoded py-object-blob-as-msg over TCP. @@ -304,21 +375,24 @@ class MsgpackTCPStream(MsgTransport): invalid msg type ''' - # __tracebackhide__: bool = hide_tb + __tracebackhide__: bool = hide_tb + + # XXX see `trio._sync.AsyncContextManagerMixin` for details + # on the `.acquire()`/`.release()` sequencing.. async with self._send_lock: # NOTE: lookup the `trio.Task.context`'s var for # the current `MsgCodec`. codec: MsgCodec = _ctxvar_MsgCodec.get() - # TODO: mask out before release? - if self._codec.pld_spec != codec.pld_spec: - self._codec = codec - log.runtime( - f'Using new codec in {self}.send()\n' - f'codec: {self._codec}\n\n' - f'msg: {msg}\n' - ) + # XXX for ctxvar debug only! + # if self._codec.pld_spec != codec.pld_spec: + # self._codec = codec + # log.runtime( + # f'Using new codec in {self}.send()\n' + # f'codec: {self._codec}\n\n' + # f'msg: {msg}\n' + # ) if type(msg) not in msgtypes.__msg_types__: if strict_types: @@ -352,6 +426,16 @@ class MsgpackTCPStream(MsgTransport): size: bytes = struct.pack(" + # except BaseException as _err: + # err = _err + # if not isinstance(err, MsgTypeError): + # __tracebackhide__: bool = False + # raise + @property def laddr(self) -> tuple[str, int]: return self._laddr @@ -361,7 +445,7 @@ class MsgpackTCPStream(MsgTransport): return self._raddr async def recv(self) -> Any: - return await self._agen.asend(None) + return await self._aiter_pkts.asend(None) async def drain(self) -> AsyncIterator[dict]: ''' @@ -378,7 +462,7 @@ class MsgpackTCPStream(MsgTransport): yield msg def __aiter__(self): - return self._agen + return self._aiter_pkts def connected(self) -> bool: return self.stream.socket.fileno() != -1 @@ -433,7 +517,7 @@ class Channel: # set after handshake - always uid of far end self.uid: tuple[str, str]|None = None - self._agen = self._aiter_recv() + self._aiter_msgs = self._iter_msgs() self._exc: Exception|None = None # set if far end actor errors self._closed: bool = False @@ -497,8 +581,6 @@ class Channel: ) return self._transport - # TODO: something simliar at the IPC-`Context` - # level so as to support @cm def apply_codec( self, @@ -517,6 +599,7 @@ class Channel: finally: self._transport.codec = orig + # TODO: do a .src/.dst: str for maddrs? def __repr__(self) -> str: if not self._transport: return '' @@ -560,27 +643,43 @@ class Channel: ) return transport + # TODO: something like, + # `pdbp.hideframe_on(errors=[MsgTypeError])` + # instead of the `try/except` hack we have rn.. + # seems like a pretty useful thing to have in general + # along with being able to filter certain stack frame(s / sets) + # possibly based on the current log-level? async def send( self, payload: Any, - # hide_tb: bool = False, + hide_tb: bool = False, ) -> None: ''' Send a coded msg-blob over the transport. ''' - # __tracebackhide__: bool = hide_tb - log.transport( - '=> send IPC msg:\n\n' - f'{pformat(payload)}\n' - ) # type: ignore - assert self._transport - await self._transport.send( - payload, - # hide_tb=hide_tb, - ) + __tracebackhide__: bool = hide_tb + try: + log.transport( + '=> send IPC msg:\n\n' + f'{pformat(payload)}\n' + ) + # assert self._transport # but why typing? + await self._transport.send( + payload, + hide_tb=hide_tb, + ) + except BaseException as _err: + err = _err # bind for introspection + if not isinstance(_err, MsgTypeError): + # assert err + __tracebackhide__: bool = False + else: + assert err.cid + + raise async def recv(self) -> Any: assert self._transport @@ -617,8 +716,11 @@ class Channel: await self.aclose(*args) def __aiter__(self): - return self._agen + return self._aiter_msgs + # ?TODO? run any reconnection sequence? + # -[ ] prolly should be impl-ed as deco-API? + # # async def _reconnect(self) -> None: # """Handle connection failures by polling until a reconnect can be # established. @@ -636,7 +738,6 @@ class Channel: # else: # log.transport("Stream connection re-established!") - # # TODO: run any reconnection sequence # # on_recon = self._recon_seq # # if on_recon: # # await on_recon(self) @@ -650,11 +751,17 @@ class Channel: # " for re-establishment") # await trio.sleep(1) - async def _aiter_recv( + async def _iter_msgs( self ) -> AsyncGenerator[Any, None]: ''' - Async iterate items from underlying stream. + Yield `MsgType` IPC msgs decoded and deliverd from + an underlying `MsgTransport` protocol. + + This is a streaming routine alo implemented as an async-gen + func (same a `MsgTransport._iter_pkts()`) gets allocated by + a `.__call__()` inside `.__init__()` where it is assigned to + the `._aiter_msgs` attr. ''' assert self._transport @@ -680,15 +787,6 @@ class Channel: case _: yield msg - # TODO: if we were gonna do this it should be - # done up at the `MsgStream` layer! - # - # sent = yield item - # if sent is not None: - # # optimization, passing None through all the - # # time is pointless - # await self._transport.send(sent) - except trio.BrokenResourceError: # if not self._autorecon: diff --git a/tractor/_rpc.py b/tractor/_rpc.py index fc8687c..1849cf6 100644 --- a/tractor/_rpc.py +++ b/tractor/_rpc.py @@ -1197,7 +1197,7 @@ async def process_messages( parent_chan=chan, ) - except TransportClosed: + except TransportClosed as tc: # channels "breaking" (for TCP streams by EOF or 104 # connection-reset) is ok since we don't have a teardown # handshake for them (yet) and instead we simply bail out of @@ -1205,12 +1205,20 @@ async def process_messages( # up.. # # TODO: maybe add a teardown handshake? and, - # -[ ] don't show this msg if it's an ephemeral discovery ep call? + # -[x] don't show this msg if it's an ephemeral discovery ep call? + # |_ see the below `.report_n_maybe_raise()` impl as well as + # tc-exc input details in `MsgpackTCPStream._iter_pkts()` + # for different read-failure cases. # -[ ] figure out how this will break with other transports? - log.runtime( - f'IPC channel closed abruptly\n' - f'<=x peer: {chan.uid}\n' - f' |_{chan.raddr}\n' + tc.report_n_maybe_raise( + message=( + f'peer IPC channel closed abruptly?\n\n' + f'<=x {chan}\n' + f' |_{chan.raddr}\n\n' + ) + + + tc.message + ) # transport **WAS** disconnected -- 2.34.1 From 3c5816c97791ee729f8d53157afc373a48f9487a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 2 Jul 2024 15:53:33 -0400 Subject: [PATCH 19/42] Add `Portal.chan` property, to wrap `._chan` attr --- tractor/_portal.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tractor/_portal.py b/tractor/_portal.py index 0f69883..3dc7f3a 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -97,7 +97,7 @@ class Portal: channel: Channel, ) -> None: - self.chan = channel + self._chan: Channel = channel # during the portal's lifetime self._final_result_pld: Any|None = None self._final_result_msg: PayloadMsg|None = None @@ -109,6 +109,10 @@ class Portal: self._streams: set[MsgStream] = set() self.actor: Actor = current_actor() + @property + def chan(self) -> Channel: + return self._chan + @property def channel(self) -> Channel: ''' -- 2.34.1 From 02812b9f519e4e873622b5fb10db126b8bb6f85a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 2 Jul 2024 16:00:26 -0400 Subject: [PATCH 20/42] Reraise RAEs in `MsgStream.receive()`; truncate tbs To avoid showing lowlevel details of exception handling around the underlying call to `return await self._ctx._pld_rx.recv_pld(ipc=self)`, any time a `RemoteActorError` is unpacked (an raised locally) we re-raise it directly from the captured `src_err` captured so as to present to the user/app caller-code an exception raised directly from the `.receive()` frame. This simplifies traceback call-stacks for any `log.exception()` or `pdb`-REPL output filtering out the lower `PldRx` frames by default. --- tractor/_streaming.py | 109 +++++++++++++++++++----------------------- 1 file changed, 50 insertions(+), 59 deletions(-) diff --git a/tractor/_streaming.py b/tractor/_streaming.py index 314a93b..bc87164 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -36,8 +36,8 @@ import warnings import trio from ._exceptions import ( - # _raise_from_no_key_in_msg, ContextCancelled, + RemoteActorError, ) from .log import get_logger from .trionics import ( @@ -101,7 +101,7 @@ class MsgStream(trio.abc.Channel): @property def ctx(self) -> Context: ''' - This stream's IPC `Context` ref. + A read-only ref to this stream's inter-actor-task `Context`. ''' return self._ctx @@ -145,9 +145,8 @@ class MsgStream(trio.abc.Channel): ''' __tracebackhide__: bool = hide_tb - # NOTE: `trio.ReceiveChannel` implements - # EOC handling as follows (aka uses it - # to gracefully exit async for loops): + # NOTE FYI: `trio.ReceiveChannel` implements EOC handling as + # follows (aka uses it to gracefully exit async for loops): # # async def __anext__(self) -> ReceiveType: # try: @@ -165,48 +164,29 @@ class MsgStream(trio.abc.Channel): src_err: Exception|None = None # orig tb try: - ctx: Context = self._ctx return await ctx._pld_rx.recv_pld(ipc=self) # XXX: the stream terminates on either of: - # - via `self._rx_chan.receive()` raising after manual closure - # by the rpc-runtime OR, - # - via a received `{'stop': ...}` msg from remote side. - # |_ NOTE: previously this was triggered by calling - # ``._rx_chan.aclose()`` on the send side of the channel inside - # `Actor._deliver_ctx_payload()`, but now the 'stop' message handling - # has been put just above inside `_raise_from_no_key_in_msg()`. - except ( - trio.EndOfChannel, - ) as eoc: - src_err = eoc + # - `self._rx_chan.receive()` raising after manual closure + # by the rpc-runtime, + # OR + # - via a `Stop`-msg received from remote peer task. + # NOTE + # |_ previously this was triggered by calling + # ``._rx_chan.aclose()`` on the send side of the channel + # inside `Actor._deliver_ctx_payload()`, but now the 'stop' + # message handling gets delegated to `PldRFx.recv_pld()` + # internals. + except trio.EndOfChannel as eoc: + # a graceful stream finished signal self._eoc = eoc + src_err = eoc - # TODO: Locally, we want to close this stream gracefully, by - # terminating any local consumers tasks deterministically. - # Once we have broadcast support, we **don't** want to be - # closing this stream and not flushing a final value to - # remaining (clone) consumers who may not have been - # scheduled to receive it yet. - # try: - # maybe_err_msg_or_res: dict = self._rx_chan.receive_nowait() - # if maybe_err_msg_or_res: - # log.warning( - # 'Discarding un-processed msg:\n' - # f'{maybe_err_msg_or_res}' - # ) - # except trio.WouldBlock: - # # no queued msgs that might be another remote - # # error, so just raise the original EoC - # pass - - # raise eoc - - # a ``ClosedResourceError`` indicates that the internal - # feeder memory receive channel was closed likely by the - # runtime after the associated transport-channel - # disconnected or broke. + # a `ClosedResourceError` indicates that the internal feeder + # memory receive channel was closed likely by the runtime + # after the associated transport-channel disconnected or + # broke. except trio.ClosedResourceError as cre: # by self._rx_chan.receive() src_err = cre log.warning( @@ -218,14 +198,15 @@ class MsgStream(trio.abc.Channel): # terminated and signal this local iterator to stop drained: list[Exception|dict] = await self.aclose() if drained: + # ?TODO? pass these to the `._ctx._drained_msgs: deque` + # and then iterate them as part of any `.wait_for_result()` call? + # # from .devx import pause # await pause() log.warning( - 'Drained context msgs during closure:\n' + 'Drained context msgs during closure\n\n' f'{drained}' ) - # TODO: pass these to the `._ctx._drained_msgs: deque` - # and then iterate them as part of any `.result()` call? # NOTE XXX: if the context was cancelled or remote-errored # but we received the stream close msg first, we @@ -238,28 +219,36 @@ class MsgStream(trio.abc.Channel): from_src_exc=src_err, ) - # propagate any error but hide low-level frame details - # from the caller by default for debug noise reduction. + # propagate any error but hide low-level frame details from + # the caller by default for console/debug-REPL noise + # reduction. if ( hide_tb + and ( - # XXX NOTE XXX don't reraise on certain - # stream-specific internal error types like, - # - # - `trio.EoC` since we want to use the exact instance - # to ensure that it is the error that bubbles upward - # for silent absorption by `Context.open_stream()`. - and not self._eoc + # XXX NOTE special conditions: don't reraise on + # certain stream-specific internal error types like, + # + # - `trio.EoC` since we want to use the exact instance + # to ensure that it is the error that bubbles upward + # for silent absorption by `Context.open_stream()`. + not self._eoc - # - `RemoteActorError` (or `ContextCancelled`) if it gets - # raised from `_raise_from_no_key_in_msg()` since we - # want the same (as the above bullet) for any - # `.open_context()` block bubbled error raised by - # any nearby ctx API remote-failures. - # and not isinstance(src_err, RemoteActorError) + # - `RemoteActorError` (or subtypes like ctxc) + # since we want to present the error as though it is + # "sourced" directly from this `.receive()` call and + # generally NOT include the stack frames raised from + # inside the `PldRx` and/or the transport stack + # layers. + or isinstance(src_err, RemoteActorError) + ) ): raise type(src_err)(*src_err.args) from src_err else: + # for any non-graceful-EOC we want to NOT hide this frame + if not self._eoc: + __tracebackhide__: bool = False + raise src_err async def aclose(self) -> list[Exception|dict]: @@ -385,6 +374,8 @@ class MsgStream(trio.abc.Channel): if not self._eoc: message: str = ( f'Stream self-closed by {self._ctx.side!r}-side before EoC\n' + # } bc a stream is a "scope"/msging-phase inside an IPC + f'x}}>\n' f'|_{self}\n' ) log.cancel(message) -- 2.34.1 From b46400a86f513a74791eec8059aa091d7b13d3b6 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 2 Jul 2024 16:31:58 -0400 Subject: [PATCH 21/42] Use `._entry` proto-ed "lifetime ops" in logging As per a WIP scribbled out TODO in `._entry.nest_from_op()`, change a bunch of "supervisor/lifetime mgmt ops" related log messages to contain some supervisor-annotation "headers" in an effort to give a terser "visual indication" of how some execution/scope/storage primitive entity (like an actor/task/ctx/connection) is being operated on (like, opening/started/closed/cancelled/erroring) from a "supervisor action" POV. Also tweak a bunch more emissions to lower levels to reduce noise around normal inter-actor operations like process and IPC ctx supervision. --- tractor/_context.py | 26 ++++++++++++++++++++------ tractor/_entry.py | 2 +- tractor/_runtime.py | 42 +++++++++++++++++++++++++----------------- tractor/_spawn.py | 25 ++++++++++++++++--------- 4 files changed, 62 insertions(+), 33 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index 6b5f2eb..cc6503e 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -933,13 +933,14 @@ class Context: self.cancel_called = True header: str = ( - f'Cancelling ctx with peer from {side.upper()} side\n\n' + f'Cancelling ctx from {side.upper()}-side\n' ) reminfo: str = ( # ' =>\n' - f'Context.cancel() => {self.chan.uid}\n' + # f'Context.cancel() => {self.chan.uid}\n' + f'c)=> {self.chan.uid}\n' # f'{self.chan.uid}\n' - f' |_ @{self.dst_maddr}\n' + f' |_ @{self.dst_maddr}\n' f' >> {self.repr_rpc}\n' # f' >> {self._nsf}() -> {codec}[dict]:\n\n' # TODO: pull msg-type from spec re #320 @@ -1267,6 +1268,12 @@ class Context: @property def maybe_error(self) -> BaseException|None: + ''' + Return the (remote) error as outcome or `None`. + + Remote errors take precedence over local ones. + + ''' le: BaseException|None = self._local_error re: RemoteActorError|ContextCancelled|None = self._remote_error @@ -2182,9 +2189,16 @@ async def open_context_from_portal( # handled in the block above ^^^ !! # await _debug.pause() # log.cancel( - log.exception( - f'{ctx.side}-side of `Context` terminated with ' - f'.outcome => {ctx.repr_outcome()}\n' + match scope_err: + case trio.Cancelled: + logmeth = log.cancel + + # XXX explicitly report on any non-graceful-taskc cases + case _: + logmeth = log.exception + + logmeth( + f'ctx {ctx.side!r}-side exited with {ctx.repr_outcome()}\n' ) if debug_mode(): diff --git a/tractor/_entry.py b/tractor/_entry.py index 60050ea..3f17452 100644 --- a/tractor/_entry.py +++ b/tractor/_entry.py @@ -265,7 +265,7 @@ def _trio_main( except BaseException as err: logmeth = log.error exit_status: str = ( - 'Main actor task crashed during exit?\n' + 'Main actor task exited due to crash?\n' + nest_from_op( input_op='x)>', # closed by error diff --git a/tractor/_runtime.py b/tractor/_runtime.py index f472c06..92afa29 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -66,10 +66,11 @@ from trio import ( ) from tractor.msg import ( - pretty_struct, - NamespacePath, - types as msgtypes, MsgType, + NamespacePath, + Stop, + pretty_struct, + types as msgtypes, ) from ._ipc import Channel from ._context import ( @@ -545,7 +546,8 @@ class Actor: ): log.cancel( 'Waiting on cancel request to peer\n' - f'`Portal.cancel_actor()` => {chan.uid}\n' + f'c)=>\n' + f' |_{chan.uid}\n' ) # XXX: this is a soft wait on the channel (and its @@ -642,12 +644,14 @@ class Actor: # and an_exit_cs.cancelled_caught ): - log.warning( + report: str = ( 'Timed out waiting on local actor-nursery to exit?\n' f'{local_nursery}\n' - f' |_{pformat(local_nursery._children)}\n' ) - # await _debug.pause() + if children := local_nursery._children: + report += f' |_{pformat(children)}\n' + + log.warning(report) if disconnected: # if the transport died and this actor is still @@ -819,14 +823,17 @@ class Actor: # side, )] except KeyError: - log.warning( + report: str = ( 'Ignoring invalid IPC ctx msg!\n\n' - f'<= sender: {uid}\n\n' - # XXX don't need right since it's always in msg? - # f'=> cid: {cid}\n\n' - - f'{pretty_struct.pformat(msg)}\n' + f'<=? {uid}\n\n' + f' |_{pretty_struct.pformat(msg)}\n' ) + match msg: + case Stop(): + log.runtime(report) + case _: + log.warning(report) + return # if isinstance(msg, MsgTypeError): @@ -1338,10 +1345,11 @@ class Actor: return True log.cancel( - 'Cancel request for RPC task\n\n' - f'<= Actor._cancel_task(): {requesting_uid}\n\n' - f'=> {ctx._task}\n' - f' |_ >> {ctx.repr_rpc}\n' + 'Rxed cancel request for RPC task\n' + f'<=c) {requesting_uid}\n' + f' |_{ctx._task}\n' + f' >> {ctx.repr_rpc}\n' + # f'=> {ctx._task}\n' # f' >> Actor._cancel_task() => {ctx._task}\n' # f' |_ {ctx._task}\n\n' diff --git a/tractor/_spawn.py b/tractor/_spawn.py index c06fa22..2c0f8fa 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -246,8 +246,9 @@ async def hard_kill( ''' log.cancel( - 'Terminating sub-proc:\n' - f'|_{proc}\n' + 'Terminating sub-proc\n' + f'>x)\n' + f' |_{proc}\n' ) # NOTE: this timeout used to do nothing since we were shielding # the ``.wait()`` inside ``new_proc()`` which will pretty much @@ -293,8 +294,8 @@ async def hard_kill( log.critical( # 'Well, the #ZOMBIE_LORD_IS_HERE# to collect\n' '#T-800 deployed to collect zombie B0\n' - f'|\n' - f'|_{proc}\n' + f'>x)\n' + f' |_{proc}\n' ) proc.kill() @@ -322,8 +323,9 @@ async def soft_kill( uid: tuple[str, str] = portal.channel.uid try: log.cancel( - 'Soft killing sub-actor via `Portal.cancel_actor()`\n' - f'|_{proc}\n' + 'Soft killing sub-actor via portal request\n' + f'c)> {portal.chan.uid}\n' + f' |_{proc}\n' ) # wait on sub-proc to signal termination await wait_func(proc) @@ -552,8 +554,9 @@ async def trio_proc( # cancel result waiter that may have been spawned in # tandem if not done already log.cancel( - 'Cancelling existing result waiter task for ' - f'{subactor.uid}' + 'Cancelling portal result reaper task\n' + f'>c)\n' + f' |_{subactor.uid}\n' ) nursery.cancel_scope.cancel() @@ -562,7 +565,11 @@ async def trio_proc( # allowed! Do this **after** cancellation/teardown to avoid # killing the process too early. if proc: - log.cancel(f'Hard reap sequence starting for {subactor.uid}') + log.cancel( + f'Hard reap sequence starting for subactor\n' + f'>x)\n' + f' |_{subactor}@{subactor.uid}\n' + ) with trio.CancelScope(shield=True): # don't clobber an ongoing pdb -- 2.34.1 From 9be821a5cfa4bf4bb7edd44ec1a14da35fd15bd5 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 2 Jul 2024 17:06:50 -0400 Subject: [PATCH 22/42] 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 1135932..113371d 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()__` -- 2.34.1 From b56352b0e4951d73be2b94af1b692bd3bd98aec3 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 3 Jul 2024 17:01:37 -0400 Subject: [PATCH 23/42] Quieter `Stop` handling on ctx result capture In the `drain_to_final_msg()` impl, since a stream terminating gracefully requires this msg, there's really no reason to `log.cancel()` about it; go `.runtime()` level instead since we're trying de-noise under "normal operation". Also, - passthrough `hide_tb` to taskc-handler's `ctx.maybe_raise()` call. - raise `MessagingError` for the `MsgType` unmatched `case _:`. - detail the doc string motivation a little more. --- tractor/msg/_ops.py | 55 +++++++++++++++++++++++++++++---------------- 1 file changed, 36 insertions(+), 19 deletions(-) diff --git a/tractor/msg/_ops.py b/tractor/msg/_ops.py index 91c0dde..f0f3b6b 100644 --- a/tractor/msg/_ops.py +++ b/tractor/msg/_ops.py @@ -374,7 +374,7 @@ class PldRx(Struct): case _: src_err = InternalError( - 'Unknown IPC msg ??\n\n' + 'Invalid IPC msg ??\n\n' f'{msg}\n' ) @@ -499,7 +499,7 @@ async def maybe_limit_plds( yield None return - # sanity on scoping + # sanity check on IPC scoping curr_ctx: Context = current_ipc_ctx() assert ctx is curr_ctx @@ -510,6 +510,8 @@ async def maybe_limit_plds( ) as msgdec: yield msgdec + # when the applied spec is unwound/removed, the same IPC-ctx + # should still be in scope. curr_ctx: Context = current_ipc_ctx() assert ctx is curr_ctx @@ -525,16 +527,26 @@ async def drain_to_final_msg( list[MsgType] ]: ''' - Drain IPC msgs delivered to the underlying IPC primitive's - rx-mem-chan (eg. `Context._rx_chan`) from the runtime in - search for a final result or error. + Drain IPC msgs delivered to the underlying IPC context's + rx-mem-chan (i.e. from `Context._rx_chan`) in search for a final + `Return` or `Error` msg. - The motivation here is to ideally capture errors during ctxc - conditions where a canc-request/or local error is sent but the - local task also excepts and enters the - `Portal.open_context().__aexit__()` block wherein we prefer to - capture and raise any remote error or ctxc-ack as part of the - `ctx.result()` cleanup and teardown sequence. + Deliver the `Return` + preceding drained msgs (`list[MsgType]`) + as a pair unless an `Error` is found, in which unpack and raise + it. + + The motivation here is to always capture any remote error relayed + by the remote peer task during a ctxc condition. + + For eg. a ctxc-request may be sent to the peer as part of the + local task's (request for) cancellation but then that same task + **also errors** before executing the teardown in the + `Portal.open_context().__aexit__()` block. In such error-on-exit + cases we want to always capture and raise any delivered remote + error (like an expected ctxc-ACK) as part of the final + `ctx.wait_for_result()` teardown sequence such that the + `Context.outcome` related state always reflect what transpired + even after ctx closure and the `.open_context()` block exit. ''' __tracebackhide__: bool = hide_tb @@ -572,7 +584,6 @@ async def drain_to_final_msg( # |_from tractor.devx._debug import pause # await pause() - # NOTE: we get here if the far end was # `ContextCancelled` in 2 cases: # 1. we requested the cancellation and thus @@ -580,13 +591,13 @@ async def drain_to_final_msg( # 2. WE DID NOT REQUEST that cancel and thus # SHOULD RAISE HERE! except trio.Cancelled as taskc: - # CASE 2: mask the local cancelled-error(s) # only when we are sure the remote error is # the source cause of this local task's # cancellation. ctx.maybe_raise( - # TODO: when use this/ + hide_tb=hide_tb, + # TODO: when use this? # from_src_exc=taskc, ) @@ -659,7 +670,7 @@ async def drain_to_final_msg( # Stop() case Stop(): pre_result_drained.append(msg) - log.cancel( + log.runtime( # normal/expected shutdown transaction 'Remote stream terminated due to "stop" msg:\n\n' f'{pretty_struct.pformat(msg)}\n' ) @@ -719,13 +730,19 @@ async def drain_to_final_msg( pre_result_drained.append(msg) # It's definitely an internal error if any other # msg type without a`'cid'` field arrives here! + report: str = ( + f'Invalid or unknown msg type {type(msg)!r}!?\n' + ) if not msg.cid: - raise InternalError( - 'Unexpected cid-missing msg?\n\n' - f'{msg}\n' + report += ( + '\nWhich also has no `.cid` field?\n' ) - raise RuntimeError('Unknown msg type: {msg}') + raise MessagingError( + report + + + f'\n{msg}\n' + ) else: log.cancel( -- 2.34.1 From 5f8f8e98ba62be422e1ae42c6c482d63aa02efd9 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 4 Jul 2024 15:06:15 -0400 Subject: [PATCH 24/42] More-n-more scops annots in logging --- tractor/_context.py | 11 ++++++---- tractor/_entry.py | 1 + tractor/_portal.py | 6 +++--- tractor/_runtime.py | 50 ++++++++++++++++++++++++++++----------------- 4 files changed, 42 insertions(+), 26 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index cc6503e..ff44a34 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -2376,8 +2376,9 @@ async def open_context_from_portal( and ctx.cancel_acked ): log.cancel( - f'Context cancelled by {ctx.side!r}-side task\n' - f'|_{ctx._task}\n\n' + f'Context cancelled by local {ctx.side!r}-side task\n' + f'c)>\n' + f' |_{ctx._task}\n\n' f'{repr(scope_err)}\n' ) @@ -2393,8 +2394,10 @@ async def open_context_from_portal( # type_only=True, ) log.cancel( - f'Context terminated due to local {ctx.side!r}-side error:\n\n' - f'{ctx.chan.uid} => {outcome_str}\n' + f'Context terminated due to {ctx.side!r}-side\n\n' + # TODO: do an x)> on err and c)> only for ctxc? + f'c)> {outcome_str}\n' + f' |_{ctx.repr_rpc}\n' ) # FINALLY, remove the context from runtime tracking and diff --git a/tractor/_entry.py b/tractor/_entry.py index 3f17452..a072706 100644 --- a/tractor/_entry.py +++ b/tractor/_entry.py @@ -243,6 +243,7 @@ def _trio_main( nest_from_op( input_op=')>', # like a "closed-to-play"-icon from super perspective tree_str=actor_info, + back_from_op=1, ) ) try: diff --git a/tractor/_portal.py b/tractor/_portal.py index 3dc7f3a..f5a6683 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -263,11 +263,11 @@ class Portal: return False reminfo: str = ( - f'Portal.cancel_actor() => {self.channel.uid}\n' - f'|_{chan}\n' + f'c)=> {self.channel.uid}\n' + f' |_{chan}\n' ) log.cancel( - f'Requesting runtime cancel for peer\n\n' + f'Requesting actor-runtime cancel for peer\n\n' f'{reminfo}' ) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 92afa29..f946cb1 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -439,10 +439,10 @@ class Actor: TransportClosed, ): - # XXX: This may propagate up from ``Channel._aiter_recv()`` - # and ``MsgpackStream._inter_packets()`` on a read from the + # XXX: This may propagate up from `Channel._aiter_recv()` + # and `MsgpackStream._inter_packets()` on a read from the # stream particularly when the runtime is first starting up - # inside ``open_root_actor()`` where there is a check for + # inside `open_root_actor()` where there is a check for # a bound listener on the "arbiter" addr. the reset will be # because the handshake was never meant took place. log.runtime( @@ -507,8 +507,9 @@ class Actor: ) except trio.Cancelled: log.cancel( - 'IPC transport msg loop was cancelled for \n' - f'|_{chan}\n' + 'IPC transport msg loop was cancelled\n' + f'c)>\n' + f' |_{chan}\n' ) raise @@ -545,9 +546,9 @@ class Actor: ): log.cancel( - 'Waiting on cancel request to peer\n' + 'Waiting on cancel request to peer..\n' f'c)=>\n' - f' |_{chan.uid}\n' + f' |_{chan.uid}\n' ) # XXX: this is a soft wait on the channel (and its @@ -646,10 +647,14 @@ class Actor: ): report: str = ( 'Timed out waiting on local actor-nursery to exit?\n' - f'{local_nursery}\n' + f'c)>\n' + f' |_{local_nursery}\n' ) if children := local_nursery._children: - report += f' |_{pformat(children)}\n' + # indent from above local-nurse repr + report += ( + f' |_{pformat(children)}\n' + ) log.warning(report) @@ -1236,8 +1241,9 @@ class Actor: # TODO: just use the new `Context.repr_rpc: str` (and # other) repr fields instead of doing this all manual.. msg: str = ( - f'Runtime cancel request from {requester_type}:\n\n' - f'<= .cancel(): {requesting_uid}\n\n' + f'Actor-runtime cancel request from {requester_type}\n\n' + f'<=c) {requesting_uid}\n' + f' |_{self}\n' ) # TODO: what happens here when we self-cancel tho? @@ -1347,7 +1353,7 @@ class Actor: log.cancel( 'Rxed cancel request for RPC task\n' f'<=c) {requesting_uid}\n' - f' |_{ctx._task}\n' + f' |_{ctx._task}\n' f' >> {ctx.repr_rpc}\n' # f'=> {ctx._task}\n' # f' >> Actor._cancel_task() => {ctx._task}\n' @@ -1465,17 +1471,17 @@ class Actor: "IPC channel's " ) rent_chan_repr: str = ( - f' |_{parent_chan}\n\n' + f' |_{parent_chan}\n\n' if parent_chan else '' ) log.cancel( f'Cancelling {descr} RPC tasks\n\n' - f'<= canceller: {req_uid}\n' + f'<=c) {req_uid} [canceller]\n' f'{rent_chan_repr}' - f'=> cancellee: {self.uid}\n' - f' |_{self}.cancel_rpc_tasks()\n' - f' |_tasks: {len(tasks)}\n' + f'c)=> {self.uid} [cancellee]\n' + f' |_{self} [with {len(tasks)} tasks]\n' + # f' |_tasks: {len(tasks)}\n' # f'{tasks_str}' ) for ( @@ -1932,9 +1938,15 @@ async def async_main( with CancelScope(shield=True): await actor._no_more_peers.wait() - teardown_report += ('-> All peer channels are complete\n') + teardown_report += ( + '-> All peer channels are complete\n' + ) - teardown_report += ('Actor runtime exited') + teardown_report += ( + 'Actor runtime exiting\n' + f'>)\n' + f'|_{actor}\n' + ) log.info(teardown_report) -- 2.34.1 From 31207f92eea49753f2c0c78a2be3301109067e38 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 4 Jul 2024 19:40:11 -0400 Subject: [PATCH 25/42] Finally implement peer-lookup optimization.. There's a been a todo for soo long for this XD Since all `Actor`'s store a set of `._peers` we can try a lookup on that table as a shortcut before pinging the registry Bo Impl deats: - add a new `._discovery.get_peer_by_name()` routine which attempts the `._peers` lookup by combining a copy of that `dict` + an entry added for `Actor._parent_chan` (since all subs have a parent and often the desired contact is just that connection). - change `.find_actor()` (for the `only_first == True` case), `.query_actor()` and `.wait_for_actor()` to call the new helper and deliver appropriate outputs if possible. Other, - deprecate `get_arbiter()` def and all usage in tests and examples. - drop lingering use of `arbiter_sockaddr` arg to various routines. - tweak the `Actor` doc str as well as some code fmting and a tweak to the `._stream_handler()`'s initial `con_status: str` logging value since the way it was could never be reached.. oh and `.warning()` on any new connections which already have a `_pre_chan: Channel` entry in `._peers` so we can start minimizing IPC duplications. --- examples/service_discovery.py | 2 +- tests/test_discovery.py | 6 +- tests/test_local.py | 2 +- tests/test_multi_program.py | 4 +- tractor/__init__.py | 2 +- tractor/_discovery.py | 170 +++++++++++++++++++--------------- tractor/_runtime.py | 90 +++++++++--------- 7 files changed, 151 insertions(+), 125 deletions(-) diff --git a/examples/service_discovery.py b/examples/service_discovery.py index 858f7f1..a0f37b8 100644 --- a/examples/service_discovery.py +++ b/examples/service_discovery.py @@ -9,7 +9,7 @@ async def main(service_name): async with tractor.open_nursery() as an: await an.start_actor(service_name) - async with tractor.get_arbiter('127.0.0.1', 1616) as portal: + async with tractor.get_registry('127.0.0.1', 1616) as portal: print(f"Arbiter is listening on {portal.channel}") async with tractor.wait_for_actor(service_name) as sockaddr: diff --git a/tests/test_discovery.py b/tests/test_discovery.py index cd9dc02..508fdbe 100644 --- a/tests/test_discovery.py +++ b/tests/test_discovery.py @@ -26,7 +26,7 @@ async def test_reg_then_unreg(reg_addr): portal = await n.start_actor('actor', enable_modules=[__name__]) uid = portal.channel.uid - async with tractor.get_arbiter(*reg_addr) as aportal: + async with tractor.get_registry(*reg_addr) as aportal: # this local actor should be the arbiter assert actor is aportal.actor @@ -160,7 +160,7 @@ async def spawn_and_check_registry( async with tractor.open_root_actor( registry_addrs=[reg_addr], ): - async with tractor.get_arbiter(*reg_addr) as portal: + async with tractor.get_registry(*reg_addr) as portal: # runtime needs to be up to call this actor = tractor.current_actor() @@ -298,7 +298,7 @@ async def close_chans_before_nursery( async with tractor.open_root_actor( registry_addrs=[reg_addr], ): - async with tractor.get_arbiter(*reg_addr) as aportal: + async with tractor.get_registry(*reg_addr) as aportal: try: get_reg = partial(unpack_reg, aportal) diff --git a/tests/test_local.py b/tests/test_local.py index a019d77..ecdad5f 100644 --- a/tests/test_local.py +++ b/tests/test_local.py @@ -38,7 +38,7 @@ async def test_self_is_registered_localportal(reg_addr): "Verify waiting on the arbiter to register itself using a local portal." actor = tractor.current_actor() assert actor.is_arbiter - async with tractor.get_arbiter(*reg_addr) as portal: + async with tractor.get_registry(*reg_addr) as portal: assert isinstance(portal, tractor._portal.LocalPortal) with trio.fail_after(0.2): diff --git a/tests/test_multi_program.py b/tests/test_multi_program.py index 0b6b5ba..27521a0 100644 --- a/tests/test_multi_program.py +++ b/tests/test_multi_program.py @@ -32,7 +32,7 @@ def test_abort_on_sigint(daemon): @tractor_test async def test_cancel_remote_arbiter(daemon, reg_addr): assert not tractor.current_actor().is_arbiter - async with tractor.get_arbiter(*reg_addr) as portal: + async with tractor.get_registry(*reg_addr) as portal: await portal.cancel_actor() time.sleep(0.1) @@ -41,7 +41,7 @@ async def test_cancel_remote_arbiter(daemon, reg_addr): # no arbiter socket should exist with pytest.raises(OSError): - async with tractor.get_arbiter(*reg_addr) as portal: + async with tractor.get_registry(*reg_addr) as portal: pass diff --git a/tractor/__init__.py b/tractor/__init__.py index bda4ac2..3bfda13 100644 --- a/tractor/__init__.py +++ b/tractor/__init__.py @@ -30,7 +30,7 @@ from ._streaming import ( stream as stream, ) from ._discovery import ( - get_arbiter as get_arbiter, + get_registry as get_registry, find_actor as find_actor, wait_for_actor as wait_for_actor, query_actor as query_actor, diff --git a/tractor/_discovery.py b/tractor/_discovery.py index 99a4dd6..a681c63 100644 --- a/tractor/_discovery.py +++ b/tractor/_discovery.py @@ -26,8 +26,8 @@ from typing import ( TYPE_CHECKING, ) from contextlib import asynccontextmanager as acm -import warnings +from tractor.log import get_logger from .trionics import gather_contexts from ._ipc import _connect_chan, Channel from ._portal import ( @@ -40,11 +40,13 @@ from ._state import ( _runtime_vars, ) - if TYPE_CHECKING: from ._runtime import Actor +log = get_logger(__name__) + + @acm async def get_registry( host: str, @@ -56,14 +58,12 @@ async def get_registry( ]: ''' Return a portal instance connected to a local or remote - arbiter. + registry-service actor; if a connection already exists re-use it + (presumably to call a `.register_actor()` registry runtime RPC + ep). ''' - actor = current_actor() - - if not actor: - raise RuntimeError("No actor instance has been defined yet?") - + actor: Actor = current_actor() if actor.is_registrar: # we're already the arbiter # (likely a re-entrant call from the arbiter actor) @@ -72,6 +72,8 @@ async def get_registry( Channel((host, port)) ) else: + # TODO: try to look pre-existing connection from + # `Actor._peers` and use it instead? async with ( _connect_chan(host, port) as chan, open_portal(chan) as regstr_ptl, @@ -80,19 +82,6 @@ async def get_registry( -# TODO: deprecate and this remove _arbiter form! -@acm -async def get_arbiter(*args, **kwargs): - warnings.warn( - '`tractor.get_arbiter()` is now deprecated!\n' - 'Use `.get_registry()` instead!', - DeprecationWarning, - stacklevel=2, - ) - async with get_registry(*args, **kwargs) as to_yield: - yield to_yield - - @acm async def get_root( **kwargs, @@ -110,22 +99,53 @@ async def get_root( yield portal +def get_peer_by_name( + name: str, + # uuid: str|None = None, + +) -> list[Channel]|None: # at least 1 + ''' + Scan for an existing connection (set) to a named actor + and return any channels from `Actor._peers`. + + This is an optimization method over querying the registrar for + the same info. + + ''' + actor: Actor = current_actor() + to_scan: dict[tuple, list[Channel]] = actor._peers.copy() + pchan: Channel|None = actor._parent_chan + if pchan: + to_scan[pchan.uid].append(pchan) + + for aid, chans in to_scan.items(): + _, peer_name = aid + if name == peer_name: + if not chans: + log.warning( + 'No IPC chans for matching peer {peer_name}\n' + ) + continue + return chans + + return None + + @acm async def query_actor( name: str, - arbiter_sockaddr: tuple[str, int] | None = None, - regaddr: tuple[str, int] | None = None, + regaddr: tuple[str, int]|None = None, ) -> AsyncGenerator[ - tuple[str, int] | None, + tuple[str, int]|None, None, ]: ''' - Make a transport address lookup for an actor name to a specific - registrar. + Lookup a transport address (by actor name) via querying a registrar + listening @ `regaddr`. - Returns the (socket) address or ``None`` if no entry under that - name exists for the given registrar listening @ `regaddr`. + Returns the transport protocol (socket) address or `None` if no + entry under that name exists. ''' actor: Actor = current_actor() @@ -137,14 +157,10 @@ async def query_actor( 'The current actor IS the registry!?' ) - if arbiter_sockaddr is not None: - warnings.warn( - '`tractor.query_actor(regaddr=)` is deprecated.\n' - 'Use `registry_addrs: list[tuple]` instead!', - DeprecationWarning, - stacklevel=2, - ) - regaddr: list[tuple[str, int]] = arbiter_sockaddr + maybe_peers: list[Channel]|None = get_peer_by_name(name) + if maybe_peers: + yield maybe_peers[0].raddr + return reg_portal: Portal regaddr: tuple[str, int] = regaddr or actor.reg_addrs[0] @@ -159,10 +175,28 @@ async def query_actor( yield sockaddr +@acm +async def maybe_open_portal( + addr: tuple[str, int], + name: str, +): + async with query_actor( + name=name, + regaddr=addr, + ) as sockaddr: + pass + + if sockaddr: + async with _connect_chan(*sockaddr) as chan: + async with open_portal(chan) as portal: + yield portal + else: + yield None + + @acm async def find_actor( name: str, - arbiter_sockaddr: tuple[str, int]|None = None, registry_addrs: list[tuple[str, int]]|None = None, only_first: bool = True, @@ -179,29 +213,12 @@ async def find_actor( known to the arbiter. ''' - if arbiter_sockaddr is not None: - warnings.warn( - '`tractor.find_actor(arbiter_sockaddr=)` is deprecated.\n' - 'Use `registry_addrs: list[tuple]` instead!', - DeprecationWarning, - stacklevel=2, - ) - registry_addrs: list[tuple[str, int]] = [arbiter_sockaddr] - - @acm - async def maybe_open_portal_from_reg_addr( - addr: tuple[str, int], - ): - async with query_actor( - name=name, - regaddr=addr, - ) as sockaddr: - if sockaddr: - async with _connect_chan(*sockaddr) as chan: - async with open_portal(chan) as portal: - yield portal - else: - yield None + # optimization path, use any pre-existing peer channel + maybe_peers: list[Channel]|None = get_peer_by_name(name) + if maybe_peers and only_first: + async with open_portal(maybe_peers[0]) as peer_portal: + yield peer_portal + return if not registry_addrs: # XXX NOTE: make sure to dynamically read the value on @@ -217,10 +234,13 @@ async def find_actor( maybe_portals: list[ AsyncContextManager[tuple[str, int]] ] = list( - maybe_open_portal_from_reg_addr(addr) + maybe_open_portal( + addr=addr, + name=name, + ) for addr in registry_addrs ) - + portals: list[Portal] async with gather_contexts( mngrs=maybe_portals, ) as portals: @@ -254,31 +274,31 @@ async def find_actor( @acm async def wait_for_actor( name: str, - arbiter_sockaddr: tuple[str, int] | None = None, registry_addr: tuple[str, int] | None = None, ) -> AsyncGenerator[Portal, None]: ''' - Wait on an actor to register with the arbiter. - - A portal to the first registered actor is returned. + Wait on at least one peer actor to register `name` with the + registrar, yield a `Portal to the first registree. ''' actor: Actor = current_actor() - if arbiter_sockaddr is not None: - warnings.warn( - '`tractor.wait_for_actor(arbiter_sockaddr=)` is deprecated.\n' - 'Use `registry_addr: tuple` instead!', - DeprecationWarning, - stacklevel=2, - ) - registry_addr: tuple[str, int] = arbiter_sockaddr + # optimization path, use any pre-existing peer channel + maybe_peers: list[Channel]|None = get_peer_by_name(name) + if maybe_peers: + async with open_portal(maybe_peers[0]) as peer_portal: + yield peer_portal + return + regaddr: tuple[str, int] = ( + registry_addr + or + actor.reg_addrs[0] + ) # TODO: use `.trionics.gather_contexts()` like # above in `find_actor()` as well? reg_portal: Portal - regaddr: tuple[str, int] = registry_addr or actor.reg_addrs[0] async with get_registry(*regaddr) as reg_portal: sockaddrs = await reg_portal.run_from_ns( 'self', diff --git a/tractor/_runtime.py b/tractor/_runtime.py index f946cb1..5615373 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -115,25 +115,26 @@ class Actor: ''' The fundamental "runtime" concurrency primitive. - An *actor* is the combination of a regular Python process executing - a ``trio`` task tree, communicating with other actors through - "memory boundary portals" - which provide a native async API around - IPC transport "channels" which themselves encapsulate various - (swappable) network protocols. + An "actor" is the combination of a regular Python process + executing a `trio.run()` task tree, communicating with other + "actors" through "memory boundary portals": `Portal`, which + provide a high-level async API around IPC "channels" (`Channel`) + which themselves encapsulate various (swappable) network + transport protocols for sending msgs between said memory domains + (processes, hosts, non-GIL threads). - - Each "actor" is ``trio.run()`` scheduled "runtime" composed of - many concurrent tasks in a single thread. The "runtime" tasks - conduct a slew of low(er) level functions to make it possible - for message passing between actors as well as the ability to - create new actors (aka new "runtimes" in new processes which - are supervised via a nursery construct). Each task which sends - messages to a task in a "peer" (not necessarily a parent-child, + Each "actor" is `trio.run()` scheduled "runtime" composed of many + concurrent tasks in a single thread. The "runtime" tasks conduct + a slew of low(er) level functions to make it possible for message + passing between actors as well as the ability to create new + actors (aka new "runtimes" in new processes which are supervised + via an "actor-nursery" construct). Each task which sends messages + to a task in a "peer" actor (not necessarily a parent-child, depth hierarchy) is able to do so via an "address", which maps IPC connections across memory boundaries, and a task request id - which allows for per-actor tasks to send and receive messages - to specific peer-actor tasks with which there is an ongoing - RPC/IPC dialog. + which allows for per-actor tasks to send and receive messages to + specific peer-actor tasks with which there is an ongoing RPC/IPC + dialog. ''' # ugh, we need to get rid of this and replace with a "registry" sys @@ -230,17 +231,20 @@ class Actor: # by the user (currently called the "arbiter") self._spawn_method: str = spawn_method - self._peers: defaultdict = defaultdict(list) + self._peers: defaultdict[ + str, # uaid + list[Channel], # IPC conns from peer + ] = defaultdict(list) self._peer_connected: dict[tuple[str, str], trio.Event] = {} self._no_more_peers = trio.Event() self._no_more_peers.set() + + # RPC state self._ongoing_rpc_tasks = trio.Event() self._ongoing_rpc_tasks.set() - - # (chan, cid) -> (cancel_scope, func) self._rpc_tasks: dict[ - tuple[Channel, str], - tuple[Context, Callable, trio.Event] + tuple[Channel, str], # (chan, cid) + tuple[Context, Callable, trio.Event] # (ctx=>, fn(), done?) ] = {} # map {actor uids -> Context} @@ -317,7 +321,10 @@ class Actor: event = self._peer_connected.setdefault(uid, trio.Event()) await event.wait() log.debug(f'{uid!r} successfully connected back to us') - return event, self._peers[uid][-1] + return ( + event, + self._peers[uid][-1], + ) def load_modules( self, @@ -408,26 +415,11 @@ class Actor: ''' self._no_more_peers = trio.Event() # unset by making new chan = Channel.from_stream(stream) - their_uid: tuple[str, str]|None = chan.uid - - con_status: str = '' - - # TODO: remove this branch since can never happen? - # NOTE: `.uid` is only set after first contact - if their_uid: - con_status = ( - 'IPC Re-connection from already known peer?\n' - ) - else: - con_status = ( - 'New inbound IPC connection <=\n' - ) - - con_status += ( + con_status: str = ( + 'New inbound IPC connection <=\n' f'|_{chan}\n' - # f' |_@{chan.raddr}\n\n' - # ^-TODO-^ remove since alfready in chan.__repr__()? ) + # send/receive initial handshake response try: uid: tuple|None = await self._do_handshake(chan) @@ -452,9 +444,22 @@ class Actor: ) return + familiar: str = 'new-peer' + if _pre_chan := self._peers.get(uid): + familiar: str = 'pre-existing-peer' + uid_short: str = f'{uid[0]}[{uid[1][-6:]}]' con_status += ( - f' -> Handshake with actor `{uid[0]}[{uid[1][-6:]}]` complete\n' + f' -> Handshake with {familiar} `{uid_short}` complete\n' ) + + if _pre_chan: + log.warning( + # con_status += ( + # ^TODO^ swap once we minimize conn duplication + f' -> Wait, we already have IPC with `{uid_short}`??\n' + f' |_{_pre_chan}\n' + ) + # IPC connection tracking for both peers and new children: # - if this is a new channel to a locally spawned # sub-actor there will be a spawn wait even registered @@ -1550,7 +1555,7 @@ class Actor: def accept_addr(self) -> tuple[str, int]: ''' Primary address to which the IPC transport server is - bound. + bound and listening for new connections. ''' # throws OSError on failure @@ -1567,6 +1572,7 @@ class Actor: def get_chans( self, uid: tuple[str, str], + ) -> list[Channel]: ''' Return all IPC channels to the actor with provided `uid`. -- 2.34.1 From e6ccfce75155bc083b114b67bd799bb0e203dc20 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 5 Jul 2024 13:31:29 -0400 Subject: [PATCH 26/42] Adjusts advanced fault tests to match new `TransportClosed` semantics --- tests/test_advanced_faults.py | 30 ++++++++++++++++++++++++------ 1 file changed, 24 insertions(+), 6 deletions(-) diff --git a/tests/test_advanced_faults.py b/tests/test_advanced_faults.py index dfaeb68..a4d1779 100644 --- a/tests/test_advanced_faults.py +++ b/tests/test_advanced_faults.py @@ -91,7 +91,8 @@ def test_ipc_channel_break_during_stream( # non-`trio` spawners should never hit the hang condition that # requires the user to do ctl-c to cancel the actor tree. - expect_final_exc = trio.ClosedResourceError + # expect_final_exc = trio.ClosedResourceError + expect_final_exc = tractor.TransportClosed mod: ModuleType = import_path( examples_dir() / 'advanced_faults' @@ -157,7 +158,7 @@ def test_ipc_channel_break_during_stream( if pre_aclose_msgstream: expect_final_exc = KeyboardInterrupt - # NOTE when the parent IPC side dies (even if the child's does as well + # NOTE when the parent IPC side dies (even if the child does as well # but the child fails BEFORE the parent) we always expect the # IPC layer to raise a closed-resource, NEVER do we expect # a stop msg since the parent-side ctx apis will error out @@ -169,7 +170,8 @@ def test_ipc_channel_break_during_stream( and ipc_break['break_child_ipc_after'] is False ): - expect_final_exc = trio.ClosedResourceError + # expect_final_exc = trio.ClosedResourceError + expect_final_exc = tractor.TransportClosed # BOTH but, PARENT breaks FIRST elif ( @@ -180,7 +182,8 @@ def test_ipc_channel_break_during_stream( ipc_break['break_parent_ipc_after'] ) ): - expect_final_exc = trio.ClosedResourceError + # expect_final_exc = trio.ClosedResourceError + expect_final_exc = tractor.TransportClosed with pytest.raises( expected_exception=( @@ -199,8 +202,8 @@ def test_ipc_channel_break_during_stream( **ipc_break, ) ) - except KeyboardInterrupt as kbi: - _err = kbi + except KeyboardInterrupt as _kbi: + kbi = _kbi if expect_final_exc is not KeyboardInterrupt: pytest.fail( 'Rxed unexpected KBI !?\n' @@ -209,6 +212,21 @@ def test_ipc_channel_break_during_stream( raise + except tractor.TransportClosed as _tc: + tc = _tc + if expect_final_exc is KeyboardInterrupt: + pytest.fail( + 'Unexpected transport failure !?\n' + f'{repr(tc)}' + ) + cause: Exception = tc.__cause__ + assert ( + type(cause) is trio.ClosedResourceError + and + cause.args[0] == 'another task closed this fd' + ) + raise + # get raw instance from pytest wrapper value = excinfo.value if isinstance(value, ExceptionGroup): -- 2.34.1 From bef3dd9e972bf25c63fefc0dbcfd5875a8a1ea29 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 5 Jul 2024 13:32:03 -0400 Subject: [PATCH 27/42] 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 113371d..ae1c46d 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' ) -- 2.34.1 From fc95c6719f7aa686d7a46752de7c21ff913b0e6f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 8 Jul 2024 20:57:41 -0400 Subject: [PATCH 28/42] 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 137710f..95472c9 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 71c691a..7acf984 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 86dd128..bb6791a 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 ae1c46d..02551fa 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() -- 2.34.1 From 131e3e81570b126dfcca3cbb7fd0cbd19dd8fcdc Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 10 Jul 2024 15:40:44 -0400 Subject: [PATCH 29/42] 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 7acf984..273451b 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 @@ -29,7 +28,7 @@ from tractor.devx._debug import ( _repl_fail_msg, ) from tractor._testing import ( - examples_dir, + mk_cmd, ) from conftest import ( _ci_env, @@ -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 fd79fe2..1f6624e 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, -- 2.34.1 From d216068713009918c7a27fe3ba48601170af86ee Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 10 Jul 2024 15:52:38 -0400 Subject: [PATCH 30/42] 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 -- 2.34.1 From a628eabb30752a763215bfd881ab8107288454d6 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 10 Jul 2024 18:17:42 -0400 Subject: [PATCH 31/42] 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 | 180 +++-------------------- tests/devx/test_tooling.py | 120 +++++++++++++++ tractor/_root.py | 12 +- tractor/devx/_stackscope.py | 166 +++++++++++++++++---- 7 files changed, 523 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 0000000..3cc084d --- /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 0000000..e69de29 diff --git a/tests/devx/conftest.py b/tests/devx/conftest.py new file mode 100644 index 0000000..b739569 --- /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 273451b..bb59045 100644 --- a/tests/devx/test_debugger.py +++ b/tests/devx/test_debugger.py @@ -16,7 +16,6 @@ import platform import time import pytest -import pexpect from pexpect.exceptions import ( TIMEOUT, EOF, @@ -27,12 +26,14 @@ from tractor.devx._debug import ( _crash_msg, _repl_fail_msg, ) -from tractor._testing import ( - mk_cmd, -) from conftest import ( _ci_env, ) +from .conftest import ( + expect, + in_prompt_msg, + assert_before, +) # TODO: The next great debugger audit could be done by you! # - recurrent entry to breakpoint() from single actor *after* and an @@ -69,154 +70,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 +137,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 +221,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 +286,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 +349,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 +492,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 +632,7 @@ def test_multi_daemon_subactors( ) child.sendline('c') - child.expect(pexpect.EOF) + child.expect(EOF) @has_nested_actors @@ -852,7 +708,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 +838,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 +880,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 +901,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 +1052,7 @@ def test_pause_from_sync( ) child.sendline('c') - child.expect(pexpect.EOF) + child.expect(EOF) def test_post_mortem_api( @@ -1301,7 +1157,7 @@ def test_post_mortem_api( # ) child.sendline('c') - child.expect(pexpect.EOF) + child.expect(EOF) def test_shield_pause( @@ -1376,7 +1232,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 0000000..3e48844 --- /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 882285a..51dbe5e 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 e8e97d1..944ae49 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 -- 2.34.1 From 8363317e112a356aee5b8f1ea8b05ba089191360 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 11 Jul 2024 12:11:31 -0400 Subject: [PATCH 32/42] 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 42eb35b..fca971d 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 a072706..19dcb9f 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 5615373..8cbf855 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -59,6 +59,7 @@ import os 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, @@ -102,6 +104,7 @@ from ._rpc import ( if TYPE_CHECKING: from ._supervise import ActorNursery + from trio._channel import MemoryChannelState log = get_logger('tractor') @@ -897,11 +900,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: @@ -1095,7 +1102,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 9f89600..a87ad36 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 fb737c1..b644ef2 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: -- 2.34.1 From f7469442e35171e77cd04b7d41224fb769e7ed0d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 11 Jul 2024 19:22:40 -0400 Subject: [PATCH 33/42] 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 5564d0e..c37c046 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 ' ) -- 2.34.1 From 1f1a3f19d50405f418398b70d36443fdb7654064 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 12 Jul 2024 15:57:41 -0400 Subject: [PATCH 34/42] 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 80ef933..4a46262 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 b739569..28a14cb 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 bb59045..ce4f1ed 100644 --- a/tests/devx/test_debugger.py +++ b/tests/devx/test_debugger.py @@ -96,14 +96,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) @@ -243,10 +244,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: @@ -265,17 +268,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 @@ -296,14 +297,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 @@ -319,9 +317,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'"] ) @@ -334,9 +331,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',] @@ -351,9 +347,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',] @@ -377,7 +372,7 @@ def test_multi_subactors( before = str(child.before.decode()) assert in_prompt_msg( - before, + child, [_pause_msg, "('breakpoint_forever'"] ) @@ -398,12 +393,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) @@ -427,9 +424,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'"] ) @@ -522,25 +518,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) @@ -609,14 +608,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( @@ -797,10 +794,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: @@ -891,9 +891,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"], ) @@ -903,8 +902,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. @@ -956,9 +953,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 @@ -1039,7 +1035,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 fca971d..f5fa0aa 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 -- 2.34.1 From 89127614d5cdc012a553abc2e73c2ddab5cfad97 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 13 Jul 2024 00:16:28 -0400 Subject: [PATCH 35/42] 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 baddfe0..f2fabdd 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 02551fa..0784e59 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 c37c046..8feaf88 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 -- 2.34.1 From 110a023a0349a5f3c6b7583a3e8d97ec9ef7519f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 29 Jul 2024 17:53:52 -0400 Subject: [PATCH 36/42] 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 f2fabdd..161435d 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( -- 2.34.1 From 3b39cce7412551225fa221c7d4e6e07e57571f6d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 29 Jul 2024 17:56:00 -0400 Subject: [PATCH 37/42] 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 0784e59..d38c223 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 -- 2.34.1 From ae95e0c83ebba4060625b8698b8239a017c72606 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 31 Jul 2024 21:34:57 -0400 Subject: [PATCH 38/42] 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 d38c223..e0a20c0 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 -- 2.34.1 From f7f738638dbbf9b94c5ad6d19704cd5ab79d0a86 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 22 Aug 2024 17:10:01 -0400 Subject: [PATCH 39/42] 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 e0a20c0..5e3b00d 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? -- 2.34.1 From 7859e743cc8e0799b1719c610d663a92afd0449d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 22 Aug 2024 17:22:53 -0400 Subject: [PATCH 40/42] Add `tb_hide: bool` ctl flag to `_open_and_supervise_one_cancels_all_nursery()` --- tractor/_supervise.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tractor/_supervise.py b/tractor/_supervise.py index b644ef2..de26807 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -374,11 +374,12 @@ class ActorNursery: @acm async def _open_and_supervise_one_cancels_all_nursery( actor: Actor, + tb_hide: bool = False, ) -> typing.AsyncGenerator[ActorNursery, None]: # normally don't need to show user by default - __tracebackhide__: bool = True + __tracebackhide__: bool = tb_hide outer_err: BaseException|None = None inner_err: BaseException|None = None -- 2.34.1 From 59f4024242818a222bb3dd8c6ada9c9da79673dd Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 22 Aug 2024 20:19:55 -0400 Subject: [PATCH 41/42] Add `indent: str` suport to `Context.pformat()` using `textwrap` --- tractor/_context.py | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index ff44a34..167aad6 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -46,6 +46,7 @@ from dataclasses import ( from functools import partial import inspect from pprint import pformat +import textwrap from typing import ( Any, AsyncGenerator, @@ -335,6 +336,7 @@ class Context: extra_fields: dict[str, Any]|None = None, # ^-TODO-^ some built-in extra state fields # we'll want in some devx specific cases? + indent: str|None = None, ) -> str: ds: str = '=' @@ -354,7 +356,6 @@ class Context: show_error_fields=True ) fmtstr: str = ( - f'\n' + f'{indent})>\n' ) + # NOTE: making this return a value that can be passed to # `eval()` is entirely **optional** dawggg B) # https://docs.python.org/3/library/functions.html#repr -- 2.34.1 From 5c2e9723152a20e0ec8cef808bb295e5e1e61a13 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 26 Aug 2024 14:29:09 -0400 Subject: [PATCH 42/42] Report any external-rent-task-canceller during msg-drain As in whenever `Context.cancel()` is not (runtime internally) called (i.e. `._cancel_called` is not set), we can attempt to detect the parent `trio` nursery/cancel-scope that is the source. Emit the report with a `.cancel()` level and attempt to repr in "sclang" form as well as unhide the stack frame for debug/traceback-in. --- tractor/msg/_ops.py | 27 ++++++++++++++++++++++++--- 1 file changed, 24 insertions(+), 3 deletions(-) diff --git a/tractor/msg/_ops.py b/tractor/msg/_ops.py index f0f3b6b..2faadb9 100644 --- a/tractor/msg/_ops.py +++ b/tractor/msg/_ops.py @@ -590,15 +590,36 @@ async def drain_to_final_msg( # SHOULD NOT raise that far end error, # 2. WE DID NOT REQUEST that cancel and thus # SHOULD RAISE HERE! - except trio.Cancelled as taskc: + except trio.Cancelled as _taskc: + taskc: trio.Cancelled = _taskc + + # report when the cancellation wasn't (ostensibly) due to + # RPC operation, some surrounding parent cancel-scope. + if not ctx._scope.cancel_called: + task: trio.lowlevel.Task = trio.lowlevel.current_task() + rent_n: trio.Nursery = task.parent_nursery + if ( + (local_cs := rent_n.cancel_scope).cancel_called + ): + log.cancel( + 'RPC-ctx cancelled by local-parent scope during drain!\n\n' + f'c}}>\n' + f' |_{rent_n}\n' + f' |_.cancel_scope = {local_cs}\n' + f' |_>c}}\n' + f' |_{ctx.pformat(indent=" "*9)}' + # ^TODO, some (other) simpler repr here? + ) + __tracebackhide__: bool = False + # CASE 2: mask the local cancelled-error(s) # only when we are sure the remote error is # the source cause of this local task's # cancellation. ctx.maybe_raise( hide_tb=hide_tb, - # TODO: when use this? - # from_src_exc=taskc, + from_src_exc=taskc, + # ?TODO? when *should* we use this? ) # CASE 1: we DID request the cancel we simply -- 2.34.1