From cfb2bc0feee62a0f24abd61d69e3e2661c120c6c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 7 Mar 2023 16:58:33 -0500 Subject: [PATCH 001/146] Enable `Context` backpressure by default; avoid startup race-crashes? --- tractor/_streaming.py | 28 ++++++++++++++++------------ 1 file changed, 16 insertions(+), 12 deletions(-) diff --git a/tractor/_streaming.py b/tractor/_streaming.py index b1129567..a0f0eaf0 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -390,9 +390,13 @@ class Context: # only set on the callee side _scope_nursery: Optional[trio.Nursery] = None - _backpressure: bool = False + _backpressure: bool = True - async def send_yield(self, data: Any) -> None: + async def send_yield( + self, + data: Any, + + ) -> None: warnings.warn( "`Context.send_yield()` is now deprecated. " @@ -466,7 +470,7 @@ class Context: async def cancel( self, - msg: Optional[str] = None, + msg: str | None = None, ) -> None: ''' @@ -535,8 +539,8 @@ class Context: async def open_stream( self, - backpressure: Optional[bool] = True, - msg_buffer_size: Optional[int] = None, + backpressure: bool | None = True, + msg_buffer_size: int | None = None, ) -> AsyncGenerator[MsgStream, None]: ''' @@ -677,7 +681,7 @@ class Context: async def started( self, - value: Optional[Any] = None + value: Any | None = None ) -> None: ''' @@ -705,10 +709,10 @@ class Context: def stream(func: Callable) -> Callable: - """Mark an async function as a streaming routine with ``@stream``. + ''' + Mark an async function as a streaming routine with ``@stream``. - """ - # annotate + ''' # TODO: apply whatever solution ``mypy`` ends up picking for this: # https://github.com/python/mypy/issues/2087#issuecomment-769266912 func._tractor_stream_function = True # type: ignore @@ -737,10 +741,10 @@ def stream(func: Callable) -> Callable: def context(func: Callable) -> Callable: - """Mark an async function as a streaming routine with ``@context``. + ''' + Mark an async function as a streaming routine with ``@context``. - """ - # annotate + ''' # TODO: apply whatever solution ``mypy`` ends up picking for this: # https://github.com/python/mypy/issues/2087#issuecomment-769266912 func._tractor_context_function = True # type: ignore From d75343106b9220ce85b83e4c3b35ae0ad1d4e2c4 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 7 Mar 2023 17:09:43 -0500 Subject: [PATCH 002/146] More single doc-strs in discovery mod --- tractor/_discovery.py | 14 ++++++++++---- tractor/_runtime.py | 1 + 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/tractor/_discovery.py b/tractor/_discovery.py index b6957ba3..03775ac2 100644 --- a/tractor/_discovery.py +++ b/tractor/_discovery.py @@ -41,8 +41,10 @@ async def get_arbiter( port: int, ) -> AsyncGenerator[Union[Portal, LocalPortal], None]: - '''Return a portal instance connected to a local or remote + ''' + Return a portal instance connected to a local or remote arbiter. + ''' actor = current_actor() @@ -134,12 +136,16 @@ async def find_actor( @acm async def wait_for_actor( name: str, - arbiter_sockaddr: tuple[str, int] | None = None + 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. + ''' + Wait on an actor to register with the arbiter. A portal to the first registered actor is returned. - """ + + ''' actor = current_actor() async with get_arbiter( diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 707b9dd6..06bc34f8 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -825,6 +825,7 @@ class Actor: 1, f'\n*** No stream open on `{self.uid[0]}` side! ***\n' ) + text = '\n'.join(lines) if ctx._backpressure: From 903537ce04cdc0ae39abb394662414453cdf9bca Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 2 Apr 2023 14:01:39 -0400 Subject: [PATCH 003/146] Tweak context doc str --- tractor/_streaming.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tractor/_streaming.py b/tractor/_streaming.py index a0f0eaf0..9908ab65 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -350,8 +350,9 @@ class Context: An inter-actor, ``trio`` task communication context. NB: This class should never be instatiated directly, it is delivered - by either runtime machinery to a remotely started task or by entering - ``Portal.open_context()``. + by either, + - runtime machinery to a remotely started task or, + - by entering ``Portal.open_context()``. Allows maintaining task or protocol specific state between 2 communicating actor tasks. A unique context is created on the From b3f9251edaa97ff0ca18eaa6ab54d82f6515710c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 4 Apr 2023 15:06:38 -0400 Subject: [PATCH 004/146] Add some log-level method doc-strings --- tractor/log.py | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/tractor/log.py b/tractor/log.py index 342257fa..1ea99315 100644 --- a/tractor/log.py +++ b/tractor/log.py @@ -82,6 +82,10 @@ class StackLevelAdapter(logging.LoggerAdapter): msg: str, ) -> None: + ''' + IPC level msg-ing. + + ''' return self.log(5, msg) def runtime( @@ -94,12 +98,20 @@ class StackLevelAdapter(logging.LoggerAdapter): self, msg: str, ) -> None: + ''' + Cancellation logging, mostly for runtime reporting. + + ''' return self.log(16, msg) def pdb( self, msg: str, ) -> None: + ''' + Debugger logging. + + ''' return self.log(500, msg) def log(self, level, msg, *args, **kwargs): From e80e0a551f3e76e42c7c83f172f5144621460003 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 7 Apr 2023 16:07:26 -0400 Subject: [PATCH 005/146] Change a bunch of log levels to cancel, including any `ContextCancelled` handling --- tractor/_entry.py | 2 +- tractor/_runtime.py | 51 +++++++++++++++++++++++++++++++++++---------- 2 files changed, 41 insertions(+), 12 deletions(-) diff --git a/tractor/_entry.py b/tractor/_entry.py index 1e7997e8..e8fb56db 100644 --- a/tractor/_entry.py +++ b/tractor/_entry.py @@ -132,7 +132,7 @@ def _trio_main( else: trio.run(trio_main) except KeyboardInterrupt: - log.warning(f"Actor {actor.uid} received KBI") + log.cancel(f"Actor {actor.uid} received KBI") finally: log.info(f"Actor {actor.uid} terminated") diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 06bc34f8..97f30253 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -221,6 +221,8 @@ async def _invoke( assert cs if cs.cancelled_caught: + # if 'brokerd.kraken' in actor.uid: + # await _debug.breakpoint() # TODO: pack in ``trio.Cancelled.__traceback__`` here # so they can be unwrapped and displayed on the caller @@ -228,11 +230,11 @@ async def _invoke( fname = func.__name__ if ctx._cancel_called: - msg = f'`{fname}()` cancelled itself' + msg = f'`{fname}()`@{actor.uid} cancelled itself' elif cs.cancel_called: msg = ( - f'`{fname}()` was remotely cancelled by its caller ' + f'`{fname}()`@{actor.uid} was remotely cancelled by its caller ' f'{ctx.chan.uid}' ) @@ -910,9 +912,15 @@ class Actor: ''' cid = str(uuid.uuid4()) assert chan.uid - ctx = self.get_context(chan, cid, msg_buffer_size=msg_buffer_size) + ctx = self.get_context( + chan, + cid, + msg_buffer_size=msg_buffer_size, + ) log.runtime(f"Sending cmd to {chan.uid}: {ns}.{func}({kwargs})") - await chan.send({'cmd': (ns, func, kwargs, self.uid, cid)}) + await chan.send( + {'cmd': (ns, func, kwargs, self.uid, cid)} + ) # Wait on first response msg and validate; this should be # immediate. @@ -922,7 +930,11 @@ class Actor: if 'error' in first_msg: raise unpack_error(first_msg, chan) - elif functype not in ('asyncfunc', 'asyncgen', 'context'): + elif functype not in ( + 'asyncfunc', + 'asyncgen', + 'context', + ): raise ValueError(f"{first_msg} is an invalid response packet?") ctx._remote_func_type = functype @@ -1130,7 +1142,7 @@ class Actor: async def cancel_rpc_tasks( self, - only_chan: Optional[Channel] = None, + only_chan: Channel | None = None, ) -> None: ''' Cancel all existing RPC responder tasks using the cancel scope @@ -1331,13 +1343,15 @@ async def async_main( ) ) log.runtime("Waiting on service nursery to complete") - log.runtime("Service nursery complete") - log.runtime("Waiting on root nursery to complete") + log.runtime( + "Service nursery complete\n" + "Waiting on root nursery to complete" + ) # 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.info("Closing all actor lifetime contexts") + log.runtime("Closing all actor lifetime contexts") actor.lifetime_stack.close() if not registered_with_arbiter: @@ -1358,7 +1372,14 @@ async def async_main( await try_ship_error_to_parent(actor._parent_chan, err) # always! - log.exception("Actor errored:") + match err: + case ContextCancelled(): + log.cancel( + f'Actor: {actor.uid} was task-context-cancelled with,\n' + f'str(err)' + ) + case _: + log.exception("Actor errored:") raise finally: @@ -1631,7 +1652,15 @@ async def process_messages( else: # ship any "internal" exception (i.e. one from internal # machinery not from an rpc task) to parent - log.exception("Actor errored:") + match err: + case ContextCancelled(): + log.cancel( + f'Actor: {actor.uid} was task-context-cancelled with,\n' + f'str(err)' + ) + case _: + log.exception("Actor errored:") + if actor._parent_chan: await try_ship_error_to_parent(actor._parent_chan, err) From 831790377b84f403bfa878dae6afdab88c455598 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 7 Apr 2023 16:08:07 -0400 Subject: [PATCH 006/146] Assign `RemoteActorError` boxed error type for context cancelleds --- tractor/_exceptions.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 5440cad0..58caada1 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -55,9 +55,11 @@ class RemoteActorError(Exception): class InternalActorError(RemoteActorError): - """Remote internal ``tractor`` error indicating + ''' + Remote internal ``tractor`` error indicating failure of some primitive or machinery. - """ + + ''' class TransportClosed(trio.ClosedResourceError): @@ -136,7 +138,7 @@ def unpack_error( if type_name == 'ContextCancelled': err_type = ContextCancelled - suberror_type = trio.Cancelled + suberror_type = RemoteActorError else: # try to lookup a suitable local error type for ns in [ From 220b244508e15381feafb41244092b7e1cafa463 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 7 Apr 2023 16:08:50 -0400 Subject: [PATCH 007/146] Log waiter task cancelling msg as cancel-level --- tractor/_spawn.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index e0472a98..b1a20f49 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -457,7 +457,7 @@ async def trio_proc( # cancel result waiter that may have been spawned in # tandem if not done already - log.warning( + log.cancel( "Cancelling existing result waiter task for " f"{subactor.uid}") nursery.cancel_scope.cancel() From 71cd4453190401573eb3dc0fb75e210b445461b5 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 12 Apr 2023 18:13:30 -0400 Subject: [PATCH 008/146] Add new set of context cancellation tests These will verify new changes to the runtime/messaging core which allows us to adopt an "ignore cancel if requested by us" style handling of `ContextCancelled` more like how `trio` does with `trio.Nursery.cancel_scope.cancel()`. We now expect a `ContextCancelled.canceller: tuple` which is set to the actor uid of the actor which requested the cancellation which eventually resulted in the remote error-msg. Also adds some experimental tweaks to the "backpressure" test which it turns out is very problematic in coordination with context cancellation since blocking on the feed mem chan to some task will block the ipc msg loop and thus handling of cancellation.. More to come to both the test and core to address this hopefully since right now this test is failing. --- tests/test_context_stream_semantics.py | 141 ++++++++++++++++++++----- 1 file changed, 116 insertions(+), 25 deletions(-) diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index c92c4407..f7c1e92e 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -13,7 +13,10 @@ from typing import Optional import pytest import trio import tractor -from tractor._exceptions import StreamOverrun +from tractor._exceptions import ( + StreamOverrun, + ContextCancelled, +) from conftest import tractor_test @@ -91,7 +94,10 @@ async def not_started_but_stream_opened( @pytest.mark.parametrize( 'target', - [too_many_starteds, not_started_but_stream_opened], + [ + too_many_starteds, + not_started_but_stream_opened, + ], ids='misuse_type={}'.format, ) def test_started_misuse(target): @@ -228,6 +234,70 @@ def test_simple_context( trio.run(main) +@pytest.mark.parametrize( + 'cancel_method', + ['ctx', 'portal'], + ids=lambda item: f'cancel_method={item}' +) +@pytest.mark.parametrize( + 'result_before_exit', + [True, False], + ids=lambda item: f'result_before_exit={item}' +) +def test_caller_cancels( + cancel_method: str, + result_before_exit: bool, +): + ''' + Verify that when the opening side of a context (aka the caller) + cancels that context, the ctx does not raise a cancelled when + either calling `.result()` or on context exit. + + ''' + + async def check_canceller( + ctx: tractor.Context, + ) -> None: + # should not raise yet return the remote + # context cancelled error. + err = await ctx.result() + assert isinstance(err, ContextCancelled) + assert ( + tuple(err.canceller) + == + tractor.current_actor().uid + ) + + async def main(): + async with tractor.open_nursery() as nursery: + portal = await nursery.start_actor( + 'simple_context', + enable_modules=[__name__], + ) + with trio.fail_after(0.5): + async with portal.open_context( + simple_setup_teardown, + data=10, + block_forever=True, + ) as (ctx, sent): + + if cancel_method == 'ctx': + await ctx.cancel() + else: + await portal.cancel_actor() + + if result_before_exit: + await check_canceller(ctx) + + if not result_before_exit: + await check_canceller(ctx) + + if cancel_method != 'portal': + await portal.cancel_actor() + + trio.run(main) + + # basic stream terminations: # - callee context closes without using stream # - caller context closes without using stream @@ -506,7 +576,6 @@ async def test_callee_cancels_before_started(): cancel_self, ) as (ctx, sent): async with ctx.open_stream(): - await trio.sleep_forever() # raises a special cancel signal @@ -610,7 +679,7 @@ def test_one_end_stream_not_opened(overrun_by): # 2 overrun cases and the no overrun case (which pushes right up to # the msg limit) - if overrunner == 'caller' or 'cance' in overrunner: + if overrunner == 'caller' or 'cancel' in overrunner: with pytest.raises(tractor.RemoteActorError) as excinfo: trio.run(main) @@ -634,7 +703,7 @@ async def echo_back_sequence( ctx: tractor.Context, seq: list[int], - msg_buffer_size: Optional[int] = None, + msg_buffer_size: int | None = None, ) -> None: ''' @@ -644,11 +713,13 @@ async def echo_back_sequence( await ctx.started() async with ctx.open_stream( msg_buffer_size=msg_buffer_size, + backpressure=True, ) as stream: seq = list(seq) # bleh, `msgpack`... count = 0 - while count < 3: + # while count < 10: + while True: batch = [] async for msg in stream: batch.append(msg) @@ -661,13 +732,17 @@ async def echo_back_sequence( count += 1 - return 'yo' + print("EXITING CALLEEE") + return 'yo' -def test_stream_backpressure(): +def test_stream_backpressure( + loglevel: str, +): ''' Demonstrate small overruns of each task back and forth - on a stream not raising any errors by default. + on a stream not raising any errors by default by setting + the ``backpressure=True``. ''' async def main(): @@ -675,16 +750,21 @@ def test_stream_backpressure(): portal = await n.start_actor( 'callee_sends_forever', enable_modules=[__name__], + loglevel=loglevel, ) seq = list(range(3)) async with portal.open_context( echo_back_sequence, seq=seq, - msg_buffer_size=1, ) as (ctx, sent): + assert sent is None - async with ctx.open_stream(msg_buffer_size=1) as stream: + async with ctx.open_stream( + msg_buffer_size=1, + backpressure=True, + # allow_overruns=True, + ) as stream: count = 0 while count < 3: for msg in seq: @@ -693,15 +773,25 @@ def test_stream_backpressure(): await trio.sleep(0.1) batch = [] + # with trio.move_on_after(1) as cs: async for msg in stream: + print(f'RX {msg}') batch.append(msg) if batch == seq: break count += 1 + # if cs.cancelled_caught: + # break + + # cancel the remote task + # print('SENDING ROOT SIDE CANCEL') + # await ctx.cancel() + # here the context should return - assert await ctx.result() == 'yo' + res = await ctx.result() + assert res == 'yo' # cancel the daemon await portal.cancel_actor() @@ -737,18 +827,18 @@ async def attach_to_sleep_forever(): finally: # XXX: previously this would trigger local # ``ContextCancelled`` to be received and raised in the - # local context overriding any local error due to - # logic inside ``_invoke()`` which checked for - # an error set on ``Context._error`` and raised it in - # under a cancellation scenario. - - # The problem is you can have a remote cancellation - # that is part of a local error and we shouldn't raise - # ``ContextCancelled`` **iff** we weren't the side of - # the context to initiate it, i.e. + # local context overriding any local error due to logic + # inside ``_invoke()`` which checked for an error set on + # ``Context._error`` and raised it in a cancellation + # scenario. + # ------ + # The problem is you can have a remote cancellation that + # is part of a local error and we shouldn't raise + # ``ContextCancelled`` **iff** we **were not** the side + # of the context to initiate it, i.e. # ``Context._cancel_called`` should **NOT** have been # set. The special logic to handle this case is now - # inside ``Context._may_raise_from_remote_msg()`` XD + # inside ``Context._maybe_raise_from_remote_msg()`` XD await peer_ctx.cancel() @@ -769,9 +859,10 @@ async def error_before_started( def test_do_not_swallow_error_before_started_by_remote_contextcancelled(): ''' - Verify that an error raised in a remote context which itself opens another - remote context, which it cancels, does not ovverride the original error that - caused the cancellation of the secondardy context. + Verify that an error raised in a remote context which itself opens + another remote context, which it cancels, does not ovverride the + original error that caused the cancellation of the secondardy + context. ''' async def main(): From 67f82c6ebd75070e1a40e3ed0273b869fca98a24 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 12 Apr 2023 18:18:46 -0400 Subject: [PATCH 009/146] Add new remote error introspection attrs To handle both remote cancellation this adds `ContextCanceled.canceller: tuple` the uid of the cancel requesting actor and is expected to be set by the runtime when servicing any remote cancel request. This makes it possible for `ContextCancelled` receivers to know whether "their actor runtime" is the source of the cancellation. Also add an explicit `RemoteActor.src_actor_uid` which better formalizes the notion of "which remote actor" the error originated from. Both of these new attrs are expected to be packed in the `.msgdata` when the errors are loaded locally. --- tractor/_exceptions.py | 41 ++++++++++++++++++++++++++++------------- 1 file changed, 28 insertions(+), 13 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 58caada1..07f4627c 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -18,18 +18,18 @@ Our classy exception set. """ +import builtins +import importlib from typing import ( Any, - Optional, Type, ) -import importlib -import builtins import traceback import exceptiongroup as eg import trio +from ._state import current_actor _this_mod = importlib.import_module(__name__) @@ -44,7 +44,7 @@ class RemoteActorError(Exception): def __init__( self, message: str, - suberror_type: Optional[Type[BaseException]] = None, + suberror_type: Type[BaseException] | None = None, **msgdata ) -> None: @@ -53,6 +53,10 @@ class RemoteActorError(Exception): self.type = suberror_type self.msgdata = msgdata + @property + def src_actor_uid(self) -> tuple[str, str] | None: + return self.msgdata.get('src_actor_uid') + class InternalActorError(RemoteActorError): ''' @@ -62,14 +66,21 @@ class InternalActorError(RemoteActorError): ''' +class ContextCancelled(RemoteActorError): + ''' + Inter-actor task context was cancelled by either a call to + ``Portal.cancel_actor()`` or ``Context.cancel()``. + + ''' + @property + def canceller(self) -> tuple[str, str] | None: + return self.msgdata.get('canceller') + + class TransportClosed(trio.ClosedResourceError): "Underlying channel transport was closed prior to use" -class ContextCancelled(RemoteActorError): - "Inter-actor task context cancelled itself on the callee side." - - class NoResult(RuntimeError): "No final result is expected for this actor" @@ -108,13 +119,17 @@ def pack_error( else: tb_str = traceback.format_exc() - return { - 'error': { - 'tb_str': tb_str, - 'type_str': type(exc).__name__, - } + error_msg = { + 'tb_str': tb_str, + 'type_str': type(exc).__name__, + 'src_actor_uid': current_actor().uid, } + if isinstance(exc, ContextCancelled): + error_msg.update(exc.msgdata) + + return {'error': error_msg} + def unpack_error( From 03644f59cccb5946215ee5bb893ea7c3b39d5b63 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 13 Apr 2023 15:14:49 -0400 Subject: [PATCH 010/146] Augment test cases for callee-returns-result early Turns out stuff was totally broken in these cases because we're either closing the underlying mem chan too early or not handling the "allow_overruns" mode's cancellation correctly.. --- tests/test_context_stream_semantics.py | 122 +++++++++++++++++-------- 1 file changed, 84 insertions(+), 38 deletions(-) diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index f7c1e92e..2531fd86 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -234,19 +234,25 @@ def test_simple_context( trio.run(main) +@pytest.mark.parametrize( + 'callee_returns_early', + [True, False], + ids=lambda item: f'callee_returns_early={item}' +) @pytest.mark.parametrize( 'cancel_method', ['ctx', 'portal'], ids=lambda item: f'cancel_method={item}' ) @pytest.mark.parametrize( - 'result_before_exit', + 'chk_ctx_result_before_exit', [True, False], - ids=lambda item: f'result_before_exit={item}' + ids=lambda item: f'chk_ctx_result_before_exit={item}' ) def test_caller_cancels( cancel_method: str, - result_before_exit: bool, + chk_ctx_result_before_exit: bool, + callee_returns_early: bool, ): ''' Verify that when the opening side of a context (aka the caller) @@ -254,19 +260,24 @@ def test_caller_cancels( either calling `.result()` or on context exit. ''' - async def check_canceller( ctx: tractor.Context, ) -> None: # should not raise yet return the remote # context cancelled error. - err = await ctx.result() - assert isinstance(err, ContextCancelled) - assert ( - tuple(err.canceller) - == - tractor.current_actor().uid - ) + res = await ctx.result() + + if callee_returns_early: + assert res == 'yo' + + else: + err = res + assert isinstance(err, ContextCancelled) + assert ( + tuple(err.canceller) + == + tractor.current_actor().uid + ) async def main(): async with tractor.open_nursery() as nursery: @@ -274,22 +285,29 @@ def test_caller_cancels( 'simple_context', enable_modules=[__name__], ) - with trio.fail_after(0.5): + timeout = 0.5 if not callee_returns_early else 2 + with trio.fail_after(timeout): async with portal.open_context( simple_setup_teardown, data=10, - block_forever=True, + block_forever=not callee_returns_early, ) as (ctx, sent): + if callee_returns_early: + # ensure we block long enough before sending + # a cancel such that the callee has already + # returned it's result. + await trio.sleep(0.5) + if cancel_method == 'ctx': await ctx.cancel() else: await portal.cancel_actor() - if result_before_exit: + if chk_ctx_result_before_exit: await check_canceller(ctx) - if not result_before_exit: + if not chk_ctx_result_before_exit: await check_canceller(ctx) if cancel_method != 'portal': @@ -703,46 +721,71 @@ async def echo_back_sequence( ctx: tractor.Context, seq: list[int], + wait_for_cancel: bool, msg_buffer_size: int | None = None, ) -> None: ''' - Send endlessly on the calleee stream. + Send endlessly on the calleee stream using a small buffer size + setting on the contex to simulate backlogging that would normally + cause overruns. ''' + # NOTE: ensure that if the caller is expecting to cancel this task + # that we stay echoing much longer then they are so we don't + # return early instead of receive the cancel msg. + total_batches: int = 1000 if wait_for_cancel else 6 + await ctx.started() async with ctx.open_stream( msg_buffer_size=msg_buffer_size, - backpressure=True, + allow_overruns=True, ) as stream: seq = list(seq) # bleh, `msgpack`... - count = 0 - # while count < 10: - while True: + for _ in range(total_batches): batch = [] async for msg in stream: batch.append(msg) if batch == seq: break + print('callee waiting on next') + for msg in batch: print(f'callee sending {msg}') await stream.send(msg) - count += 1 - - print("EXITING CALLEEE") + print( + 'EXITING CALLEEE:\n' + f'{ctx.cancel_called_remote}' + ) return 'yo' -def test_stream_backpressure( +@pytest.mark.parametrize( + 'cancel_ctx', + [True, False], + ids=lambda item: f'cancel_ctx={item}' +) +def test_allow_overruns_stream( + cancel_ctx: bool, loglevel: str, ): ''' Demonstrate small overruns of each task back and forth on a stream not raising any errors by default by setting - the ``backpressure=True``. + the ``allow_overruns=True``. + + The original idea here was to show that if you set the feeder mem + chan to a size smaller then the # of msgs sent you could could not + get a `StreamOverrun` crash plus maybe get all the msgs that were + sent. The problem with the "real backpressure" case is that due to + the current arch it can result in the msg loop being blocked and thus + blocking cancellation - which is like super bad. So instead this test + had to be adjusted to more or less just "not send overrun errors" so + as to handle the case where the sender just moreso cares about not getting + errored out when it send to fast.. ''' async def main(): @@ -756,42 +799,45 @@ def test_stream_backpressure( async with portal.open_context( echo_back_sequence, seq=seq, + wait_for_cancel=cancel_ctx, ) as (ctx, sent): assert sent is None async with ctx.open_stream( msg_buffer_size=1, - backpressure=True, - # allow_overruns=True, + allow_overruns=True, ) as stream: count = 0 while count < 3: for msg in seq: - print(f'caller sending {msg}') + print(f'root tx {msg}') await stream.send(msg) await trio.sleep(0.1) batch = [] - # with trio.move_on_after(1) as cs: async for msg in stream: - print(f'RX {msg}') + print(f'root rx {msg}') batch.append(msg) if batch == seq: break count += 1 - # if cs.cancelled_caught: - # break + if cancel_ctx: + # cancel the remote task + print('sending root side cancel') + await ctx.cancel() - # cancel the remote task - # print('SENDING ROOT SIDE CANCEL') - # await ctx.cancel() - - # here the context should return res = await ctx.result() - assert res == 'yo' + + if cancel_ctx: + assert isinstance(res, ContextCancelled) + assert tuple(res.canceller) == tractor.current_actor().uid + + else: + print(f'RX ROOT SIDE RESULT {res}') + assert res == 'yo' # cancel the daemon await portal.cancel_actor() From f54c415060f711ed5414d4e791a06834f8c17518 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 13 Apr 2023 15:16:39 -0400 Subject: [PATCH 011/146] Move `NoRuntime` import inside `current_actor()` to avoid cycle --- tractor/_state.py | 24 +++++++++++++----------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/tractor/_state.py b/tractor/_state.py index 28fa16e7..f94c3ebb 100644 --- a/tractor/_state.py +++ b/tractor/_state.py @@ -23,11 +23,6 @@ from typing import ( Any, ) -import trio - -from ._exceptions import NoRuntime - - _current_actor: Optional['Actor'] = None # type: ignore # noqa _runtime_vars: dict[str, Any] = { '_debug_mode': False, @@ -37,8 +32,11 @@ _runtime_vars: dict[str, Any] = { def current_actor(err_on_no_runtime: bool = True) -> 'Actor': # type: ignore # noqa - """Get the process-local actor instance. - """ + ''' + Get the process-local actor instance. + + ''' + from ._exceptions import NoRuntime if _current_actor is None and err_on_no_runtime: raise NoRuntime("No local actor has been initialized yet") @@ -46,16 +44,20 @@ def current_actor(err_on_no_runtime: bool = True) -> 'Actor': # type: ignore # def is_main_process() -> bool: - """Bool determining if this actor is running in the top-most process. - """ + ''' + Bool determining if this actor is running in the top-most process. + + ''' import multiprocessing as mp return mp.current_process().name == 'MainProcess' def debug_mode() -> bool: - """Bool determining if "debug mode" is on which enables + ''' + Bool determining if "debug mode" is on which enables remote subactor pdb entry on crashes. - """ + + ''' return bool(_runtime_vars['_debug_mode']) From 90e41016b9c20f88dcd4bb467e40e341259cba63 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 13 Apr 2023 15:18:00 -0400 Subject: [PATCH 012/146] Only tuplize `.canceller` if non-`None` --- tractor/_exceptions.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 07f4627c..6da2e657 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -74,7 +74,9 @@ class ContextCancelled(RemoteActorError): ''' @property def canceller(self) -> tuple[str, str] | None: - return self.msgdata.get('canceller') + value = self.msgdata.get('canceller') + if value: + return tuple(value) class TransportClosed(trio.ClosedResourceError): From c72026091e036a7ecb8485550b4ede7fecaeb44b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 13 Apr 2023 16:03:35 -0400 Subject: [PATCH 013/146] Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`. --- tractor/_portal.py | 88 ++++--- tractor/_root.py | 4 +- tractor/_runtime.py | 417 ++++++++++++++++++++-------------- tractor/_streaming.py | 518 ++++++++++++++++++++++++++++++++++-------- 4 files changed, 724 insertions(+), 303 deletions(-) diff --git a/tractor/_portal.py b/tractor/_portal.py index 17871aa2..e61ac37a 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -103,7 +103,7 @@ class Portal: # When set to a ``Context`` (when _submit_for_result is called) # it is expected that ``result()`` will be awaited at some # point. - self._expect_result: Optional[Context] = None + self._expect_result: Context | None = None self._streams: set[MsgStream] = set() self.actor = current_actor() @@ -209,7 +209,10 @@ class Portal: try: # send cancel cmd - might not get response # XXX: sure would be nice to make this work with a proper shield - with trio.move_on_after(timeout or self.cancel_timeout) as cs: + with trio.move_on_after( + timeout + or self.cancel_timeout + ) as cs: cs.shield = True await self.run_from_ns('self', 'cancel') @@ -330,7 +333,9 @@ class Portal: f'{async_gen_func} must be an async generator function!') fn_mod_path, fn_name = NamespacePath.from_ref( - async_gen_func).to_tuple() + async_gen_func + ).to_tuple() + ctx = await self.actor.start_remote_task( self.channel, fn_mod_path, @@ -396,13 +401,16 @@ class Portal: raise TypeError( f'{func} must be an async generator function!') + # TODO: i think from here onward should probably + # just be factored into an `@acm` inside a new + # a new `_context.py` mod. fn_mod_path, fn_name = NamespacePath.from_ref(func).to_tuple() ctx = await self.actor.start_remote_task( self.channel, fn_mod_path, fn_name, - kwargs + kwargs, ) assert ctx._remote_func_type == 'context' @@ -426,29 +434,47 @@ class Portal: f' but received a non-error msg:\n{pformat(msg)}' ) - _err: Optional[BaseException] = None - ctx._portal = self + _err: BaseException | None = None + ctx._portal: Portal = self - uid = self.channel.uid - cid = ctx.cid - etype: Optional[Type[BaseException]] = None + uid: tuple = self.channel.uid + cid: str = ctx.cid + etype: Type[BaseException] | None = None - # deliver context instance and .started() msg value in open tuple. + # deliver context instance and .started() msg value in enter + # tuple. try: - async with trio.open_nursery() as scope_nursery: - ctx._scope_nursery = scope_nursery - - # do we need this? - # await trio.lowlevel.checkpoint() + async with trio.open_nursery() as nurse: + ctx._scope_nursery = nurse + ctx._scope = nurse.cancel_scope yield ctx, first + # when in allow_ovveruns mode there may be lingering + # overflow sender tasks remaining? + if nurse.child_tasks: + # ensure we are in overrun state with + # ``._allow_overruns=True`` bc otherwise + # there should be no tasks in this nursery! + if ( + not ctx._allow_overruns + or len(nurse.child_tasks) > 1 + ): + raise RuntimeError( + 'Context has sub-tasks but is ' + 'not in `allow_overruns=True` Mode!?' + ) + ctx._scope.cancel() + except ContextCancelled as err: _err = err + + # swallow and mask cross-actor task context cancels that + # were initiated by *this* side's task. if not ctx._cancel_called: - # context was cancelled at the far end but was - # not part of this end requesting that cancel - # so raise for the local task to respond and handle. + # XXX: this should NEVER happen! + # from ._debug import breakpoint + # await breakpoint() raise # if the context was cancelled by client code @@ -468,17 +494,17 @@ class Portal: ) as err: etype = type(err) - # the context cancels itself on any cancel - # causing error. - if ctx.chan.connected(): - log.cancel( - 'Context cancelled for task, sending cancel request..\n' - f'task:{cid}\n' - f'actor:{uid}' - ) + # cancel ourselves on any error. + log.cancel( + 'Context cancelled for task, sending cancel request..\n' + f'task:{cid}\n' + f'actor:{uid}' + ) + try: + await ctx.cancel() - else: + except trio.BrokenResourceError: log.warning( 'IPC connection for context is broken?\n' f'task:{cid}\n' @@ -487,12 +513,7 @@ class Portal: raise - finally: - # in the case where a runtime nursery (due to internal bug) - # or a remote actor transmits an error we want to be - # sure we get the error the underlying feeder mem chan. - # if it's not raised here it *should* be raised from the - # msg loop nursery right? + else: if ctx.chan.connected(): log.info( 'Waiting on final context-task result for\n' @@ -505,6 +526,7 @@ class Portal: f'value from callee `{result}`' ) + finally: # though it should be impossible for any tasks # operating *in* this scope to have survived # we tear down the runtime feeder chan last diff --git a/tractor/_root.py b/tractor/_root.py index 64652a19..a2d31586 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -254,7 +254,9 @@ async def open_root_actor( # tempn.start_soon(an.exited.wait) logger.cancel("Shutting down root actor") - await actor.cancel() + await actor.cancel( + requesting_uid=actor.uid, + ) finally: _state._current_actor = None diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 97f30253..9d9427d6 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -28,9 +28,11 @@ import inspect import signal import sys from typing import ( - Any, Optional, - Union, TYPE_CHECKING, + Any, Callable, + Optional, + Union, + TYPE_CHECKING, ) import uuid from types import ModuleType @@ -44,7 +46,10 @@ import trio # type: ignore from trio_typing import TaskStatus from ._ipc import Channel -from ._streaming import Context +from ._streaming import ( + mk_context, + Context, +) from .log import get_logger from ._exceptions import ( pack_error, @@ -53,7 +58,6 @@ from ._exceptions import ( is_multi_cancelled, ContextCancelled, TransportClosed, - StreamOverrun, ) from . import _debug from ._discovery import get_arbiter @@ -79,7 +83,7 @@ async def _invoke( is_rpc: bool = True, task_status: TaskStatus[ - Union[trio.CancelScope, BaseException] + Union[Context, BaseException] ] = trio.TASK_STATUS_IGNORED, ): ''' @@ -99,7 +103,14 @@ async def _invoke( # activated cancel scope ref cs: Optional[trio.CancelScope] = None - ctx = actor.get_context(chan, cid) + ctx = actor.get_context( + chan, + cid, + # We shouldn't ever need to pass this through right? + # it's up to the soon-to-be called rpc task to + # open the stream with this option. + # allow_overruns=True, + ) context: bool = False if getattr(func, '_tractor_stream_function', False): @@ -138,7 +149,10 @@ async def _invoke( ): raise TypeError(f'{func} must be an async function!') - coro = func(**kwargs) + try: + coro = func(**kwargs) + except TypeError: + raise if inspect.isasyncgen(coro): await chan.send({'functype': 'asyncgen', 'cid': cid}) @@ -150,7 +164,8 @@ async def _invoke( # of the async gen in order to be sure the cancel # is propagated! with cancel_scope as cs: - task_status.started(cs) + ctx._scope = cs + task_status.started(ctx) async with aclosing(coro) as agen: async for item in agen: # TODO: can we send values back in here? @@ -176,7 +191,8 @@ async def _invoke( # manualy construct the response dict-packet-responses as # above with cancel_scope as cs: - task_status.started(cs) + ctx._scope = cs + task_status.started(ctx) await coro if not cs.cancelled_caught: @@ -189,19 +205,25 @@ async def _invoke( await chan.send({'functype': 'context', 'cid': cid}) try: - async with trio.open_nursery() as scope_nursery: - ctx._scope_nursery = scope_nursery - cs = scope_nursery.cancel_scope - task_status.started(cs) + with cancel_scope as cs: + ctx._scope = cs + task_status.started(ctx) res = await coro await chan.send({'return': res, 'cid': cid}) - except BaseExceptionGroup: + # XXX: do we ever trigger this block any more? + except ( + BaseExceptionGroup, + trio.Cancelled, + ): # if a context error was set then likely # thei multierror was raised due to that - if ctx._error is not None: - raise ctx._error from None + if ctx._remote_error is not None: + raise ctx._remote_error + # maybe TODO: pack in ``trio.Cancelled.__traceback__`` here + # so they can be unwrapped and displayed on the caller + # side? raise finally: @@ -213,7 +235,6 @@ async def _invoke( # associated child isn't in debug any more await _debug.maybe_wait_for_debugger() ctx = actor._contexts.pop((chan.uid, cid)) - if ctx: log.runtime( f'Context entrypoint {func} was terminated:\n{ctx}' @@ -221,32 +242,60 @@ async def _invoke( assert cs if cs.cancelled_caught: - # if 'brokerd.kraken' in actor.uid: - # await _debug.breakpoint() - # TODO: pack in ``trio.Cancelled.__traceback__`` here - # so they can be unwrapped and displayed on the caller - # side! + # first check for and raise any remote error + # before raising any context cancelled case + # so that real remote errors don't get masked as + # ``ContextCancelled``s. + re = ctx._remote_error + if re: + ctx._maybe_raise_remote_err(re) fname = func.__name__ - if ctx._cancel_called: - msg = f'`{fname}()`@{actor.uid} cancelled itself' + if cs.cancel_called: + canceller = ctx._cancel_called_remote + # await _debug.breakpoint() - elif cs.cancel_called: - msg = ( - f'`{fname}()`@{actor.uid} was remotely cancelled by its caller ' - f'{ctx.chan.uid}' + # NOTE / TODO: if we end up having + # ``Actor._cancel_task()`` call + # ``Context.cancel()`` directly, we're going to + # need to change this logic branch since it will + # always enter.. + if ctx._cancel_called: + msg = f'`{fname}()`@{actor.uid} cancelled itself' + + else: + msg = ( + f'`{fname}()`@{actor.uid} ' + 'was remotely cancelled by ' + ) + + # if the channel which spawned the ctx is the + # one that cancelled it then we report that, vs. + # it being some other random actor that for ex. + # some actor who calls `Portal.cancel_actor()` + # and by side-effect cancels this ctx. + if canceller == ctx.chan.uid: + msg += f'its caller {canceller}' + else: + msg += f'remote actor {canceller}' + + # TODO: does this ever get set any more or can + # we remove it? + if ctx._cancel_msg: + msg += f' with msg:\n{ctx._cancel_msg}' + + # task-contex was either cancelled by request using + # ``Portal.cancel_actor()`` or ``Context.cancel()`` + # on the far end, or it was cancelled by the local + # (callee) task, so relay this cancel signal to the + # other side. + raise ContextCancelled( + msg, + suberror_type=trio.Cancelled, + canceller=canceller, ) - if ctx._cancel_msg: - msg += f' with msg:\n{ctx._cancel_msg}' - - # task-contex was cancelled so relay to the cancel to caller - raise ContextCancelled( - msg, - suberror_type=trio.Cancelled, - ) - else: # regular async function try: @@ -261,12 +310,17 @@ async def _invoke( ) with cancel_scope as cs: - task_status.started(cs) + ctx._scope = cs + task_status.started(ctx) result = await coro - log.cancel(f'result: {result}') + fname = func.__name__ + log.runtime(f'{fname}() result: {result}') if not failed_resp: # only send result if we know IPC isn't down - await chan.send({'return': result, 'cid': cid}) + await chan.send( + {'return': result, + 'cid': cid} + ) except ( Exception, @@ -309,6 +363,7 @@ async def _invoke( # always ship errors back to caller err_msg = pack_error(err, tb=tb) err_msg['cid'] = cid + try: await chan.send(err_msg) @@ -325,14 +380,21 @@ async def _invoke( f"Failed to ship error to caller @ {chan.uid} !?" ) + # error is probably from above coro running code *not from the + # underlyingn rpc invocation* since a scope was never allocated + # around actual coroutine await. if cs is None: - # error is from above code not from rpc invocation + # we don't ever raise directly here to allow the + # msg-loop-scheduler to continue running for this + # channel. task_status.started(err) finally: # RPC task bookeeping try: - scope, func, is_complete = actor._rpc_tasks.pop((chan, cid)) + ctx, func, is_complete = actor._rpc_tasks.pop( + (chan, cid) + ) is_complete.set() except KeyError: @@ -341,6 +403,9 @@ async def _invoke( # cancel scope will not have been inserted yet log.warning( f"Task {func} likely errored or cancelled before start") + else: + log.cancel(f'{func.__name__}({kwargs}) failed?') + finally: if not actor._rpc_tasks: log.runtime("All RPC tasks have completed") @@ -439,6 +504,7 @@ class Actor: self.uid = (name, uid or str(uuid.uuid4())) self._cancel_complete = trio.Event() + self._cancel_called_remote: tuple[str, tuple] | None = None self._cancel_called: bool = False # retreive and store parent `__main__` data which @@ -477,7 +543,7 @@ class Actor: # (chan, cid) -> (cancel_scope, func) self._rpc_tasks: dict[ tuple[Channel, str], - tuple[trio.CancelScope, Callable, trio.Event] + tuple[Context, Callable, trio.Event] ] = {} # map {actor uids -> Context} @@ -652,8 +718,8 @@ class Actor: if ( local_nursery ): - - log.cancel(f"Waiting on cancel request to peer {chan.uid}") + if chan._cancel_called: + log.cancel(f"Waiting on cancel request to peer {chan.uid}") # XXX: this is a soft wait on the channel (and its # underlying transport protocol) to close from the # remote peer side since we presume that any channel @@ -786,76 +852,15 @@ class Actor: f'\n{msg}') return - send_chan = ctx._send_chan - - log.runtime(f"Delivering {msg} from {chan.uid} to caller {cid}") - - # XXX: we do **not** maintain backpressure and instead - # opt to relay stream overrun errors to the sender. - try: - send_chan.send_nowait(msg) - # if an error is deteced we should always - # expect it to be raised by any context (stream) - # consumer task - await ctx._maybe_raise_from_remote_msg(msg) - - except trio.BrokenResourceError: - # TODO: what is the right way to handle the case where the - # local task has already sent a 'stop' / StopAsyncInteration - # to the other side but and possibly has closed the local - # feeder mem chan? Do we wait for some kind of ack or just - # let this fail silently and bubble up (currently)? - - # XXX: local consumer has closed their side - # so cancel the far end streaming task - log.warning(f"{send_chan} consumer is already closed") - return - - except trio.WouldBlock: - # XXX: always push an error even if the local - # receiver is in overrun state. - await ctx._maybe_raise_from_remote_msg(msg) - - uid = chan.uid - lines = [ - 'Task context stream was overrun', - f'local task: {cid} @ {self.uid}', - f'remote sender: {uid}', - ] - if not ctx._stream_opened: - lines.insert( - 1, - f'\n*** No stream open on `{self.uid[0]}` side! ***\n' - ) - - text = '\n'.join(lines) - - if ctx._backpressure: - log.warning(text) - try: - await send_chan.send(msg) - except trio.BrokenResourceError: - # XXX: local consumer has closed their side - # so cancel the far end streaming task - log.warning(f"{chan} is already closed") - else: - try: - raise StreamOverrun(text) from None - except StreamOverrun as err: - err_msg = pack_error(err) - err_msg['cid'] = cid - try: - await chan.send(err_msg) - except trio.BrokenResourceError: - # XXX: local consumer has closed their side - # so cancel the far end streaming task - log.warning(f"{chan} is already closed") + return await ctx._deliver_msg(msg) def get_context( self, chan: Channel, cid: str, - msg_buffer_size: Optional[int] = None, + + msg_buffer_size: int | None = None, + allow_overruns: bool = False, ) -> Context: ''' @@ -871,6 +876,7 @@ class Actor: assert actor_uid try: ctx = self._contexts[(actor_uid, cid)] + ctx._allow_overruns = allow_overruns # adjust buffer size if specified state = ctx._send_chan._state # type: ignore @@ -878,15 +884,11 @@ class Actor: state.max_buffer_size = msg_buffer_size except KeyError: - send_chan: trio.MemorySendChannel - recv_chan: trio.MemoryReceiveChannel - send_chan, recv_chan = trio.open_memory_channel( - msg_buffer_size or self.msg_buffer_size) - ctx = Context( + ctx = mk_context( chan, cid, - _send_chan=send_chan, - _recv_chan=recv_chan, + msg_buffer_size=msg_buffer_size or self.msg_buffer_size, + _allow_overruns=allow_overruns, ) self._contexts[(actor_uid, cid)] = ctx @@ -898,7 +900,8 @@ class Actor: ns: str, func: str, kwargs: dict, - msg_buffer_size: Optional[int] = None, + msg_buffer_size: int | None = None, + allow_overruns: bool = False, ) -> Context: ''' @@ -916,6 +919,7 @@ class Actor: chan, cid, msg_buffer_size=msg_buffer_size, + allow_overruns=allow_overruns, ) log.runtime(f"Sending cmd to {chan.uid}: {ns}.{func}({kwargs})") await chan.send( @@ -1046,7 +1050,11 @@ class Actor: assert self._service_n self._service_n.start_soon(self.cancel) - async def cancel(self) -> bool: + async def cancel( + self, + requesting_uid: tuple[str, str], + + ) -> bool: ''' Cancel this actor's runtime. @@ -1060,6 +1068,7 @@ class Actor: ''' log.cancel(f"{self.uid} is trying to cancel") + self._cancel_called_remote: tuple = requesting_uid self._cancel_called = True # cancel all ongoing rpc tasks @@ -1073,7 +1082,7 @@ class Actor: dbcs.cancel() # kill all ongoing tasks - await self.cancel_rpc_tasks() + await self.cancel_rpc_tasks(requesting_uid=requesting_uid) # stop channel server self.cancel_server() @@ -1099,7 +1108,13 @@ class Actor: # for n in root.child_nurseries: # n.cancel_scope.cancel() - async def _cancel_task(self, cid, chan): + async def _cancel_task( + self, + cid: str, + chan: Channel, + + requesting_uid: tuple[str, str] | None = None, + ) -> bool: ''' Cancel a local task by call-id / channel. @@ -1114,35 +1129,51 @@ class Actor: try: # this ctx based lookup ensures the requested task to # be cancelled was indeed spawned by a request from this channel - scope, func, is_complete = self._rpc_tasks[(chan, cid)] + ctx, func, is_complete = self._rpc_tasks[(chan, cid)] + scope = ctx._scope except KeyError: log.cancel(f"{cid} has already completed/terminated?") - return + return True log.cancel( f"Cancelling task:\ncid: {cid}\nfunc: {func}\n" f"peer: {chan.uid}\n") + if ( + ctx._cancel_called_remote is None + and requesting_uid + ): + ctx._cancel_called_remote: tuple = requesting_uid + # don't allow cancelling this function mid-execution # (is this necessary?) if func is self._cancel_task: - return + return True + # TODO: shouldn't we eventually be calling ``Context.cancel()`` + # directly here instead (since that method can handle both + # side's calls into it? scope.cancel() # wait for _invoke to mark the task complete log.runtime( - f"Waiting on task to cancel:\ncid: {cid}\nfunc: {func}\n" - f"peer: {chan.uid}\n") + 'Waiting on task to cancel:\n' + f'cid: {cid}\nfunc: {func}\n' + f'peer: {chan.uid}\n' + ) await is_complete.wait() log.runtime( f"Sucessfully cancelled task:\ncid: {cid}\nfunc: {func}\n" f"peer: {chan.uid}\n") + return True + async def cancel_rpc_tasks( self, only_chan: Channel | None = None, + requesting_uid: tuple[str, str] | None = None, + ) -> None: ''' Cancel all existing RPC responder tasks using the cancel scope @@ -1154,7 +1185,7 @@ class Actor: log.cancel(f"Cancelling all {len(tasks)} rpc tasks:\n{tasks} ") for ( (chan, cid), - (scope, func, is_complete), + (ctx, func, is_complete), ) in tasks.copy().items(): if only_chan is not None: if only_chan != chan: @@ -1162,7 +1193,11 @@ class Actor: # TODO: this should really done in a nursery batch if func != self._cancel_task: - await self._cancel_task(cid, chan) + await self._cancel_task( + cid, + chan, + requesting_uid=requesting_uid, + ) log.cancel( f"Waiting for remaining rpc tasks to complete {tasks}") @@ -1248,8 +1283,8 @@ async def async_main( Actor runtime entrypoint; start the IPC channel server, maybe connect back to the parent, and startup all core machinery tasks. - A "root-most" (or "top-level") nursery for this actor is opened here - and when cancelled effectively cancels the actor. + A "root" (or "top-level") nursery for this actor is opened here and + when cancelled/terminated effectively closes the actor's "runtime". ''' # attempt to retreive ``trio``'s sigint handler and stash it @@ -1446,15 +1481,16 @@ async def process_messages( ) -> bool: ''' - Process messages for the IPC transport channel async-RPC style. + This is the per-channel, low level RPC task scheduler loop. - Receive multiplexed RPC requests, spawn handler tasks and deliver - responses over or boxed errors back to the "caller" task. + Receive multiplexed RPC request messages from some remote process, + spawn handler tasks depending on request type and deliver responses + or boxed errors back to the remote caller (task). ''' # TODO: once https://github.com/python-trio/trio/issues/467 gets # worked out we'll likely want to use that! - msg = None + msg: dict | None = None nursery_cancelled_before_task: bool = False log.runtime(f"Entering msg loop for {chan} from {chan.uid}") @@ -1476,7 +1512,10 @@ async def process_messages( for (channel, cid) in actor._rpc_tasks.copy(): if channel is chan: - await actor._cancel_task(cid, channel) + await actor._cancel_task( + cid, + channel, + ) log.runtime( f"Msg loop signalled to terminate for" @@ -1490,12 +1529,14 @@ async def process_messages( cid = msg.get('cid') if cid: # deliver response to local caller/waiter + # via its per-remote-context memory channel. await actor._push_result(chan, cid, msg) log.runtime( f"Waiting on next msg for {chan} from {chan.uid}") continue + # TODO: implement with ``match:`` syntax? # process command request try: ns, funcname, kwargs, actorid, cid = msg['cmd'] @@ -1515,13 +1556,12 @@ async def process_messages( f"{ns}.{funcname}({kwargs})") if ns == 'self': - func = getattr(actor, funcname) - if funcname == 'cancel': + func = actor.cancel + kwargs['requesting_uid'] = chan.uid - # don't start entire actor runtime - # cancellation if this actor is in debug - # mode + # don't start entire actor runtime cancellation + # if this actor is currently in debug mode! pdb_complete = _debug.Lock.local_pdb_complete if pdb_complete: await pdb_complete.wait() @@ -1533,43 +1573,56 @@ async def process_messages( # msg loop and break out into # ``async_main()`` log.cancel( - f"Actor {actor.uid} was remotely cancelled " + "Actor runtime for was remotely cancelled " f"by {chan.uid}" ) await _invoke( - actor, cid, chan, func, kwargs, is_rpc=False + actor, + cid, + chan, + func, + kwargs, + is_rpc=False, ) + log.cancel( + f'Cancelling msg loop for {chan.uid}' + ) loop_cs.cancel() break if funcname == '_cancel_task': + func = actor._cancel_task # we immediately start the runtime machinery # shutdown - with trio.CancelScope(shield=True): - # actor.cancel() was called so kill this - # msg loop and break out into - # ``async_main()`` - kwargs['chan'] = chan - log.cancel( - f'Remote request to cancel task\n' - f'remote actor: {chan.uid}\n' - f'task: {cid}' + # with trio.CancelScope(shield=True): + kwargs['chan'] = chan + target_cid = kwargs['cid'] + kwargs['requesting_uid'] = chan.uid + log.cancel( + f'Remote request to cancel task\n' + f'remote actor: {chan.uid}\n' + f'task: {target_cid}' + ) + try: + await _invoke( + actor, + cid, + chan, + func, + kwargs, + is_rpc=False, ) - try: - await _invoke( - actor, - cid, - chan, - func, - kwargs, - is_rpc=False, - ) - except BaseException: - log.exception("failed to cancel task?") + except BaseException: + log.exception("failed to cancel task?") + + continue + else: + # normally registry methods, eg. + # ``.register_actor()`` etc. + func = getattr(actor, funcname) - continue else: # complain to client about restricted modules try: @@ -1584,34 +1637,49 @@ async def process_messages( log.runtime(f"Spawning task for {func}") assert actor._service_n try: - cs = await actor._service_n.start( - partial(_invoke, actor, cid, chan, func, kwargs), + ctx: Context = await actor._service_n.start( + partial( + _invoke, + actor, + cid, + chan, + func, + kwargs, + ), name=funcname, ) + except ( RuntimeError, BaseExceptionGroup, ): # avoid reporting a benign race condition # during actor runtime teardown. - nursery_cancelled_before_task = True + nursery_cancelled_before_task: bool = True break - # never allow cancelling cancel requests (results in - # deadlock and other weird behaviour) - # if func != actor.cancel: - if isinstance(cs, Exception): + # in the lone case where a ``Context`` is not + # delivered, it's likely going to be a locally + # scoped exception from ``_invoke()`` itself. + if isinstance(ctx, Exception): log.warning( f"Task for RPC func {func} failed with" - f"{cs}") + f"{ctx}" + ) + continue + else: # mark that we have ongoing rpc tasks actor._ongoing_rpc_tasks = trio.Event() log.runtime(f"RPC func is {func}") + # store cancel scope such that the rpc task can be # cancelled gracefully if requested actor._rpc_tasks[(chan, cid)] = ( - cs, func, trio.Event()) + ctx, + func, + trio.Event(), + ) log.runtime( f"Waiting on next msg for {chan} from {chan.uid}") @@ -1655,7 +1723,7 @@ async def process_messages( match err: case ContextCancelled(): log.cancel( - f'Actor: {actor.uid} was task-context-cancelled with,\n' + f'Actor: {actor.uid} was context-cancelled with,\n' f'str(err)' ) case _: @@ -1672,7 +1740,8 @@ async def process_messages( # msg debugging for when he machinery is brokey log.runtime( f"Exiting msg loop for {chan} from {chan.uid} " - f"with last msg:\n{msg}") + f"with last msg:\n{msg}" + ) # transport **was not** disconnected return False diff --git a/tractor/_streaming.py b/tractor/_streaming.py index 9908ab65..450c712e 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -21,24 +21,41 @@ Message stream types and APIs. from __future__ import annotations import inspect from contextlib import asynccontextmanager -from dataclasses import dataclass +from collections import deque +from dataclasses import ( + dataclass, + field, +) +from functools import partial +from pprint import pformat from typing import ( Any, Optional, Callable, AsyncGenerator, - AsyncIterator + AsyncIterator, + TYPE_CHECKING, ) - import warnings import trio from ._ipc import Channel -from ._exceptions import unpack_error, ContextCancelled -from ._state import current_actor +from ._exceptions import ( + unpack_error, + pack_error, + ContextCancelled, + StreamOverrun, +) from .log import get_logger -from .trionics import broadcast_receiver, BroadcastReceiver +from ._state import current_actor +from .trionics import ( + broadcast_receiver, + BroadcastReceiver, +) + +if TYPE_CHECKING: + from ._portal import Portal log = get_logger(__name__) @@ -335,8 +352,8 @@ class MsgStream(trio.abc.Channel): Send a message over this stream to the far end. ''' - if self._ctx._error: - raise self._ctx._error # from None + if self._ctx._remote_error: + raise self._ctx._remote_error # from None if self._closed: raise trio.ClosedResourceError('This stream was already closed') @@ -373,25 +390,61 @@ class Context: _recv_chan: trio.MemoryReceiveChannel _send_chan: trio.MemorySendChannel - _remote_func_type: Optional[str] = None + _remote_func_type: str | None = None # only set on the caller side - _portal: Optional['Portal'] = None # type: ignore # noqa - _result: Optional[Any] = False - _error: Optional[BaseException] = None + _portal: Portal | None = None # type: ignore # noqa + _result: Any | int = None + _remote_error: BaseException | None = None - # status flags + # cancellation state _cancel_called: bool = False - _cancel_msg: Optional[str] = None + _cancel_called_remote: tuple | None = None + _cancel_msg: str | None = None + _scope: trio.CancelScope | None = None _enter_debugger_on_cancel: bool = True + + @property + def cancel_called(self) -> bool: + ''' + Records whether cancellation has been requested for this context + by either an explicit call to ``.cancel()`` or an implicit call + due to an error caught inside the ``Portal.open_context()`` + block. + + ''' + return self._cancel_called + + @property + def cancel_called_remote(self) -> tuple[str, str] | None: + ''' + ``Actor.uid`` of the remote actor who's task was cancelled + causing this side of the context to also be cancelled. + + ''' + remote_uid = self._cancel_called_remote + if remote_uid: + return tuple(remote_uid) + + # init and streaming state _started_called: bool = False _started_received: bool = False _stream_opened: bool = False - # only set on the callee side - _scope_nursery: Optional[trio.Nursery] = None - - _backpressure: bool = True + # overrun handling machinery + # NOTE: none of this provides "backpressure" to the remote + # task, only an ability to not lose messages when the local + # task is configured to NOT transmit ``StreamOverrun``s back + # to the other side. + _overflow_q: deque[dict] = field( + default_factory=partial( + deque, + maxlen=616, + ) + ) + _scope_nursery: trio.Nursery | None = None + _in_overrun: bool = False + _allow_overruns: bool = False async def send_yield( self, @@ -410,9 +463,9 @@ class Context: async def send_stop(self) -> None: await self.chan.send({'stop': True, 'cid': self.cid}) - async def _maybe_raise_from_remote_msg( + async def _maybe_cancel_and_set_remote_error( self, - msg: dict[str, Any], + error_msg: dict[str, Any], ) -> None: ''' @@ -423,55 +476,77 @@ class Context: in the corresponding remote callee task. ''' - error = msg.get('error') - if error: - # If this is an error message from a context opened by - # ``Portal.open_context()`` we want to interrupt any ongoing - # (child) tasks within that context to be notified of the remote - # error relayed here. - # - # The reason we may want to raise the remote error immediately - # is that there is no guarantee the associated local task(s) - # will attempt to read from any locally opened stream any time - # soon. - # - # NOTE: this only applies when - # ``Portal.open_context()`` has been called since it is assumed - # (currently) that other portal APIs (``Portal.run()``, - # ``.run_in_actor()``) do their own error checking at the point - # of the call and result processing. - log.error( - f'Remote context error for {self.chan.uid}:{self.cid}:\n' - f'{msg["error"]["tb_str"]}' + # If this is an error message from a context opened by + # ``Portal.open_context()`` we want to interrupt any ongoing + # (child) tasks within that context to be notified of the remote + # error relayed here. + # + # The reason we may want to raise the remote error immediately + # is that there is no guarantee the associated local task(s) + # will attempt to read from any locally opened stream any time + # soon. + # + # NOTE: this only applies when + # ``Portal.open_context()`` has been called since it is assumed + # (currently) that other portal APIs (``Portal.run()``, + # ``.run_in_actor()``) do their own error checking at the point + # of the call and result processing. + error = unpack_error( + error_msg, + self.chan, + ) + + # XXX: set the remote side's error so that after we cancel + # whatever task is the opener of this context it can raise + # that error as the reason. + self._remote_error = error + + if ( + isinstance(error, ContextCancelled) + ): + log.cancel( + 'Remote task-context sucessfully cancelled for ' + f'{self.chan.uid}:{self.cid}' ) - error = unpack_error(msg, self.chan) - if ( - isinstance(error, ContextCancelled) and - self._cancel_called - ): + + if self._cancel_called: # this is an expected cancel request response message # and we don't need to raise it in scope since it will # potentially override a real error return + else: + log.error( + f'Remote context error for {self.chan.uid}:{self.cid}:\n' + f'{error_msg["error"]["tb_str"]}' + ) + # TODO: tempted to **not** do this by-reraising in a + # nursery and instead cancel a surrounding scope, detect + # the cancellation, then lookup the error that was set? + # YES! this is way better and simpler! + if ( + self._scope + ): + # from trio.testing import wait_all_tasks_blocked + # await wait_all_tasks_blocked() + self._cancel_called_remote = self.chan.uid + self._scope.cancel() - self._error = error + # NOTE: this usage actually works here B) + # from ._debug import breakpoint + # await breakpoint() - # TODO: tempted to **not** do this by-reraising in a - # nursery and instead cancel a surrounding scope, detect - # the cancellation, then lookup the error that was set? - if self._scope_nursery: - async def raiser(): - raise self._error from None - - # from trio.testing import wait_all_tasks_blocked - # await wait_all_tasks_blocked() - if not self._scope_nursery._closed: # type: ignore - self._scope_nursery.start_soon(raiser) + # XXX: this will break early callee results sending + # since when `.result()` is finally called, this + # chan will be closed.. + # if self._recv_chan: + # await self._recv_chan.aclose() async def cancel( self, msg: str | None = None, + timeout: float = 0.5, + # timeout: float = 1000, ) -> None: ''' @@ -488,6 +563,8 @@ class Context: log.cancel(f'Cancelling {side} side of context to {self.chan.uid}') self._cancel_called = True + # await _debug.breakpoint() + # breakpoint() if side == 'caller': if not self._portal: @@ -496,8 +573,8 @@ class Context: ) cid = self.cid - with trio.move_on_after(0.5) as cs: - cs.shield = True + with trio.move_on_after(timeout) as cs: + # cs.shield = True log.cancel( f"Cancelling stream {cid} to " f"{self._portal.channel.uid}") @@ -505,7 +582,12 @@ class Context: # NOTE: we're telling the far end actor to cancel a task # corresponding to *this actor*. The far end local channel # instance is passed to `Actor._cancel_task()` implicitly. - await self._portal.run_from_ns('self', '_cancel_task', cid=cid) + await self._portal.run_from_ns( + 'self', + '_cancel_task', + cid=cid, + ) + # print("EXITING CANCEL CALL") if cs.cancelled_caught: # XXX: there's no way to know if the remote task was indeed @@ -530,17 +612,14 @@ class Context: # {'error': trio.Cancelled, cid: "blah"} enough? # This probably gets into the discussion in # https://github.com/goodboy/tractor/issues/36 - assert self._scope_nursery - self._scope_nursery.cancel_scope.cancel() - - if self._recv_chan: - await self._recv_chan.aclose() + assert self._scope + self._scope.cancel() @asynccontextmanager async def open_stream( self, - backpressure: bool | None = True, + allow_overruns: bool | None = False, msg_buffer_size: int | None = None, ) -> AsyncGenerator[MsgStream, None]: @@ -592,8 +671,9 @@ class Context: self.chan, self.cid, msg_buffer_size=msg_buffer_size, + allow_overruns=allow_overruns, ) - ctx._backpressure = backpressure + ctx._allow_overruns = allow_overruns assert ctx is self # XXX: If the underlying channel feeder receive mem chan has @@ -637,48 +717,115 @@ class Context: f'ctx id: {self.cid}' ) - async def result(self) -> Any: + def _maybe_raise_remote_err( + self, + err: Exception, + ) -> None: + # NOTE: whenever the context's "opener" side (task) **is** + # the side which requested the cancellation (likekly via + # ``Context.cancel()``), we don't want to re-raise that + # cancellation signal locally (would be akin to + # a ``trio.Nursery`` nursery raising ``trio.Cancelled`` + # whenever ``CancelScope.cancel()`` was called) and instead + # silently reap the expected cancellation "error"-msg. + # if 'pikerd' in err.msgdata['tb_str']: + # # from . import _debug + # # await _debug.breakpoint() + # breakpoint() + + if ( + isinstance(err, ContextCancelled) + and ( + self._cancel_called + or self.chan._cancel_called + or tuple(err.canceller) == current_actor().uid + ) + ): + return err + + raise err from None + + async def result(self) -> Any | Exception: ''' - From a caller side, wait for and return the final result from - the callee side task. + From some (caller) side task, wait for and return the final + result from the remote (callee) side's task. + + This provides a mechanism for one task running in some actor to wait + on another task at the other side, in some other actor, to terminate. + + If the remote task is still in a streaming state (it is delivering + values from inside a ``Context.open_stream():`` block, then those + msgs are drained but discarded since it is presumed this side of + the context has already finished with its own streaming logic. + + If the remote context (or its containing actor runtime) was + canceled, either by a local task calling one of + ``Context.cancel()`` or `Portal.cancel_actor()``, we ignore the + received ``ContextCancelled`` exception if the context or + underlying IPC channel is marked as having been "cancel called". + This is similar behavior to using ``trio.Nursery.cancel()`` + wherein tasks which raise ``trio.Cancel`` are silently reaped; + the main different in this API is in the "cancel called" case, + instead of just not raising, we also return the exception *as + the result* since client code may be interested in the details + of the remote cancellation. ''' assert self._portal, "Context.result() can not be called from callee!" assert self._recv_chan - if self._result is False: + # from . import _debug + # await _debug.breakpoint() - if not self._recv_chan._closed: # type: ignore + re = self._remote_error + if re: + self._maybe_raise_remote_err(re) + return re - # wait for a final context result consuming - # and discarding any bi dir stream msgs still - # in transit from the far end. - while True: + if ( + self._result == id(self) + and not self._remote_error + and not self._recv_chan._closed # type: ignore + ): + # wait for a final context result consuming + # and discarding any bi dir stream msgs still + # in transit from the far end. + while True: + msg = await self._recv_chan.receive() + try: + self._result = msg['return'] - msg = await self._recv_chan.receive() - try: - self._result = msg['return'] - break - except KeyError as msgerr: + # NOTE: we don't need to do this right? + # XXX: only close the rx mem chan AFTER + # a final result is retreived. + # if self._recv_chan: + # await self._recv_chan.aclose() - if 'yield' in msg: - # far end task is still streaming to us so discard - log.warning(f'Discarding stream delivered {msg}') - continue + break + except KeyError: # as msgerr: - elif 'stop' in msg: - log.debug('Remote stream terminated') - continue + if 'yield' in msg: + # far end task is still streaming to us so discard + log.warning(f'Discarding stream delivered {msg}') + continue - # internal error should never get here - assert msg.get('cid'), ( - "Received internal error at portal?") + elif 'stop' in msg: + log.debug('Remote stream terminated') + continue - raise unpack_error( - msg, self._portal.channel - ) from msgerr + # internal error should never get here + assert msg.get('cid'), ( + "Received internal error at portal?") - return self._result + err = unpack_error( + msg, + self._portal.channel + ) # from msgerr + + err = self._maybe_raise_remote_err(err) + self._remote_err = err + + return self._remote_error or self._result async def started( self, @@ -708,6 +855,187 @@ class Context: # async def restart(self) -> None: # pass + async def _drain_overflows( + self, + ) -> None: + ''' + Private task spawned to push newly received msgs to the local + task which getting overrun by the remote side. + + In order to not block the rpc msg loop, but also not discard + msgs received in this context, we need to async push msgs in + a new task which only runs for as long as the local task is in + an overrun state. + + ''' + self._in_overrun = True + try: + while self._overflow_q: + # NOTE: these msgs should never be errors since we always do + # the check prior to checking if we're in an overrun state + # inside ``.deliver_msg()``. + msg = self._overflow_q.popleft() + try: + await self._send_chan.send(msg) + except trio.BrokenResourceError: + log.warning( + f"{self._send_chan} consumer is already closed" + ) + return + except trio.Cancelled: + # we are obviously still in overrun + # but the context is being closed anyway + # so we just warn that there are un received + # msgs still.. + self._overflow_q.appendleft(msg) + fmt_msgs = '' + for msg in self._overflow_q: + fmt_msgs += f'{pformat(msg)}\n' + + log.warning( + f'Context for {self.cid} is being closed while ' + 'in an overrun state!\n' + 'Discarding the following msgs:\n' + f'{fmt_msgs}\n' + ) + raise + + finally: + # task is now finished with the backlog so mark us as + # no longer in backlog. + self._in_overrun = False + + async def _deliver_msg( + self, + msg: dict, + + draining: bool = False, + + ) -> bool: + + cid = self.cid + chan = self.chan + uid = chan.uid + send_chan: trio.MemorySendChannel = self._send_chan + + log.runtime( + f"Delivering {msg} from {uid} to caller {cid}" + ) + + error = msg.get('error') + if error: + await self._maybe_cancel_and_set_remote_error(msg) + + if ( + self._in_overrun + ): + self._overflow_q.append(msg) + return False + + try: + send_chan.send_nowait(msg) + return True + # if an error is deteced we should always + # expect it to be raised by any context (stream) + # consumer task + + except trio.BrokenResourceError: + # TODO: what is the right way to handle the case where the + # local task has already sent a 'stop' / StopAsyncInteration + # to the other side but and possibly has closed the local + # feeder mem chan? Do we wait for some kind of ack or just + # let this fail silently and bubble up (currently)? + + # XXX: local consumer has closed their side + # so cancel the far end streaming task + log.warning(f"{send_chan} consumer is already closed") + return False + + # NOTE XXX: by default we do **not** maintain context-stream + # backpressure and instead opt to relay stream overrun errors to + # the sender; the main motivation is that using bp can block the + # msg handling loop which calls into this method! + except trio.WouldBlock: + # XXX: always push an error even if the local + # receiver is in overrun state. + # await self._maybe_cancel_and_set_remote_error(msg) + + local_uid = current_actor().uid + lines = [ + f'Actor-task context {cid}@{local_uid} was overrun by remote!', + f'sender actor: {uid}', + ] + if not self._stream_opened: + lines.insert( + 1, + f'\n*** No stream open on `{local_uid[0]}` side! ***\n' + ) + + text = '\n'.join(lines) + + # XXX: lul, this really can't be backpressure since any + # blocking here will block the entire msg loop rpc sched for + # a whole channel.. maybe we should rename it? + if self._allow_overruns: + text += f'\nStarting overflow queuing task on msg: {msg}' + log.warning(text) + if ( + not self._in_overrun + ): + self._overflow_q.append(msg) + n = self._scope_nursery + if n.child_tasks: + from . import _debug + await _debug.breakpoint() + assert not n.child_tasks + n.start_soon( + self._drain_overflows, + ) + else: + try: + raise StreamOverrun(text) + except StreamOverrun as err: + err_msg = pack_error(err) + err_msg['cid'] = cid + try: + await chan.send(err_msg) + except trio.BrokenResourceError: + # XXX: local consumer has closed their side + # so cancel the far end streaming task + log.warning(f"{chan} is already closed") + + return False + + +def mk_context( + chan: Channel, + cid: str, + msg_buffer_size: int = 2**6, + + **kwargs, + +) -> Context: + ''' + Internal factory to create an inter-actor task ``Context``. + + This is called by internals and should generally never be called + by user code. + + ''' + send_chan: trio.MemorySendChannel + recv_chan: trio.MemoryReceiveChannel + send_chan, recv_chan = trio.open_memory_channel(msg_buffer_size) + + ctx = Context( + chan, + cid, + _send_chan=send_chan, + _recv_chan=recv_chan, + **kwargs, + ) + ctx._result = id(ctx) + return ctx + def stream(func: Callable) -> Callable: ''' From 6994d2026d2e3dd30f50dea536732938188fb61e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 13 Apr 2023 17:24:27 -0400 Subject: [PATCH 014/146] Drop brackpressure usage from fan out tests --- tests/test_task_broadcasting.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/test_task_broadcasting.py b/tests/test_task_broadcasting.py index 9f4a1fed..5e18e10a 100644 --- a/tests/test_task_broadcasting.py +++ b/tests/test_task_broadcasting.py @@ -86,7 +86,7 @@ async def open_sequence_streamer( ) as (ctx, first): assert first is None - async with ctx.open_stream(backpressure=True) as stream: + async with ctx.open_stream(allow_overruns=True) as stream: yield stream await portal.cancel_actor() @@ -413,8 +413,8 @@ def test_ensure_slow_consumers_lag_out( seq = brx._state.subs[brx.key] assert seq == len(brx._state.queue) - 1 - # all backpressured entries in the underlying - # channel should have been copied into the caster + # all no_overruns entries in the underlying + # channel should have been copied into the bcaster # queue trailing-window async for i in rx: print(f'bped: {i}') From 6db656fecf960a2ed8bda09144f20c7a269949bf Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 13 Apr 2023 17:47:13 -0400 Subject: [PATCH 015/146] Flip allocate log msgs to debug --- tractor/trionics/_mngrs.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tractor/trionics/_mngrs.py b/tractor/trionics/_mngrs.py index 5621f79d..89db895b 100644 --- a/tractor/trionics/_mngrs.py +++ b/tractor/trionics/_mngrs.py @@ -237,7 +237,7 @@ async def maybe_open_context( yielded = _Cache.values[ctx_key] except KeyError: - log.info(f'Allocating new {acm_func} for {ctx_key}') + log.debug(f'Allocating new {acm_func} for {ctx_key}') mngr = acm_func(**kwargs) resources = _Cache.resources assert not resources.get(ctx_key), f'Resource exists? {ctx_key}' @@ -265,7 +265,7 @@ async def maybe_open_context( if yielded is not None: # if no more consumers, teardown the client if _Cache.users <= 0: - log.info(f'De-allocating resource for {ctx_key}') + log.debug(f'De-allocating resource for {ctx_key}') # XXX: if we're cancelled we the entry may have never # been entered since the nursery task was killed. From f1e9c0be93bbed36880769e0ab00f08029cf8489 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 13 Apr 2023 17:48:57 -0400 Subject: [PATCH 016/146] Fix cluster test to use `allow_overruns` --- tests/test_clustering.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/test_clustering.py b/tests/test_clustering.py index df3d8357..02b1f8fa 100644 --- a/tests/test_clustering.py +++ b/tests/test_clustering.py @@ -49,7 +49,7 @@ async def worker( await ctx.started() async with ctx.open_stream( - backpressure=True, + allow_overruns=True, ) as stream: # TODO: this with the below assert causes a hang bug? From 63adf73b4b5ba414369c30528828ecfa81c3a849 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 13 Apr 2023 18:06:44 -0400 Subject: [PATCH 017/146] Adjust aio test for silent cancellation by parent --- tests/test_infected_asyncio.py | 20 +++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 775ee98d..dd9d681a 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -15,6 +15,7 @@ import tractor from tractor import ( to_asyncio, RemoteActorError, + ContextCancelled, ) from tractor.trionics import BroadcastReceiver @@ -224,14 +225,23 @@ def test_context_spawns_aio_task_that_errors( await trio.sleep_forever() - with pytest.raises(RemoteActorError) as excinfo: - trio.run(main) + return await ctx.result() - err = excinfo.value - assert isinstance(err, RemoteActorError) if parent_cancels: - assert err.type == trio.Cancelled + # bc the parent made the cancel request, + # the error is not raised locally but instead + # the context is exited silently + res = trio.run(main) + assert isinstance(res, ContextCancelled) + assert 'root' in res.canceller[0] + else: + expect = RemoteActorError + with pytest.raises(expect) as excinfo: + trio.run(main) + + err = excinfo.value + assert isinstance(err, expect) assert err.type == AssertionError From f9911c22a40c699245db29adf3006696ac913915 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 13 Apr 2023 19:46:35 -0400 Subject: [PATCH 018/146] Seriously cover all overrun cases This actually caught further runtime bugs so it's gud i tried.. Add overrun-ignore enabled / disabled cases and error catching for all of them. More or less this should cover every possible outcome when it comes to setting `allow_overruns: bool` i hope XD --- tests/test_context_stream_semantics.py | 107 +++++++++++++++++++++---- tractor/_streaming.py | 6 +- 2 files changed, 98 insertions(+), 15 deletions(-) diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index 2531fd86..615ccd7a 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -722,7 +722,9 @@ async def echo_back_sequence( ctx: tractor.Context, seq: list[int], wait_for_cancel: bool, - msg_buffer_size: int | None = None, + allow_overruns_side: str, + be_slow: bool = False, + msg_buffer_size: int = 1, ) -> None: ''' @@ -737,12 +739,22 @@ async def echo_back_sequence( total_batches: int = 1000 if wait_for_cancel else 6 await ctx.started() + # await tractor.breakpoint() async with ctx.open_stream( msg_buffer_size=msg_buffer_size, - allow_overruns=True, + + # literally the point of this test XD + allow_overruns=(allow_overruns_side in {'child', 'both'}), ) as stream: - seq = list(seq) # bleh, `msgpack`... + # ensure mem chan settings are correct + assert ( + ctx._send_chan._state.max_buffer_size + == + msg_buffer_size + ) + + seq = list(seq) # bleh, msgpack sometimes ain't decoded right for _ in range(total_batches): batch = [] async for msg in stream: @@ -750,6 +762,9 @@ async def echo_back_sequence( if batch == seq: break + if be_slow: + await trio.sleep(0.05) + print('callee waiting on next') for msg in batch: @@ -763,13 +778,29 @@ async def echo_back_sequence( return 'yo' +@pytest.mark.parametrize( + # aka the side that will / should raise + # and overrun under normal conditions. + 'allow_overruns_side', + ['parent', 'child', 'none', 'both'], + ids=lambda item: f'allow_overruns_side={item}' +) +@pytest.mark.parametrize( + # aka the side that will / should raise + # and overrun under normal conditions. + 'slow_side', + ['parent', 'child'], + ids=lambda item: f'slow_side={item}' +) @pytest.mark.parametrize( 'cancel_ctx', [True, False], ids=lambda item: f'cancel_ctx={item}' ) -def test_allow_overruns_stream( +def test_maybe_allow_overruns_stream( cancel_ctx: bool, + slow_side: str, + allow_overruns_side: str, loglevel: str, ): ''' @@ -794,26 +825,35 @@ def test_allow_overruns_stream( 'callee_sends_forever', enable_modules=[__name__], loglevel=loglevel, + + # debug_mode=True, ) - seq = list(range(3)) + seq = list(range(10)) async with portal.open_context( echo_back_sequence, seq=seq, wait_for_cancel=cancel_ctx, + be_slow=(slow_side == 'child'), + allow_overruns_side=allow_overruns_side, ) as (ctx, sent): assert sent is None async with ctx.open_stream( - msg_buffer_size=1, - allow_overruns=True, + msg_buffer_size=1 if slow_side == 'parent' else None, + allow_overruns=(allow_overruns_side in {'parent', 'both'}), ) as stream: - count = 0 - while count < 3: + + total_batches: int = 2 + for _ in range(total_batches): for msg in seq: - print(f'root tx {msg}') + # print(f'root tx {msg}') await stream.send(msg) - await trio.sleep(0.1) + if slow_side == 'parent': + # NOTE: we make the parent slightly + # slower, when it is slow, to make sure + # that in the overruns everywhere case + await trio.sleep(0.16) batch = [] async for msg in stream: @@ -822,8 +862,6 @@ def test_allow_overruns_stream( if batch == seq: break - count += 1 - if cancel_ctx: # cancel the remote task print('sending root side cancel') @@ -842,7 +880,48 @@ def test_allow_overruns_stream( # cancel the daemon await portal.cancel_actor() - trio.run(main) + if ( + allow_overruns_side == 'both' + or slow_side == allow_overruns_side + ): + trio.run(main) + + elif ( + slow_side != allow_overruns_side + ): + + with pytest.raises(tractor.RemoteActorError) as excinfo: + trio.run(main) + + err = excinfo.value + + if ( + allow_overruns_side == 'none' + ): + # depends on timing is is racy which side will + # overrun first :sadkitty: + + # NOTE: i tried to isolate to a deterministic case here + # based on timeing, but i was kinda wasted, and i don't + # think it's sane to catch them.. + assert err.type in ( + tractor.RemoteActorError, + StreamOverrun, + ) + + elif ( + slow_side == 'child' + ): + assert err.type == StreamOverrun + + elif slow_side == 'parent': + assert err.type == tractor.RemoteActorError + assert 'StreamOverrun' in err.msgdata['tb_str'] + + else: + # if this hits the logic blocks from above are not + # exhaustive.. + pytest.fail('PARAMETRIZED CASE GEN PROBLEM YO') @tractor.context diff --git a/tractor/_streaming.py b/tractor/_streaming.py index 450c712e..05984436 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -426,6 +426,10 @@ class Context: if remote_uid: return tuple(remote_uid) + @property + def cancelled_caught(self) -> bool: + return self._scope.cancelled_caught + # init and streaming state _started_called: bool = False _started_received: bool = False @@ -743,7 +747,7 @@ class Context: ): return err - raise err from None + raise err # from None async def result(self) -> Any | Exception: ''' From 968f13f9ef6fff342164b0170d072b035a060cb3 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 13 Apr 2023 23:10:39 -0400 Subject: [PATCH 019/146] Set `Context._scope_nursery` on callee side too Because obviously we probably want to support `allow_overruns` on the remote callee side as well XD Only found the bugs fixed in this patch this thanks to writing a much more exhaustive test set for overrun cases B) --- tractor/_runtime.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 9d9427d6..5bc8b1fd 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -205,8 +205,9 @@ async def _invoke( await chan.send({'functype': 'context', 'cid': cid}) try: - with cancel_scope as cs: - ctx._scope = cs + async with trio.open_nursery() as nurse: + ctx._scope_nursery = nurse + ctx._scope = nurse.cancel_scope task_status.started(ctx) res = await coro await chan.send({'return': res, 'cid': cid}) @@ -240,8 +241,7 @@ async def _invoke( f'Context entrypoint {func} was terminated:\n{ctx}' ) - assert cs - if cs.cancelled_caught: + if ctx.cancelled_caught: # first check for and raise any remote error # before raising any context cancelled case @@ -252,6 +252,7 @@ async def _invoke( ctx._maybe_raise_remote_err(re) fname = func.__name__ + cs: trio.CancelScope = ctx._scope if cs.cancel_called: canceller = ctx._cancel_called_remote # await _debug.breakpoint() @@ -383,7 +384,7 @@ async def _invoke( # error is probably from above coro running code *not from the # underlyingn rpc invocation* since a scope was never allocated # around actual coroutine await. - if cs is None: + if ctx._scope is None: # we don't ever raise directly here to allow the # msg-loop-scheduler to continue running for this # channel. From 04e4397a8f5ab078935b7964e078ad389815e977 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 13 Apr 2023 23:45:08 -0400 Subject: [PATCH 020/146] Ignore drainer-task nursery RTE during context exit --- tractor/_streaming.py | 25 ++++++++++++++++--------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/tractor/_streaming.py b/tractor/_streaming.py index 05984436..4bf6d1c7 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -539,7 +539,6 @@ class Context: # from ._debug import breakpoint # await breakpoint() - # XXX: this will break early callee results sending # since when `.result()` is finally called, this # chan will be closed.. @@ -966,8 +965,12 @@ class Context: local_uid = current_actor().uid lines = [ - f'Actor-task context {cid}@{local_uid} was overrun by remote!', - f'sender actor: {uid}', + f'OVERRUN on actor-task context {cid}@{local_uid}!\n' + # TODO: put remote task name here if possible? + f'remote sender actor: {uid}', + # TODO: put task func name here and maybe an arrow + # from sender to overrunner? + # f'local task {self.func_name}' ] if not self._stream_opened: lines.insert( @@ -988,13 +991,17 @@ class Context: ): self._overflow_q.append(msg) n = self._scope_nursery - if n.child_tasks: - from . import _debug - await _debug.breakpoint() assert not n.child_tasks - n.start_soon( - self._drain_overflows, - ) + try: + n.start_soon( + self._drain_overflows, + ) + except RuntimeError: + # if the nursery is already cancelled due to + # this context exiting or in error, we ignore + # the nursery error since we never expected + # anything different. + return False else: try: raise StreamOverrun(text) From 041d7da721c9e69de3aaedcaa3e8e240b15c2e9d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 14 Apr 2023 15:46:21 -0400 Subject: [PATCH 021/146] Drop caller cancels overrun test; covered in new tests --- tests/test_context_stream_semantics.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index 615ccd7a..4efc6319 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -646,7 +646,6 @@ async def keep_sending_from_callee( 'overrun_by', [ ('caller', 1, never_open_stream), - ('cancel_caller_during_overrun', 1, never_open_stream), ('callee', 0, keep_sending_from_callee), ], ids='overrun_condition={}'.format, @@ -676,14 +675,13 @@ def test_one_end_stream_not_opened(overrun_by): if 'caller' in overrunner: async with ctx.open_stream() as stream: + + # itersend +1 msg more then the buffer size + # to cause the most basic overrun. for i in range(buf_size): print(f'sending {i}') await stream.send(i) - if 'cancel' in overrunner: - # without this we block waiting on the child side - await ctx.cancel() - else: # expect overrun error to be relayed back # and this sleep interrupted @@ -697,7 +695,9 @@ def test_one_end_stream_not_opened(overrun_by): # 2 overrun cases and the no overrun case (which pushes right up to # the msg limit) - if overrunner == 'caller' or 'cancel' in overrunner: + if ( + overrunner == 'caller' + ): with pytest.raises(tractor.RemoteActorError) as excinfo: trio.run(main) From 20d75ff9343af8a08585a4b9dca296fee534c1f2 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 14 Apr 2023 16:23:47 -0400 Subject: [PATCH 022/146] Move move context code into new `._context` mod --- tractor/__init__.py | 8 +- tractor/_context.py | 767 ++++++++++++++++++++++++++++++++ tractor/_portal.py | 6 +- tractor/_runtime.py | 2 +- tractor/_streaming.py | 736 +----------------------------- tractor/experimental/_pubsub.py | 8 +- 6 files changed, 789 insertions(+), 738 deletions(-) create mode 100644 tractor/_context.py diff --git a/tractor/__init__.py b/tractor/__init__.py index 12123a2d..aa262105 100644 --- a/tractor/__init__.py +++ b/tractor/__init__.py @@ -15,18 +15,20 @@ # along with this program. If not, see . """ -tractor: structured concurrent "actors". +tractor: structured concurrent ``trio``-"actors". """ from exceptiongroup import BaseExceptionGroup from ._clustering import open_actor_cluster from ._ipc import Channel -from ._streaming import ( +from ._context import ( Context, + context, +) +from ._streaming import ( MsgStream, stream, - context, ) from ._discovery import ( get_arbiter, diff --git a/tractor/_context.py b/tractor/_context.py new file mode 100644 index 00000000..4e52b218 --- /dev/null +++ b/tractor/_context.py @@ -0,0 +1,767 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +''' +The fundamental cross process SC abstraction: an inter-actor, +cancel-scope linked task "context". + +A ``Context`` is very similar to the ``trio.Nursery.cancel_scope`` built +into each ``trio.Nursery`` except it links the lifetimes of memory space +disjoint, parallel executing tasks in separate actors. + +''' +from __future__ import annotations +from collections import deque +from contextlib import asynccontextmanager as acm +from dataclasses import ( + dataclass, + field, +) +from functools import partial +import inspect +from pprint import pformat +from typing import ( + Any, + Callable, + AsyncGenerator, + TYPE_CHECKING, +) +import warnings + +import trio + +from ._exceptions import ( + unpack_error, + pack_error, + ContextCancelled, + StreamOverrun, +) +from .log import get_logger +from ._ipc import Channel +from ._streaming import MsgStream +from ._state import current_actor + +if TYPE_CHECKING: + from ._portal import Portal + + +log = get_logger(__name__) + + +@dataclass +class Context: + ''' + An inter-actor, ``trio`` task communication context. + + NB: This class should never be instatiated directly, it is delivered + by either, + - runtime machinery to a remotely started task or, + - by entering ``Portal.open_context()``. + + Allows maintaining task or protocol specific state between + 2 communicating actor tasks. A unique context is created on the + callee side/end for every request to a remote actor from a portal. + + A context can be cancelled and (possibly eventually restarted) from + either side of the underlying IPC channel, open task oriented + message streams and acts as an IPC aware inter-actor-task cancel + scope. + + ''' + chan: Channel + cid: str + + # these are the "feeder" channels for delivering + # message values to the local task from the runtime + # msg processing loop. + _recv_chan: trio.MemoryReceiveChannel + _send_chan: trio.MemorySendChannel + + _remote_func_type: str | None = None + + # only set on the caller side + _portal: Portal | None = None # type: ignore # noqa + _result: Any | int = None + _remote_error: BaseException | None = None + + # cancellation state + _cancel_called: bool = False + _cancel_called_remote: tuple | None = None + _cancel_msg: str | None = None + _scope: trio.CancelScope | None = None + _enter_debugger_on_cancel: bool = True + + @property + def cancel_called(self) -> bool: + ''' + Records whether cancellation has been requested for this context + by either an explicit call to ``.cancel()`` or an implicit call + due to an error caught inside the ``Portal.open_context()`` + block. + + ''' + return self._cancel_called + + @property + def cancel_called_remote(self) -> tuple[str, str] | None: + ''' + ``Actor.uid`` of the remote actor who's task was cancelled + causing this side of the context to also be cancelled. + + ''' + remote_uid = self._cancel_called_remote + if remote_uid: + return tuple(remote_uid) + + @property + def cancelled_caught(self) -> bool: + return self._scope.cancelled_caught + + # init and streaming state + _started_called: bool = False + _started_received: bool = False + _stream_opened: bool = False + + # overrun handling machinery + # NOTE: none of this provides "backpressure" to the remote + # task, only an ability to not lose messages when the local + # task is configured to NOT transmit ``StreamOverrun``s back + # to the other side. + _overflow_q: deque[dict] = field( + default_factory=partial( + deque, + maxlen=616, + ) + ) + _scope_nursery: trio.Nursery | None = None + _in_overrun: bool = False + _allow_overruns: bool = False + + async def send_yield( + self, + data: Any, + + ) -> None: + + warnings.warn( + "`Context.send_yield()` is now deprecated. " + "Use ``MessageStream.send()``. ", + DeprecationWarning, + stacklevel=2, + ) + await self.chan.send({'yield': data, 'cid': self.cid}) + + async def send_stop(self) -> None: + await self.chan.send({'stop': True, 'cid': self.cid}) + + async def _maybe_cancel_and_set_remote_error( + self, + error_msg: dict[str, Any], + + ) -> None: + ''' + (Maybe) unpack and raise a msg error into the local scope + nursery for this context. + + Acts as a form of "relay" for a remote error raised + in the corresponding remote callee task. + + ''' + # If this is an error message from a context opened by + # ``Portal.open_context()`` we want to interrupt any ongoing + # (child) tasks within that context to be notified of the remote + # error relayed here. + # + # The reason we may want to raise the remote error immediately + # is that there is no guarantee the associated local task(s) + # will attempt to read from any locally opened stream any time + # soon. + # + # NOTE: this only applies when + # ``Portal.open_context()`` has been called since it is assumed + # (currently) that other portal APIs (``Portal.run()``, + # ``.run_in_actor()``) do their own error checking at the point + # of the call and result processing. + error = unpack_error( + error_msg, + self.chan, + ) + + # XXX: set the remote side's error so that after we cancel + # whatever task is the opener of this context it can raise + # that error as the reason. + self._remote_error = error + + if ( + isinstance(error, ContextCancelled) + ): + log.cancel( + 'Remote task-context sucessfully cancelled for ' + f'{self.chan.uid}:{self.cid}' + ) + + if self._cancel_called: + # this is an expected cancel request response message + # and we don't need to raise it in scope since it will + # potentially override a real error + return + else: + log.error( + f'Remote context error for {self.chan.uid}:{self.cid}:\n' + f'{error_msg["error"]["tb_str"]}' + ) + # TODO: tempted to **not** do this by-reraising in a + # nursery and instead cancel a surrounding scope, detect + # the cancellation, then lookup the error that was set? + # YES! this is way better and simpler! + if ( + self._scope + ): + # from trio.testing import wait_all_tasks_blocked + # await wait_all_tasks_blocked() + self._cancel_called_remote = self.chan.uid + self._scope.cancel() + + # NOTE: this usage actually works here B) + # from ._debug import breakpoint + # await breakpoint() + + # XXX: this will break early callee results sending + # since when `.result()` is finally called, this + # chan will be closed.. + # if self._recv_chan: + # await self._recv_chan.aclose() + + async def cancel( + self, + msg: str | None = None, + timeout: float = 0.5, + # timeout: float = 1000, + + ) -> None: + ''' + Cancel this inter-actor-task context. + + Request that the far side cancel it's current linked context, + Timeout quickly in an attempt to sidestep 2-generals... + + ''' + side = 'caller' if self._portal else 'callee' + if msg: + assert side == 'callee', 'Only callee side can provide cancel msg' + + log.cancel(f'Cancelling {side} side of context to {self.chan.uid}') + + self._cancel_called = True + # await _debug.breakpoint() + # breakpoint() + + if side == 'caller': + if not self._portal: + raise RuntimeError( + "No portal found, this is likely a callee side context" + ) + + cid = self.cid + with trio.move_on_after(timeout) as cs: + # cs.shield = True + log.cancel( + f"Cancelling stream {cid} to " + f"{self._portal.channel.uid}") + + # NOTE: we're telling the far end actor to cancel a task + # corresponding to *this actor*. The far end local channel + # instance is passed to `Actor._cancel_task()` implicitly. + await self._portal.run_from_ns( + 'self', + '_cancel_task', + cid=cid, + ) + # print("EXITING CANCEL CALL") + + if cs.cancelled_caught: + # XXX: there's no way to know if the remote task was indeed + # cancelled in the case where the connection is broken or + # some other network error occurred. + # if not self._portal.channel.connected(): + if not self.chan.connected(): + log.cancel( + "May have failed to cancel remote task " + f"{cid} for {self._portal.channel.uid}") + else: + log.cancel( + "Timed out on cancelling remote task " + f"{cid} for {self._portal.channel.uid}") + + # callee side remote task + else: + self._cancel_msg = msg + + # TODO: should we have an explicit cancel message + # or is relaying the local `trio.Cancelled` as an + # {'error': trio.Cancelled, cid: "blah"} enough? + # This probably gets into the discussion in + # https://github.com/goodboy/tractor/issues/36 + assert self._scope + self._scope.cancel() + + @acm + async def open_stream( + + self, + allow_overruns: bool | None = False, + msg_buffer_size: int | None = None, + + ) -> AsyncGenerator[MsgStream, None]: + ''' + Open a ``MsgStream``, a bi-directional stream connected to the + cross-actor (far end) task for this ``Context``. + + This context manager must be entered on both the caller and + callee for the stream to logically be considered "connected". + + A ``MsgStream`` is currently "one-shot" use, meaning if you + close it you can not "re-open" it for streaming and instead you + must re-establish a new surrounding ``Context`` using + ``Portal.open_context()``. In the future this may change but + currently there seems to be no obvious reason to support + "re-opening": + - pausing a stream can be done with a message. + - task errors will normally require a restart of the entire + scope of the inter-actor task context due to the nature of + ``trio``'s cancellation system. + + ''' + actor = current_actor() + + # here we create a mem chan that corresponds to the + # far end caller / callee. + + # Likewise if the surrounding context has been cancelled we error here + # since it likely means the surrounding block was exited or + # killed + + if self._cancel_called: + task = trio.lowlevel.current_task().name + raise ContextCancelled( + f'Context around {actor.uid[0]}:{task} was already cancelled!' + ) + + if not self._portal and not self._started_called: + raise RuntimeError( + 'Context.started()` must be called before opening a stream' + ) + + # NOTE: in one way streaming this only happens on the + # caller side inside `Actor.start_remote_task()` so if you try + # to send a stop from the caller to the callee in the + # single-direction-stream case you'll get a lookup error + # currently. + ctx = actor.get_context( + self.chan, + self.cid, + msg_buffer_size=msg_buffer_size, + allow_overruns=allow_overruns, + ) + ctx._allow_overruns = allow_overruns + assert ctx is self + + # XXX: If the underlying channel feeder receive mem chan has + # been closed then likely client code has already exited + # a ``.open_stream()`` block prior or there was some other + # unanticipated error or cancellation from ``trio``. + + if ctx._recv_chan._closed: + raise trio.ClosedResourceError( + 'The underlying channel for this stream was already closed!?') + + async with MsgStream( + ctx=self, + rx_chan=ctx._recv_chan, + ) as stream: + + if self._portal: + self._portal._streams.add(stream) + + try: + self._stream_opened = True + + # XXX: do we need this? + # ensure we aren't cancelled before yielding the stream + # await trio.lowlevel.checkpoint() + yield stream + + # NOTE: Make the stream "one-shot use". On exit, signal + # ``trio.EndOfChannel``/``StopAsyncIteration`` to the + # far end. + await stream.aclose() + + finally: + if self._portal: + try: + self._portal._streams.remove(stream) + except KeyError: + log.warning( + f'Stream was already destroyed?\n' + f'actor: {self.chan.uid}\n' + f'ctx id: {self.cid}' + ) + + def _maybe_raise_remote_err( + self, + err: Exception, + ) -> None: + # NOTE: whenever the context's "opener" side (task) **is** + # the side which requested the cancellation (likekly via + # ``Context.cancel()``), we don't want to re-raise that + # cancellation signal locally (would be akin to + # a ``trio.Nursery`` nursery raising ``trio.Cancelled`` + # whenever ``CancelScope.cancel()`` was called) and instead + # silently reap the expected cancellation "error"-msg. + # if 'pikerd' in err.msgdata['tb_str']: + # # from . import _debug + # # await _debug.breakpoint() + # breakpoint() + + if ( + isinstance(err, ContextCancelled) + and ( + self._cancel_called + or self.chan._cancel_called + or tuple(err.canceller) == current_actor().uid + ) + ): + return err + + raise err # from None + + async def result(self) -> Any | Exception: + ''' + From some (caller) side task, wait for and return the final + result from the remote (callee) side's task. + + This provides a mechanism for one task running in some actor to wait + on another task at the other side, in some other actor, to terminate. + + If the remote task is still in a streaming state (it is delivering + values from inside a ``Context.open_stream():`` block, then those + msgs are drained but discarded since it is presumed this side of + the context has already finished with its own streaming logic. + + If the remote context (or its containing actor runtime) was + canceled, either by a local task calling one of + ``Context.cancel()`` or `Portal.cancel_actor()``, we ignore the + received ``ContextCancelled`` exception if the context or + underlying IPC channel is marked as having been "cancel called". + This is similar behavior to using ``trio.Nursery.cancel()`` + wherein tasks which raise ``trio.Cancel`` are silently reaped; + the main different in this API is in the "cancel called" case, + instead of just not raising, we also return the exception *as + the result* since client code may be interested in the details + of the remote cancellation. + + ''' + assert self._portal, "Context.result() can not be called from callee!" + assert self._recv_chan + + # from . import _debug + # await _debug.breakpoint() + + re = self._remote_error + if re: + self._maybe_raise_remote_err(re) + return re + + if ( + self._result == id(self) + and not self._remote_error + and not self._recv_chan._closed # type: ignore + ): + # wait for a final context result consuming + # and discarding any bi dir stream msgs still + # in transit from the far end. + while True: + msg = await self._recv_chan.receive() + try: + self._result = msg['return'] + + # NOTE: we don't need to do this right? + # XXX: only close the rx mem chan AFTER + # a final result is retreived. + # if self._recv_chan: + # await self._recv_chan.aclose() + + break + except KeyError: # as msgerr: + + if 'yield' in msg: + # far end task is still streaming to us so discard + log.warning(f'Discarding stream delivered {msg}') + continue + + elif 'stop' in msg: + log.debug('Remote stream terminated') + continue + + # internal error should never get here + assert msg.get('cid'), ( + "Received internal error at portal?") + + err = unpack_error( + msg, + self._portal.channel + ) # from msgerr + + err = self._maybe_raise_remote_err(err) + self._remote_err = err + + return self._remote_error or self._result + + async def started( + self, + value: Any | None = None + + ) -> None: + ''' + Indicate to calling actor's task that this linked context + has started and send ``value`` to the other side. + + On the calling side ``value`` is the second item delivered + in the tuple returned by ``Portal.open_context()``. + + ''' + if self._portal: + raise RuntimeError( + f"Caller side context {self} can not call started!") + + elif self._started_called: + raise RuntimeError( + f"called 'started' twice on context with {self.chan.uid}") + + await self.chan.send({'started': value, 'cid': self.cid}) + self._started_called = True + + # TODO: do we need a restart api? + # async def restart(self) -> None: + # pass + + async def _drain_overflows( + self, + ) -> None: + ''' + Private task spawned to push newly received msgs to the local + task which getting overrun by the remote side. + + In order to not block the rpc msg loop, but also not discard + msgs received in this context, we need to async push msgs in + a new task which only runs for as long as the local task is in + an overrun state. + + ''' + self._in_overrun = True + try: + while self._overflow_q: + # NOTE: these msgs should never be errors since we always do + # the check prior to checking if we're in an overrun state + # inside ``.deliver_msg()``. + msg = self._overflow_q.popleft() + try: + await self._send_chan.send(msg) + except trio.BrokenResourceError: + log.warning( + f"{self._send_chan} consumer is already closed" + ) + return + except trio.Cancelled: + # we are obviously still in overrun + # but the context is being closed anyway + # so we just warn that there are un received + # msgs still.. + self._overflow_q.appendleft(msg) + fmt_msgs = '' + for msg in self._overflow_q: + fmt_msgs += f'{pformat(msg)}\n' + + log.warning( + f'Context for {self.cid} is being closed while ' + 'in an overrun state!\n' + 'Discarding the following msgs:\n' + f'{fmt_msgs}\n' + ) + raise + + finally: + # task is now finished with the backlog so mark us as + # no longer in backlog. + self._in_overrun = False + + async def _deliver_msg( + self, + msg: dict, + + draining: bool = False, + + ) -> bool: + + cid = self.cid + chan = self.chan + uid = chan.uid + send_chan: trio.MemorySendChannel = self._send_chan + + log.runtime( + f"Delivering {msg} from {uid} to caller {cid}" + ) + + error = msg.get('error') + if error: + await self._maybe_cancel_and_set_remote_error(msg) + + if ( + self._in_overrun + ): + self._overflow_q.append(msg) + return False + + try: + send_chan.send_nowait(msg) + return True + # if an error is deteced we should always + # expect it to be raised by any context (stream) + # consumer task + + except trio.BrokenResourceError: + # TODO: what is the right way to handle the case where the + # local task has already sent a 'stop' / StopAsyncInteration + # to the other side but and possibly has closed the local + # feeder mem chan? Do we wait for some kind of ack or just + # let this fail silently and bubble up (currently)? + + # XXX: local consumer has closed their side + # so cancel the far end streaming task + log.warning(f"{send_chan} consumer is already closed") + return False + + # NOTE XXX: by default we do **not** maintain context-stream + # backpressure and instead opt to relay stream overrun errors to + # the sender; the main motivation is that using bp can block the + # msg handling loop which calls into this method! + except trio.WouldBlock: + # XXX: always push an error even if the local + # receiver is in overrun state. + # await self._maybe_cancel_and_set_remote_error(msg) + + local_uid = current_actor().uid + lines = [ + f'OVERRUN on actor-task context {cid}@{local_uid}!\n' + # TODO: put remote task name here if possible? + f'remote sender actor: {uid}', + # TODO: put task func name here and maybe an arrow + # from sender to overrunner? + # f'local task {self.func_name}' + ] + if not self._stream_opened: + lines.insert( + 1, + f'\n*** No stream open on `{local_uid[0]}` side! ***\n' + ) + + text = '\n'.join(lines) + + # XXX: lul, this really can't be backpressure since any + # blocking here will block the entire msg loop rpc sched for + # a whole channel.. maybe we should rename it? + if self._allow_overruns: + text += f'\nStarting overflow queuing task on msg: {msg}' + log.warning(text) + if ( + not self._in_overrun + ): + self._overflow_q.append(msg) + n = self._scope_nursery + assert not n.child_tasks + try: + n.start_soon( + self._drain_overflows, + ) + except RuntimeError: + # if the nursery is already cancelled due to + # this context exiting or in error, we ignore + # the nursery error since we never expected + # anything different. + return False + else: + try: + raise StreamOverrun(text) + except StreamOverrun as err: + err_msg = pack_error(err) + err_msg['cid'] = cid + try: + await chan.send(err_msg) + except trio.BrokenResourceError: + # XXX: local consumer has closed their side + # so cancel the far end streaming task + log.warning(f"{chan} is already closed") + + return False + + +def mk_context( + chan: Channel, + cid: str, + msg_buffer_size: int = 2**6, + + **kwargs, + +) -> Context: + ''' + Internal factory to create an inter-actor task ``Context``. + + This is called by internals and should generally never be called + by user code. + + ''' + send_chan: trio.MemorySendChannel + recv_chan: trio.MemoryReceiveChannel + send_chan, recv_chan = trio.open_memory_channel(msg_buffer_size) + + ctx = Context( + chan, + cid, + _send_chan=send_chan, + _recv_chan=recv_chan, + **kwargs, + ) + ctx._result = id(ctx) + return ctx + + +def context(func: Callable) -> Callable: + ''' + Mark an async function as a streaming routine with ``@context``. + + ''' + # TODO: apply whatever solution ``mypy`` ends up picking for this: + # https://github.com/python/mypy/issues/2087#issuecomment-769266912 + func._tractor_context_function = True # type: ignore + + sig = inspect.signature(func) + params = sig.parameters + if 'ctx' not in params: + raise TypeError( + "The first argument to the context function " + f"{func.__name__} must be `ctx: tractor.Context`" + ) + return func diff --git a/tractor/_portal.py b/tractor/_portal.py index e61ac37a..bf3e385c 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -45,10 +45,8 @@ from ._exceptions import ( NoResult, ContextCancelled, ) -from ._streaming import ( - Context, - MsgStream, -) +from ._context import Context +from ._streaming import MsgStream log = get_logger(__name__) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 5bc8b1fd..84583e9a 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -46,7 +46,7 @@ import trio # type: ignore from trio_typing import TaskStatus from ._ipc import Channel -from ._streaming import ( +from ._context import ( mk_context, Context, ) diff --git a/tractor/_streaming.py b/tractor/_streaming.py index 4bf6d1c7..3045b835 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -14,25 +14,18 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . -""" +''' Message stream types and APIs. -""" +The machinery and types behind ``Context.open_stream()`` + +''' from __future__ import annotations import inspect -from contextlib import asynccontextmanager -from collections import deque -from dataclasses import ( - dataclass, - field, -) -from functools import partial -from pprint import pformat +from contextlib import asynccontextmanager as acm from typing import ( Any, - Optional, Callable, - AsyncGenerator, AsyncIterator, TYPE_CHECKING, ) @@ -40,22 +33,17 @@ import warnings import trio -from ._ipc import Channel from ._exceptions import ( unpack_error, - pack_error, - ContextCancelled, - StreamOverrun, ) from .log import get_logger -from ._state import current_actor from .trionics import ( broadcast_receiver, BroadcastReceiver, ) if TYPE_CHECKING: - from ._portal import Portal + from ._context import Context log = get_logger(__name__) @@ -87,9 +75,9 @@ class MsgStream(trio.abc.Channel): ''' def __init__( self, - ctx: 'Context', # typing: ignore # noqa + ctx: Context, # typing: ignore # noqa rx_chan: trio.MemoryReceiveChannel, - _broadcaster: Optional[BroadcastReceiver] = None, + _broadcaster: BroadcastReceiver | None = None, ) -> None: self._ctx = ctx @@ -292,7 +280,7 @@ class MsgStream(trio.abc.Channel): # still need to consume msgs that are "in transit" from the far # end (eg. for ``Context.result()``). - @asynccontextmanager + @acm async def subscribe( self, @@ -361,693 +349,6 @@ class MsgStream(trio.abc.Channel): await self._ctx.chan.send({'yield': data, 'cid': self._ctx.cid}) -@dataclass -class Context: - ''' - An inter-actor, ``trio`` task communication context. - - NB: This class should never be instatiated directly, it is delivered - by either, - - runtime machinery to a remotely started task or, - - by entering ``Portal.open_context()``. - - Allows maintaining task or protocol specific state between - 2 communicating actor tasks. A unique context is created on the - callee side/end for every request to a remote actor from a portal. - - A context can be cancelled and (possibly eventually restarted) from - either side of the underlying IPC channel, open task oriented - message streams and acts as an IPC aware inter-actor-task cancel - scope. - - ''' - chan: Channel - cid: str - - # these are the "feeder" channels for delivering - # message values to the local task from the runtime - # msg processing loop. - _recv_chan: trio.MemoryReceiveChannel - _send_chan: trio.MemorySendChannel - - _remote_func_type: str | None = None - - # only set on the caller side - _portal: Portal | None = None # type: ignore # noqa - _result: Any | int = None - _remote_error: BaseException | None = None - - # cancellation state - _cancel_called: bool = False - _cancel_called_remote: tuple | None = None - _cancel_msg: str | None = None - _scope: trio.CancelScope | None = None - _enter_debugger_on_cancel: bool = True - - @property - def cancel_called(self) -> bool: - ''' - Records whether cancellation has been requested for this context - by either an explicit call to ``.cancel()`` or an implicit call - due to an error caught inside the ``Portal.open_context()`` - block. - - ''' - return self._cancel_called - - @property - def cancel_called_remote(self) -> tuple[str, str] | None: - ''' - ``Actor.uid`` of the remote actor who's task was cancelled - causing this side of the context to also be cancelled. - - ''' - remote_uid = self._cancel_called_remote - if remote_uid: - return tuple(remote_uid) - - @property - def cancelled_caught(self) -> bool: - return self._scope.cancelled_caught - - # init and streaming state - _started_called: bool = False - _started_received: bool = False - _stream_opened: bool = False - - # overrun handling machinery - # NOTE: none of this provides "backpressure" to the remote - # task, only an ability to not lose messages when the local - # task is configured to NOT transmit ``StreamOverrun``s back - # to the other side. - _overflow_q: deque[dict] = field( - default_factory=partial( - deque, - maxlen=616, - ) - ) - _scope_nursery: trio.Nursery | None = None - _in_overrun: bool = False - _allow_overruns: bool = False - - async def send_yield( - self, - data: Any, - - ) -> None: - - warnings.warn( - "`Context.send_yield()` is now deprecated. " - "Use ``MessageStream.send()``. ", - DeprecationWarning, - stacklevel=2, - ) - await self.chan.send({'yield': data, 'cid': self.cid}) - - async def send_stop(self) -> None: - await self.chan.send({'stop': True, 'cid': self.cid}) - - async def _maybe_cancel_and_set_remote_error( - self, - error_msg: dict[str, Any], - - ) -> None: - ''' - (Maybe) unpack and raise a msg error into the local scope - nursery for this context. - - Acts as a form of "relay" for a remote error raised - in the corresponding remote callee task. - - ''' - # If this is an error message from a context opened by - # ``Portal.open_context()`` we want to interrupt any ongoing - # (child) tasks within that context to be notified of the remote - # error relayed here. - # - # The reason we may want to raise the remote error immediately - # is that there is no guarantee the associated local task(s) - # will attempt to read from any locally opened stream any time - # soon. - # - # NOTE: this only applies when - # ``Portal.open_context()`` has been called since it is assumed - # (currently) that other portal APIs (``Portal.run()``, - # ``.run_in_actor()``) do their own error checking at the point - # of the call and result processing. - error = unpack_error( - error_msg, - self.chan, - ) - - # XXX: set the remote side's error so that after we cancel - # whatever task is the opener of this context it can raise - # that error as the reason. - self._remote_error = error - - if ( - isinstance(error, ContextCancelled) - ): - log.cancel( - 'Remote task-context sucessfully cancelled for ' - f'{self.chan.uid}:{self.cid}' - ) - - if self._cancel_called: - # this is an expected cancel request response message - # and we don't need to raise it in scope since it will - # potentially override a real error - return - else: - log.error( - f'Remote context error for {self.chan.uid}:{self.cid}:\n' - f'{error_msg["error"]["tb_str"]}' - ) - # TODO: tempted to **not** do this by-reraising in a - # nursery and instead cancel a surrounding scope, detect - # the cancellation, then lookup the error that was set? - # YES! this is way better and simpler! - if ( - self._scope - ): - # from trio.testing import wait_all_tasks_blocked - # await wait_all_tasks_blocked() - self._cancel_called_remote = self.chan.uid - self._scope.cancel() - - # NOTE: this usage actually works here B) - # from ._debug import breakpoint - # await breakpoint() - - # XXX: this will break early callee results sending - # since when `.result()` is finally called, this - # chan will be closed.. - # if self._recv_chan: - # await self._recv_chan.aclose() - - async def cancel( - self, - msg: str | None = None, - timeout: float = 0.5, - # timeout: float = 1000, - - ) -> None: - ''' - Cancel this inter-actor-task context. - - Request that the far side cancel it's current linked context, - Timeout quickly in an attempt to sidestep 2-generals... - - ''' - side = 'caller' if self._portal else 'callee' - if msg: - assert side == 'callee', 'Only callee side can provide cancel msg' - - log.cancel(f'Cancelling {side} side of context to {self.chan.uid}') - - self._cancel_called = True - # await _debug.breakpoint() - # breakpoint() - - if side == 'caller': - if not self._portal: - raise RuntimeError( - "No portal found, this is likely a callee side context" - ) - - cid = self.cid - with trio.move_on_after(timeout) as cs: - # cs.shield = True - log.cancel( - f"Cancelling stream {cid} to " - f"{self._portal.channel.uid}") - - # NOTE: we're telling the far end actor to cancel a task - # corresponding to *this actor*. The far end local channel - # instance is passed to `Actor._cancel_task()` implicitly. - await self._portal.run_from_ns( - 'self', - '_cancel_task', - cid=cid, - ) - # print("EXITING CANCEL CALL") - - if cs.cancelled_caught: - # XXX: there's no way to know if the remote task was indeed - # cancelled in the case where the connection is broken or - # some other network error occurred. - # if not self._portal.channel.connected(): - if not self.chan.connected(): - log.cancel( - "May have failed to cancel remote task " - f"{cid} for {self._portal.channel.uid}") - else: - log.cancel( - "Timed out on cancelling remote task " - f"{cid} for {self._portal.channel.uid}") - - # callee side remote task - else: - self._cancel_msg = msg - - # TODO: should we have an explicit cancel message - # or is relaying the local `trio.Cancelled` as an - # {'error': trio.Cancelled, cid: "blah"} enough? - # This probably gets into the discussion in - # https://github.com/goodboy/tractor/issues/36 - assert self._scope - self._scope.cancel() - - @asynccontextmanager - async def open_stream( - - self, - allow_overruns: bool | None = False, - msg_buffer_size: int | None = None, - - ) -> AsyncGenerator[MsgStream, None]: - ''' - Open a ``MsgStream``, a bi-directional stream connected to the - cross-actor (far end) task for this ``Context``. - - This context manager must be entered on both the caller and - callee for the stream to logically be considered "connected". - - A ``MsgStream`` is currently "one-shot" use, meaning if you - close it you can not "re-open" it for streaming and instead you - must re-establish a new surrounding ``Context`` using - ``Portal.open_context()``. In the future this may change but - currently there seems to be no obvious reason to support - "re-opening": - - pausing a stream can be done with a message. - - task errors will normally require a restart of the entire - scope of the inter-actor task context due to the nature of - ``trio``'s cancellation system. - - ''' - actor = current_actor() - - # here we create a mem chan that corresponds to the - # far end caller / callee. - - # Likewise if the surrounding context has been cancelled we error here - # since it likely means the surrounding block was exited or - # killed - - if self._cancel_called: - task = trio.lowlevel.current_task().name - raise ContextCancelled( - f'Context around {actor.uid[0]}:{task} was already cancelled!' - ) - - if not self._portal and not self._started_called: - raise RuntimeError( - 'Context.started()` must be called before opening a stream' - ) - - # NOTE: in one way streaming this only happens on the - # caller side inside `Actor.start_remote_task()` so if you try - # to send a stop from the caller to the callee in the - # single-direction-stream case you'll get a lookup error - # currently. - ctx = actor.get_context( - self.chan, - self.cid, - msg_buffer_size=msg_buffer_size, - allow_overruns=allow_overruns, - ) - ctx._allow_overruns = allow_overruns - assert ctx is self - - # XXX: If the underlying channel feeder receive mem chan has - # been closed then likely client code has already exited - # a ``.open_stream()`` block prior or there was some other - # unanticipated error or cancellation from ``trio``. - - if ctx._recv_chan._closed: - raise trio.ClosedResourceError( - 'The underlying channel for this stream was already closed!?') - - async with MsgStream( - ctx=self, - rx_chan=ctx._recv_chan, - ) as stream: - - if self._portal: - self._portal._streams.add(stream) - - try: - self._stream_opened = True - - # XXX: do we need this? - # ensure we aren't cancelled before yielding the stream - # await trio.lowlevel.checkpoint() - yield stream - - # NOTE: Make the stream "one-shot use". On exit, signal - # ``trio.EndOfChannel``/``StopAsyncIteration`` to the - # far end. - await stream.aclose() - - finally: - if self._portal: - try: - self._portal._streams.remove(stream) - except KeyError: - log.warning( - f'Stream was already destroyed?\n' - f'actor: {self.chan.uid}\n' - f'ctx id: {self.cid}' - ) - - def _maybe_raise_remote_err( - self, - err: Exception, - ) -> None: - # NOTE: whenever the context's "opener" side (task) **is** - # the side which requested the cancellation (likekly via - # ``Context.cancel()``), we don't want to re-raise that - # cancellation signal locally (would be akin to - # a ``trio.Nursery`` nursery raising ``trio.Cancelled`` - # whenever ``CancelScope.cancel()`` was called) and instead - # silently reap the expected cancellation "error"-msg. - # if 'pikerd' in err.msgdata['tb_str']: - # # from . import _debug - # # await _debug.breakpoint() - # breakpoint() - - if ( - isinstance(err, ContextCancelled) - and ( - self._cancel_called - or self.chan._cancel_called - or tuple(err.canceller) == current_actor().uid - ) - ): - return err - - raise err # from None - - async def result(self) -> Any | Exception: - ''' - From some (caller) side task, wait for and return the final - result from the remote (callee) side's task. - - This provides a mechanism for one task running in some actor to wait - on another task at the other side, in some other actor, to terminate. - - If the remote task is still in a streaming state (it is delivering - values from inside a ``Context.open_stream():`` block, then those - msgs are drained but discarded since it is presumed this side of - the context has already finished with its own streaming logic. - - If the remote context (or its containing actor runtime) was - canceled, either by a local task calling one of - ``Context.cancel()`` or `Portal.cancel_actor()``, we ignore the - received ``ContextCancelled`` exception if the context or - underlying IPC channel is marked as having been "cancel called". - This is similar behavior to using ``trio.Nursery.cancel()`` - wherein tasks which raise ``trio.Cancel`` are silently reaped; - the main different in this API is in the "cancel called" case, - instead of just not raising, we also return the exception *as - the result* since client code may be interested in the details - of the remote cancellation. - - ''' - assert self._portal, "Context.result() can not be called from callee!" - assert self._recv_chan - - # from . import _debug - # await _debug.breakpoint() - - re = self._remote_error - if re: - self._maybe_raise_remote_err(re) - return re - - if ( - self._result == id(self) - and not self._remote_error - and not self._recv_chan._closed # type: ignore - ): - # wait for a final context result consuming - # and discarding any bi dir stream msgs still - # in transit from the far end. - while True: - msg = await self._recv_chan.receive() - try: - self._result = msg['return'] - - # NOTE: we don't need to do this right? - # XXX: only close the rx mem chan AFTER - # a final result is retreived. - # if self._recv_chan: - # await self._recv_chan.aclose() - - break - except KeyError: # as msgerr: - - if 'yield' in msg: - # far end task is still streaming to us so discard - log.warning(f'Discarding stream delivered {msg}') - continue - - elif 'stop' in msg: - log.debug('Remote stream terminated') - continue - - # internal error should never get here - assert msg.get('cid'), ( - "Received internal error at portal?") - - err = unpack_error( - msg, - self._portal.channel - ) # from msgerr - - err = self._maybe_raise_remote_err(err) - self._remote_err = err - - return self._remote_error or self._result - - async def started( - self, - value: Any | None = None - - ) -> None: - ''' - Indicate to calling actor's task that this linked context - has started and send ``value`` to the other side. - - On the calling side ``value`` is the second item delivered - in the tuple returned by ``Portal.open_context()``. - - ''' - if self._portal: - raise RuntimeError( - f"Caller side context {self} can not call started!") - - elif self._started_called: - raise RuntimeError( - f"called 'started' twice on context with {self.chan.uid}") - - await self.chan.send({'started': value, 'cid': self.cid}) - self._started_called = True - - # TODO: do we need a restart api? - # async def restart(self) -> None: - # pass - - async def _drain_overflows( - self, - ) -> None: - ''' - Private task spawned to push newly received msgs to the local - task which getting overrun by the remote side. - - In order to not block the rpc msg loop, but also not discard - msgs received in this context, we need to async push msgs in - a new task which only runs for as long as the local task is in - an overrun state. - - ''' - self._in_overrun = True - try: - while self._overflow_q: - # NOTE: these msgs should never be errors since we always do - # the check prior to checking if we're in an overrun state - # inside ``.deliver_msg()``. - msg = self._overflow_q.popleft() - try: - await self._send_chan.send(msg) - except trio.BrokenResourceError: - log.warning( - f"{self._send_chan} consumer is already closed" - ) - return - except trio.Cancelled: - # we are obviously still in overrun - # but the context is being closed anyway - # so we just warn that there are un received - # msgs still.. - self._overflow_q.appendleft(msg) - fmt_msgs = '' - for msg in self._overflow_q: - fmt_msgs += f'{pformat(msg)}\n' - - log.warning( - f'Context for {self.cid} is being closed while ' - 'in an overrun state!\n' - 'Discarding the following msgs:\n' - f'{fmt_msgs}\n' - ) - raise - - finally: - # task is now finished with the backlog so mark us as - # no longer in backlog. - self._in_overrun = False - - async def _deliver_msg( - self, - msg: dict, - - draining: bool = False, - - ) -> bool: - - cid = self.cid - chan = self.chan - uid = chan.uid - send_chan: trio.MemorySendChannel = self._send_chan - - log.runtime( - f"Delivering {msg} from {uid} to caller {cid}" - ) - - error = msg.get('error') - if error: - await self._maybe_cancel_and_set_remote_error(msg) - - if ( - self._in_overrun - ): - self._overflow_q.append(msg) - return False - - try: - send_chan.send_nowait(msg) - return True - # if an error is deteced we should always - # expect it to be raised by any context (stream) - # consumer task - - except trio.BrokenResourceError: - # TODO: what is the right way to handle the case where the - # local task has already sent a 'stop' / StopAsyncInteration - # to the other side but and possibly has closed the local - # feeder mem chan? Do we wait for some kind of ack or just - # let this fail silently and bubble up (currently)? - - # XXX: local consumer has closed their side - # so cancel the far end streaming task - log.warning(f"{send_chan} consumer is already closed") - return False - - # NOTE XXX: by default we do **not** maintain context-stream - # backpressure and instead opt to relay stream overrun errors to - # the sender; the main motivation is that using bp can block the - # msg handling loop which calls into this method! - except trio.WouldBlock: - # XXX: always push an error even if the local - # receiver is in overrun state. - # await self._maybe_cancel_and_set_remote_error(msg) - - local_uid = current_actor().uid - lines = [ - f'OVERRUN on actor-task context {cid}@{local_uid}!\n' - # TODO: put remote task name here if possible? - f'remote sender actor: {uid}', - # TODO: put task func name here and maybe an arrow - # from sender to overrunner? - # f'local task {self.func_name}' - ] - if not self._stream_opened: - lines.insert( - 1, - f'\n*** No stream open on `{local_uid[0]}` side! ***\n' - ) - - text = '\n'.join(lines) - - # XXX: lul, this really can't be backpressure since any - # blocking here will block the entire msg loop rpc sched for - # a whole channel.. maybe we should rename it? - if self._allow_overruns: - text += f'\nStarting overflow queuing task on msg: {msg}' - log.warning(text) - if ( - not self._in_overrun - ): - self._overflow_q.append(msg) - n = self._scope_nursery - assert not n.child_tasks - try: - n.start_soon( - self._drain_overflows, - ) - except RuntimeError: - # if the nursery is already cancelled due to - # this context exiting or in error, we ignore - # the nursery error since we never expected - # anything different. - return False - else: - try: - raise StreamOverrun(text) - except StreamOverrun as err: - err_msg = pack_error(err) - err_msg['cid'] = cid - try: - await chan.send(err_msg) - except trio.BrokenResourceError: - # XXX: local consumer has closed their side - # so cancel the far end streaming task - log.warning(f"{chan} is already closed") - - return False - - -def mk_context( - chan: Channel, - cid: str, - msg_buffer_size: int = 2**6, - - **kwargs, - -) -> Context: - ''' - Internal factory to create an inter-actor task ``Context``. - - This is called by internals and should generally never be called - by user code. - - ''' - send_chan: trio.MemorySendChannel - recv_chan: trio.MemoryReceiveChannel - send_chan, recv_chan = trio.open_memory_channel(msg_buffer_size) - - ctx = Context( - chan, - cid, - _send_chan=send_chan, - _recv_chan=recv_chan, - **kwargs, - ) - ctx._result = id(ctx) - return ctx - - def stream(func: Callable) -> Callable: ''' Mark an async function as a streaming routine with ``@stream``. @@ -1078,22 +379,3 @@ def stream(func: Callable) -> Callable: "(Or ``to_trio`` if using ``asyncio`` in guest mode)." ) return func - - -def context(func: Callable) -> Callable: - ''' - Mark an async function as a streaming routine with ``@context``. - - ''' - # TODO: apply whatever solution ``mypy`` ends up picking for this: - # https://github.com/python/mypy/issues/2087#issuecomment-769266912 - func._tractor_context_function = True # type: ignore - - sig = inspect.signature(func) - params = sig.parameters - if 'ctx' not in params: - raise TypeError( - "The first argument to the context function " - f"{func.__name__} must be `ctx: tractor.Context`" - ) - return func diff --git a/tractor/experimental/_pubsub.py b/tractor/experimental/_pubsub.py index 99117b01..89f286d2 100644 --- a/tractor/experimental/_pubsub.py +++ b/tractor/experimental/_pubsub.py @@ -37,7 +37,7 @@ import trio import wrapt from ..log import get_logger -from .._streaming import Context +from .._context import Context __all__ = ['pub'] @@ -148,7 +148,8 @@ def pub( *, tasks: set[str] = set(), ): - """Publisher async generator decorator. + ''' + Publisher async generator decorator. A publisher can be called multiple times from different actors but will only spawn a finite set of internal tasks to stream values to @@ -227,7 +228,8 @@ def pub( running in a single actor to stream data to an arbitrary number of subscribers. If you are ok to have a new task running for every call to ``pub_service()`` then probably don't need this. - """ + + ''' global _pubtask2lock # handle the decorator not called with () case From 7293b82bccbe3683710ee21647e5205b779963b8 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 14 Apr 2023 18:08:08 -0400 Subject: [PATCH 023/146] Tweak doc string --- tractor/_context.py | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index 4e52b218..e95cd2b1 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -64,21 +64,25 @@ log = get_logger(__name__) @dataclass class Context: ''' - An inter-actor, ``trio`` task communication context. + An inter-actor, ``trio``-task communication context. NB: This class should never be instatiated directly, it is delivered by either, - runtime machinery to a remotely started task or, - by entering ``Portal.open_context()``. + and is always constructed using ``mkt_context()``. + Allows maintaining task or protocol specific state between - 2 communicating actor tasks. A unique context is created on the - callee side/end for every request to a remote actor from a portal. + 2 communicating, parallel executing actor tasks. A unique context is + allocated on each side of any task RPC-linked msg dialog, for + every request to a remote actor from a portal. On the "callee" + side a context is always allocated inside ``._runtime._invoke()``. A context can be cancelled and (possibly eventually restarted) from - either side of the underlying IPC channel, open task oriented - message streams and acts as an IPC aware inter-actor-task cancel - scope. + either side of the underlying IPC channel, it can also open task + oriented message streams, and acts more or less as an IPC aware + inter-actor-task ``trio.CancelScope``. ''' chan: Channel @@ -744,7 +748,7 @@ def mk_context( _recv_chan=recv_chan, **kwargs, ) - ctx._result = id(ctx) + ctx._result: int | Any = id(ctx) return ctx From 60791ed54667ec35188fd4bd952883eb939f86fc Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 20 Apr 2023 19:13:35 -0400 Subject: [PATCH 024/146] Oof, fix remaining `Actor.cancel()` in `Actor._from_parent()` --- tractor/_runtime.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 84583e9a..78f9e4fd 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -998,7 +998,7 @@ class Actor: log.warning( f"Failed to connect to parent @ {parent_addr}," " closing server") - await self.cancel() + await self.cancel(requesting_uid=self.uid) raise async def _serve_forever( From ead9e418de8c131dc48c3ad3565e9f1752bfdf10 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 12 May 2023 18:28:47 -0400 Subject: [PATCH 025/146] Expose `allow_overruns` to `Portal.open_context()` Turns out you can get a case where you might be opening multiple ctx-streams concurrently and during the context opening phase you block for all contexts to open, but then when you eventually start opening streams some slow to start context has caused the others become in an overrun state.. so we need to let the caller control whether that's an error ;) This also needs a test! --- tractor/_portal.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/tractor/_portal.py b/tractor/_portal.py index bf3e385c..60293716 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -380,6 +380,7 @@ class Portal: self, func: Callable, + allow_overruns: bool = False, **kwargs, ) -> AsyncGenerator[tuple[Context, Any], None]: @@ -409,6 +410,16 @@ class Portal: fn_mod_path, fn_name, kwargs, + + # NOTE: it's imporant to expose this since you might + # get the case where the parent who opened the context does + # not open a stream until after some slow startup/init + # period, in which case when the first msg is read from + # the feeder mem chan, say when first calling + # `Context.open_stream(allow_overruns=True)`, the overrun condition will be + # raised before any ignoring of overflow msgs can take + # place.. + allow_overruns=allow_overruns, ) assert ctx._remote_func_type == 'context' From a0276f41c23db8ee56b24de7dc9e51322857b48e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 18 May 2023 18:50:46 -0400 Subject: [PATCH 026/146] Remote cancellation runtime-internal vars renames - `Context._cancel_called_remote` -> `._cancelled_remote` since "called" implies the cancellation was "requested" when it could be due to another error and the actor uid is the value - only set once the far end task scope is terminated due to either error or cancel, which has nothing to do with *what* caused the cancellation. - `Actor._cancel_called_remote` -> `._cancel_called_by_remote` which emphasizes that this variable is **only set** IFF some remote actor **requested that** this actor's runtime be cancelled via `Actor.cancel()`. --- tractor/_context.py | 21 ++++++++++++++------- tractor/_runtime.py | 10 +++++----- 2 files changed, 19 insertions(+), 12 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index e95cd2b1..054f7832 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -103,7 +103,7 @@ class Context: # cancellation state _cancel_called: bool = False - _cancel_called_remote: tuple | None = None + _cancelled_remote: tuple | None = None _cancel_msg: str | None = None _scope: trio.CancelScope | None = None _enter_debugger_on_cancel: bool = True @@ -126,7 +126,7 @@ class Context: causing this side of the context to also be cancelled. ''' - remote_uid = self._cancel_called_remote + remote_uid = self._cancelled_remote if remote_uid: return tuple(remote_uid) @@ -209,6 +209,10 @@ class Context: # that error as the reason. self._remote_error = error + # always record the remote actor's uid since its cancellation + # state is directly linked to ours (the local one). + self._cancelled_remote = self.chan.uid + if ( isinstance(error, ContextCancelled) ): @@ -218,9 +222,12 @@ class Context: ) if self._cancel_called: + # from ._debug import breakpoint + # await breakpoint() + # this is an expected cancel request response message - # and we don't need to raise it in scope since it will - # potentially override a real error + # and we **don't need to raise it** in local cancel + # scope since it will potentially override a real error. return else: log.error( @@ -236,7 +243,7 @@ class Context: ): # from trio.testing import wait_all_tasks_blocked # await wait_all_tasks_blocked() - self._cancel_called_remote = self.chan.uid + # self._cancelled_remote = self.chan.uid self._scope.cancel() # NOTE: this usage actually works here B) @@ -252,7 +259,7 @@ class Context: async def cancel( self, msg: str | None = None, - timeout: float = 0.5, + timeout: float = 0.616, # timeout: float = 1000, ) -> None: @@ -281,7 +288,7 @@ class Context: cid = self.cid with trio.move_on_after(timeout) as cs: - # cs.shield = True + cs.shield = True log.cancel( f"Cancelling stream {cid} to " f"{self._portal.channel.uid}") diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 78f9e4fd..93168375 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -254,7 +254,7 @@ async def _invoke( fname = func.__name__ cs: trio.CancelScope = ctx._scope if cs.cancel_called: - canceller = ctx._cancel_called_remote + canceller = ctx._cancelled_remote # await _debug.breakpoint() # NOTE / TODO: if we end up having @@ -505,7 +505,7 @@ class Actor: self.uid = (name, uid or str(uuid.uuid4())) self._cancel_complete = trio.Event() - self._cancel_called_remote: tuple[str, tuple] | None = None + self._cancel_called_by_remote: tuple[str, tuple] | None = None self._cancel_called: bool = False # retreive and store parent `__main__` data which @@ -1069,7 +1069,7 @@ class Actor: ''' log.cancel(f"{self.uid} is trying to cancel") - self._cancel_called_remote: tuple = requesting_uid + self._cancel_called_by_remote: tuple = requesting_uid self._cancel_called = True # cancel all ongoing rpc tasks @@ -1141,10 +1141,10 @@ class Actor: f"peer: {chan.uid}\n") if ( - ctx._cancel_called_remote is None + ctx._cancelled_remote is None and requesting_uid ): - ctx._cancel_called_remote: tuple = requesting_uid + ctx._cancelled_remote: tuple = requesting_uid # don't allow cancelling this function mid-execution # (is this necessary?) From 64956887302b6450c9503b7d04eef28053270bd6 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 25 May 2023 16:00:05 -0400 Subject: [PATCH 027/146] Drop `Optional` style from runtime mod --- tractor/_runtime.py | 53 +++++++++++++++++++++++++-------------------- 1 file changed, 29 insertions(+), 24 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 93168375..08ddabc4 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -30,7 +30,6 @@ import sys from typing import ( Any, Callable, - Optional, Union, TYPE_CHECKING, ) @@ -101,7 +100,7 @@ async def _invoke( cancel_scope = trio.CancelScope() # activated cancel scope ref - cs: Optional[trio.CancelScope] = None + cs: trio.CancelScope | None = None ctx = actor.get_context( chan, @@ -468,16 +467,20 @@ class Actor: msg_buffer_size: int = 2**6 # nursery placeholders filled in by `async_main()` after fork - _root_n: Optional[trio.Nursery] = None - _service_n: Optional[trio.Nursery] = None - _server_n: Optional[trio.Nursery] = None + _root_n: trio.Nursery | None = None + _service_n: trio.Nursery | None = None + _server_n: trio.Nursery | None = None # Information about `__main__` from parent _parent_main_data: dict[str, str] - _parent_chan_cs: Optional[trio.CancelScope] = None + _parent_chan_cs: trio.CancelScope | None = None # syncs for setup/teardown sequences - _server_down: Optional[trio.Event] = None + _server_down: trio.Event | None = None + + # user toggled crash handling (including monkey-patched in + # `trio.open_nursery()` via `.trionics._supervisor` B) + _debug_mode: bool = False # if started on ``asycio`` running ``trio`` in guest mode _infected_aio: bool = False @@ -493,8 +496,8 @@ class Actor: enable_modules: list[str] = [], uid: str | None = None, loglevel: str | None = None, - arbiter_addr: Optional[tuple[str, int]] = None, - spawn_method: Optional[str] = None + arbiter_addr: tuple[str, int] | None = None, + spawn_method: str | None = None ) -> None: ''' This constructor is called in the parent actor **before** the spawning @@ -554,9 +557,8 @@ class Actor: ] = {} self._listeners: list[trio.abc.Listener] = [] - self._parent_chan: Optional[Channel] = None - self._forkserver_info: Optional[ - tuple[Any, Any, Any, Any, Any]] = None + self._parent_chan: Channel | None = None + self._forkserver_info: tuple | None = None self._actoruid2nursery: dict[ tuple[str, str], ActorNursery | None, @@ -647,7 +649,7 @@ class Actor: self._no_more_peers = trio.Event() # unset chan = Channel.from_stream(stream) - uid: Optional[tuple[str, str]] = chan.uid + uid: tuple[str, str] | None = chan.uid log.runtime(f"New connection to us {chan}") # send/receive initial handshake response @@ -695,7 +697,7 @@ class Actor: # append new channel self._peers[uid].append(chan) - local_nursery: Optional[ActorNursery] = None # noqa + local_nursery: ActorNursery | None = None # noqa disconnected: bool = False # Begin channel management - respond to remote requests and @@ -947,8 +949,8 @@ class Actor: async def _from_parent( self, - parent_addr: Optional[tuple[str, int]], - ) -> tuple[Channel, Optional[tuple[str, int]]]: + parent_addr: tuple[str, int] | None, + ) -> tuple[Channel, tuple[str, int] | None]: try: # Connect back to the parent actor and conduct initial # handshake. From this point on if we error, we @@ -961,7 +963,7 @@ class Actor: # Initial handshake: swap names. await self._do_handshake(chan) - accept_addr: Optional[tuple[str, int]] = None + accept_addr: tuple[str, int] | None = None if self._spawn_method == "trio": # Receive runtime state from our parent @@ -1020,7 +1022,7 @@ class Actor: self._server_down = trio.Event() try: async with trio.open_nursery() as server_n: - l: list[trio.abc.Listener] = await server_n.start( + listeners: list[trio.abc.Listener] = await server_n.start( partial( trio.serve_tcp, self._stream_handler, @@ -1031,10 +1033,13 @@ class Actor: host=accept_host, ) ) + sockets: list[trio.socket] = [ + getattr(listener, 'socket', 'unknown socket') + for listener in listeners + ] log.runtime( - "Started tcp server(s) on" - f" {[getattr(l, 'socket', 'unknown socket') for l in l]}") - self._listeners.extend(l) + f'Started tcp server(s) on {sockets}') + self._listeners.extend(listeners) task_status.started(server_n) finally: # signal the server is down since nursery above terminated @@ -1215,7 +1220,7 @@ class Actor: self._server_n.cancel_scope.cancel() @property - def accept_addr(self) -> Optional[tuple[str, int]]: + def accept_addr(self) -> tuple[str, int] | None: ''' Primary address to which the channel server is bound. @@ -1267,7 +1272,7 @@ class Actor: async def async_main( actor: Actor, - accept_addr: Optional[tuple[str, int]] = None, + accept_addr: tuple[str, int] | None = None, # XXX: currently ``parent_addr`` is only needed for the # ``multiprocessing`` backend (which pickles state sent to @@ -1276,7 +1281,7 @@ async def async_main( # change this to a simple ``is_subactor: bool`` which will # be False when running as root actor and True when as # a subactor. - parent_addr: Optional[tuple[str, int]] = None, + parent_addr: tuple[str, int] | None = None, task_status: TaskStatus[None] = trio.TASK_STATUS_IGNORED, ) -> None: From 17ae44916056d840eea48bba843d008e6d97281d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 14 Jun 2023 15:31:52 -0400 Subject: [PATCH 028/146] Tidy up `typing` imports in broadcaster mod --- tractor/trionics/_broadcast.py | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/tractor/trionics/_broadcast.py b/tractor/trionics/_broadcast.py index 42b1704b..244a42d4 100644 --- a/tractor/trionics/_broadcast.py +++ b/tractor/trionics/_broadcast.py @@ -25,8 +25,16 @@ from collections import deque from contextlib import asynccontextmanager from functools import partial from operator import ne -from typing import Optional, Callable, Awaitable, Any, AsyncIterator, Protocol -from typing import Generic, TypeVar +from typing import ( + Optional, + Callable, + Awaitable, + Any, + AsyncIterator, + Protocol, + Generic, + TypeVar, +) import trio from trio._core._run import Task From 7507e269ec2f769e4f6cc208dec51f69c62558e9 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 14 Jun 2023 15:32:15 -0400 Subject: [PATCH 029/146] Just import `mp` top level in `._spawn` --- tractor/_spawn.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index b1a20f49..db465421 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -19,6 +19,7 @@ Machinery for actor process spawning using multiple backends. """ from __future__ import annotations +import multiprocessing as mp import sys import platform from typing import ( @@ -53,7 +54,6 @@ from ._exceptions import ActorFailure if TYPE_CHECKING: from ._supervise import ActorNursery - import multiprocessing as mp ProcessType = TypeVar('ProcessType', mp.Process, trio.Process) log = get_logger('tractor') @@ -70,7 +70,6 @@ _spawn_method: SpawnMethodKey = 'trio' if platform.system() == 'Windows': - import multiprocessing as mp _ctx = mp.get_context("spawn") async def proc_waiter(proc: mp.Process) -> None: From f33f689f342ce2c04e82bb141b289439da4f263e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 7 Jul 2023 14:48:37 -0400 Subject: [PATCH 030/146] .log: more correct handling for `get_logger(__name__)` usage --- tractor/log.py | 42 +++++++++++++++++++++++++++++++++++------- 1 file changed, 35 insertions(+), 7 deletions(-) diff --git a/tractor/log.py b/tractor/log.py index 1ea99315..5710e83e 100644 --- a/tractor/log.py +++ b/tractor/log.py @@ -193,15 +193,39 @@ def get_logger( ''' log = rlog = logging.getLogger(_root_name) - if name and name != _proj_name: + if ( + name + and name != _proj_name + ): - # handling for modules that use ``get_logger(__name__)`` to - # avoid duplicate project-package token in msg output - rname, _, tail = name.partition('.') - if rname == _root_name: - name = tail + # NOTE: for handling for modules that use ``get_logger(__name__)`` + # we make the following stylistic choice: + # - always avoid duplicate project-package token + # in msg output: i.e. tractor.tractor _ipc.py in header + # looks ridiculous XD + # - never show the leaf module name in the {name} part + # since in python the {filename} is always this same + # module-file. + + sub_name: None | str = None + rname, _, sub_name = name.partition('.') + pkgpath, _, modfilename = sub_name.rpartition('.') + + # NOTE: for tractor itself never include the last level + # module key in the name such that something like: eg. + # 'tractor.trionics._broadcast` only includes the first + # 2 tokens in the (coloured) name part. + if rname == 'tractor': + sub_name = pkgpath + + if _root_name in sub_name: + duplicate, _, sub_name = sub_name.partition('.') + + if not sub_name: + log = rlog + else: + log = rlog.getChild(sub_name) - log = rlog.getChild(name) log.level = rlog.level # add our actor-task aware adapter which will dynamically look up @@ -254,3 +278,7 @@ def get_console_log( def get_loglevel() -> str: return _default_loglevel + + +# global module logger for tractor itself +log = get_logger('tractor') From 89b84ed6c0872cbcca12a92d85d4c09a724cf830 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 12 Jul 2023 13:07:30 -0400 Subject: [PATCH 031/146] Make `NamespacePath` work on object refs Detect if the input ref is a non-func (like an `object` instance) in which case grab its type name using `type()`. Wrap all the name-getting into a new `_mk_fqpn()` static meth: gets the "fully qualified path name" and returns path and name in tuple; port other methds to use it. Refine and update the docs B) --- tractor/msg.py | 56 +++++++++++++++++++++++++++++++++++--------------- 1 file changed, 40 insertions(+), 16 deletions(-) diff --git a/tractor/msg.py b/tractor/msg.py index 9af3ccd7..ca34dba8 100644 --- a/tractor/msg.py +++ b/tractor/msg.py @@ -43,38 +43,62 @@ Built-in messaging patterns, types, APIs and helpers. # - https://github.com/msgpack/msgpack-python#packingunpacking-of-custom-data-type from __future__ import annotations +from inspect import isfunction from pkgutil import resolve_name class NamespacePath(str): ''' - A serializeable description of a (function) Python object location - described by the target's module path and namespace key meant as - a message-native "packet" to allows actors to point-and-load objects - by absolute reference. + A serializeable description of a (function) Python object + location described by the target's module path and namespace + key meant as a message-native "packet" to allows actors to + point-and-load objects by an absolute ``str`` (and thus + serializable) reference. ''' - _ref: object = None + _ref: object | type | None = None - def load_ref(self) -> object: + def load_ref(self) -> object | type: if self._ref is None: self._ref = resolve_name(self) return self._ref - def to_tuple( - self, + @staticmethod + def _mk_fqnp(ref: type | object) -> tuple[str, str]: + ''' + Generate a minial ``str`` pair which describes a python + object's namespace path and object/type name. - ) -> tuple[str, str]: - ref = self.load_ref() - return ref.__module__, getattr(ref, '__name__', '') + In more precise terms something like: + - 'py.namespace.path:object_name', + - eg.'tractor.msg:NamespacePath' will be the ``str`` form + of THIS type XD + + ''' + if ( + isinstance(ref, object) + and not isfunction(ref) + ): + name: str = type(ref).__name__ + else: + name: str = getattr(ref, '__name__') + + # fully qualified namespace path, tuple. + fqnp: tuple[str, str] = ( + ref.__module__, + name, + ) + return fqnp @classmethod def from_ref( cls, - ref, + ref: type | object, ) -> NamespacePath: - return cls(':'.join( - (ref.__module__, - getattr(ref, '__name__', '')) - )) + + fqnp: tuple[str, str] = cls._mk_fqnp(ref) + return cls(':'.join(fqnp)) + + def to_tuple(self) -> tuple[str, str]: + return self._mk_fqnp(self.load_ref()) From ee94d6d62c3e8bbb67c934aee123d85cf8be41b5 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 18 Aug 2023 10:10:36 -0400 Subject: [PATCH 032/146] Teensie tidy up on actor doc string --- tractor/_runtime.py | 23 ++++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 08ddabc4..0337b2ea 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -448,17 +448,18 @@ class Actor: (swappable) network protocols. - 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, depth hierarchy)) is able - to do so via an "address", which maps IPC connections across memory - boundaries, and 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. + 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, + 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. ''' # ugh, we need to get rid of this and replace with a "registry" sys From cff69d07fe9905a3fc9a9bf471fc80d344431646 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 27 Sep 2023 14:05:22 -0400 Subject: [PATCH 033/146] Mk `gather_contexts()` support `@acm`s yielding `None` We were using a `all()` condition which obviously won't work if the batched managers yield any non-truthy value. So instead see the `unwrapped: dict` with the `id(mngrs)` and only unblock once all values have been filled in to be something that is not that value. --- tractor/trionics/_mngrs.py | 26 +++++++++++++++++++++----- 1 file changed, 21 insertions(+), 5 deletions(-) diff --git a/tractor/trionics/_mngrs.py b/tractor/trionics/_mngrs.py index 89db895b..801b138b 100644 --- a/tractor/trionics/_mngrs.py +++ b/tractor/trionics/_mngrs.py @@ -70,6 +70,7 @@ async def _enter_and_wait( unwrapped: dict[int, T], all_entered: trio.Event, parent_exit: trio.Event, + seed: int, ) -> None: ''' @@ -80,7 +81,10 @@ async def _enter_and_wait( async with mngr as value: unwrapped[id(mngr)] = value - if all(unwrapped.values()): + if all( + val != seed + for val in unwrapped.values() + ): all_entered.set() await parent_exit.wait() @@ -91,7 +95,13 @@ async def gather_contexts( mngrs: Sequence[AsyncContextManager[T]], -) -> AsyncGenerator[tuple[Optional[T], ...], None]: +) -> AsyncGenerator[ + tuple[ + T | None, + ... + ], + None, +]: ''' Concurrently enter a sequence of async context managers, each in a separate ``trio`` task and deliver the unwrapped values in the @@ -104,7 +114,11 @@ async def gather_contexts( entered and exited, and cancellation just works. ''' - unwrapped: dict[int, Optional[T]] = {}.fromkeys(id(mngr) for mngr in mngrs) + seed: int = id(mngrs) + unwrapped: dict[int, T | None] = {}.fromkeys( + (id(mngr) for mngr in mngrs), + seed, + ) all_entered = trio.Event() parent_exit = trio.Event() @@ -116,8 +130,9 @@ async def gather_contexts( if not mngrs: raise ValueError( - 'input mngrs is empty?\n' - 'Did try to use inline generator syntax?' + '`.trionics.gather_contexts()` input mngrs is empty?\n' + 'Did try to use inline generator syntax?\n' + 'Use a non-lazy iterator or sequence type intead!' ) async with trio.open_nursery() as n: @@ -128,6 +143,7 @@ async def gather_contexts( unwrapped, all_entered, parent_exit, + seed, ) # deliver control once all managers have started up From 51fdf3524cf682881eb8966585299a03548149cf Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 5 Oct 2023 19:45:46 -0400 Subject: [PATCH 034/146] Start inter-peer cancellation test mod Move over relevant test from the "context semantics" test module which was already verifying peer-caused-`ContextCancelled.canceller: tuple` error info and propagation during an inter-peer cancellation scenario. Also begin a more general set of inter-peer cancellation tests starting with the simplest case where when a peer is cancelled the parent should NOT get an "muted" `trio.Cancelled` and instead a `tractor.ContextCancelled` with a `.canceller: tuple` which points to the sibling actor which requested the peer cancel. --- tests/test_context_stream_semantics.py | 94 +---------- tests/test_inter_peer_cancellation.py | 209 +++++++++++++++++++++++++ 2 files changed, 211 insertions(+), 92 deletions(-) create mode 100644 tests/test_inter_peer_cancellation.py diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index 4efc6319..a0d291d7 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -1,8 +1,8 @@ ''' ``async with ():`` inlined context-stream cancellation testing. -Verify the we raise errors when streams are opened prior to sync-opening -a ``tractor.Context`` beforehand. +Verify the we raise errors when streams are opened prior to +sync-opening a ``tractor.Context`` beforehand. ''' from contextlib import asynccontextmanager as acm @@ -922,93 +922,3 @@ def test_maybe_allow_overruns_stream( # if this hits the logic blocks from above are not # exhaustive.. pytest.fail('PARAMETRIZED CASE GEN PROBLEM YO') - - -@tractor.context -async def sleep_forever( - ctx: tractor.Context, -) -> None: - await ctx.started() - async with ctx.open_stream(): - await trio.sleep_forever() - - -@acm -async def attach_to_sleep_forever(): - ''' - Cancel a context **before** any underlying error is raised in order - to trigger a local reception of a ``ContextCancelled`` which **should not** - be re-raised in the local surrounding ``Context`` *iff* the cancel was - requested by **this** side of the context. - - ''' - async with tractor.wait_for_actor('sleeper') as p2: - async with ( - p2.open_context(sleep_forever) as (peer_ctx, first), - peer_ctx.open_stream(), - ): - try: - yield - finally: - # XXX: previously this would trigger local - # ``ContextCancelled`` to be received and raised in the - # local context overriding any local error due to logic - # inside ``_invoke()`` which checked for an error set on - # ``Context._error`` and raised it in a cancellation - # scenario. - # ------ - # The problem is you can have a remote cancellation that - # is part of a local error and we shouldn't raise - # ``ContextCancelled`` **iff** we **were not** the side - # of the context to initiate it, i.e. - # ``Context._cancel_called`` should **NOT** have been - # set. The special logic to handle this case is now - # inside ``Context._maybe_raise_from_remote_msg()`` XD - await peer_ctx.cancel() - - -@tractor.context -async def error_before_started( - ctx: tractor.Context, -) -> None: - ''' - This simulates exactly an original bug discovered in: - https://github.com/pikers/piker/issues/244 - - ''' - async with attach_to_sleep_forever(): - # send an unserializable type which should raise a type error - # here and **NOT BE SWALLOWED** by the surrounding acm!!?! - await ctx.started(object()) - - -def test_do_not_swallow_error_before_started_by_remote_contextcancelled(): - ''' - Verify that an error raised in a remote context which itself opens - another remote context, which it cancels, does not ovverride the - original error that caused the cancellation of the secondardy - context. - - ''' - async def main(): - async with tractor.open_nursery() as n: - portal = await n.start_actor( - 'errorer', - enable_modules=[__name__], - ) - await n.start_actor( - 'sleeper', - enable_modules=[__name__], - ) - - async with ( - portal.open_context( - error_before_started - ) as (ctx, sent), - ): - await trio.sleep_forever() - - with pytest.raises(tractor.RemoteActorError) as excinfo: - trio.run(main) - - assert excinfo.value.type == TypeError diff --git a/tests/test_inter_peer_cancellation.py b/tests/test_inter_peer_cancellation.py new file mode 100644 index 00000000..bf93372b --- /dev/null +++ b/tests/test_inter_peer_cancellation.py @@ -0,0 +1,209 @@ +''' +Codify the cancellation request semantics in terms +of one remote actor cancelling another. + +''' +from contextlib import asynccontextmanager as acm + +import pytest +import trio +import tractor +from tractor._exceptions import ( + StreamOverrun, + ContextCancelled, +) + + +def test_self_cancel(): + ''' + 2 cases: + - calls `Actor.cancel()` locally in some task + - calls LocalPortal.cancel_actor()` ? + + ''' + ... + + +@tractor.context +async def sleep_forever( + ctx: tractor.Context, +) -> None: + ''' + Sync the context, open a stream then just sleep. + + ''' + await ctx.started() + async with ctx.open_stream(): + await trio.sleep_forever() + + +@acm +async def attach_to_sleep_forever(): + ''' + Cancel a context **before** any underlying error is raised in order + to trigger a local reception of a ``ContextCancelled`` which **should not** + be re-raised in the local surrounding ``Context`` *iff* the cancel was + requested by **this** side of the context. + + ''' + async with tractor.wait_for_actor('sleeper') as p2: + async with ( + p2.open_context(sleep_forever) as (peer_ctx, first), + peer_ctx.open_stream(), + ): + try: + yield + finally: + # XXX: previously this would trigger local + # ``ContextCancelled`` to be received and raised in the + # local context overriding any local error due to logic + # inside ``_invoke()`` which checked for an error set on + # ``Context._error`` and raised it in a cancellation + # scenario. + # ------ + # The problem is you can have a remote cancellation that + # is part of a local error and we shouldn't raise + # ``ContextCancelled`` **iff** we **were not** the side + # of the context to initiate it, i.e. + # ``Context._cancel_called`` should **NOT** have been + # set. The special logic to handle this case is now + # inside ``Context._maybe_raise_from_remote_msg()`` XD + await peer_ctx.cancel() + + +@tractor.context +async def error_before_started( + ctx: tractor.Context, +) -> None: + ''' + This simulates exactly an original bug discovered in: + https://github.com/pikers/piker/issues/244 + + ''' + async with attach_to_sleep_forever(): + + # XXX NOTE XXX: THIS sends an UNSERIALIZABLE TYPE which + # should raise a `TypeError` and **NOT BE SWALLOWED** by + # the surrounding acm!!?! + await ctx.started(object()) + + +def test_do_not_swallow_error_before_started_by_remote_contextcancelled(): + ''' + Verify that an error raised in a remote context which itself + opens YET ANOTHER remote context, which it then cancels, does not + override the original error that caused the cancellation of the + secondary context. + + ''' + async def main(): + async with tractor.open_nursery() as n: + portal = await n.start_actor( + 'errorer', + enable_modules=[__name__], + ) + await n.start_actor( + 'sleeper', + enable_modules=[__name__], + ) + + async with ( + portal.open_context( + error_before_started + ) as (ctx, sent), + ): + await trio.sleep_forever() + + with pytest.raises(tractor.RemoteActorError) as excinfo: + trio.run(main) + + assert excinfo.value.type == TypeError + + +@tractor.context +async def sleep_a_bit_then_cancel_sleeper( + ctx: tractor.Context, +) -> None: + async with tractor.wait_for_actor('sleeper') as sleeper: + await ctx.started() + # await trio.sleep_forever() + await trio.sleep(3) + # async with tractor.wait_for_actor('sleeper') as sleeper: + await sleeper.cancel_actor() + + +def test_peer_canceller(): + ''' + Verify that a cancellation triggered by a peer (whether in tree + or not) results in a cancelled error with + a `ContextCancelled.errorer` matching the requesting actor. + + cases: + - some arbitrary remote peer cancels via Portal.cancel_actor(). + => all other connected peers should get that cancel requesting peer's + uid in the ctx-cancelled error msg. + + - peer spawned a sub-actor which (also) spawned a failing task + which was unhandled and propagated up to the immediate + parent, the peer to the actor that also spawned a remote task + task in that same peer-parent. + + - peer cancelled itself - so other peers should + get errors reflecting that the peer was itself the .canceller? + + - WE cancelled the peer and thus should not see any raised + `ContextCancelled` as it should be reaped silently? + => pretty sure `test_context_stream_semantics::test_caller_cancels()` + already covers this case? + + ''' + + async def main(): + async with tractor.open_nursery() as n: + canceller: tractor.Portal = await n.start_actor( + 'canceller', + enable_modules=[__name__], + ) + sleeper: tractor.Portal = await n.start_actor( + 'sleeper', + enable_modules=[__name__], + ) + + async with ( + sleeper.open_context( + sleep_forever, + ) as (sleeper_ctx, sent), + + canceller.open_context( + sleep_a_bit_then_cancel_sleeper, + ) as (canceller_ctx, sent), + ): + # await tractor.pause() + try: + print('PRE CONTEXT RESULT') + await sleeper_ctx.result() + + # TODO: not sure why this isn't catching + # but maybe we need an `ExceptionGroup` and + # the whole except *errs: thinger in 3.11? + except ( + ContextCancelled, + ) as berr: + print('CAUGHT REMOTE CONTEXT CANCEL') + + # canceller should not have been remotely + # cancelled. + assert canceller_ctx.cancel_called_remote is None + assert sleeper_ctx.canceller == 'canceller' + await tractor.pause(shield=True) + assert not sleep_ctx.cancelled_caught + + raise + else: + raise RuntimeError('NEVER RXED EXPECTED `ContextCancelled`') + + + with pytest.raises(tractor.ContextCancelled) as excinfo: + trio.run(main) + + assert excinfo.value.type == ContextCancelled From 144d1f4d9479e54f863cfea3c5f42aaf12136535 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 7 Oct 2023 18:51:03 -0400 Subject: [PATCH 035/146] Msg-ified `ContextCancelled`s sub-error type should always be just, its type.. --- tractor/_exceptions.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 6da2e657..9de27bdf 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -39,8 +39,11 @@ class ActorFailure(Exception): class RemoteActorError(Exception): + ''' + Remote actor exception bundled locally + + ''' # TODO: local recontruction of remote exception deats - "Remote actor exception bundled locally" def __init__( self, message: str, @@ -149,13 +152,13 @@ def unpack_error( error = msg['error'] tb_str = error.get('tb_str', '') - message = f"{chan.uid}\n" + tb_str + message = f'{chan.uid}\n' + tb_str type_name = error['type_str'] suberror_type: Type[BaseException] = Exception if type_name == 'ContextCancelled': err_type = ContextCancelled - suberror_type = RemoteActorError + suberror_type = err_type else: # try to lookup a suitable local error type for ns in [ From 81776a6238c0fd73bf882c6b8d1e84ec4b1500ab Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 7 Oct 2023 18:51:59 -0400 Subject: [PATCH 036/146] Drop pause line from ctx cancel handler block in test --- tests/test_inter_peer_cancellation.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/test_inter_peer_cancellation.py b/tests/test_inter_peer_cancellation.py index bf93372b..aeb70e81 100644 --- a/tests/test_inter_peer_cancellation.py +++ b/tests/test_inter_peer_cancellation.py @@ -194,8 +194,12 @@ def test_peer_canceller(): # canceller should not have been remotely # cancelled. assert canceller_ctx.cancel_called_remote is None + + # NOTE: will only enter if you wrap in + # a shielded cs.. + # await tractor.pause() # TODO: shield=True) + assert sleeper_ctx.canceller == 'canceller' - await tractor.pause(shield=True) assert not sleep_ctx.cancelled_caught raise From fa9b57bae08b6eafdfb6321927559103261b0853 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 8 Oct 2023 15:57:18 -0400 Subject: [PATCH 037/146] Write more comprehensive `Portal.cancel_actor()` doc str --- tractor/_portal.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/tractor/_portal.py b/tractor/_portal.py index 60293716..7ad4eb85 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -191,7 +191,15 @@ class Portal: ) -> bool: ''' - Cancel the actor on the other end of this portal. + Cancel the actor runtime (and thus process) on the far + end of this portal. + + **NOTE** THIS CANCELS THE ENTIRE RUNTIME AND THE + SUBPROCESS, it DOES NOT just cancel the remote task. If you + want to have a handle to cancel a remote ``tri.Task`` look + at `.open_context()` and the definition of + `._context.Context.cancel()` which CAN be used for this + purpose. ''' if not self.channel.connected(): From 6d10f0c5162cc1f922c0f07a8c8e0da9747dc8d5 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 10 Oct 2023 09:45:49 -0400 Subject: [PATCH 038/146] Always raise remote (cancelled) error if set Previously we weren't raising a remote error if the local scope was cancelled during a call to `Context.result()` which is problematic if the caller WAS NOT the requester for said remote cancellation; in that case we still want a `ContextCancelled` raised with the `.canceller: str` set to the cancelling actor uid. Further fix a naming bug where the (seemingly older) `._remote_err` was being set to such an error instead of `._remote_error` XD --- tractor/_context.py | 35 +++++++++++++++++++++++++---------- 1 file changed, 25 insertions(+), 10 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index 054f7832..ac28e56e 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -102,10 +102,14 @@ class Context: _remote_error: BaseException | None = None # cancellation state - _cancel_called: bool = False - _cancelled_remote: tuple | None = None + _cancel_called: bool = False # did WE cancel the far end? + _cancelled_remote: tuple[str, str] | None = None _cancel_msg: str | None = None _scope: trio.CancelScope | None = None + + # NOTE: this is set by the `.devx._debug` machinery + # to indicate whether code in `._runtime` should handle + # cancelled context crashes in the pdbp REPL. _enter_debugger_on_cancel: bool = True @property @@ -207,7 +211,7 @@ class Context: # XXX: set the remote side's error so that after we cancel # whatever task is the opener of this context it can raise # that error as the reason. - self._remote_error = error + self._remote_error: BaseException = error # always record the remote actor's uid since its cancellation # state is directly linked to ours (the local one). @@ -488,11 +492,7 @@ class Context: assert self._portal, "Context.result() can not be called from callee!" assert self._recv_chan - # from . import _debug - # await _debug.breakpoint() - - re = self._remote_error - if re: + if re := self._remote_error: self._maybe_raise_remote_err(re) return re @@ -507,7 +507,7 @@ class Context: while True: msg = await self._recv_chan.receive() try: - self._result = msg['return'] + self._result: Any = msg['return'] # NOTE: we don't need to do this right? # XXX: only close the rx mem chan AFTER @@ -516,6 +516,21 @@ class Context: # await self._recv_chan.aclose() break + + # NOTE: we get here if the far end was + # `ContextCancelled` in 2 cases: + # - we requested the cancellation and thus + # SHOULD NOT raise that far end error, + # - WE DID NOT REQUEST that cancel and thus + # SHOULD RAISE HERE! + except trio.Cancelled: + if not self._cancel_called: + raise self._remote_error + else: + # if we DID request the cancel we simply + # continue as normal. + raise + except KeyError: # as msgerr: if 'yield' in msg: @@ -537,7 +552,7 @@ class Context: ) # from msgerr err = self._maybe_raise_remote_err(err) - self._remote_err = err + self._remote_error = err return self._remote_error or self._result From d4d09b60712f57bc820d5a75a785a32c936548ed Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 16 Oct 2023 15:35:16 -0400 Subject: [PATCH 039/146] Factor non-yield stream msg processing into helper Since both `MsgStream.receive()` and `.receive_nowait()` need the same raising logic when a non-stream msg arrives (so that maybe an appropriate IPC translated error can be raised) move the `KeyError` handler code into a new `._streaming._raise_from_no_yield_msg()` func and call it from both methods to make the error-interface-raising symmetrical across both methods. --- tractor/_streaming.py | 112 +++++++++++++++++++++++++++--------------- 1 file changed, 72 insertions(+), 40 deletions(-) diff --git a/tractor/_streaming.py b/tractor/_streaming.py index 3045b835..e449fefe 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -54,6 +54,60 @@ log = get_logger(__name__) # messages? class ReceiveChannel(AsyncResource, Generic[ReceiveType]): # - use __slots__ on ``Context``? +def _raise_from_no_yield_msg( + stream: MsgStream, + msg: dict, + src_err: KeyError, + +) -> bool: + ''' + Raise an appopriate local error when a `MsgStream` msg arrives + which does not contain the expected (under normal operation) + `'yield'` field. + + ''' + # internal error should never get here + assert msg.get('cid'), ("Received internal error at portal?") + + # TODO: handle 2 cases with 3.10+ match syntax + # - 'stop' + # - 'error' + # possibly just handle msg['stop'] here! + + if stream._closed: + raise trio.ClosedResourceError('This stream was closed') + + if msg.get('stop') or stream._eoc: + log.debug(f"{stream} was stopped at remote end") + + # XXX: important to set so that a new ``.receive()`` + # call (likely by another task using a broadcast receiver) + # doesn't accidentally pull the ``return`` message + # value out of the underlying feed mem chan! + stream._eoc = True + + # # when the send is closed we assume the stream has + # # terminated and signal this local iterator to stop + # await stream.aclose() + + # XXX: this causes ``ReceiveChannel.__anext__()`` to + # raise a ``StopAsyncIteration`` **and** in our catch + # block below it will trigger ``.aclose()``. + raise trio.EndOfChannel from src_err + + # TODO: test that shows stream raising an expected error!!! + elif msg.get('error'): + # raise the error message + raise unpack_error(msg, stream._ctx.chan) + + # always re-raise the source error if no translation error + # case is activated above. + raise src_err + # raise RuntimeError( + # 'Unknown non-yield stream msg?\n' + # f'{msg}' + # ) + class MsgStream(trio.abc.Channel): ''' @@ -91,11 +145,20 @@ class MsgStream(trio.abc.Channel): # delegate directly to underlying mem channel def receive_nowait(self): msg = self._rx_chan.receive_nowait() - return msg['yield'] + try: + return msg['yield'] + except KeyError as kerr: + _raise_from_no_yield_msg( + stream=self, + msg=msg, + src_err=kerr, + ) async def receive(self): - '''Async receive a single msg from the IPC transport, the next - in sequence for this stream. + ''' + Receive a single msg from the IPC transport, the next in + sequence sent by the far end task (possibly in order as + determined by the underlying protocol). ''' # see ``.aclose()`` for notes on the old behaviour prior to @@ -110,43 +173,12 @@ class MsgStream(trio.abc.Channel): msg = await self._rx_chan.receive() return msg['yield'] - except KeyError as err: - # internal error should never get here - assert msg.get('cid'), ("Received internal error at portal?") - - # TODO: handle 2 cases with 3.10 match syntax - # - 'stop' - # - 'error' - # possibly just handle msg['stop'] here! - - if self._closed: - raise trio.ClosedResourceError('This stream was closed') - - if msg.get('stop') or self._eoc: - log.debug(f"{self} was stopped at remote end") - - # XXX: important to set so that a new ``.receive()`` - # call (likely by another task using a broadcast receiver) - # doesn't accidentally pull the ``return`` message - # value out of the underlying feed mem chan! - self._eoc = True - - # # when the send is closed we assume the stream has - # # terminated and signal this local iterator to stop - # await self.aclose() - - # XXX: this causes ``ReceiveChannel.__anext__()`` to - # raise a ``StopAsyncIteration`` **and** in our catch - # block below it will trigger ``.aclose()``. - raise trio.EndOfChannel from err - - # TODO: test that shows stream raising an expected error!!! - elif msg.get('error'): - # raise the error message - raise unpack_error(msg, self._ctx.chan) - - else: - raise + except KeyError as kerr: + _raise_from_no_yield_msg( + stream=self, + msg=msg, + src_err=kerr, + ) except ( trio.ClosedResourceError, # by self._rx_chan From a18663213aaf04f0cfe33548435ee975e2a4bd89 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 16 Oct 2023 15:45:02 -0400 Subject: [PATCH 040/146] Add comments around diff between `C/context` refs --- tractor/__init__.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tractor/__init__.py b/tractor/__init__.py index aa262105..64c81e99 100644 --- a/tractor/__init__.py +++ b/tractor/__init__.py @@ -23,8 +23,8 @@ from exceptiongroup import BaseExceptionGroup from ._clustering import open_actor_cluster from ._ipc import Channel from ._context import ( - Context, - context, + Context, # the type + context, # a func-decorator ) from ._streaming import ( MsgStream, From 11bbf15817980f69aea39d0742e5e1fb8f32a1a8 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 16 Oct 2023 16:23:30 -0400 Subject: [PATCH 041/146] `._exceptions`: typing and error unpacking updates Bump type annotations to 3.10+ style throughout module as well as fill out doc strings a bit. Inside `unpack_error()` pop any `error_dict: dict` and, - return `None` early if not found, - versus pass directly as `**error_dict` to the error constructor instead of a double field read. --- tractor/_exceptions.py | 53 ++++++++++++++++++++++++++++-------------- 1 file changed, 35 insertions(+), 18 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 9de27bdf..d9e1d17f 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -113,18 +113,24 @@ class AsyncioCancelled(Exception): def pack_error( exc: BaseException, - tb=None, + tb: str | None = None, -) -> dict[str, Any]: - """Create an "error message" for tranmission over - a channel (aka the wire). - """ +) -> dict[str, dict]: + ''' + Create an "error message" encoded for wire transport via an IPC + `Channel`; expected to be unpacked on the receiver side using + `unpack_error()` below. + + ''' if tb: tb_str = ''.join(traceback.format_tb(tb)) else: tb_str = traceback.format_exc() - error_msg = { + error_msg: dict[ + str, + str | tuple[str, str] + ] = { 'tb_str': tb_str, 'type_str': type(exc).__name__, 'src_actor_uid': current_actor().uid, @@ -142,18 +148,28 @@ def unpack_error( chan=None, err_type=RemoteActorError -) -> Exception: +) -> None | Exception: ''' Unpack an 'error' message from the wire - into a local ``RemoteActorError``. + into a local `RemoteActorError` (subtype). + + NOTE: this routine DOES not RAISE the embedded remote error, + which is the responsibilitiy of the caller. ''' - __tracebackhide__ = True - error = msg['error'] + __tracebackhide__: bool = True - tb_str = error.get('tb_str', '') - message = f'{chan.uid}\n' + tb_str - type_name = error['type_str'] + error_dict: dict[str, dict] | None + if ( + error_dict := msg.get('error') + ) is None: + # no error field, nothing to unpack. + return None + + # retrieve the remote error's msg encoded details + tb_str: str = error_dict.get('tb_str', '') + message: str = f'{chan.uid}\n' + tb_str + type_name: str = error_dict['type_str'] suberror_type: Type[BaseException] = Exception if type_name == 'ContextCancelled': @@ -167,18 +183,19 @@ def unpack_error( eg, trio, ]: - try: - suberror_type = getattr(ns, type_name) + if suberror_type := getattr( + ns, + type_name, + False, + ): break - except AttributeError: - continue exc = err_type( message, suberror_type=suberror_type, # unpack other fields into error type init - **msg['error'], + **error_dict, ) return exc From 89ed8b67ff5f0d1cf9529b5e20707d91ddf25087 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 17 Oct 2023 15:30:16 -0400 Subject: [PATCH 042/146] Drop `msg` kwarg from `Context.cancel()` Well first off, turns out it's never used and generally speaking doesn't seem to help much with "runtime hacking/debugging"; why would we need to "fabricate" a msg when `.cancel()` is called to self-cancel? Also (and since `._maybe_cancel_and_set_remote_error()` now takes an `error: BaseException` as input and thus expects error-msg unpacking prior to being called), we now manually set `Context._cancel_msg: dict` just prior to any remote error assignment - so any case where we would have fabbed a "cancel msg" near calling `.cancel()`, just do the manual assign. In this vein some other subtle changes: - obviously don't set `._cancel_msg` in `.cancel()` since it's no longer an input. - generally do walrus-style `error := unpack_error()` before applying and setting remote error-msg state. - always raise any `._remote_error` in `.result()` instead of returning the exception instance and check before AND after the underlying mem chan read. - add notes/todos around `raise self._remote_error from None` masking of (runtime) errors in `._maybe_raise_remote_err()` and use it inside `.result()` since we had the inverse duplicate logic there anyway.. Further, this adds and extends a ton of (internal) interface docs and details comments around the `Context` API including many subtleties pertaining to calling `._maybe_cancel_and_set_remote_error()`. --- tractor/_context.py | 288 ++++++++++++++++++++++++++++---------------- 1 file changed, 183 insertions(+), 105 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index ac28e56e..0aadec49 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -86,30 +86,51 @@ class Context: ''' chan: Channel - cid: str + cid: str # "context id", more or less a unique linked-task-pair id - # these are the "feeder" channels for delivering - # message values to the local task from the runtime - # msg processing loop. + # the "feeder" channels for delivering message values to the + # local task from the runtime's msg processing loop. _recv_chan: trio.MemoryReceiveChannel _send_chan: trio.MemorySendChannel + # the "invocation type" of the far end task-entry-point + # function, normally matching a logic block inside + # `._runtime.invoke()`. _remote_func_type: str | None = None - # only set on the caller side - _portal: Portal | None = None # type: ignore # noqa + # NOTE: (for now) only set (a portal) on the caller side since + # the callee doesn't generally need a ref to one and should + # normally need to explicitly ask for handle to its peer if + # more the the `Context` is needed? + _portal: Portal | None = None + + # 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. + _scope: trio.CancelScope | None = None _result: Any | int = None _remote_error: BaseException | None = None # cancellation state _cancel_called: bool = False # did WE cancel the far end? _cancelled_remote: tuple[str, str] | None = None - _cancel_msg: str | None = None - _scope: trio.CancelScope | None = None - # NOTE: this is set by the `.devx._debug` machinery - # to indicate whether code in `._runtime` should handle - # cancelled context crashes in the pdbp REPL. + # NOTE: we try to ensure assignment of a "cancel msg" since + # there's always going to be an "underlying reason" that any + # context was closed due to either a remote side error or + # a call to `.cancel()` which triggers `ContextCancelled`. + _cancel_msg: str | dict | None = None + + # NOTE: this state var used by the runtime to determine if the + # `pdbp` REPL is allowed to engage on contexts terminated via + # a `ContextCancelled` due to a call to `.cancel()` triggering + # "graceful closure" on either side: + # - `._runtime._invoke()` will check this flag before engaging + # the crash handler REPL in such cases where the "callee" + # raises the cancellation, + # - `.devx._debug.lock_tty_for_child()` will set it to `False` if + # the global tty-lock has been configured to filter out some + # actors from being able to acquire the debugger lock. _enter_debugger_on_cancel: bool = True @property @@ -177,36 +198,71 @@ class Context: async def _maybe_cancel_and_set_remote_error( self, - error_msg: dict[str, Any], + error: BaseException, ) -> None: ''' - (Maybe) unpack and raise a msg error into the local scope - nursery for this context. + (Maybe) cancel this local scope due to a received remote + error (normally via an IPC msg) which the actor runtime + routes to this context. - Acts as a form of "relay" for a remote error raised - in the corresponding remote callee task. + Acts as a form of "relay" for a remote error raised in the + corresponding remote task's `Context` wherein the next time + the local task exectutes a checkpoint, a `trio.Cancelled` + will be raised and depending on the type and source of the + original remote error, and whether or not the local task + called `.cancel()` itself prior, an equivalent + `ContextCancelled` or `RemoteActorError` wrapping the + remote error may be raised here by any of, + + - `Portal.open_context()` + - `Portal.result()` + - `Context.open_stream()` + - `Context.result()` + + when called/closed by actor local task(s). + + NOTEs & TODOs: + - It is expected that the caller has previously unwrapped + the remote error using a call to `unpack_error()` and + provides that output exception value as the input + `error` argument here. + - If this is an error message from a context opened by + `Portal.open_context()` we want to interrupt any + ongoing local tasks operating within that `Context`'s + cancel-scope so as to be notified ASAP of the remote + error and engage any caller handling (eg. for + cross-process task supervision). + - In some cases we may want to raise the remote error + immediately since there is no guarantee the locally + operating task(s) will attempt to execute a checkpoint + any time soon; in such cases there are 2 possible + approaches depending on the current task's work and + wrapping "thread" type: + + - `trio`-native-and-graceful: only ever wait for tasks + to exec a next `trio.lowlevel.checkpoint()` assuming + that any such task must do so to interact with the + actor runtime and IPC interfaces. + + - (NOT IMPLEMENTED) system-level-aggressive: maybe we + could eventually interrupt sync code (invoked using + `trio.to_thread` or some other adapter layer) with + a signal (a custom unix one for example? + https://stackoverflow.com/a/5744185) depending on the + task's wrapping thread-type such that long running + sync code should never cause the delay of actor + supervision tasks such as cancellation and respawn + logic. ''' - # If this is an error message from a context opened by - # ``Portal.open_context()`` we want to interrupt any ongoing - # (child) tasks within that context to be notified of the remote - # error relayed here. - # - # The reason we may want to raise the remote error immediately - # is that there is no guarantee the associated local task(s) - # will attempt to read from any locally opened stream any time - # soon. - # - # NOTE: this only applies when - # ``Portal.open_context()`` has been called since it is assumed - # (currently) that other portal APIs (``Portal.run()``, - # ``.run_in_actor()``) do their own error checking at the point - # of the call and result processing. - error = unpack_error( - error_msg, - self.chan, - ) + # XXX: currently this should only be used when + # `Portal.open_context()` has been opened since it's + # assumed that other portal APIs like, + # - `Portal.run()`, + # - `ActorNursery.run_in_actor()` + # do their own error checking at their own call points and + # result processing. # XXX: set the remote side's error so that after we cancel # whatever task is the opener of this context it can raise @@ -236,35 +292,25 @@ class Context: else: log.error( f'Remote context error for {self.chan.uid}:{self.cid}:\n' - f'{error_msg["error"]["tb_str"]}' + f'{error}' ) # TODO: tempted to **not** do this by-reraising in a # nursery and instead cancel a surrounding scope, detect # the cancellation, then lookup the error that was set? # YES! this is way better and simpler! - if ( - self._scope - ): + if self._scope: # from trio.testing import wait_all_tasks_blocked # await wait_all_tasks_blocked() # self._cancelled_remote = self.chan.uid self._scope.cancel() - # NOTE: this usage actually works here B) - # from ._debug import breakpoint - # await breakpoint() - - # XXX: this will break early callee results sending - # since when `.result()` is finally called, this - # chan will be closed.. - # if self._recv_chan: - # await self._recv_chan.aclose() + # this REPL usage actually works here BD + # from .devx._debug import pause + # await pause() async def cancel( self, - msg: str | None = None, timeout: float = 0.616, - # timeout: float = 1000, ) -> None: ''' @@ -274,15 +320,12 @@ class Context: Timeout quickly in an attempt to sidestep 2-generals... ''' - side = 'caller' if self._portal else 'callee' - if msg: - assert side == 'callee', 'Only callee side can provide cancel msg' + side: str = 'caller' if self._portal else 'callee' + log.cancel( + f'Cancelling {side} side of context to {self.chan.uid}' + ) - log.cancel(f'Cancelling {side} side of context to {self.chan.uid}') - - self._cancel_called = True - # await _debug.breakpoint() - # breakpoint() + self._cancel_called: bool = True if side == 'caller': if not self._portal: @@ -290,12 +333,13 @@ class Context: "No portal found, this is likely a callee side context" ) - cid = self.cid + cid: str = self.cid with trio.move_on_after(timeout) as cs: cs.shield = True log.cancel( - f"Cancelling stream {cid} to " - f"{self._portal.channel.uid}") + f'Cancelling stream {cid} to ' + f'{self._portal.channel.uid}' + ) # NOTE: we're telling the far end actor to cancel a task # corresponding to *this actor*. The far end local channel @@ -314,17 +358,17 @@ class Context: # if not self._portal.channel.connected(): if not self.chan.connected(): log.cancel( - "May have failed to cancel remote task " - f"{cid} for {self._portal.channel.uid}") + 'May have failed to cancel remote task ' + f'{cid} for {self._portal.channel.uid}' + ) else: log.cancel( - "Timed out on cancelling remote task " - f"{cid} for {self._portal.channel.uid}") + 'Timed out on cancel request of remote task ' + f'{cid} for {self._portal.channel.uid}' + ) # callee side remote task else: - self._cancel_msg = msg - # TODO: should we have an explicit cancel message # or is relaying the local `trio.Cancelled` as an # {'error': trio.Cancelled, cid: "blah"} enough? @@ -335,7 +379,6 @@ class Context: @acm async def open_stream( - self, allow_overruns: bool | None = False, msg_buffer_size: int | None = None, @@ -354,10 +397,10 @@ class Context: ``Portal.open_context()``. In the future this may change but currently there seems to be no obvious reason to support "re-opening": - - pausing a stream can be done with a message. - - task errors will normally require a restart of the entire - scope of the inter-actor task context due to the nature of - ``trio``'s cancellation system. + - pausing a stream can be done with a message. + - task errors will normally require a restart of the entire + scope of the inter-actor task context due to the nature of + ``trio``'s cancellation system. ''' actor = current_actor() @@ -439,18 +482,19 @@ class Context: self, err: Exception, ) -> None: + ''' + Maybe raise a remote error depending on who (which task from + which actor) requested a cancellation (if any). + + ''' # NOTE: whenever the context's "opener" side (task) **is** # the side which requested the cancellation (likekly via # ``Context.cancel()``), we don't want to re-raise that # cancellation signal locally (would be akin to # a ``trio.Nursery`` nursery raising ``trio.Cancelled`` - # whenever ``CancelScope.cancel()`` was called) and instead - # silently reap the expected cancellation "error"-msg. - # if 'pikerd' in err.msgdata['tb_str']: - # # from . import _debug - # # await _debug.breakpoint() - # breakpoint() - + # whenever ``CancelScope.cancel()`` was called) and + # instead silently reap the expected cancellation + # "error"-msg. if ( isinstance(err, ContextCancelled) and ( @@ -461,7 +505,18 @@ class Context: ): return err - raise err # from None + # NOTE: currently we are masking underlying runtime errors + # which are often superfluous to user handler code. not + # sure if this is still needed / desired for all operation? + # TODO: maybe we can only NOT mask if: + # - [ ] debug mode is enabled or, + # - [ ] a certain log level is set? + # - [ ] consider using `.with_traceback()` to filter out + # runtime frames from the tb explicitly? + # https://docs.python.org/3/reference/simple_stmts.html#the-raise-statement + # https://stackoverflow.com/a/24752607 + __tracebackhide__: bool = True + raise err from None async def result(self) -> Any | Exception: ''' @@ -489,12 +544,12 @@ class Context: of the remote cancellation. ''' + __tracebackhide__: bool = True assert self._portal, "Context.result() can not be called from callee!" assert self._recv_chan if re := self._remote_error: - self._maybe_raise_remote_err(re) - return re + return self._maybe_raise_remote_err(re) if ( self._result == id(self) @@ -505,8 +560,8 @@ class Context: # and discarding any bi dir stream msgs still # in transit from the far end. while True: - msg = await self._recv_chan.receive() try: + msg = await self._recv_chan.receive() self._result: Any = msg['return'] # NOTE: we don't need to do this right? @@ -519,17 +574,22 @@ class Context: # NOTE: we get here if the far end was # `ContextCancelled` in 2 cases: - # - we requested the cancellation and thus - # SHOULD NOT raise that far end error, - # - WE DID NOT REQUEST that cancel and thus - # SHOULD RAISE HERE! + # 1. we requested the cancellation and thus + # SHOULD NOT raise that far end error, + # 2. WE DID NOT REQUEST that cancel and thus + # SHOULD RAISE HERE! except trio.Cancelled: - if not self._cancel_called: - raise self._remote_error - else: - # if we DID request the cancel we simply - # continue as normal. - raise + + # CASE 2: mask the local cancelled-error(s) + # only when we are sure the remote error is the + # (likely) source cause of this local runtime + # task's cancellation. + if re := self._remote_error: + self._maybe_raise_remote_err(re) + + # CASE 1: we DID request the cancel we simply + # continue to bubble up as normal. + raise except KeyError: # as msgerr: @@ -544,7 +604,8 @@ class Context: # internal error should never get here assert msg.get('cid'), ( - "Received internal error at portal?") + "Received internal error at portal?" + ) err = unpack_error( msg, @@ -554,7 +615,10 @@ class Context: err = self._maybe_raise_remote_err(err) self._remote_error = err - return self._remote_error or self._result + if re := self._remote_error: + return self._maybe_raise_remote_err(re) + + return self._result async def started( self, @@ -563,7 +627,7 @@ class Context: ) -> None: ''' Indicate to calling actor's task that this linked context - has started and send ``value`` to the other side. + has started and send ``value`` to the other side via IPC. On the calling side ``value`` is the second item delivered in the tuple returned by ``Portal.open_context()``. @@ -571,19 +635,17 @@ class Context: ''' if self._portal: raise RuntimeError( - f"Caller side context {self} can not call started!") + f'Caller side context {self} can not call started!' + ) elif self._started_called: raise RuntimeError( - f"called 'started' twice on context with {self.chan.uid}") + f'called `.started()` twice on context with {self.chan.uid}' + ) await self.chan.send({'started': value, 'cid': self.cid}) self._started_called = True - # TODO: do we need a restart api? - # async def restart(self) -> None: - # pass - async def _drain_overflows( self, ) -> None: @@ -638,10 +700,21 @@ class Context: self, msg: dict, - draining: bool = False, + # draining: bool = False, ) -> bool: + ''' + Deliver an IPC msg received from a transport-channel to + this context's underlying mem chan for handling by + user operating tasks; deliver a bool indicating whether the + msg was immediately sent. + If `._allow_overruns == True` (maybe) append the msg to an + "overflow queue" and start a "drainer task" (inside the + `._scope_nursery: trio.Nursery`) which ensures that such + messages are eventually sent if possible. + + ''' cid = self.cid chan = self.chan uid = chan.uid @@ -652,8 +725,12 @@ class Context: ) error = msg.get('error') - if error: - await self._maybe_cancel_and_set_remote_error(msg) + if error := unpack_error( + msg, + self.chan, + ): + self._cancel_msg = msg + await self._maybe_cancel_and_set_remote_error(error) if ( self._in_overrun @@ -685,6 +762,7 @@ class Context: # the sender; the main motivation is that using bp can block the # msg handling loop which calls into this method! except trio.WouldBlock: + # XXX: always push an error even if the local # receiver is in overrun state. # await self._maybe_cancel_and_set_remote_error(msg) From 22e4b324b1cbf9dacf9f26b2c972c8d1ecf8eddb Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 17 Oct 2023 17:22:57 -0400 Subject: [PATCH 043/146] Get mega-pedantic in `Portal.open_context()` Specifically in the `.__aexit__()` phase to ensure remote, runtime-internal, and locally raised error-during-cancelled-handling exceptions are NEVER masked by a local `ContextCancelled` or any exception group of `trio.Cancelled`s. Also adds a ton of details to doc strings including extreme detail surrounding the `ContextCancelled` raising cases and their processing inside `.open_context()`'s exception handler blocks. Details, details: - internal rename `err`/`_err` stuff to just be `scope_err` since it's effectively the error bubbled up from the context's surrounding (and cross-actor) "scope". - always shield `._recv_chan.aclose()` to avoid any `Cancelled` from masking the `scope_err` with a runtime related `trio.Cancelled`. - explicitly catch the specific set of `scope_err: BaseException` that we can reasonably expect to handle instead of the catch-all parent type including exception groups, cancels and KBIs. --- tractor/_portal.py | 238 +++++++++++++++++++++++++++++++++------------ 1 file changed, 177 insertions(+), 61 deletions(-) diff --git a/tractor/_portal.py b/tractor/_portal.py index 7ad4eb85..6a2ec27a 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -15,8 +15,12 @@ # along with this program. If not, see . ''' -Memory boundary "Portals": an API for structured -concurrency linked tasks running in disparate memory domains. +Memory "portal" contruct. + +"Memory portals" are both an API and set of IPC wrapping primitives +for managing structured concurrency "cancel-scope linked" tasks +running in disparate virtual memory domains - at least in different +OS processes, possibly on different (hardware) hosts. ''' from __future__ import annotations @@ -66,20 +70,21 @@ def _unwrap_msg( raise unpack_error(msg, channel) from None +# TODO: maybe move this to ._exceptions? class MessagingError(Exception): 'Some kind of unexpected SC messaging dialog issue' class Portal: ''' - A 'portal' to a(n) (remote) ``Actor``. + A 'portal' to a memory-domain-separated `Actor`. A portal is "opened" (and eventually closed) by one side of an inter-actor communication context. The side which opens the portal is equivalent to a "caller" in function parlance and usually is either the called actor's parent (in process tree hierarchy terms) or a client interested in scheduling work to be done remotely in a - far process. + process which has a separate (virtual) memory domain. The portal api allows the "caller" actor to invoke remote routines and receive results through an underlying ``tractor.Channel`` as @@ -89,9 +94,9 @@ class Portal: like having a "portal" between the seperate actor memory spaces. ''' - # the timeout for a remote cancel request sent to - # a(n) (peer) actor. - cancel_timeout = 0.5 + # global timeout for remote cancel requests sent to + # connected (peer) actors. + cancel_timeout: float = 0.5 def __init__(self, channel: Channel) -> None: self.channel = channel @@ -393,12 +398,32 @@ class Portal: ) -> AsyncGenerator[tuple[Context, Any], None]: ''' - Open an inter-actor task context. + Open an inter-actor "task context"; a remote task is + scheduled and cancel-scope-state-linked to a `trio.run()` across + memory boundaries in another actor's runtime. - This is a synchronous API which allows for deterministic - setup/teardown of a remote task. The yielded ``Context`` further - allows for opening bidirectional streams, explicit cancellation - and synchronized final result collection. See ``tractor.Context``. + This is an `@acm` API which allows for deterministic setup + and teardown of a remotely scheduled task in another remote + actor. Once opened, the 2 now "linked" tasks run completely + in parallel in each actor's runtime with their enclosing + `trio.CancelScope`s kept in a synced state wherein if + either side errors or cancels an equivalent error is + relayed to the other side via an SC-compat IPC protocol. + + The yielded `tuple` is a pair delivering a `tractor.Context` + and any first value "sent" by the "callee" task via a call + to `Context.started()`; this side of the + context does not unblock until the "callee" task calls + `.started()` in similar style to `trio.Nursery.start()`. + 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. + + The yielded ``Context`` instance further allows for opening + bidirectional streams, explicit cancellation and + structurred-concurrency-synchronized final result-msg + collection. See ``tractor.Context`` for more details. ''' # conduct target func method structural checks @@ -431,47 +456,52 @@ class Portal: ) assert ctx._remote_func_type == 'context' - msg = await ctx._recv_chan.receive() + msg: dict = await ctx._recv_chan.receive() try: # the "first" value here is delivered by the callee's # ``Context.started()`` call. first = msg['started'] - ctx._started_called = True + ctx._started_called: bool = True except KeyError: - assert msg.get('cid'), ("Received internal error at context?") + if not (cid := msg.get('cid')): + raise MessagingError( + 'Received internal error at context?\n' + 'No call-id (cid) in startup msg?' + ) if msg.get('error'): - # raise kerr from unpack_error(msg, self.channel) + # NOTE: mask the key error with the remote one raise unpack_error(msg, self.channel) from None else: raise MessagingError( - f'Context for {ctx.cid} was expecting a `started` message' - f' but received a non-error msg:\n{pformat(msg)}' + f'Context for {cid} was expecting a `started` message' + ' but received a non-error msg:\n' + f'{pformat(msg)}' ) - _err: BaseException | None = None ctx._portal: Portal = self - uid: tuple = self.channel.uid cid: str = ctx.cid - etype: Type[BaseException] | None = None - # deliver context instance and .started() msg value in enter - # tuple. + # placeholder for any exception raised in the runtime + # or by user tasks which cause this context's closure. + scope_err: BaseException | None = None try: async with trio.open_nursery() as nurse: - ctx._scope_nursery = nurse - ctx._scope = nurse.cancel_scope + ctx._scope_nursery: trio.Nursery = nurse + ctx._scope: trio.CancelScope = nurse.cancel_scope + # deliver context instance and .started() msg value + # in enter tuple. yield ctx, first - # when in allow_ovveruns mode there may be lingering - # overflow sender tasks remaining? + # when in allow_overruns mode there may be + # lingering overflow sender tasks remaining? if nurse.child_tasks: - # ensure we are in overrun state with - # ``._allow_overruns=True`` bc otherwise + # XXX: ensure we are in overrun state + # with ``._allow_overruns=True`` bc otherwise # there should be no tasks in this nursery! if ( not ctx._allow_overruns @@ -479,47 +509,72 @@ class Portal: ): raise RuntimeError( 'Context has sub-tasks but is ' - 'not in `allow_overruns=True` Mode!?' + 'not in `allow_overruns=True` mode!?' ) + + # ensure cancel of all overflow sender tasks + # started in the ctx nursery. ctx._scope.cancel() - except ContextCancelled as err: - _err = err + # XXX: (maybe) shield/mask context-cancellations that were + # initiated by any of the context's 2 tasks. There are + # subsequently 2 operating cases for a "graceful cancel" + # of a `Context`: + # + # 1.*this* side's task called `Context.cancel()`, in + # which case we mask the `ContextCancelled` from bubbling + # to the opener (much like how `trio.Nursery` swallows + # any `trio.Cancelled` bubbled by a call to + # `Nursery.cancel_scope.cancel()`) + # + # 2.*the other* side's (callee/spawned) task cancelled due + # to a self or peer cancellation request in which case we + # DO let the error bubble to the opener. + except ContextCancelled as ctxc: + scope_err = ctxc - # swallow and mask cross-actor task context cancels that - # were initiated by *this* side's task. + # CASE 1: this context was never cancelled + # via a local task's call to `Context.cancel()`. if not ctx._cancel_called: # XXX: this should NEVER happen! # from ._debug import breakpoint # await breakpoint() raise - # if the context was cancelled by client code - # then we don't need to raise since user code - # is expecting this and the block should exit. + # CASE 2: context was cancelled by local task calling + # `.cancel()`, we don't raise and the exit block should + # exit silently. else: - log.debug(f'Context {ctx} cancelled gracefully') + log.debug( + f'Context {ctx} cancelled gracefully with:\n' + f'{ctxc}' + ) except ( - BaseException, + # - a standard error in the caller/yieldee + Exception, - # more specifically, we need to handle these but not - # sure it's worth being pedantic: - # Exception, - # trio.Cancelled, - # KeyboardInterrupt, + # - a runtime teardown exception-group and/or + # cancellation request from a caller task. + BaseExceptionGroup, + trio.Cancelled, + KeyboardInterrupt, ) as err: - etype = type(err) + scope_err = err - # cancel ourselves on any error. + # XXX: request cancel of this context on any error. + # NOTE: `Context.cancel()` is conversely NOT called in + # the `ContextCancelled` "cancellation requested" case + # above. log.cancel( - 'Context cancelled for task, sending cancel request..\n' + 'Context cancelled for task due to\n' + f'{err}\n' + 'Sending cancel request..\n' f'task:{cid}\n' f'actor:{uid}' ) try: - await ctx.cancel() except trio.BrokenResourceError: log.warning( @@ -528,8 +583,9 @@ class Portal: f'actor:{uid}' ) - raise + raise # duh + # no scope error case else: if ctx.chan.connected(): log.info( @@ -537,10 +593,20 @@ class Portal: f'task: {cid}\n' f'actor: {uid}' ) + # XXX NOTE XXX: the below call to + # `Context.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 + # via a call to + # `Context._maybe_cancel_and_set_remote_error()` + # which IS SET any time the far end fails and + # causes "caller side" cancellation via + # a `ContextCancelled` here. result = await ctx.result() log.runtime( - f'Context {fn_name} returned ' - f'value from callee `{result}`' + f'Context {fn_name} returned value from callee:\n' + f'`{result}`' ) finally: @@ -548,22 +614,73 @@ class Portal: # operating *in* this scope to have survived # we tear down the runtime feeder chan last # to avoid premature stream clobbers. - if ctx._recv_chan is not None: - # should we encapsulate this in the context api? - await ctx._recv_chan.aclose() + rxchan: trio.ReceiveChannel = ctx._recv_chan + if ( + rxchan - if etype: + # maybe TODO: yes i know the below check is + # touching `trio` memchan internals..BUT, there are + # only a couple ways to avoid a `trio.Cancelled` + # bubbling from the `.aclose()` call below: + # + # - catch and mask it via the cancel-scope-shielded call + # as we are rn (manual and frowned upon) OR, + # - specially handle the case where `scope_err` is + # one of {`BaseExceptionGroup`, `trio.Cancelled`} + # and then presume that the `.aclose()` call will + # raise a `trio.Cancelled` and just don't call it + # in those cases.. + # + # that latter approach is more logic, LOC, and more + # convoluted so for now stick with the first + # psuedo-hack-workaround where we just try to avoid + # the shielded call as much as we can detect from + # the memchan's `._closed` state.. + # + # XXX MOTIVATION XXX-> we generally want to raise + # any underlying actor-runtime/internals error that + # surfaces from a bug in tractor itself so it can + # be easily detected/fixed AND, we also want to + # minimize noisy runtime tracebacks (normally due + # to the cross-actor linked task scope machinery + # teardown) displayed to user-code and instead only + # displaying `ContextCancelled` traces where the + # cause of crash/exit IS due to something in + # user/app code on either end of the context. + and not rxchan._closed + ): + # XXX NOTE XXX: and again as per above, we mask any + # `trio.Cancelled` raised here so as to NOT mask + # out any exception group or legit (remote) ctx + # error that sourced from the remote task or its + # runtime. + with trio.CancelScope(shield=True): + await ctx._recv_chan.aclose() + + # XXX: since we always (maybe) re-raise (and thus also + # mask runtime machinery related + # multi-`trio.Cancelled`s) any scope error which was + # the underlying cause of this context's exit, add + # different log msgs for each of the (2) cases. + if scope_err is not None: + etype: Type[BaseException] = type(scope_err) + + # CASE 2 if ctx._cancel_called: log.cancel( - f'Context {fn_name} cancelled by caller with\n{etype}' + f'Context {fn_name} cancelled by caller with\n' + f'{etype}' ) - elif _err is not None: + + # CASE 1 + else: log.cancel( - f'Context for task cancelled by callee with {etype}\n' + f'Context cancelled by callee with {etype}\n' f'target: `{fn_name}`\n' f'task:{cid}\n' f'actor:{uid}' ) + # XXX: (MEGA IMPORTANT) if this is a root opened process we # wait for any immediate child in debug before popping the # context from the runtime msg loop otherwise inside @@ -572,10 +689,9 @@ class Portal: # a "stop" msg for a stream), this can result in a deadlock # where the root is waiting on the lock to clear but the # child has already cleared it and clobbered IPC. - from ._debug import maybe_wait_for_debugger - await maybe_wait_for_debugger() - # remove the context from runtime tracking + # FINALLY, remove the context from runtime tracking and + # exit Bo self.actor._contexts.pop( (self.channel.uid, ctx.cid), None, From a7c36a9cbe438cdb69654685a3a60f095ebc7a5f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 18 Oct 2023 13:19:34 -0400 Subject: [PATCH 044/146] Tidy/clarify another `._runtime` comment --- tractor/_runtime.py | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 0337b2ea..067fd81e 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -347,14 +347,13 @@ async def _invoke( and ctx._enter_debugger_on_cancel ) ): - # XXX: is there any case where we'll want to debug IPC - # disconnects as a default? - # - # I can't think of a reason that inspecting - # this type of failure will be useful for respawns or - # recovery logic - the only case is some kind of strange bug - # in our transport layer itself? Going to keep this - # open ended for now. + # XXX QUESTION XXX: is there any case where we'll + # want to debug IPC disconnects as a default? + # => I can't think of a reason that inspecting this + # type of failure will be useful for respawns or + # recovery logic - the only case is some kind of + # strange bug in our transport layer itself? Going + # to keep this open ended for now. entered_debug = await _debug._maybe_enter_pm(err) if not entered_debug: From ca1a1476bb22eb61406705d4f719482d39c13c86 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 18 Oct 2023 13:59:08 -0400 Subject: [PATCH 045/146] Add a first serious inter-peer remote cancel suite Tests that appropriate `Context` exit state, the relay of a `ContextCancelled` error and its `.canceller: tuple[str, str]` value are set when an inter-peer cancellation happens via an "out of band" request method (in this case using `Portal.cancel_actor()` and that cancellation is propagated "horizontally" to other peers. Verify that any such cancellation scenario which also experiences an "error during `ContextCancelled` handling" DOES NOT result in that further error being suppressed and that the user's exception bubbles out of the `Context.open_context()` block(s) appropriately! Likely more tests to come as well as some factoring of the teardown state checks where possible. Pertains to serious testing the major work landing in #357 --- tests/test_inter_peer_cancellation.py | 434 ++++++++++++++++++++------ 1 file changed, 336 insertions(+), 98 deletions(-) diff --git a/tests/test_inter_peer_cancellation.py b/tests/test_inter_peer_cancellation.py index aeb70e81..46ca5758 100644 --- a/tests/test_inter_peer_cancellation.py +++ b/tests/test_inter_peer_cancellation.py @@ -3,30 +3,32 @@ Codify the cancellation request semantics in terms of one remote actor cancelling another. ''' -from contextlib import asynccontextmanager as acm +# from contextlib import asynccontextmanager as acm +import itertools import pytest import trio import tractor -from tractor._exceptions import ( - StreamOverrun, +from tractor import ( # typing + Portal, + Context, ContextCancelled, ) -def test_self_cancel(): - ''' - 2 cases: - - calls `Actor.cancel()` locally in some task - - calls LocalPortal.cancel_actor()` ? +# def test_self_cancel(): +# ''' +# 2 cases: +# - calls `Actor.cancel()` locally in some task +# - calls LocalPortal.cancel_actor()` ? - ''' - ... +# ''' +# ... @tractor.context async def sleep_forever( - ctx: tractor.Context, + ctx: Context, ) -> None: ''' Sync the context, open a stream then just sleep. @@ -37,13 +39,19 @@ async def sleep_forever( await trio.sleep_forever() -@acm -async def attach_to_sleep_forever(): +@tractor.context +async def error_before_started( + ctx: Context, +) -> None: ''' - Cancel a context **before** any underlying error is raised in order - to trigger a local reception of a ``ContextCancelled`` which **should not** - be re-raised in the local surrounding ``Context`` *iff* the cancel was - requested by **this** side of the context. + This simulates exactly an original bug discovered in: + https://github.com/pikers/piker/issues/244 + + Cancel a context **before** any underlying error is raised so + as to trigger a local reception of a ``ContextCancelled`` which + SHOULD NOT be re-raised in the local surrounding ``Context`` + *iff* the cancel was requested by **this** (callee) side of + the context. ''' async with tractor.wait_for_actor('sleeper') as p2: @@ -51,8 +59,16 @@ async def attach_to_sleep_forever(): p2.open_context(sleep_forever) as (peer_ctx, first), peer_ctx.open_stream(), ): + # NOTE: this WAS inside an @acm body but i factored it + # out and just put it inline here since i don't think + # the mngr part really matters, though maybe it could? try: - yield + # XXX NOTE XXX: THIS sends an UNSERIALIZABLE TYPE which + # should raise a `TypeError` and **NOT BE SWALLOWED** by + # the surrounding try/finally (normally inside the + # body of some acm).. + await ctx.started(object()) + # yield finally: # XXX: previously this would trigger local # ``ContextCancelled`` to be received and raised in the @@ -71,23 +87,6 @@ async def attach_to_sleep_forever(): await peer_ctx.cancel() -@tractor.context -async def error_before_started( - ctx: tractor.Context, -) -> None: - ''' - This simulates exactly an original bug discovered in: - https://github.com/pikers/piker/issues/244 - - ''' - async with attach_to_sleep_forever(): - - # XXX NOTE XXX: THIS sends an UNSERIALIZABLE TYPE which - # should raise a `TypeError` and **NOT BE SWALLOWED** by - # the surrounding acm!!?! - await ctx.started(object()) - - def test_do_not_swallow_error_before_started_by_remote_contextcancelled(): ''' Verify that an error raised in a remote context which itself @@ -121,93 +120,332 @@ def test_do_not_swallow_error_before_started_by_remote_contextcancelled(): @tractor.context -async def sleep_a_bit_then_cancel_sleeper( - ctx: tractor.Context, +async def sleep_a_bit_then_cancel_peer( + ctx: Context, + peer_name: str = 'sleeper', + cancel_after: float = .5, + ) -> None: - async with tractor.wait_for_actor('sleeper') as sleeper: - await ctx.started() - # await trio.sleep_forever() - await trio.sleep(3) - # async with tractor.wait_for_actor('sleeper') as sleeper: - await sleeper.cancel_actor() - - -def test_peer_canceller(): ''' - Verify that a cancellation triggered by a peer (whether in tree - or not) results in a cancelled error with - a `ContextCancelled.errorer` matching the requesting actor. + Connect to peer, sleep as per input delay, cancel the peer. - cases: - - some arbitrary remote peer cancels via Portal.cancel_actor(). - => all other connected peers should get that cancel requesting peer's - uid in the ctx-cancelled error msg. + ''' + peer: Portal + async with tractor.wait_for_actor(peer_name) as peer: + await ctx.started() + await trio.sleep(cancel_after) + await peer.cancel_actor() - - peer spawned a sub-actor which (also) spawned a failing task - which was unhandled and propagated up to the immediate - parent, the peer to the actor that also spawned a remote task - task in that same peer-parent. - - peer cancelled itself - so other peers should - get errors reflecting that the peer was itself the .canceller? +@tractor.context +async def stream_ints( + ctx: Context, +): + await ctx.started() + async with ctx.open_stream() as stream: + for i in itertools.count(): + await stream.send(i) + + +@tractor.context +async def stream_from_peer( + ctx: Context, + peer_name: str = 'sleeper', +) -> None: + + peer: Portal + try: + async with ( + tractor.wait_for_actor(peer_name) as peer, + peer.open_context(stream_ints) as (peer_ctx, first), + peer_ctx.open_stream() as stream, + ): + await ctx.started() + # XXX TODO: big set of questions for this + # - should we raise `ContextCancelled` or `Cancelled` (rn + # it does that) here?! + # - test the `ContextCancelled` OUTSIDE the + # `.open_context()` call? + try: + async for msg in stream: + print(msg) + + except trio.Cancelled: + assert not ctx.cancel_called + assert not ctx.cancelled_caught + + assert not peer_ctx.cancel_called + assert not peer_ctx.cancelled_caught + + assert 'root' in ctx.cancel_called_remote + + raise # XXX MUST NEVER MASK IT!! + + with trio.CancelScope(shield=True): + await tractor.pause() + # pass + # pytest.fail( + raise RuntimeError( + 'peer never triggered local `[Context]Cancelled`?!?' + ) + + # NOTE: cancellation of the (sleeper) peer should always + # cause a `ContextCancelled` raise in this streaming + # actor. + except ContextCancelled as ctxerr: + assert ctxerr.canceller == 'canceller' + assert ctxerr._remote_error is ctxerr + + # CASE 1: we were cancelled by our parent, the root actor. + # TODO: there are other cases depending on how the root + # actor and it's caller side task are written: + # - if the root does not req us to cancel then an + # IPC-transport related error should bubble from the async + # for loop and thus cause local cancellation both here + # and in the root (since in that case this task cancels the + # context with the root, not the other way around) + assert ctx.cancel_called_remote[0] == 'root' + raise + + # except BaseException as err: + + # raise + +# cases: +# - some arbitrary remote peer cancels via Portal.cancel_actor(). +# => all other connected peers should get that cancel requesting peer's +# uid in the ctx-cancelled error msg. + +# - peer spawned a sub-actor which (also) spawned a failing task +# which was unhandled and propagated up to the immediate +# parent, the peer to the actor that also spawned a remote task +# task in that same peer-parent. + +# - peer cancelled itself - so other peers should +# get errors reflecting that the peer was itself the .canceller? + +# - WE cancelled the peer and thus should not see any raised +# `ContextCancelled` as it should be reaped silently? +# => pretty sure `test_context_stream_semantics::test_caller_cancels()` +# already covers this case? + +@pytest.mark.parametrize( + 'error_during_ctxerr_handling', + [False, True], +) +def test_peer_canceller( + error_during_ctxerr_handling: bool, +): + ''' + Verify that a cancellation triggered by an in-actor-tree peer + results in a cancelled errors with all other actors which have + opened contexts to that same actor. + + legend: + name> + a "play button" that indicates a new runtime instance, + an individual actor with `name`. + + .subname> + a subactor who's parent should be on some previous + line and be less indented. + + .actor0> ()-> .actor1> + a inter-actor task context opened (by `async with `Portal.open_context()`) + from actor0 *into* actor1. + + .actor0> ()<=> .actor1> + a inter-actor task context opened (as above) + from actor0 *into* actor1 which INCLUDES an additional + stream open using `async with Context.open_stream()`. + + + ------ - ------ + supervision view + ------ - ------ + root> + .sleeper> TODO: SOME SYNTAX SHOWING JUST SLEEPING + .just_caller> ()=> .sleeper> + .canceller> ()-> .sleeper> + TODO: how define calling `Portal.cancel_actor()` + + In this case a `ContextCancelled` with `.errorer` set to the + requesting actor, in this case 'canceller', should be relayed + to all other actors who have also opened a (remote task) + context with that now cancelled actor. + + ------ - ------ + task view + ------ - ------ + So there are 5 context open in total with 3 from the root to + its children and 2 from children to their peers: + 1. root> ()-> .sleeper> + 2. root> ()-> .streamer> + 3. root> ()-> .canceller> + + 4. .streamer> ()<=> .sleep> + 5. .canceller> ()-> .sleeper> + - calls `Portal.cancel_actor()` - - WE cancelled the peer and thus should not see any raised - `ContextCancelled` as it should be reaped silently? - => pretty sure `test_context_stream_semantics::test_caller_cancels()` - already covers this case? ''' async def main(): - async with tractor.open_nursery() as n: - canceller: tractor.Portal = await n.start_actor( + async with tractor.open_nursery() as an: + canceller: Portal = await an.start_actor( 'canceller', enable_modules=[__name__], ) - sleeper: tractor.Portal = await n.start_actor( + sleeper: Portal = await an.start_actor( 'sleeper', enable_modules=[__name__], ) + just_caller: Portal = await an.start_actor( + 'just_caller', # but i just met her? + enable_modules=[__name__], + ) - async with ( - sleeper.open_context( - sleep_forever, - ) as (sleeper_ctx, sent), + try: + async with ( + sleeper.open_context( + sleep_forever, + ) as (sleeper_ctx, sent), - canceller.open_context( - sleep_a_bit_then_cancel_sleeper, - ) as (canceller_ctx, sent), - ): - # await tractor.pause() - try: - print('PRE CONTEXT RESULT') - await sleeper_ctx.result() + just_caller.open_context( + stream_from_peer, + ) as (caller_ctx, sent), - # TODO: not sure why this isn't catching - # but maybe we need an `ExceptionGroup` and - # the whole except *errs: thinger in 3.11? - except ( - ContextCancelled, - ) as berr: - print('CAUGHT REMOTE CONTEXT CANCEL') + canceller.open_context( + sleep_a_bit_then_cancel_peer, + ) as (canceller_ctx, sent), - # canceller should not have been remotely - # cancelled. - assert canceller_ctx.cancel_called_remote is None + ): + ctxs: list[Context] = [ + sleeper_ctx, + caller_ctx, + canceller_ctx, + ] - # NOTE: will only enter if you wrap in - # a shielded cs.. - # await tractor.pause() # TODO: shield=True) + try: + print('PRE CONTEXT RESULT') + await sleeper_ctx.result() - assert sleeper_ctx.canceller == 'canceller' - assert not sleep_ctx.cancelled_caught + # should never get here + pytest.fail( + 'Context.result() did not raise ctx-cancelled?' + ) + + # TODO: not sure why this isn't catching + # but maybe we need an `ExceptionGroup` and + # the whole except *errs: thinger in 3.11? + except ContextCancelled as ctxerr: + print(f'CAUGHT REMOTE CONTEXT CANCEL {ctxerr}') + + # canceller and caller peers should not + # have been remotely cancelled. + assert canceller_ctx.cancel_called_remote is None + assert caller_ctx.cancel_called_remote is None + + assert ctxerr.canceller[0] == 'canceller' + + # XXX NOTE XXX: since THIS `ContextCancelled` + # HAS NOT YET bubbled up to the + # `sleeper.open_context().__aexit__()` this + # value is not yet set, however outside this + # block it should be. + assert not sleeper_ctx.cancelled_caught + + # TODO: a test which ensures this error is + # bubbled and caught (NOT MASKED) by the + # runtime!!! + if error_during_ctxerr_handling: + raise RuntimeError('Simulated error during teardown') + + raise + + # SHOULD NEVER GET HERE! + except BaseException: + pytest.fail('did not rx ctx-cancelled error?') + else: + pytest.fail('did not rx ctx-cancelled error?') + + except ( + ContextCancelled, + RuntimeError, + )as ctxerr: + _err = ctxerr + + if error_during_ctxerr_handling: + assert isinstance(ctxerr, RuntimeError) + + # NOTE: this root actor task should have + # called `Context.cancel()` on the + # `.__aexit__()` to every opened ctx. + for ctx in ctxs: + assert ctx.cancel_called + + # each context should have received + # a silently absorbed context cancellation + # from its peer actor's task. + assert ctx.chan.uid == ctx.cancel_called_remote + + # this root actor task should have + # cancelled all opened contexts except + # the sleeper which is cancelled by its + # peer "canceller" + if ctx is not sleeper_ctx: + assert ctx._remote_error.canceller[0] == 'root' - raise else: - raise RuntimeError('NEVER RXED EXPECTED `ContextCancelled`') + assert ctxerr.canceller[0] == 'canceller' + # the sleeper's remote error is the error bubbled + # out of the context-stack above! + re = sleeper_ctx._remote_error + assert re is ctxerr - with pytest.raises(tractor.ContextCancelled) as excinfo: - trio.run(main) + for ctx in ctxs: - assert excinfo.value.type == ContextCancelled + if ctx is sleeper_ctx: + assert not ctx.cancel_called + assert ctx.cancelled_caught + else: + assert ctx.cancel_called + assert not ctx.cancelled_caught + + # each context should have received + # a silently absorbed context cancellation + # from its peer actor's task. + assert ctx.chan.uid == ctx.cancel_called_remote + + # NOTE: when an inter-peer cancellation + # occurred, we DO NOT expect this + # root-actor-task to have requested a cancel of + # the context since cancellation was caused by + # the "canceller" peer and thus + # `Context.cancel()` SHOULD NOT have been + # called inside + # `Portal.open_context().__aexit__()`. + assert not sleeper_ctx.cancel_called + + # XXX NOTE XXX: and see matching comment above but, + # this flag is set only AFTER the `.open_context()` + # has exited and should be set in both outcomes + # including the case where ctx-cancel handling + # itself errors. + assert sleeper_ctx.cancelled_caught + assert sleeper_ctx.cancel_called_remote[0] == 'sleeper' + + # await tractor.pause() + raise # always to ensure teardown + + if error_during_ctxerr_handling: + with pytest.raises(RuntimeError) as excinfo: + trio.run(main) + else: + + with pytest.raises(ContextCancelled) as excinfo: + trio.run(main) + + assert excinfo.value.type == ContextCancelled + assert excinfo.value.canceller[0] == 'canceller' From f895c9660051bf44f592d35f0b8fda9bd26f1acb Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 18 Oct 2023 15:29:43 -0400 Subject: [PATCH 046/146] Add masked super timeout line to `do_hard_kill()` for would-be runtime hackers --- tractor/_spawn.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index db465421..8f2615bd 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -199,6 +199,10 @@ async def do_hard_kill( proc: trio.Process, terminate_after: int = 3, + # NOTE: for mucking with `.pause()`-ing inside the runtime + # whilst also hacking on it XD + # terminate_after: int = 99999, + ) -> None: # NOTE: this timeout used to do nothing since we were shielding # the ``.wait()`` inside ``new_proc()`` which will pretty much From fdf0c43bfa1d94e0c6a6be9a090d1eaad3ae7408 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 18 Oct 2023 15:36:00 -0400 Subject: [PATCH 047/146] Type out the full-fledged streaming ex. --- examples/full_fledged_streaming_service.py | 21 ++++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/examples/full_fledged_streaming_service.py b/examples/full_fledged_streaming_service.py index 1650b583..c93df242 100644 --- a/examples/full_fledged_streaming_service.py +++ b/examples/full_fledged_streaming_service.py @@ -65,21 +65,28 @@ async def aggregate(seed): print("AGGREGATOR COMPLETE!") -# this is the main actor and *arbiter* -async def main(): - # a nursery which spawns "actors" - async with tractor.open_nursery( - arbiter_addr=('127.0.0.1', 1616) - ) as nursery: +async def main() -> list[int]: + ''' + This is the "root" actor's main task's entrypoint. + + By default (and if not otherwise specified) that root process + also acts as a "registry actor" / "registrar" on the localhost + for the purposes of multi-actor "service discovery". + + ''' + # yes, a nursery which spawns `trio`-"actors" B) + nursery: tractor.ActorNursery + async with tractor.open_nursery() as nursery: seed = int(1e3) pre_start = time.time() - portal = await nursery.start_actor( + portal: tractor.Portal = await nursery.start_actor( name='aggregator', enable_modules=[__name__], ) + stream: tractor.MsgStream async with portal.open_stream_from( aggregate, seed=seed, From 715348c5c2d2d0ec793fb2dc47479c38b8a54c49 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 18 Oct 2023 15:39:20 -0400 Subject: [PATCH 048/146] Port all tests to new `reg_addr` fixture name --- tests/test_cancellation.py | 29 ++++---- tests/test_child_manages_service_nursery.py | 2 +- tests/test_context_stream_semantics.py | 2 +- tests/test_debugger.py | 2 +- tests/test_discovery.py | 76 +++++++++++++-------- tests/test_infected_asyncio.py | 26 +++---- tests/test_legacy_one_way_streaming.py | 22 +++--- tests/test_local.py | 12 ++-- tests/test_multi_program.py | 10 +-- tests/test_pubsub.py | 8 +-- tests/test_rpc.py | 4 +- tests/test_spawning.py | 14 ++-- tests/test_task_broadcasting.py | 18 ++--- 13 files changed, 126 insertions(+), 99 deletions(-) diff --git a/tests/test_cancellation.py b/tests/test_cancellation.py index 657ab8e4..ce396ace 100644 --- a/tests/test_cancellation.py +++ b/tests/test_cancellation.py @@ -47,7 +47,7 @@ async def do_nuthin(): ], ids=['no_args', 'unexpected_args'], ) -def test_remote_error(arb_addr, args_err): +def test_remote_error(reg_addr, args_err): """Verify an error raised in a subactor that is propagated to the parent nursery, contains the underlying boxed builtin error type info and causes cancellation and reraising all the @@ -57,7 +57,7 @@ def test_remote_error(arb_addr, args_err): async def main(): async with tractor.open_nursery( - arbiter_addr=arb_addr, + registry_addrs=[reg_addr], ) as nursery: # on a remote type error caused by bad input args @@ -97,7 +97,7 @@ def test_remote_error(arb_addr, args_err): assert exc.type == errtype -def test_multierror(arb_addr): +def test_multierror(reg_addr): ''' Verify we raise a ``BaseExceptionGroup`` out of a nursery where more then one actor errors. @@ -105,7 +105,7 @@ def test_multierror(arb_addr): ''' async def main(): async with tractor.open_nursery( - arbiter_addr=arb_addr, + registry_addrs=[reg_addr], ) as nursery: await nursery.run_in_actor(assert_err, name='errorer1') @@ -130,14 +130,14 @@ def test_multierror(arb_addr): @pytest.mark.parametrize( 'num_subactors', range(25, 26), ) -def test_multierror_fast_nursery(arb_addr, start_method, num_subactors, delay): +def test_multierror_fast_nursery(reg_addr, start_method, num_subactors, delay): """Verify we raise a ``BaseExceptionGroup`` out of a nursery where more then one actor errors and also with a delay before failure to test failure during an ongoing spawning. """ async def main(): async with tractor.open_nursery( - arbiter_addr=arb_addr, + registry_addrs=[reg_addr], ) as nursery: for i in range(num_subactors): @@ -175,15 +175,20 @@ async def do_nothing(): @pytest.mark.parametrize('mechanism', ['nursery_cancel', KeyboardInterrupt]) -def test_cancel_single_subactor(arb_addr, mechanism): - """Ensure a ``ActorNursery.start_actor()`` spawned subactor +def test_cancel_single_subactor(reg_addr, mechanism): + ''' + Ensure a ``ActorNursery.start_actor()`` spawned subactor cancels when the nursery is cancelled. - """ + + ''' async def spawn_actor(): - """Spawn an actor that blocks indefinitely. - """ + ''' + Spawn an actor that blocks indefinitely then cancel via + either `ActorNursery.cancel()` or an exception raise. + + ''' async with tractor.open_nursery( - arbiter_addr=arb_addr, + registry_addrs=[reg_addr], ) as nursery: portal = await nursery.start_actor( diff --git a/tests/test_child_manages_service_nursery.py b/tests/test_child_manages_service_nursery.py index 806e6d7e..fd1ceb80 100644 --- a/tests/test_child_manages_service_nursery.py +++ b/tests/test_child_manages_service_nursery.py @@ -141,7 +141,7 @@ async def open_actor_local_nursery( ) def test_actor_managed_trio_nursery_task_error_cancels_aio( asyncio_mode: bool, - arb_addr + reg_addr: tuple, ): ''' Verify that a ``trio`` nursery created managed in a child actor diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index a0d291d7..29d50e84 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -5,7 +5,7 @@ Verify the we raise errors when streams are opened prior to sync-opening a ``tractor.Context`` beforehand. ''' -from contextlib import asynccontextmanager as acm +# from contextlib import asynccontextmanager as acm from itertools import count import platform from typing import Optional diff --git a/tests/test_debugger.py b/tests/test_debugger.py index a44a3138..e7bb0d73 100644 --- a/tests/test_debugger.py +++ b/tests/test_debugger.py @@ -78,7 +78,7 @@ has_nested_actors = pytest.mark.has_nested_actors def spawn( start_method, testdir, - arb_addr, + reg_addr, ) -> 'pexpect.spawn': if start_method != 'trio': diff --git a/tests/test_discovery.py b/tests/test_discovery.py index 8ba4ebee..8b47700c 100644 --- a/tests/test_discovery.py +++ b/tests/test_discovery.py @@ -15,19 +15,19 @@ from conftest import tractor_test @tractor_test -async def test_reg_then_unreg(arb_addr): +async def test_reg_then_unreg(reg_addr): actor = tractor.current_actor() assert actor.is_arbiter assert len(actor._registry) == 1 # only self is registered async with tractor.open_nursery( - arbiter_addr=arb_addr, + registry_addrs=[reg_addr], ) as n: portal = await n.start_actor('actor', enable_modules=[__name__]) uid = portal.channel.uid - async with tractor.get_arbiter(*arb_addr) as aportal: + async with tractor.get_arbiter(*reg_addr) as aportal: # this local actor should be the arbiter assert actor is aportal.actor @@ -53,15 +53,27 @@ async def hi(): return the_line.format(tractor.current_actor().name) -async def say_hello(other_actor): +async def say_hello( + other_actor: str, + reg_addr: tuple[str, int], +): await trio.sleep(1) # wait for other actor to spawn - async with tractor.find_actor(other_actor) as portal: + async with tractor.find_actor( + other_actor, + registry_addrs=[reg_addr], + ) as portal: assert portal is not None return await portal.run(__name__, 'hi') -async def say_hello_use_wait(other_actor): - async with tractor.wait_for_actor(other_actor) as portal: +async def say_hello_use_wait( + other_actor: str, + reg_addr: tuple[str, int], +): + async with tractor.wait_for_actor( + other_actor, + registry_addr=reg_addr, + ) as portal: assert portal is not None result = await portal.run(__name__, 'hi') return result @@ -69,21 +81,29 @@ async def say_hello_use_wait(other_actor): @tractor_test @pytest.mark.parametrize('func', [say_hello, say_hello_use_wait]) -async def test_trynamic_trio(func, start_method, arb_addr): - """Main tractor entry point, the "master" process (for now - acts as the "director"). - """ +async def test_trynamic_trio( + func, + start_method, + reg_addr, +): + ''' + Root actor acting as the "director" and running one-shot-task-actors + for the directed subs. + + ''' async with tractor.open_nursery() as n: print("Alright... Action!") donny = await n.run_in_actor( func, other_actor='gretchen', + reg_addr=reg_addr, name='donny', ) gretchen = await n.run_in_actor( func, other_actor='donny', + reg_addr=reg_addr, name='gretchen', ) print(await gretchen.result()) @@ -131,7 +151,7 @@ async def unpack_reg(actor_or_portal): async def spawn_and_check_registry( - arb_addr: tuple, + reg_addr: tuple, use_signal: bool, remote_arbiter: bool = False, with_streaming: bool = False, @@ -139,9 +159,9 @@ async def spawn_and_check_registry( ) -> None: async with tractor.open_root_actor( - arbiter_addr=arb_addr, + registry_addrs=[reg_addr], ): - async with tractor.get_arbiter(*arb_addr) as portal: + async with tractor.get_arbiter(*reg_addr) as portal: # runtime needs to be up to call this actor = tractor.current_actor() @@ -213,17 +233,19 @@ async def spawn_and_check_registry( def test_subactors_unregister_on_cancel( start_method, use_signal, - arb_addr, + reg_addr, with_streaming, ): - """Verify that cancelling a nursery results in all subactors + ''' + Verify that cancelling a nursery results in all subactors deregistering themselves with the arbiter. - """ + + ''' with pytest.raises(KeyboardInterrupt): trio.run( partial( spawn_and_check_registry, - arb_addr, + reg_addr, use_signal, remote_arbiter=False, with_streaming=with_streaming, @@ -237,7 +259,7 @@ def test_subactors_unregister_on_cancel_remote_daemon( daemon, start_method, use_signal, - arb_addr, + reg_addr, with_streaming, ): """Verify that cancelling a nursery results in all subactors @@ -248,7 +270,7 @@ def test_subactors_unregister_on_cancel_remote_daemon( trio.run( partial( spawn_and_check_registry, - arb_addr, + reg_addr, use_signal, remote_arbiter=True, with_streaming=with_streaming, @@ -262,7 +284,7 @@ async def streamer(agen): async def close_chans_before_nursery( - arb_addr: tuple, + reg_addr: tuple, use_signal: bool, remote_arbiter: bool = False, ) -> None: @@ -275,9 +297,9 @@ async def close_chans_before_nursery( entries_at_end = 1 async with tractor.open_root_actor( - arbiter_addr=arb_addr, + registry_addrs=[reg_addr], ): - async with tractor.get_arbiter(*arb_addr) as aportal: + async with tractor.get_arbiter(*reg_addr) as aportal: try: get_reg = partial(unpack_reg, aportal) @@ -329,7 +351,7 @@ async def close_chans_before_nursery( def test_close_channel_explicit( start_method, use_signal, - arb_addr, + reg_addr, ): """Verify that closing a stream explicitly and killing the actor's "root nursery" **before** the containing nursery tears down also @@ -339,7 +361,7 @@ def test_close_channel_explicit( trio.run( partial( close_chans_before_nursery, - arb_addr, + reg_addr, use_signal, remote_arbiter=False, ), @@ -351,7 +373,7 @@ def test_close_channel_explicit_remote_arbiter( daemon, start_method, use_signal, - arb_addr, + reg_addr, ): """Verify that closing a stream explicitly and killing the actor's "root nursery" **before** the containing nursery tears down also @@ -361,7 +383,7 @@ def test_close_channel_explicit_remote_arbiter( trio.run( partial( close_chans_before_nursery, - arb_addr, + reg_addr, use_signal, remote_arbiter=True, ), diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index dd9d681a..76744198 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -47,7 +47,7 @@ async def trio_cancels_single_aio_task(): await tractor.to_asyncio.run_task(sleep_forever) -def test_trio_cancels_aio_on_actor_side(arb_addr): +def test_trio_cancels_aio_on_actor_side(reg_addr): ''' Spawn an infected actor that is cancelled by the ``trio`` side task using std cancel scope apis. @@ -55,7 +55,7 @@ def test_trio_cancels_aio_on_actor_side(arb_addr): ''' async def main(): async with tractor.open_nursery( - arbiter_addr=arb_addr + registry_addrs=[reg_addr] ) as n: await n.run_in_actor( trio_cancels_single_aio_task, @@ -94,7 +94,7 @@ async def asyncio_actor( raise -def test_aio_simple_error(arb_addr): +def test_aio_simple_error(reg_addr): ''' Verify a simple remote asyncio error propagates back through trio to the parent actor. @@ -103,7 +103,7 @@ def test_aio_simple_error(arb_addr): ''' async def main(): async with tractor.open_nursery( - arbiter_addr=arb_addr + registry_addrs=[reg_addr] ) as n: await n.run_in_actor( asyncio_actor, @@ -120,7 +120,7 @@ def test_aio_simple_error(arb_addr): assert err.type == AssertionError -def test_tractor_cancels_aio(arb_addr): +def test_tractor_cancels_aio(reg_addr): ''' Verify we can cancel a spawned asyncio task gracefully. @@ -139,7 +139,7 @@ def test_tractor_cancels_aio(arb_addr): trio.run(main) -def test_trio_cancels_aio(arb_addr): +def test_trio_cancels_aio(reg_addr): ''' Much like the above test with ``tractor.Portal.cancel_actor()`` except we just use a standard ``trio`` cancellation api. @@ -194,7 +194,7 @@ async def trio_ctx( ids='parent_actor_cancels_child={}'.format ) def test_context_spawns_aio_task_that_errors( - arb_addr, + reg_addr, parent_cancels: bool, ): ''' @@ -258,7 +258,7 @@ async def aio_cancel(): await sleep_forever() -def test_aio_cancelled_from_aio_causes_trio_cancelled(arb_addr): +def test_aio_cancelled_from_aio_causes_trio_cancelled(reg_addr): async def main(): async with tractor.open_nursery() as n: @@ -395,7 +395,7 @@ async def stream_from_aio( 'fan_out', [False, True], ids='fan_out_w_chan_subscribe={}'.format ) -def test_basic_interloop_channel_stream(arb_addr, fan_out): +def test_basic_interloop_channel_stream(reg_addr, fan_out): async def main(): async with tractor.open_nursery() as n: portal = await n.run_in_actor( @@ -409,7 +409,7 @@ def test_basic_interloop_channel_stream(arb_addr, fan_out): # TODO: parametrize the above test and avoid the duplication here? -def test_trio_error_cancels_intertask_chan(arb_addr): +def test_trio_error_cancels_intertask_chan(reg_addr): async def main(): async with tractor.open_nursery() as n: portal = await n.run_in_actor( @@ -428,7 +428,7 @@ def test_trio_error_cancels_intertask_chan(arb_addr): assert exc.type == Exception -def test_trio_closes_early_and_channel_exits(arb_addr): +def test_trio_closes_early_and_channel_exits(reg_addr): async def main(): async with tractor.open_nursery() as n: portal = await n.run_in_actor( @@ -443,7 +443,7 @@ def test_trio_closes_early_and_channel_exits(arb_addr): trio.run(main) -def test_aio_errors_and_channel_propagates_and_closes(arb_addr): +def test_aio_errors_and_channel_propagates_and_closes(reg_addr): async def main(): async with tractor.open_nursery() as n: portal = await n.run_in_actor( @@ -520,7 +520,7 @@ async def trio_to_aio_echo_server( ids='raise_error={}'.format, ) def test_echoserver_detailed_mechanics( - arb_addr, + reg_addr, raise_error_mid_stream, ): diff --git a/tests/test_legacy_one_way_streaming.py b/tests/test_legacy_one_way_streaming.py index 17e94ba3..0cbda4d8 100644 --- a/tests/test_legacy_one_way_streaming.py +++ b/tests/test_legacy_one_way_streaming.py @@ -55,7 +55,7 @@ async def context_stream( async def stream_from_single_subactor( - arb_addr, + reg_addr, start_method, stream_func, ): @@ -64,7 +64,7 @@ async def stream_from_single_subactor( # only one per host address, spawns an actor if None async with tractor.open_nursery( - arbiter_addr=arb_addr, + registry_addrs=[reg_addr], start_method=start_method, ) as nursery: @@ -115,13 +115,13 @@ async def stream_from_single_subactor( @pytest.mark.parametrize( 'stream_func', [async_gen_stream, context_stream] ) -def test_stream_from_single_subactor(arb_addr, start_method, stream_func): +def test_stream_from_single_subactor(reg_addr, start_method, stream_func): """Verify streaming from a spawned async generator. """ trio.run( partial( stream_from_single_subactor, - arb_addr, + reg_addr, start_method, stream_func=stream_func, ), @@ -225,14 +225,14 @@ async def a_quadruple_example(): return result_stream -async def cancel_after(wait, arb_addr): - async with tractor.open_root_actor(arbiter_addr=arb_addr): +async def cancel_after(wait, reg_addr): + async with tractor.open_root_actor(registry_addrs=[reg_addr]): with trio.move_on_after(wait): return await a_quadruple_example() @pytest.fixture(scope='module') -def time_quad_ex(arb_addr, ci_env, spawn_backend): +def time_quad_ex(reg_addr, ci_env, spawn_backend): if spawn_backend == 'mp': """no idea but the mp *nix runs are flaking out here often... """ @@ -240,7 +240,7 @@ def time_quad_ex(arb_addr, ci_env, spawn_backend): timeout = 7 if platform.system() in ('Windows', 'Darwin') else 4 start = time.time() - results = trio.run(cancel_after, timeout, arb_addr) + results = trio.run(cancel_after, timeout, reg_addr) diff = time.time() - start assert results return results, diff @@ -260,14 +260,14 @@ def test_a_quadruple_example(time_quad_ex, ci_env, spawn_backend): list(map(lambda i: i/10, range(3, 9))) ) def test_not_fast_enough_quad( - arb_addr, time_quad_ex, cancel_delay, ci_env, spawn_backend + reg_addr, time_quad_ex, cancel_delay, ci_env, spawn_backend ): """Verify we can cancel midway through the quad example and all actors cancel gracefully. """ results, diff = time_quad_ex delay = max(diff - cancel_delay, 0) - results = trio.run(cancel_after, delay, arb_addr) + results = trio.run(cancel_after, delay, reg_addr) system = platform.system() if system in ('Windows', 'Darwin') and results is not None: # In CI envoirments it seems later runs are quicker then the first @@ -280,7 +280,7 @@ def test_not_fast_enough_quad( @tractor_test async def test_respawn_consumer_task( - arb_addr, + reg_addr, spawn_backend, loglevel, ): diff --git a/tests/test_local.py b/tests/test_local.py index 97a83285..009d0d71 100644 --- a/tests/test_local.py +++ b/tests/test_local.py @@ -24,7 +24,7 @@ async def test_no_runtime(): @tractor_test -async def test_self_is_registered(arb_addr): +async def test_self_is_registered(reg_addr): "Verify waiting on the arbiter to register itself using the standard api." actor = tractor.current_actor() assert actor.is_arbiter @@ -34,20 +34,20 @@ async def test_self_is_registered(arb_addr): @tractor_test -async def test_self_is_registered_localportal(arb_addr): +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(*arb_addr) as portal: + async with tractor.get_arbiter(*reg_addr) as portal: assert isinstance(portal, tractor._portal.LocalPortal) with trio.fail_after(0.2): sockaddr = await portal.run_from_ns( 'self', 'wait_for_actor', name='root') - assert sockaddr[0] == arb_addr + assert sockaddr[0] == reg_addr -def test_local_actor_async_func(arb_addr): +def test_local_actor_async_func(reg_addr): """Verify a simple async function in-process. """ nums = [] @@ -55,7 +55,7 @@ def test_local_actor_async_func(arb_addr): async def print_loop(): async with tractor.open_root_actor( - arbiter_addr=arb_addr, + registry_addrs=[reg_addr], ): # arbiter is started in-proc if dne assert tractor.current_actor().is_arbiter diff --git a/tests/test_multi_program.py b/tests/test_multi_program.py index e7a3ac5c..d1ee0f5e 100644 --- a/tests/test_multi_program.py +++ b/tests/test_multi_program.py @@ -28,9 +28,9 @@ def test_abort_on_sigint(daemon): @tractor_test -async def test_cancel_remote_arbiter(daemon, arb_addr): +async def test_cancel_remote_arbiter(daemon, reg_addr): assert not tractor.current_actor().is_arbiter - async with tractor.get_arbiter(*arb_addr) as portal: + async with tractor.get_arbiter(*reg_addr) as portal: await portal.cancel_actor() time.sleep(0.1) @@ -39,16 +39,16 @@ async def test_cancel_remote_arbiter(daemon, arb_addr): # no arbiter socket should exist with pytest.raises(OSError): - async with tractor.get_arbiter(*arb_addr) as portal: + async with tractor.get_arbiter(*reg_addr) as portal: pass -def test_register_duplicate_name(daemon, arb_addr): +def test_register_duplicate_name(daemon, reg_addr): async def main(): async with tractor.open_nursery( - arbiter_addr=arb_addr, + registry_addrs=[reg_addr], ) as n: assert not tractor.current_actor().is_arbiter diff --git a/tests/test_pubsub.py b/tests/test_pubsub.py index ababcb51..20554fa5 100644 --- a/tests/test_pubsub.py +++ b/tests/test_pubsub.py @@ -160,7 +160,7 @@ async def test_required_args(callwith_expecterror): ) def test_multi_actor_subs_arbiter_pub( loglevel, - arb_addr, + reg_addr, pub_actor, ): """Try out the neato @pub decorator system. @@ -170,7 +170,7 @@ def test_multi_actor_subs_arbiter_pub( async def main(): async with tractor.open_nursery( - arbiter_addr=arb_addr, + registry_addrs=[reg_addr], enable_modules=[__name__], ) as n: @@ -255,12 +255,12 @@ def test_multi_actor_subs_arbiter_pub( def test_single_subactor_pub_multitask_subs( loglevel, - arb_addr, + reg_addr, ): async def main(): async with tractor.open_nursery( - arbiter_addr=arb_addr, + registry_addrs=[reg_addr], enable_modules=[__name__], ) as n: diff --git a/tests/test_rpc.py b/tests/test_rpc.py index 6d158961..7ede231b 100644 --- a/tests/test_rpc.py +++ b/tests/test_rpc.py @@ -45,7 +45,7 @@ async def short_sleep(): ids=['no_mods', 'this_mod', 'this_mod_bad_func', 'fail_to_import', 'fail_on_syntax'], ) -def test_rpc_errors(arb_addr, to_call, testdir): +def test_rpc_errors(reg_addr, to_call, testdir): """Test errors when making various RPC requests to an actor that either doesn't have the requested module exposed or doesn't define the named function. @@ -77,7 +77,7 @@ def test_rpc_errors(arb_addr, to_call, testdir): # spawn a subactor which calls us back async with tractor.open_nursery( - arbiter_addr=arb_addr, + arbiter_addr=reg_addr, enable_modules=exposed_mods.copy(), ) as n: diff --git a/tests/test_spawning.py b/tests/test_spawning.py index 17798c09..0f6a8cfe 100644 --- a/tests/test_spawning.py +++ b/tests/test_spawning.py @@ -16,14 +16,14 @@ data_to_pass_down = {'doggy': 10, 'kitty': 4} async def spawn( is_arbiter: bool, data: dict, - arb_addr: tuple[str, int], + reg_addr: tuple[str, int], ): namespaces = [__name__] await trio.sleep(0.1) async with tractor.open_root_actor( - arbiter_addr=arb_addr, + arbiter_addr=reg_addr, ): actor = tractor.current_actor() @@ -41,7 +41,7 @@ async def spawn( is_arbiter=False, name='sub-actor', data=data, - arb_addr=arb_addr, + reg_addr=reg_addr, enable_modules=namespaces, ) @@ -55,12 +55,12 @@ async def spawn( return 10 -def test_local_arbiter_subactor_global_state(arb_addr): +def test_local_arbiter_subactor_global_state(reg_addr): result = trio.run( spawn, True, data_to_pass_down, - arb_addr, + reg_addr, ) assert result == 10 @@ -140,7 +140,7 @@ async def check_loglevel(level): def test_loglevel_propagated_to_subactor( start_method, capfd, - arb_addr, + reg_addr, ): if start_method == 'mp_forkserver': pytest.skip( @@ -152,7 +152,7 @@ def test_loglevel_propagated_to_subactor( async with tractor.open_nursery( name='arbiter', start_method=start_method, - arbiter_addr=arb_addr, + arbiter_addr=reg_addr, ) as tn: await tn.run_in_actor( diff --git a/tests/test_task_broadcasting.py b/tests/test_task_broadcasting.py index 5e18e10a..d7a29134 100644 --- a/tests/test_task_broadcasting.py +++ b/tests/test_task_broadcasting.py @@ -66,13 +66,13 @@ async def ensure_sequence( async def open_sequence_streamer( sequence: list[int], - arb_addr: tuple[str, int], + reg_addr: tuple[str, int], start_method: str, ) -> tractor.MsgStream: async with tractor.open_nursery( - arbiter_addr=arb_addr, + arbiter_addr=reg_addr, start_method=start_method, ) as tn: @@ -93,7 +93,7 @@ async def open_sequence_streamer( def test_stream_fan_out_to_local_subscriptions( - arb_addr, + reg_addr, start_method, ): @@ -103,7 +103,7 @@ def test_stream_fan_out_to_local_subscriptions( async with open_sequence_streamer( sequence, - arb_addr, + reg_addr, start_method, ) as stream: @@ -138,7 +138,7 @@ def test_stream_fan_out_to_local_subscriptions( ] ) def test_consumer_and_parent_maybe_lag( - arb_addr, + reg_addr, start_method, task_delays, ): @@ -150,7 +150,7 @@ def test_consumer_and_parent_maybe_lag( async with open_sequence_streamer( sequence, - arb_addr, + reg_addr, start_method, ) as stream: @@ -211,7 +211,7 @@ def test_consumer_and_parent_maybe_lag( def test_faster_task_to_recv_is_cancelled_by_slower( - arb_addr, + reg_addr, start_method, ): ''' @@ -225,7 +225,7 @@ def test_faster_task_to_recv_is_cancelled_by_slower( async with open_sequence_streamer( sequence, - arb_addr, + reg_addr, start_method, ) as stream: @@ -302,7 +302,7 @@ def test_subscribe_errors_after_close(): def test_ensure_slow_consumers_lag_out( - arb_addr, + reg_addr, start_method, ): '''This is a pure local task test; no tractor From 5035617adf27d65193eb07132c8282e0da3e119c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 18 Oct 2023 19:09:07 -0400 Subject: [PATCH 049/146] Dump `.msgdata` in `RemoteActorError.__repr__()` --- tractor/_exceptions.py | 37 +++++++++++++++++++++++++++++++++---- 1 file changed, 33 insertions(+), 4 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index d9e1d17f..0bb4552b 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -20,6 +20,7 @@ Our classy exception set. """ import builtins import importlib +from pprint import pformat from typing import ( Any, Type, @@ -38,12 +39,17 @@ class ActorFailure(Exception): "General actor failure" +# TODO: rename to just `RemoteError`? class RemoteActorError(Exception): ''' - Remote actor exception bundled locally + A box(ing) type which bundles a remote actor `BaseException` for + (near identical, and only if possible,) local object/instance + re-construction in the local process memory domain. + + Normally each instance is expected to be constructed from + a special "error" IPC msg sent by some remote actor-runtime. ''' - # TODO: local recontruction of remote exception deats def __init__( self, message: str, @@ -53,13 +59,36 @@ class RemoteActorError(Exception): ) -> None: super().__init__(message) - self.type = suberror_type - self.msgdata = msgdata + # TODO: maybe a better name? + # - .errtype + # - .retype + # - .boxed_errtype + # - .boxed_type + # - .remote_type + # also pertains to our long long oustanding issue XD + # https://github.com/goodboy/tractor/issues/5 + self.type: str = suberror_type + self.msgdata: dict[str, Any] = msgdata @property def src_actor_uid(self) -> tuple[str, str] | None: return self.msgdata.get('src_actor_uid') + def __repr__(self) -> str: + if remote_tb := self.msgdata.get('tb_str'): + pformat(remote_tb) + return ( + f'{type(self).__name__}(\n' + f'msgdata={pformat(self.msgdata)}\n' + ')' + ) + + return super().__repr__(self) + + # TODO: local recontruction of remote exception deats + # def unbox(self) -> BaseException: + # ... + class InternalActorError(RemoteActorError): ''' From 8cb8390201962e5c3799b851a0f3b35c6f4a551d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 23 Oct 2023 14:17:36 -0400 Subject: [PATCH 050/146] Move `MessagingError` into `._exceptions` set --- tractor/_exceptions.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 0bb4552b..214dc88a 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -83,7 +83,7 @@ class RemoteActorError(Exception): ')' ) - return super().__repr__(self) + return super().__repr__() # TODO: local recontruction of remote exception deats # def unbox(self) -> BaseException: @@ -139,6 +139,9 @@ class AsyncioCancelled(Exception): ''' +class MessagingError(Exception): + 'Some kind of unexpected SC messaging dialog issue' + def pack_error( exc: BaseException, From 04217f319a9e404c2cedbec9e5ffe8358cd8f81f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 23 Oct 2023 14:34:12 -0400 Subject: [PATCH 051/146] Raise a `MessagingError` from the src error on msging edge cases --- tractor/_streaming.py | 44 ++++++++++++++++++++++++++++++++----------- 1 file changed, 33 insertions(+), 11 deletions(-) diff --git a/tractor/_streaming.py b/tractor/_streaming.py index e449fefe..f02197b8 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -23,6 +23,7 @@ The machinery and types behind ``Context.open_stream()`` from __future__ import annotations import inspect from contextlib import asynccontextmanager as acm +from pprint import pformat from typing import ( Any, Callable, @@ -35,6 +36,7 @@ import trio from ._exceptions import ( unpack_error, + MessagingError, ) from .log import get_logger from .trionics import ( @@ -66,6 +68,8 @@ def _raise_from_no_yield_msg( `'yield'` field. ''' + __tracebackhide__: bool = True + # internal error should never get here assert msg.get('cid'), ("Received internal error at portal?") @@ -73,18 +77,22 @@ def _raise_from_no_yield_msg( # - 'stop' # - 'error' # possibly just handle msg['stop'] here! + # breakpoint() if stream._closed: raise trio.ClosedResourceError('This stream was closed') - if msg.get('stop') or stream._eoc: - log.debug(f"{stream} was stopped at remote end") + if ( + msg.get('stop') + or stream._eoc + ): + log.debug(f'{stream} was stopped at remote end') # XXX: important to set so that a new ``.receive()`` # call (likely by another task using a broadcast receiver) # doesn't accidentally pull the ``return`` message # value out of the underlying feed mem chan! - stream._eoc = True + stream._eoc: bool = True # # when the send is closed we assume the stream has # # terminated and signal this local iterator to stop @@ -93,20 +101,24 @@ def _raise_from_no_yield_msg( # XXX: this causes ``ReceiveChannel.__anext__()`` to # raise a ``StopAsyncIteration`` **and** in our catch # block below it will trigger ``.aclose()``. - raise trio.EndOfChannel from src_err + raise trio.EndOfChannel( + 'Stream ended due to msg:\n' + f'{pformat(msg)}' + ) from src_err # TODO: test that shows stream raising an expected error!!! elif msg.get('error'): # raise the error message raise unpack_error(msg, stream._ctx.chan) - # always re-raise the source error if no translation error - # case is activated above. - raise src_err - # raise RuntimeError( - # 'Unknown non-yield stream msg?\n' - # f'{msg}' - # ) + # always re-raise the source error if no translation error case + # is activated above. + raise MessagingError( + f'Context received unexpected non-error msg!?\n' + f'cid: {cid}\n' + 'received msg:\n' + f'{pformat(msg)}' + ) from src_err class MsgStream(trio.abc.Channel): @@ -161,6 +173,16 @@ class MsgStream(trio.abc.Channel): determined by the underlying protocol). ''' + # NOTE: `trio.ReceiveChannel` implements + # EOC handling as follows (aka uses it + # to gracefully exit async for loops): + # + # async def __anext__(self) -> ReceiveType: + # try: + # return await self.receive() + # except trio.EndOfChannel: + # raise StopAsyncIteration + # see ``.aclose()`` for notes on the old behaviour prior to # introducing this if self._eoc: From f5fcd8ca2e0ff0975e18e6c74e90d9499b01ef86 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 23 Oct 2023 14:35:36 -0400 Subject: [PATCH 052/146] Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method --- tractor/_context.py | 232 +++++++++++++++++++++++++++++++++----------- tractor/_runtime.py | 120 ++++++++++++++--------- 2 files changed, 248 insertions(+), 104 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index 0aadec49..fb1fe585 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -56,6 +56,7 @@ from ._state import current_actor if TYPE_CHECKING: from ._portal import Portal + from ._runtime import Actor log = get_logger(__name__) @@ -64,20 +65,26 @@ log = get_logger(__name__) @dataclass class Context: ''' - An inter-actor, ``trio``-task communication context. + An inter-actor, SC transitive, `trio.Task` communication context. - NB: This class should never be instatiated directly, it is delivered - by either, - - runtime machinery to a remotely started task or, - - by entering ``Portal.open_context()``. + 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 "caller" task or, + - by the RPC machinery's `._runtime._invoke()` as a `ctx` arg + to a remotely scheduled "callee" function. - and is always constructed using ``mkt_context()``. + AND is always constructed using the below ``mk_context()``. Allows maintaining task or protocol specific state between - 2 communicating, parallel executing actor tasks. A unique context is - allocated on each side of any task RPC-linked msg dialog, for - every request to a remote actor from a portal. On the "callee" - side a context is always allocated inside ``._runtime._invoke()``. + 2 cancel-scope-linked, communicating and parallel executing + `trio.Task`s. Contexts are allocated on each side of any task + RPC-linked msg dialog, i.e. for every request to a remote + actor from a `Portal`. On the "callee" side a context is + always allocated inside ``._runtime._invoke()``. + + # TODO: more detailed writeup on cancellation, error and + # streaming semantics.. A context can be cancelled and (possibly eventually restarted) from either side of the underlying IPC channel, it can also open task @@ -108,12 +115,31 @@ class Context: # which is exactly the primitive that allows for # cross-actor-task-supervision and thus SC. _scope: trio.CancelScope | None = None + + # on a clean exit there should be a final value + # delivered from the far end "callee" task, so + # this value is only set on one side. _result: Any | int = None + + # if the local "caller" task errors this + # value is always set to the error that was + # captured in the `Portal.open_context().__aexit__()` + # teardown. + _local_error: BaseException | None = None + + # if the either side gets an error from the other + # this value is set to that error unpacked from an + # IPC msg. _remote_error: BaseException | None = None - # cancellation state + # only set if the local task called `.cancel()` _cancel_called: bool = False # did WE cancel the far end? - _cancelled_remote: tuple[str, str] | None = None + + # TODO: do we even need this? we can assume that if we're + # cancelled that the other side is as well, so maybe we should + # instead just have a `.canceller` pulled from the + # `ContextCancelled`? + _canceller: tuple[str, str] | None = None # NOTE: we try to ensure assignment of a "cancel msg" since # there's always going to be an "underlying reason" that any @@ -145,23 +171,47 @@ class Context: return self._cancel_called @property - def cancel_called_remote(self) -> tuple[str, str] | None: + def canceller(self) -> tuple[str, str] | None: ''' - ``Actor.uid`` of the remote actor who's task was cancelled - causing this side of the context to also be cancelled. + ``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. ''' - remote_uid = self._cancelled_remote - if remote_uid: - return tuple(remote_uid) + return self._canceller @property def cancelled_caught(self) -> bool: - return self._scope.cancelled_caught + return ( + # the local scope was cancelled either by + # remote error or self-request + self._scope.cancelled_caught + + # the local scope was never cancelled + # and instead likely we received a remote side + # cancellation that was raised inside `.result()` + or ( + (se := self._local_error) + and + isinstance(se, ContextCancelled) + and ( + se.canceller == self.canceller + or + se is self._remote_error + ) + ) + ) + + @property + def side(self) -> str: + ''' + Return string indicating which task this instance is wrapping. + + ''' + return 'caller' if self._portal else 'callee' # init and streaming state _started_called: bool = False - _started_received: bool = False _stream_opened: bool = False # overrun handling machinery @@ -196,7 +246,7 @@ class Context: async def send_stop(self) -> None: await self.chan.send({'stop': True, 'cid': self.cid}) - async def _maybe_cancel_and_set_remote_error( + def _maybe_cancel_and_set_remote_error( self, error: BaseException, @@ -269,16 +319,19 @@ class Context: # that error as the reason. self._remote_error: BaseException = error - # always record the remote actor's uid since its cancellation - # state is directly linked to ours (the local one). - self._cancelled_remote = self.chan.uid - if ( isinstance(error, ContextCancelled) ): + # always record the cancelling actor's uid since its cancellation + # state is linked and we want to know which process was + # the cause / requester of the cancellation. + self._canceller = error.canceller + log.cancel( - 'Remote task-context sucessfully cancelled for ' - f'{self.chan.uid}:{self.cid}' + 'Remote task-context was cancelled for ' + f'actor: {self.chan.uid}\n' + f'task: {self.cid}\n' + f'canceller: {error.canceller}\n' ) if self._cancel_called: @@ -289,22 +342,37 @@ class Context: # and we **don't need to raise it** in local cancel # scope since it will potentially override a real error. return + else: log.error( - f'Remote context error for {self.chan.uid}:{self.cid}:\n' + f'Remote context error,\n' + f'remote actor: {self.chan.uid}\n' + f'task: {self.cid}\n' f'{error}' ) + self._canceller = self.chan.uid + # TODO: tempted to **not** do this by-reraising in a # nursery and instead cancel a surrounding scope, detect # the cancellation, then lookup the error that was set? # YES! this is way better and simpler! - if self._scope: + cs: trio.CancelScope = self._scope + if ( + cs + and not cs.cancel_called + and not cs.cancelled_caught + ): + + # TODO: we can for sure drop this right? # from trio.testing import wait_all_tasks_blocked # await wait_all_tasks_blocked() - # self._cancelled_remote = self.chan.uid + + # TODO: it'd sure be handy to inject our own + # `trio.Cancelled` subtype here ;) + # https://github.com/goodboy/tractor/issues/368 self._scope.cancel() - # this REPL usage actually works here BD + # NOTE: this REPL usage actually works here dawg! Bo # from .devx._debug import pause # await pause() @@ -320,13 +388,19 @@ class Context: Timeout quickly in an attempt to sidestep 2-generals... ''' - side: str = 'caller' if self._portal else 'callee' + side: str = self.side log.cancel( f'Cancelling {side} side of context to {self.chan.uid}' ) - self._cancel_called: bool = True + # caller side who entered `Portal.open_context()` + # NOTE: on the call side we never manually call + # `._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 + # `Portal.open_context().__aexit__()`) if side == 'caller': if not self._portal: raise RuntimeError( @@ -349,7 +423,6 @@ class Context: '_cancel_task', cid=cid, ) - # print("EXITING CANCEL CALL") if cs.cancelled_caught: # XXX: there's no way to know if the remote task was indeed @@ -368,6 +441,9 @@ class Context: ) # callee side remote task + # NOTE: on this side we ALWAYS cancel the local scope since + # the caller expects a `ContextCancelled` to be sent from + # `._runtime._invoke()` back to the other side. else: # TODO: should we have an explicit cancel message # or is relaying the local `trio.Cancelled` as an @@ -403,7 +479,7 @@ class Context: ``trio``'s cancellation system. ''' - actor = current_actor() + actor: Actor = current_actor() # here we create a mem chan that corresponds to the # far end caller / callee. @@ -413,12 +489,34 @@ class Context: # killed if self._cancel_called: - task = trio.lowlevel.current_task().name - raise ContextCancelled( - f'Context around {actor.uid[0]}:{task} was already cancelled!' + + # XXX NOTE: ALWAYS RAISE any remote error here even if + # it's an expected `ContextCancelled` (after some local + # task having called `.cancel()` ! + # + # WHY: we expect the error to always bubble up to the + # surrounding `Portal.open_context()` call and be + # absorbed there (silently) and we DO NOT want to + # actually try to stream - a cancel msg was already + # sent to the other side! + if re := self._remote_error: + raise self._remote_error + + # XXX NOTE: if no `ContextCancelled` has been responded + # back from the other side (yet), we raise a different + # runtime error indicating that this task's usage of + # `Context.cancel()` and then `.open_stream()` is WRONG! + task: str = trio.lowlevel.current_task().name + raise RuntimeError( + 'Stream opened after `Context.cancel()` called..?\n' + f'task: {actor.uid[0]}:{task}\n' + f'{self}' ) - if not self._portal and not self._started_called: + if ( + not self._portal + and not self._started_called + ): raise RuntimeError( 'Context.started()` must be called before opening a stream' ) @@ -434,7 +532,7 @@ class Context: msg_buffer_size=msg_buffer_size, allow_overruns=allow_overruns, ) - ctx._allow_overruns = allow_overruns + ctx._allow_overruns: bool = allow_overruns assert ctx is self # XXX: If the underlying channel feeder receive mem chan has @@ -444,27 +542,32 @@ class Context: if ctx._recv_chan._closed: raise trio.ClosedResourceError( - 'The underlying channel for this stream was already closed!?') + 'The underlying channel for this stream was already closed!?' + ) async with MsgStream( ctx=self, rx_chan=ctx._recv_chan, ) as stream: + # NOTE: we track all existing streams per portal for + # the purposes of attempting graceful closes on runtime + # cancel requests. if self._portal: self._portal._streams.add(stream) try: - self._stream_opened = True + self._stream_opened: bool = True # XXX: do we need this? # ensure we aren't cancelled before yielding the stream # await trio.lowlevel.checkpoint() yield stream - # NOTE: Make the stream "one-shot use". On exit, signal - # ``trio.EndOfChannel``/``StopAsyncIteration`` to the - # far end. + # NOTE: Make the stream "one-shot use". On exit, + # signal + # ``trio.EndOfChannel``/``StopAsyncIteration`` to + # the far end. await stream.aclose() finally: @@ -495,14 +598,22 @@ class Context: # whenever ``CancelScope.cancel()`` was called) and # instead silently reap the expected cancellation # "error"-msg. + our_uid: tuple[str, str] = current_actor().uid if ( isinstance(err, ContextCancelled) and ( self._cancel_called or self.chan._cancel_called - or tuple(err.canceller) == current_actor().uid + or self.canceller == our_uid + or tuple(err.canceller) == our_uid ) ): + # NOTE: we set the local scope error to any "self + # cancellation" error-response thus "absorbing" + # the error silently B) + if self._local_error is None: + self._local_error = err + return err # NOTE: currently we are masking underlying runtime errors @@ -515,7 +626,7 @@ class Context: # runtime frames from the tb explicitly? # https://docs.python.org/3/reference/simple_stmts.html#the-raise-statement # https://stackoverflow.com/a/24752607 - __tracebackhide__: bool = True + # __tracebackhide__: bool = True raise err from None async def result(self) -> Any | Exception: @@ -544,7 +655,6 @@ class Context: of the remote cancellation. ''' - __tracebackhide__: bool = True assert self._portal, "Context.result() can not be called from callee!" assert self._recv_chan @@ -607,13 +717,15 @@ class Context: "Received internal error at portal?" ) - err = unpack_error( + if err:= unpack_error( msg, self._portal.channel - ) # from msgerr + ): # from msgerr + self._maybe_cancel_and_set_remote_error(err) + self._maybe_raise_remote_err(err) - err = self._maybe_raise_remote_err(err) - self._remote_error = err + else: + raise if re := self._remote_error: return self._maybe_raise_remote_err(re) @@ -724,13 +836,17 @@ class Context: f"Delivering {msg} from {uid} to caller {cid}" ) - error = msg.get('error') - if error := unpack_error( - msg, - self.chan, + if ( + msg.get('error') # check for field + and ( + error := unpack_error( + msg, + self.chan, + ) + ) ): self._cancel_msg = msg - await self._maybe_cancel_and_set_remote_error(error) + self._maybe_cancel_and_set_remote_error(error) if ( self._in_overrun @@ -765,7 +881,7 @@ class Context: # XXX: always push an error even if the local # receiver is in overrun state. - # await self._maybe_cancel_and_set_remote_error(msg) + # self._maybe_cancel_and_set_remote_error(msg) local_uid = current_actor().uid lines = [ diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 067fd81e..729ead6b 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -86,12 +86,14 @@ async def _invoke( ] = trio.TASK_STATUS_IGNORED, ): ''' - Invoke local func and deliver result(s) over provided channel. + Schedule a `trio` task-as-func and deliver result(s) over + connected IPC channel. - This is the core "RPC task" starting machinery. + This is the core "RPC" `trio.Task` scheduling machinery used to start every + remotely invoked function, normally in `Actor._service_n: trio.Nursery`. ''' - __tracebackhide__ = True + __tracebackhide__: bool = True treat_as_gen: bool = False failed_resp: bool = False @@ -199,6 +201,8 @@ async def _invoke( # far end async gen to tear down await chan.send({'stop': True, 'cid': cid}) + # TODO: every other "func type" should be implemented from + # a special case of a context eventually! elif context: # context func with support for bi-dir streaming await chan.send({'functype': 'context', 'cid': cid}) @@ -209,21 +213,30 @@ async def _invoke( ctx._scope = nurse.cancel_scope task_status.started(ctx) res = await coro - await chan.send({'return': res, 'cid': cid}) + await chan.send({ + 'return': res, + 'cid': cid + }) # XXX: do we ever trigger this block any more? except ( BaseExceptionGroup, trio.Cancelled, - ): - # if a context error was set then likely - # thei multierror was raised due to that - if ctx._remote_error is not None: - raise ctx._remote_error + ) as scope_error: - # maybe TODO: pack in ``trio.Cancelled.__traceback__`` here - # so they can be unwrapped and displayed on the caller - # side? + # always set this (callee) side's exception as the + # local error on the context + ctx._local_error: BaseException = scope_error + + # if a remote error was set then likely the + # exception group was raised due to that, so + # and we instead raise that error immediately! + if re := ctx._remote_error: + ctx._maybe_raise_remote_err(re) + + # maybe TODO: pack in + # ``trio.Cancelled.__traceback__`` here so they can + # be unwrapped and displayed on the caller side? raise finally: @@ -234,11 +247,11 @@ async def _invoke( # don't pop the local context until we know the # associated child isn't in debug any more await _debug.maybe_wait_for_debugger() - ctx = actor._contexts.pop((chan.uid, cid)) - if ctx: - log.runtime( - f'Context entrypoint {func} was terminated:\n{ctx}' - ) + ctx: Context = actor._contexts.pop((chan.uid, cid)) + log.runtime( + f'Context entrypoint {func} was terminated:\n' + f'{ctx}' + ) if ctx.cancelled_caught: @@ -246,44 +259,43 @@ async def _invoke( # before raising any context cancelled case # so that real remote errors don't get masked as # ``ContextCancelled``s. - re = ctx._remote_error - if re: + if re := ctx._remote_error: ctx._maybe_raise_remote_err(re) - fname = func.__name__ + fname: str = func.__name__ cs: trio.CancelScope = ctx._scope if cs.cancel_called: - canceller = ctx._cancelled_remote - # await _debug.breakpoint() + canceller: tuple = ctx.canceller + msg: str = ( + f'`{fname}()`@{actor.uid} cancelled by ' + ) # NOTE / TODO: if we end up having # ``Actor._cancel_task()`` call # ``Context.cancel()`` directly, we're going to - # need to change this logic branch since it will - # always enter.. + # need to change this logic branch since it + # will always enter.. if ctx._cancel_called: - msg = f'`{fname}()`@{actor.uid} cancelled itself' - - else: - msg = ( - f'`{fname}()`@{actor.uid} ' - 'was remotely cancelled by ' - ) + msg += 'itself ' # if the channel which spawned the ctx is the # one that cancelled it then we report that, vs. # it being some other random actor that for ex. # some actor who calls `Portal.cancel_actor()` # and by side-effect cancels this ctx. - if canceller == ctx.chan.uid: - msg += f'its caller {canceller}' + elif canceller == ctx.chan.uid: + msg += f'its caller {canceller} ' + else: msg += f'remote actor {canceller}' # TODO: does this ever get set any more or can # we remove it? if ctx._cancel_msg: - msg += f' with msg:\n{ctx._cancel_msg}' + msg += ( + ' with msg:\n' + f'{ctx._cancel_msg}' + ) # task-contex was either cancelled by request using # ``Portal.cancel_actor()`` or ``Context.cancel()`` @@ -296,10 +308,13 @@ async def _invoke( canceller=canceller, ) + # regular async function else: - # regular async function try: - await chan.send({'functype': 'asyncfunc', 'cid': cid}) + await chan.send({ + 'functype': 'asyncfunc', + 'cid': cid + }) except trio.BrokenResourceError: failed_resp = True if is_rpc: @@ -313,7 +328,7 @@ async def _invoke( ctx._scope = cs task_status.started(ctx) result = await coro - fname = func.__name__ + fname: str = func.__name__ log.runtime(f'{fname}() result: {result}') if not failed_resp: # only send result if we know IPC isn't down @@ -1073,7 +1088,12 @@ class Actor: - return control the parent channel message loop ''' - log.cancel(f"{self.uid} is trying to cancel") + log.cancel( + f'{self.uid} requested to cancel by:\n' + f'{requesting_uid}' + ) + + # TODO: what happens here when we self-cancel tho? self._cancel_called_by_remote: tuple = requesting_uid self._cancel_called = True @@ -1088,7 +1108,9 @@ class Actor: dbcs.cancel() # kill all ongoing tasks - await self.cancel_rpc_tasks(requesting_uid=requesting_uid) + await self.cancel_rpc_tasks( + requesting_uid=requesting_uid, + ) # stop channel server self.cancel_server() @@ -1118,8 +1140,8 @@ class Actor: self, cid: str, chan: Channel, - requesting_uid: tuple[str, str] | None = None, + ) -> bool: ''' Cancel a local task by call-id / channel. @@ -1136,7 +1158,7 @@ class Actor: # this ctx based lookup ensures the requested task to # be cancelled was indeed spawned by a request from this channel ctx, func, is_complete = self._rpc_tasks[(chan, cid)] - scope = ctx._scope + scope: trio.CancelScope = ctx._scope except KeyError: log.cancel(f"{cid} has already completed/terminated?") return True @@ -1146,10 +1168,10 @@ class Actor: f"peer: {chan.uid}\n") if ( - ctx._cancelled_remote is None + ctx._canceller is None and requesting_uid ): - ctx._cancelled_remote: tuple = requesting_uid + ctx._canceller: tuple = requesting_uid # don't allow cancelling this function mid-execution # (is this necessary?) @@ -1159,6 +1181,7 @@ class Actor: # TODO: shouldn't we eventually be calling ``Context.cancel()`` # directly here instead (since that method can handle both # side's calls into it? + # await ctx.cancel() scope.cancel() # wait for _invoke to mark the task complete @@ -1186,9 +1209,12 @@ class Actor: registered for each. ''' - tasks = self._rpc_tasks + tasks: dict = self._rpc_tasks if tasks: - log.cancel(f"Cancelling all {len(tasks)} rpc tasks:\n{tasks} ") + log.cancel( + f'Cancelling all {len(tasks)} rpc tasks:\n' + f'{tasks}' + ) for ( (chan, cid), (ctx, func, is_complete), @@ -1206,7 +1232,9 @@ class Actor: ) log.cancel( - f"Waiting for remaining rpc tasks to complete {tasks}") + 'Waiting for remaining rpc tasks to complete:\n' + f'{tasks}' + ) await self._ongoing_rpc_tasks.wait() def cancel_server(self) -> None: From 7d5fda44854cb3c89c9d35cef91f98c480b49f06 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 23 Oct 2023 17:34:28 -0400 Subject: [PATCH 053/146] Be ultra-correct in `Portal.open_context()` This took way too long to get right but hopefully will give us grok-able and correct context exit semantics going forward B) The main fixes were: - always shielding the `MsgStream.aclose()` call on teardown to avoid bubbling a `Cancelled`. - properly absorbing any `ContextCancelled` in cases due to "self cancellation" using the new `Context.canceller` in the logic. - capturing any error raised by the `Context.result()` call in the "normal exit, result received" case and setting it as the `Context._local_error` so that self-cancels can be easily measured via `Context.cancelled_caught` in same way as remote-error caused cancellations. - extremely detailed comments around all of the cancellation-error cases to avoid ever getting confused about the control flow in the future XD --- tractor/_portal.py | 173 +++++++++++++++++++++++++++++++-------------- 1 file changed, 118 insertions(+), 55 deletions(-) diff --git a/tractor/_portal.py b/tractor/_portal.py index 6a2ec27a..2e29bcd7 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -48,6 +48,7 @@ from ._exceptions import ( unpack_error, NoResult, ContextCancelled, + MessagingError, ) from ._context import Context from ._streaming import MsgStream @@ -70,11 +71,6 @@ def _unwrap_msg( raise unpack_error(msg, channel) from None -# TODO: maybe move this to ._exceptions? -class MessagingError(Exception): - 'Some kind of unexpected SC messaging dialog issue' - - class Portal: ''' A 'portal' to a memory-domain-separated `Actor`. @@ -219,14 +215,18 @@ class Portal: try: # send cancel cmd - might not get response - # XXX: sure would be nice to make this work with a proper shield + # XXX: sure would be nice to make this work with + # a proper shield with trio.move_on_after( timeout or self.cancel_timeout ) as cs: cs.shield = True - await self.run_from_ns('self', 'cancel') + await self.run_from_ns( + 'self', + 'cancel', + ) return True if cs.cancelled_caught: @@ -461,10 +461,14 @@ class Portal: try: # the "first" value here is delivered by the callee's # ``Context.started()`` call. - first = msg['started'] + first: Any = msg['started'] ctx._started_called: bool = True except KeyError: + + # TODO: can we maybe factor this into the new raiser + # `_streaming._raise_from_no_yield_msg()` and make that + # helper more generic, say with a `_no__msg()`? if not (cid := msg.get('cid')): raise MessagingError( 'Received internal error at context?\n' @@ -516,57 +520,102 @@ class Portal: # started in the ctx nursery. ctx._scope.cancel() - # XXX: (maybe) shield/mask context-cancellations that were - # initiated by any of the context's 2 tasks. There are - # subsequently 2 operating cases for a "graceful cancel" - # of a `Context`: - # - # 1.*this* side's task called `Context.cancel()`, in - # which case we mask the `ContextCancelled` from bubbling - # to the opener (much like how `trio.Nursery` swallows - # any `trio.Cancelled` bubbled by a call to - # `Nursery.cancel_scope.cancel()`) + # XXX NOTE XXX: maybe shield against + # self-context-cancellation (which raises a local + # `ContextCancelled`) when requested (via + # `Context.cancel()`) by the same task (tree) which entered + # THIS `.open_context()`. # - # 2.*the other* side's (callee/spawned) task cancelled due - # to a self or peer cancellation request in which case we - # DO let the error bubble to the opener. + # NOTE: There are 2 operating cases for a "graceful cancel" + # of a `Context`. In both cases any `ContextCancelled` + # raised in this scope-block came from a transport msg + # relayed from some remote-actor-task which our runtime set + # as a `Context._remote_error` + # + # the CASES: + # + # - if that context IS THE SAME ONE that called + # `Context.cancel()`, we want to absorb the error + # silently and let this `.open_context()` block to exit + # without raising. + # + # - if it is from some OTHER context (we did NOT call + # `.cancel()`), we want to re-RAISE IT whilst also + # setting our own ctx's "reason for cancel" to be that + # other context's cancellation condition; we set our + # `.canceller: tuple[str, str]` to be same value as + # caught here in a `ContextCancelled.canceller`. + # + # Again, there are 2 cases: + # + # 1-some other context opened in this `.open_context()` + # block cancelled due to a self or peer cancellation + # request in which case we DO let the error bubble to the + # opener. + # + # 2-THIS "caller" task somewhere invoked `Context.cancel()` + # and received a `ContextCanclled` from the "callee" + # task, in which case we mask the `ContextCancelled` from + # bubbling to this "caller" (much like how `trio.Nursery` + # swallows any `trio.Cancelled` bubbled by a call to + # `Nursery.cancel_scope.cancel()`) except ContextCancelled as ctxc: scope_err = ctxc - # CASE 1: this context was never cancelled - # via a local task's call to `Context.cancel()`. - if not ctx._cancel_called: - # XXX: this should NEVER happen! - # from ._debug import breakpoint - # await breakpoint() - raise - # CASE 2: context was cancelled by local task calling # `.cancel()`, we don't raise and the exit block should # exit silently. - else: + if ( + ctx._cancel_called + and ( + ctxc is ctx._remote_error + or + ctxc.canceller is self.canceller + ) + ): log.debug( f'Context {ctx} cancelled gracefully with:\n' f'{ctxc}' ) + # CASE 1: this context was never cancelled via a local + # task (tree) having called `Context.cancel()`, raise + # the error since it was caused by someone else! + else: + raise + # the above `._scope` can be cancelled due to: + # 1. an explicit self cancel via `Context.cancel()` or + # `Actor.cancel()`, + # 2. any "callee"-side remote error, possibly also a cancellation + # request by some peer, + # 3. any "caller" (aka THIS scope's) local error raised in the above `yield` except ( - # - a standard error in the caller/yieldee + # CASE 3: standard local error in this caller/yieldee Exception, - # - a runtime teardown exception-group and/or - # cancellation request from a caller task. - BaseExceptionGroup, - trio.Cancelled, + # CASES 1 & 2: normally manifested as + # a `Context._scope_nursery` raised + # exception-group of, + # 1.-`trio.Cancelled`s, since + # `._scope.cancel()` will have been called and any + # `ContextCancelled` absorbed and thus NOT RAISED in + # any `Context._maybe_raise_remote_err()`, + # 2.-`BaseExceptionGroup[ContextCancelled | RemoteActorError]` + # from any error raised in the "callee" side with + # a group only raised if there was any more then one + # task started here in the "caller" in the + # `yield`-ed to task. + BaseExceptionGroup, # since overrun handler tasks may have been spawned + trio.Cancelled, # NOTE: NOT from inside the ctx._scope KeyboardInterrupt, ) as err: scope_err = err - # XXX: request cancel of this context on any error. - # NOTE: `Context.cancel()` is conversely NOT called in - # the `ContextCancelled` "cancellation requested" case - # above. + # XXX: ALWAYS request the context to CANCEL ON any ERROR. + # NOTE: `Context.cancel()` is conversely NEVER CALLED in + # the `ContextCancelled` "self cancellation absorbed" case + # handled in the block above! log.cancel( 'Context cancelled for task due to\n' f'{err}\n' @@ -585,7 +634,7 @@ class Portal: raise # duh - # no scope error case + # no local scope error, the "clean exit with a result" case. else: if ctx.chan.connected(): log.info( @@ -599,15 +648,27 @@ class Portal: # `Context._maybe_raise_remote_err()`) IFF # a `Context._remote_error` was set by the runtime # via a call to - # `Context._maybe_cancel_and_set_remote_error()` - # which IS SET any time the far end fails and - # causes "caller side" cancellation via - # a `ContextCancelled` here. - result = await ctx.result() - log.runtime( - f'Context {fn_name} returned value from callee:\n' - f'`{result}`' - ) + # `Context._maybe_cancel_and_set_remote_error()`. + # As per `Context._deliver_msg()`, that error IS + # ALWAYS SET any time "callee" side fails and causes "caller + # side" cancellation via a `ContextCancelled` here. + # result = await ctx.result() + try: + result = await ctx.result() + log.runtime( + f'Context {fn_name} returned value from callee:\n' + f'`{result}`' + ) + except BaseException as berr: + # on normal teardown, if we get some error + # raised in `Context.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 + # this task didn't know until final teardown + # / value collection. + scope_err = berr + raise finally: # though it should be impossible for any tasks @@ -657,12 +718,14 @@ class Portal: with trio.CancelScope(shield=True): await ctx._recv_chan.aclose() - # XXX: since we always (maybe) re-raise (and thus also - # mask runtime machinery related - # multi-`trio.Cancelled`s) any scope error which was - # the underlying cause of this context's exit, add - # different log msgs for each of the (2) cases. + # XXX: we always raise remote errors locally and + # generally speaking mask runtime-machinery related + # multi-`trio.Cancelled`s. As such, any `scope_error` + # which was the underlying cause of this context's exit + # should be stored as the `Context._local_error` and + # used in determining `Context.cancelled_caught: bool`. if scope_err is not None: + ctx._local_error: BaseException = scope_err etype: Type[BaseException] = type(scope_err) # CASE 2 @@ -691,7 +754,7 @@ class Portal: # child has already cleared it and clobbered IPC. # FINALLY, remove the context from runtime tracking and - # exit Bo + # exit! self.actor._contexts.pop( (self.channel.uid, ctx.cid), None, From 0222180c112e3187a0db2f80d187f8bf404c7b56 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 23 Oct 2023 17:47:55 -0400 Subject: [PATCH 054/146] Tweak `Channel._cancel_called` comment --- tractor/_ipc.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tractor/_ipc.py b/tractor/_ipc.py index ebfd261c..4d3f291f 100644 --- a/tractor/_ipc.py +++ b/tractor/_ipc.py @@ -294,9 +294,11 @@ class Channel: self._agen = self._aiter_recv() self._exc: Optional[Exception] = None # set if far end actor errors self._closed: bool = False - # flag set on ``Portal.cancel_actor()`` indicating - # remote (peer) cancellation of the far end actor runtime. - self._cancel_called: bool = False # set on ``Portal.cancel_actor()`` + + # flag set by ``Portal.cancel_actor()`` indicating remote + # (possibly peer) cancellation of the far end actor + # runtime. + self._cancel_called: bool = False @classmethod def from_stream( From abe31e9e2c7bd36f8bc05af0ba2e7d96b74a6be2 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 23 Oct 2023 17:48:34 -0400 Subject: [PATCH 055/146] Fix `Context.result()` call to be in runtime scope --- tests/test_infected_asyncio.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 76744198..56b5fde5 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -225,7 +225,7 @@ def test_context_spawns_aio_task_that_errors( await trio.sleep_forever() - return await ctx.result() + return await ctx.result() if parent_cancels: # bc the parent made the cancel request, From bac9523ecf70eba27a1a9904bed64dd3d22c113e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 23 Oct 2023 17:49:02 -0400 Subject: [PATCH 056/146] Adjust test details where `Context.cancel()` is called We can now make asserts on `.cancelled_caught` and `_remote_error` vs. `_local_error`. Expect a runtime error when `Context.open_stream()` is called AFTER `.cancel()` and the remote `ContextCancelled` hasn't arrived (yet). Adjust to `'itself'` string in self-cancel case. --- tests/test_context_stream_semantics.py | 67 ++++++++++++++++++++------ 1 file changed, 51 insertions(+), 16 deletions(-) diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index 29d50e84..dda096ce 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -13,6 +13,11 @@ from typing import Optional import pytest import trio import tractor +from tractor import ( + Actor, + Context, + current_actor, +) from tractor._exceptions import ( StreamOverrun, ContextCancelled, @@ -193,9 +198,6 @@ def test_simple_context( else: assert await ctx.result() == 'yo' - if not error_parent: - await ctx.cancel() - if pointlessly_open_stream: async with ctx.open_stream(): if error_parent: @@ -208,10 +210,15 @@ def test_simple_context( # 'stop' msg to the far end which needs # to be ignored pass + else: if error_parent: raise error_parent + # cancel AFTER we open a stream + # to avoid a cancel raised inside + # `.open_stream()` + await ctx.cancel() finally: # after cancellation @@ -276,7 +283,7 @@ def test_caller_cancels( assert ( tuple(err.canceller) == - tractor.current_actor().uid + current_actor().uid ) async def main(): @@ -430,9 +437,11 @@ async def test_caller_closes_ctx_after_callee_opens_stream( ): 'caller context closes without using stream' - async with tractor.open_nursery() as n: + async with tractor.open_nursery() as an: - portal = await n.start_actor( + root: Actor = current_actor() + + portal = await an.start_actor( 'ctx_cancelled', enable_modules=[__name__], ) @@ -440,10 +449,10 @@ async def test_caller_closes_ctx_after_callee_opens_stream( async with portal.open_context( expect_cancelled, ) as (ctx, sent): - await portal.run(assert_state, value=True) - assert sent is None + await portal.run(assert_state, value=True) + # call cancel explicitly if use_ctx_cancel_method: @@ -454,8 +463,21 @@ async def test_caller_closes_ctx_after_callee_opens_stream( async for msg in stream: pass - except tractor.ContextCancelled: - raise # XXX: must be propagated to __aexit__ + except tractor.ContextCancelled as ctxc: + # XXX: the cause is US since we call + # `Context.cancel()` just above! + assert ( + ctxc.canceller + == + current_actor().uid + == + root.uid + ) + + # XXX: must be propagated to __aexit__ + # and should be silently absorbed there + # since we called `.cancel()` just above ;) + raise else: assert 0, "Should have context cancelled?" @@ -472,7 +494,13 @@ async def test_caller_closes_ctx_after_callee_opens_stream( await ctx.result() assert 0, "Callee should have blocked!?" except trio.TooSlowError: + # NO-OP -> since already called above await ctx.cancel() + + # local scope should have absorbed the cancellation + assert ctx.cancelled_caught + assert ctx._remote_error is ctx._local_error + try: async with ctx.open_stream() as stream: async for msg in stream: @@ -551,19 +579,25 @@ async def cancel_self( global _state _state = True + # since we call this the below `.open_stream()` should always + # error! await ctx.cancel() # should inline raise immediately try: async with ctx.open_stream(): pass - except tractor.ContextCancelled: + # except tractor.ContextCancelled: + except RuntimeError: # suppress for now so we can do checkpoint tests below - pass + print('Got expected runtime error for stream-after-cancel') + else: raise RuntimeError('Context didnt cancel itself?!') - # check a real ``trio.Cancelled`` is raised on a checkpoint + # check that``trio.Cancelled`` is now raised on any further + # checkpoints since the self cancel above will have cancelled + # the `Context._scope.cancel_scope: trio.CancelScope` try: with trio.fail_after(0.1): await trio.sleep_forever() @@ -574,6 +608,7 @@ async def cancel_self( # should never get here assert 0 + raise RuntimeError('Context didnt cancel itself?!') @tractor_test async def test_callee_cancels_before_started(): @@ -601,7 +636,7 @@ async def test_callee_cancels_before_started(): ce.type == trio.Cancelled # the traceback should be informative - assert 'cancelled itself' in ce.msgdata['tb_str'] + assert 'itself' in ce.msgdata['tb_str'] # teardown the actor await portal.cancel_actor() @@ -773,7 +808,7 @@ async def echo_back_sequence( print( 'EXITING CALLEEE:\n' - f'{ctx.cancel_called_remote}' + f'{ctx.canceller}' ) return 'yo' @@ -871,7 +906,7 @@ def test_maybe_allow_overruns_stream( if cancel_ctx: assert isinstance(res, ContextCancelled) - assert tuple(res.canceller) == tractor.current_actor().uid + assert tuple(res.canceller) == current_actor().uid else: print(f'RX ROOT SIDE RESULT {res}') From a1bf4db1e32494cae82833be9e2eb6583bce0b20 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 23 Oct 2023 18:24:20 -0400 Subject: [PATCH 057/146] Get inter-peer suite passing with all `Context` state checks! Definitely needs some cleaning and refinement but this gets us to stage 1 of being pretty frickin correct i'd say :dancer: --- tests/test_inter_peer_cancellation.py | 283 ++++++++++++++++++-------- 1 file changed, 199 insertions(+), 84 deletions(-) diff --git a/tests/test_inter_peer_cancellation.py b/tests/test_inter_peer_cancellation.py index 46ca5758..09f11b87 100644 --- a/tests/test_inter_peer_cancellation.py +++ b/tests/test_inter_peer_cancellation.py @@ -15,6 +15,26 @@ from tractor import ( # typing ContextCancelled, ) +# XXX TODO cases: +# - [ ] peer cancelled itself - so other peers should +# get errors reflecting that the peer was itself the .canceller? + +# - [x] WE cancelled the peer and thus should not see any raised +# `ContextCancelled` as it should be reaped silently? +# => pretty sure `test_context_stream_semantics::test_caller_cancels()` +# already covers this case? + +# - [x] INTER-PEER: some arbitrary remote peer cancels via +# Portal.cancel_actor(). +# => all other connected peers should get that cancel requesting peer's +# uid in the ctx-cancelled error msg raised in all open ctxs +# with that peer. + +# - [ ] PEER-FAILS-BY-CHILD-ERROR: peer spawned a sub-actor which +# (also) spawned a failing task which was unhandled and +# propagated up to the immediate parent - the peer to the actor +# that also spawned a remote task task in that same peer-parent. + # def test_self_cancel(): # ''' @@ -29,14 +49,30 @@ from tractor import ( # typing @tractor.context async def sleep_forever( ctx: Context, + expect_ctxc: bool = False, ) -> None: ''' Sync the context, open a stream then just sleep. + Allow checking for (context) cancellation locally. + ''' - await ctx.started() - async with ctx.open_stream(): - await trio.sleep_forever() + try: + await ctx.started() + async with ctx.open_stream(): + await trio.sleep_forever() + + except BaseException as berr: + + # TODO: it'd sure be nice to be able to inject our own + # `ContextCancelled` here instead of of `trio.Cancelled` + # so that our runtime can expect it and this "user code" + # would be able to tell the diff between a generic trio + # cancel and a tractor runtime-IPC cancel. + if expect_ctxc: + assert isinstance(berr, trio.Cancelled) + + raise @tractor.context @@ -145,6 +181,7 @@ async def stream_ints( async with ctx.open_stream() as stream: for i in itertools.count(): await stream.send(i) + await trio.sleep(0.01) @tractor.context @@ -157,77 +194,111 @@ async def stream_from_peer( try: async with ( tractor.wait_for_actor(peer_name) as peer, - peer.open_context(stream_ints) as (peer_ctx, first), - peer_ctx.open_stream() as stream, + # peer.open_context(stream_ints) as (peer_ctx, first), + # peer_ctx.open_stream() as stream, ): - await ctx.started() - # XXX TODO: big set of questions for this - # - should we raise `ContextCancelled` or `Cancelled` (rn - # it does that) here?! - # - test the `ContextCancelled` OUTSIDE the - # `.open_context()` call? - try: - async for msg in stream: - print(msg) + async with ( + peer.open_context(stream_ints) as (peer_ctx, first), + # peer_ctx.open_stream() as stream, + ): + # # try: + async with ( + peer_ctx.open_stream() as stream, + ): - except trio.Cancelled: - assert not ctx.cancel_called - assert not ctx.cancelled_caught + await ctx.started() + # XXX QUESTIONS & TODO: for further details around this + # in the longer run.. + # https://github.com/goodboy/tractor/issues/368 + # - should we raise `ContextCancelled` or `Cancelled` (rn + # it does latter) and should/could it be implemented + # as a general injection override for `trio` such + # that ANY next checkpoint would raise the "cancel + # error type" of choice? + # - should the `ContextCancelled` bubble from + # all `Context` and `MsgStream` apis wherein it + # prolly makes the most sense to make it + # a `trio.Cancelled` subtype? + # - what about IPC-transport specific errors, should + # they bubble from the async for and trigger + # other special cases? + # try: + # NOTE: current ctl flow: + # - stream raises `trio.EndOfChannel` and + # exits the loop + # - `.open_context()` will raise the ctxcanc + # received from the sleeper. + async for msg in stream: + assert msg is not None + print(msg) + # finally: + # await trio.sleep(0.1) + # from tractor import pause + # await pause() - assert not peer_ctx.cancel_called - assert not peer_ctx.cancelled_caught + # except BaseException as berr: + # with trio.CancelScope(shield=True): + # await tractor.pause() + # raise - assert 'root' in ctx.cancel_called_remote - - raise # XXX MUST NEVER MASK IT!! - - with trio.CancelScope(shield=True): - await tractor.pause() - # pass - # pytest.fail( - raise RuntimeError( - 'peer never triggered local `[Context]Cancelled`?!?' - ) + # except trio.Cancelled: + # with trio.CancelScope(shield=True): + # await tractor.pause() + # raise # XXX NEVER MASK IT + # from tractor import pause + # await pause() # NOTE: cancellation of the (sleeper) peer should always # cause a `ContextCancelled` raise in this streaming # actor. except ContextCancelled as ctxerr: - assert ctxerr.canceller == 'canceller' - assert ctxerr._remote_error is ctxerr + err = ctxerr + assert peer_ctx._remote_error is ctxerr + assert peer_ctx.canceller == ctxerr.canceller - # CASE 1: we were cancelled by our parent, the root actor. - # TODO: there are other cases depending on how the root - # actor and it's caller side task are written: - # - if the root does not req us to cancel then an - # IPC-transport related error should bubble from the async - # for loop and thus cause local cancellation both here - # and in the root (since in that case this task cancels the - # context with the root, not the other way around) - assert ctx.cancel_called_remote[0] == 'root' + # caller peer should not be the cancel requester + assert not ctx.cancel_called + # XXX can never be true since `._invoke` only + # sets this AFTER the nursery block this task + # was started in, exits. + assert not ctx.cancelled_caught + + # we never requested cancellation + assert not peer_ctx.cancel_called + # the `.open_context()` exit definitely + # caught a cancellation in the internal `Context._scope` + # since likely the runtime called `_deliver_msg()` + # after receiving the remote error from the streaming + # task. + assert peer_ctx.cancelled_caught + + # TODO / NOTE `.canceller` won't have been set yet + # here because that machinery is inside + # `.open_context().__aexit__()` BUT, if we had + # a way to know immediately (from the last + # checkpoint) that cancellation was due to + # a remote, we COULD assert this here..see, + # https://github.com/goodboy/tractor/issues/368 + + # root/parent actor task should NEVER HAVE cancelled us! + assert not ctx.canceller + assert 'canceller' in peer_ctx.canceller + + # TODO: IN THEORY we could have other cases depending on + # who cancels first, the root actor or the canceller peer. + # + # 1- when the peer request is first then the `.canceller` + # field should obvi be set to the 'canceller' uid, + # + # 2-if the root DOES req cancel then we should see the same + # `trio.Cancelled` implicitly raised + # assert ctx.canceller[0] == 'root' + # assert peer_ctx.canceller[0] == 'sleeper' raise - # except BaseException as err: - - # raise - -# cases: -# - some arbitrary remote peer cancels via Portal.cancel_actor(). -# => all other connected peers should get that cancel requesting peer's -# uid in the ctx-cancelled error msg. - -# - peer spawned a sub-actor which (also) spawned a failing task -# which was unhandled and propagated up to the immediate -# parent, the peer to the actor that also spawned a remote task -# task in that same peer-parent. - -# - peer cancelled itself - so other peers should -# get errors reflecting that the peer was itself the .canceller? - -# - WE cancelled the peer and thus should not see any raised -# `ContextCancelled` as it should be reaped silently? -# => pretty sure `test_context_stream_semantics::test_caller_cancels()` -# already covers this case? + raise RuntimeError( + 'peer never triggered local `ContextCancelled`?' + ) @pytest.mark.parametrize( 'error_during_ctxerr_handling', @@ -251,8 +322,8 @@ def test_peer_canceller( line and be less indented. .actor0> ()-> .actor1> - a inter-actor task context opened (by `async with `Portal.open_context()`) - from actor0 *into* actor1. + a inter-actor task context opened (by `async with + `Portal.open_context()`) from actor0 *into* actor1. .actor0> ()<=> .actor1> a inter-actor task context opened (as above) @@ -287,11 +358,11 @@ def test_peer_canceller( 5. .canceller> ()-> .sleeper> - calls `Portal.cancel_actor()` - ''' - async def main(): - async with tractor.open_nursery() as an: + async with tractor.open_nursery( + # debug_mode=True + ) as an: canceller: Portal = await an.start_actor( 'canceller', enable_modules=[__name__], @@ -305,10 +376,13 @@ def test_peer_canceller( enable_modules=[__name__], ) + root = tractor.current_actor() + try: async with ( sleeper.open_context( sleep_forever, + expect_ctxc=True, ) as (sleeper_ctx, sent), just_caller.open_context( @@ -328,6 +402,7 @@ def test_peer_canceller( try: print('PRE CONTEXT RESULT') + # await tractor.pause() await sleeper_ctx.result() # should never get here @@ -343,8 +418,8 @@ def test_peer_canceller( # canceller and caller peers should not # have been remotely cancelled. - assert canceller_ctx.cancel_called_remote is None - assert caller_ctx.cancel_called_remote is None + assert canceller_ctx.canceller is None + assert caller_ctx.canceller is None assert ctxerr.canceller[0] == 'canceller' @@ -363,8 +438,9 @@ def test_peer_canceller( raise - # SHOULD NEVER GET HERE! - except BaseException: + # XXX SHOULD NEVER EVER GET HERE XXX + except BaseException as berr: + err = berr pytest.fail('did not rx ctx-cancelled error?') else: pytest.fail('did not rx ctx-cancelled error?') @@ -375,6 +451,19 @@ def test_peer_canceller( )as ctxerr: _err = ctxerr + # NOTE: the main state to check on `Context` is: + # - `.cancelled_caught` (maps to nursery cs) + # - `.cancel_called` (bool of whether this side + # requested) + # - `.canceller` (uid of cancel-causing actor-task) + # - `._remote_error` (any `RemoteActorError` + # instance from other side of context) + # - `._cancel_msg` (any msg that caused the + # cancel) + + # CASE: error raised during handling of + # `ContextCancelled` inside `.open_context()` + # block if error_during_ctxerr_handling: assert isinstance(ctxerr, RuntimeError) @@ -384,20 +473,30 @@ def test_peer_canceller( for ctx in ctxs: assert ctx.cancel_called + # this root actor task should have + # cancelled all opened contexts except the + # sleeper which is obvi by the "canceller" + # peer. + re = ctx._remote_error + if ( + ctx is sleeper_ctx + or ctx is caller_ctx + ): + assert re.canceller == canceller.channel.uid + + else: + assert re.canceller == root.uid + # each context should have received # a silently absorbed context cancellation # from its peer actor's task. - assert ctx.chan.uid == ctx.cancel_called_remote - - # this root actor task should have - # cancelled all opened contexts except - # the sleeper which is cancelled by its - # peer "canceller" - if ctx is not sleeper_ctx: - assert ctx._remote_error.canceller[0] == 'root' + # assert ctx.chan.uid == ctx.canceller + # CASE: standard teardown inside in `.open_context()` block else: - assert ctxerr.canceller[0] == 'canceller' + assert ctxerr.canceller == sleeper_ctx.canceller + # assert ctxerr.canceller[0] == 'canceller' + # assert sleeper_ctx.canceller[0] == 'canceller' # the sleeper's remote error is the error bubbled # out of the context-stack above! @@ -405,18 +504,35 @@ def test_peer_canceller( assert re is ctxerr for ctx in ctxs: + re: BaseException | None = ctx._remote_error + assert re + # root doesn't cancel sleeper since it's + # cancelled by its peer. + # match ctx: + # case sleeper_ctx: if ctx is sleeper_ctx: assert not ctx.cancel_called + # wait WHY? assert ctx.cancelled_caught + + elif ctx is caller_ctx: + # since its context was remotely + # cancelled, we never needed to + # call `Context.cancel()` bc our + # context was already remotely + # cancelled by the time we'd do it. + assert ctx.cancel_called + else: assert ctx.cancel_called assert not ctx.cancelled_caught - # each context should have received + # TODO: do we even need this flag? + # -> each context should have received # a silently absorbed context cancellation - # from its peer actor's task. - assert ctx.chan.uid == ctx.cancel_called_remote + # in its remote nursery scope. + # assert ctx.chan.uid == ctx.canceller # NOTE: when an inter-peer cancellation # occurred, we DO NOT expect this @@ -434,7 +550,6 @@ def test_peer_canceller( # including the case where ctx-cancel handling # itself errors. assert sleeper_ctx.cancelled_caught - assert sleeper_ctx.cancel_called_remote[0] == 'sleeper' # await tractor.pause() raise # always to ensure teardown From fea111e88243d5d1d09f6e3459ced81f06cf8d50 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 25 Oct 2023 15:21:41 -0400 Subject: [PATCH 058/146] Tons of interpeer test cleanup Drop all the nested `@acm` blocks and defunct comments from initial validations. Add some todos for cases that are still unclear such as whether the caller / streamer should have `.cancelled_caught == True` in it's teardown. --- tests/test_inter_peer_cancellation.py | 154 ++++++++++++-------------- 1 file changed, 72 insertions(+), 82 deletions(-) diff --git a/tests/test_inter_peer_cancellation.py b/tests/test_inter_peer_cancellation.py index 09f11b87..5e1a4cad 100644 --- a/tests/test_inter_peer_cancellation.py +++ b/tests/test_inter_peer_cancellation.py @@ -194,59 +194,33 @@ async def stream_from_peer( try: async with ( tractor.wait_for_actor(peer_name) as peer, - # peer.open_context(stream_ints) as (peer_ctx, first), - # peer_ctx.open_stream() as stream, + peer.open_context(stream_ints) as (peer_ctx, first), + peer_ctx.open_stream() as stream, ): - async with ( - peer.open_context(stream_ints) as (peer_ctx, first), - # peer_ctx.open_stream() as stream, - ): - # # try: - async with ( - peer_ctx.open_stream() as stream, - ): - - await ctx.started() - # XXX QUESTIONS & TODO: for further details around this - # in the longer run.. - # https://github.com/goodboy/tractor/issues/368 - # - should we raise `ContextCancelled` or `Cancelled` (rn - # it does latter) and should/could it be implemented - # as a general injection override for `trio` such - # that ANY next checkpoint would raise the "cancel - # error type" of choice? - # - should the `ContextCancelled` bubble from - # all `Context` and `MsgStream` apis wherein it - # prolly makes the most sense to make it - # a `trio.Cancelled` subtype? - # - what about IPC-transport specific errors, should - # they bubble from the async for and trigger - # other special cases? - # try: - # NOTE: current ctl flow: - # - stream raises `trio.EndOfChannel` and - # exits the loop - # - `.open_context()` will raise the ctxcanc - # received from the sleeper. - async for msg in stream: - assert msg is not None - print(msg) - # finally: - # await trio.sleep(0.1) - # from tractor import pause - # await pause() - - # except BaseException as berr: - # with trio.CancelScope(shield=True): - # await tractor.pause() - # raise - - # except trio.Cancelled: - # with trio.CancelScope(shield=True): - # await tractor.pause() - # raise # XXX NEVER MASK IT - # from tractor import pause - # await pause() + await ctx.started() + # XXX QUESTIONS & TODO: for further details around this + # in the longer run.. + # https://github.com/goodboy/tractor/issues/368 + # - should we raise `ContextCancelled` or `Cancelled` (rn + # it does latter) and should/could it be implemented + # as a general injection override for `trio` such + # that ANY next checkpoint would raise the "cancel + # error type" of choice? + # - should the `ContextCancelled` bubble from + # all `Context` and `MsgStream` apis wherein it + # prolly makes the most sense to make it + # a `trio.Cancelled` subtype? + # - what about IPC-transport specific errors, should + # they bubble from the async for and trigger + # other special cases? + # NOTE: current ctl flow: + # - stream raises `trio.EndOfChannel` and + # exits the loop + # - `.open_context()` will raise the ctxcanc + # received from the sleeper. + async for msg in stream: + assert msg is not None + print(msg) # NOTE: cancellation of the (sleeper) peer should always # cause a `ContextCancelled` raise in this streaming @@ -265,11 +239,10 @@ async def stream_from_peer( # we never requested cancellation assert not peer_ctx.cancel_called - # the `.open_context()` exit definitely - # caught a cancellation in the internal `Context._scope` - # since likely the runtime called `_deliver_msg()` - # after receiving the remote error from the streaming - # task. + # the `.open_context()` exit definitely caught + # a cancellation in the internal `Context._scope` since + # likely the runtime called `_deliver_msg()` after + # receiving the remote error from the streaming task. assert peer_ctx.cancelled_caught # TODO / NOTE `.canceller` won't have been set yet @@ -284,8 +257,9 @@ async def stream_from_peer( assert not ctx.canceller assert 'canceller' in peer_ctx.canceller + raise # TODO: IN THEORY we could have other cases depending on - # who cancels first, the root actor or the canceller peer. + # who cancels first, the root actor or the canceller peer?. # # 1- when the peer request is first then the `.canceller` # field should obvi be set to the 'canceller' uid, @@ -294,12 +268,12 @@ async def stream_from_peer( # `trio.Cancelled` implicitly raised # assert ctx.canceller[0] == 'root' # assert peer_ctx.canceller[0] == 'sleeper' - raise raise RuntimeError( 'peer never triggered local `ContextCancelled`?' ) + @pytest.mark.parametrize( 'error_during_ctxerr_handling', [False, True], @@ -361,6 +335,7 @@ def test_peer_canceller( ''' async def main(): async with tractor.open_nursery( + # NOTE: to halt the peer tasks on ctxc, uncomment this. # debug_mode=True ) as an: canceller: Portal = await an.start_actor( @@ -402,7 +377,6 @@ def test_peer_canceller( try: print('PRE CONTEXT RESULT') - # await tractor.pause() await sleeper_ctx.result() # should never get here @@ -410,9 +384,8 @@ def test_peer_canceller( 'Context.result() did not raise ctx-cancelled?' ) - # TODO: not sure why this isn't catching - # but maybe we need an `ExceptionGroup` and - # the whole except *errs: thinger in 3.11? + # should always raise since this root task does + # not request the sleeper cancellation ;) except ContextCancelled as ctxerr: print(f'CAUGHT REMOTE CONTEXT CANCEL {ctxerr}') @@ -430,9 +403,6 @@ def test_peer_canceller( # block it should be. assert not sleeper_ctx.cancelled_caught - # TODO: a test which ensures this error is - # bubbled and caught (NOT MASKED) by the - # runtime!!! if error_during_ctxerr_handling: raise RuntimeError('Simulated error during teardown') @@ -458,6 +428,7 @@ def test_peer_canceller( # - `.canceller` (uid of cancel-causing actor-task) # - `._remote_error` (any `RemoteActorError` # instance from other side of context) + # TODO: are we really planning to use this tho? # - `._cancel_msg` (any msg that caused the # cancel) @@ -482,21 +453,33 @@ def test_peer_canceller( ctx is sleeper_ctx or ctx is caller_ctx ): - assert re.canceller == canceller.channel.uid + assert ( + re.canceller + == + ctx.canceller + == + canceller.channel.uid + ) else: - assert re.canceller == root.uid - - # each context should have received - # a silently absorbed context cancellation - # from its peer actor's task. - # assert ctx.chan.uid == ctx.canceller + assert ( + re.canceller + == + ctx.canceller + == + root.uid + ) # CASE: standard teardown inside in `.open_context()` block else: assert ctxerr.canceller == sleeper_ctx.canceller - # assert ctxerr.canceller[0] == 'canceller' - # assert sleeper_ctx.canceller[0] == 'canceller' + assert ( + ctxerr.canceller[0] + == + sleeper_ctx.canceller[0] + == + 'canceller' + ) # the sleeper's remote error is the error bubbled # out of the context-stack above! @@ -509,21 +492,29 @@ def test_peer_canceller( # root doesn't cancel sleeper since it's # cancelled by its peer. - # match ctx: - # case sleeper_ctx: if ctx is sleeper_ctx: assert not ctx.cancel_called - # wait WHY? + # since sleeper_ctx.result() IS called + # above we should have (silently) + # absorbed the corresponding + # `ContextCancelled` for it and thus + # the logic inside `.cancelled_caught` + # should trigger! assert ctx.cancelled_caught elif ctx is caller_ctx: # since its context was remotely # cancelled, we never needed to - # call `Context.cancel()` bc our - # context was already remotely - # cancelled by the time we'd do it. + # call `Context.cancel()` bc it was + # done by the peer and also we never assert ctx.cancel_called + # TODO: figure out the details of + # this.. + # if you look the `._local_error` here + # is a multi of ctxc + 2 Cancelleds? + # assert not ctx.cancelled_caught + else: assert ctx.cancel_called assert not ctx.cancelled_caught @@ -551,7 +542,6 @@ def test_peer_canceller( # itself errors. assert sleeper_ctx.cancelled_caught - # await tractor.pause() raise # always to ensure teardown if error_during_ctxerr_handling: From 315f0fc7eb2fc7fea4d6907206ede9047fbd936c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 11 Dec 2023 18:17:42 -0500 Subject: [PATCH 059/146] More thurough hard kill doc strings --- tractor/_spawn.py | 45 ++++++++++++++++++++++++++++++++++++++------- 1 file changed, 38 insertions(+), 7 deletions(-) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index 8f2615bd..09e98bb7 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -204,6 +204,21 @@ async def do_hard_kill( # terminate_after: int = 99999, ) -> None: + ''' + Un-gracefully terminate an OS level `trio.Process` after timeout. + + Used in 2 main cases: + + - "unknown remote runtime state": a hanging/stalled actor that + isn't responding after sending a (graceful) runtime cancel + request via an IPC msg. + - "cancelled during spawn": a process who's actor runtime was + cancelled before full startup completed (such that + cancel-request-handling machinery was never fully + initialized) and thus a "cancel request msg" is never going + to be handled. + + ''' # NOTE: this timeout used to do nothing since we were shielding # the ``.wait()`` inside ``new_proc()`` which will pretty much # never release until the process exits, now it acts as @@ -219,6 +234,9 @@ async def do_hard_kill( # and wait for it to exit. If cancelled, kills the process and # waits for it to finish exiting before propagating the # cancellation. + # + # This code was originally triggred by ``proc.__aexit__()`` + # but now must be called manually. with trio.CancelScope(shield=True): if proc.stdin is not None: await proc.stdin.aclose() @@ -234,10 +252,14 @@ async def do_hard_kill( with trio.CancelScope(shield=True): await proc.wait() + # XXX NOTE XXX: zombie squad dispatch: + # (should ideally never, but) If we do get here it means + # graceful termination of a process failed and we need to + # resort to OS level signalling to interrupt and cancel the + # (presumably stalled or hung) actor. Since we never allow + # zombies (as a feature) we ask the OS to do send in the + # removal swad as the last resort. if cs.cancelled_caught: - # XXX: should pretty much never get here unless we have - # to move the bits from ``proc.__aexit__()`` out and - # into here. log.critical(f"#ZOMBIE_LORD_IS_HERE: {proc}") proc.kill() @@ -252,10 +274,13 @@ async def soft_wait( portal: Portal, ) -> None: - # Wait for proc termination but **dont' yet** call - # ``trio.Process.__aexit__()`` (it tears down stdio - # which will kill any waiting remote pdb trace). - # This is a "soft" (cancellable) join/reap. + ''' + Wait for proc termination but **dont' yet** teardown + std-streams (since it will clobber any ongoing pdb REPL + session). This is our "soft" (and thus itself cancellable) + join/reap on an actor-runtime-in-process. + + ''' uid = portal.channel.uid try: log.cancel(f'Soft waiting on actor:\n{uid}') @@ -278,7 +303,13 @@ async def soft_wait( await wait_func(proc) n.cancel_scope.cancel() + # start a task to wait on the termination of the + # process by itself waiting on a (caller provided) wait + # function which should unblock when the target process + # has terminated. n.start_soon(cancel_on_proc_deth) + + # send the actor-runtime a cancel request. await portal.cancel_actor() if proc.poll() is None: # type: ignore From 13fbcc723f65e3d265a021f18b93d938bb18ffdb Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 2 Jan 2024 09:08:39 -0500 Subject: [PATCH 060/146] Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout. --- tractor/_runtime.py | 184 +++++++++++++++++++++++++++++++------------- 1 file changed, 129 insertions(+), 55 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 729ead6b..46bdd2b5 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -15,7 +15,10 @@ # along with this program. If not, see . """ -Actor primitives and helpers +The fundamental core machinery implementing every "actor" including +the process-local (python-interpreter global) `Actor` state-type +primitive(s), RPC-in-task scheduling, and IPC connectivity and +low-level transport msg handling. """ from __future__ import annotations @@ -41,8 +44,14 @@ import warnings from async_generator import aclosing from exceptiongroup import BaseExceptionGroup -import trio # type: ignore -from trio_typing import TaskStatus +import trio +from trio import ( + CancelScope, +) +from trio_typing import ( + Nursery, + TaskStatus, +) from ._ipc import Channel from ._context import ( @@ -90,19 +99,18 @@ async def _invoke( connected IPC channel. This is the core "RPC" `trio.Task` scheduling machinery used to start every - remotely invoked function, normally in `Actor._service_n: trio.Nursery`. + remotely invoked function, normally in `Actor._service_n: Nursery`. ''' - __tracebackhide__: bool = True treat_as_gen: bool = False failed_resp: bool = False # possibly a traceback (not sure what typing is for this..) tb = None - cancel_scope = trio.CancelScope() + cancel_scope = CancelScope() # activated cancel scope ref - cs: trio.CancelScope | None = None + cs: CancelScope | None = None ctx = actor.get_context( chan, @@ -114,6 +122,7 @@ async def _invoke( ) context: bool = False + # TODO: deprecate this style.. if getattr(func, '_tractor_stream_function', False): # handle decorated ``@tractor.stream`` async functions sig = inspect.signature(func) @@ -155,6 +164,7 @@ async def _invoke( except TypeError: raise + # TODO: can we unify this with the `context=True` impl below? if inspect.isasyncgen(coro): await chan.send({'functype': 'asyncgen', 'cid': cid}) # XXX: massive gotcha! If the containing scope @@ -185,6 +195,7 @@ async def _invoke( await chan.send({'stop': True, 'cid': cid}) # one way @stream func that gets treated like an async gen + # TODO: can we unify this with the `context=True` impl below? elif treat_as_gen: await chan.send({'functype': 'asyncgen', 'cid': cid}) # XXX: the async-func may spawn further tasks which push @@ -201,8 +212,20 @@ async def _invoke( # far end async gen to tear down await chan.send({'stop': True, 'cid': cid}) + # our most general case: a remote SC-transitive, + # IPC-linked, cross-actor-task "context" + # ------ - ------ # TODO: every other "func type" should be implemented from - # a special case of a context eventually! + # a special case of this impl eventually! + # -[ ] streaming funcs should instead of being async-for + # handled directly here wrapped in + # a async-with-open_stream() closure that does the + # normal thing you'd expect a far end streaming context + # to (if written by the app-dev). + # -[ ] one off async funcs can literally just be called + # here and awaited directly, possibly just with a small + # wrapper that calls `Context.started()` and then does + # the `await coro()`? elif context: # context func with support for bi-dir streaming await chan.send({'functype': 'context', 'cid': cid}) @@ -263,11 +286,12 @@ async def _invoke( ctx._maybe_raise_remote_err(re) fname: str = func.__name__ - cs: trio.CancelScope = ctx._scope + cs: CancelScope = ctx._scope if cs.cancel_called: + our_uid: tuple = actor.uid canceller: tuple = ctx.canceller msg: str = ( - f'`{fname}()`@{actor.uid} cancelled by ' + f'`{fname}()`@{our_uid} cancelled by ' ) # NOTE / TODO: if we end up having @@ -276,6 +300,8 @@ async def _invoke( # need to change this logic branch since it # will always enter.. if ctx._cancel_called: + # TODO: test for this!!!!! + canceller: tuple = our_uid msg += 'itself ' # if the channel which spawned the ctx is the @@ -308,40 +334,76 @@ async def _invoke( canceller=canceller, ) - # regular async function + # 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. else: try: await chan.send({ 'functype': 'asyncfunc', 'cid': cid }) - except trio.BrokenResourceError: + except ( + trio.ClosedResourceError, + trio.BrokenResourceError, + BrokenPipeError, + ) as ipc_err: failed_resp = True if is_rpc: raise else: + # TODO: should this be an `.exception()` call? log.warning( - f'Failed to respond to non-rpc request: {func}' + f'Failed to respond to non-rpc request: {func}\n' + f'{ipc_err}' ) with cancel_scope as cs: - ctx._scope = cs + ctx._scope: CancelScope = cs task_status.started(ctx) result = await coro fname: str = func.__name__ log.runtime(f'{fname}() result: {result}') - if not failed_resp: - # only send result if we know IPC isn't down - await chan.send( - {'return': result, - 'cid': cid} - ) + + # NOTE: only send result if we know IPC isn't down + if ( + not failed_resp + and chan.connected() + ): + try: + await chan.send( + {'return': result, + 'cid': cid} + ) + except ( + BrokenPipeError, + trio.BrokenResourceError, + ): + log.warning( + 'Failed to return result:\n' + f'{func}@{actor.uid}\n' + f'remote chan: {chan.uid}' + ) except ( Exception, BaseExceptionGroup, ) as err: + # always hide this frame from debug REPL if the crash + # originated from an rpc task and we DID NOT fail + # due to an IPC transport error! + if ( + is_rpc + and chan.connected() + ): + __tracebackhide__: bool = True + if not is_multi_cancelled(err): # TODO: maybe we'll want different "levels" of debugging @@ -375,24 +437,31 @@ async def _invoke( log.exception("Actor crashed:") # always ship errors back to caller - err_msg = pack_error(err, tb=tb) + err_msg: dict[str, dict] = pack_error( + err, + tb=tb, + ) err_msg['cid'] = cid - try: - await chan.send(err_msg) + if is_rpc: + try: + await chan.send(err_msg) - # TODO: tests for this scenario: - # - RPC caller closes connection before getting a response - # should **not** crash this actor.. - except ( - trio.ClosedResourceError, - trio.BrokenResourceError, - BrokenPipeError, - ): - # if we can't propagate the error that's a big boo boo - log.exception( - f"Failed to ship error to caller @ {chan.uid} !?" - ) + # TODO: tests for this scenario: + # - RPC caller closes connection before getting a response + # should **not** crash this actor.. + except ( + trio.ClosedResourceError, + trio.BrokenResourceError, + BrokenPipeError, + ) as ipc_err: + + # if we can't propagate the error that's a big boo boo + log.exception( + f"Failed to ship error to caller @ {chan.uid} !?\n" + f'{ipc_err}' + + ) # error is probably from above coro running code *not from the # underlyingn rpc invocation* since a scope was never allocated @@ -418,7 +487,11 @@ async def _invoke( log.warning( f"Task {func} likely errored or cancelled before start") else: - log.cancel(f'{func.__name__}({kwargs}) failed?') + log.cancel( + 'Failed to de-alloc internal task!?\n' + f'cid: {cid}\n' + f'{func.__name__}({kwargs})' + ) finally: if not actor._rpc_tasks: @@ -435,7 +508,7 @@ async def try_ship_error_to_parent( err: Union[Exception, BaseExceptionGroup], ) -> None: - with trio.CancelScope(shield=True): + with CancelScope(shield=True): try: # internal error so ship to parent without cid await channel.send(pack_error(err)) @@ -482,13 +555,13 @@ class Actor: msg_buffer_size: int = 2**6 # nursery placeholders filled in by `async_main()` after fork - _root_n: trio.Nursery | None = None - _service_n: trio.Nursery | None = None - _server_n: trio.Nursery | None = None + _root_n: Nursery | None = None + _service_n: Nursery | None = None + _server_n: Nursery | None = None # Information about `__main__` from parent _parent_main_data: dict[str, str] - _parent_chan_cs: trio.CancelScope | None = None + _parent_chan_cs: CancelScope | None = None # syncs for setup/teardown sequences _server_down: trio.Event | None = None @@ -1020,7 +1093,7 @@ class Actor: async def _serve_forever( self, - handler_nursery: trio.Nursery, + handler_nursery: Nursery, *, # (host, port) to bind for channel server accept_host: tuple[str, int] | None = None, @@ -1098,7 +1171,7 @@ class Actor: self._cancel_called = True # cancel all ongoing rpc tasks - with trio.CancelScope(shield=True): + with CancelScope(shield=True): # kill any debugger request task to avoid deadlock # with the root actor in this tree @@ -1158,7 +1231,7 @@ class Actor: # this ctx based lookup ensures the requested task to # be cancelled was indeed spawned by a request from this channel ctx, func, is_complete = self._rpc_tasks[(chan, cid)] - scope: trio.CancelScope = ctx._scope + scope: CancelScope = ctx._scope except KeyError: log.cancel(f"{cid} has already completed/terminated?") return True @@ -1464,7 +1537,7 @@ async def async_main( # block it might be actually possible to debug THIS # machinery in the same way as user task code? # if actor.name == 'brokerd.ib': - # with trio.CancelScope(shield=True): + # with CancelScope(shield=True): # await _debug.breakpoint() actor.lifetime_stack.close() @@ -1500,7 +1573,7 @@ async def async_main( ): log.runtime( f"Waiting for remaining peers {actor._peers} to clear") - with trio.CancelScope(shield=True): + with CancelScope(shield=True): await actor._no_more_peers.wait() log.runtime("All peer channels are complete") @@ -1511,7 +1584,7 @@ async def process_messages( actor: Actor, chan: Channel, shield: bool = False, - task_status: TaskStatus[trio.CancelScope] = trio.TASK_STATUS_IGNORED, + task_status: TaskStatus[CancelScope] = trio.TASK_STATUS_IGNORED, ) -> bool: ''' @@ -1529,7 +1602,7 @@ async def process_messages( log.runtime(f"Entering msg loop for {chan} from {chan.uid}") try: - with trio.CancelScope(shield=shield) as loop_cs: + with CancelScope(shield=shield) as loop_cs: # this internal scope allows for keeping this message # loop running despite the current task having been # cancelled (eg. `open_portal()` may call this method from @@ -1591,18 +1664,18 @@ async def process_messages( if ns == 'self': if funcname == 'cancel': - func = actor.cancel + func: Callable = actor.cancel kwargs['requesting_uid'] = chan.uid # don't start entire actor runtime cancellation # if this actor is currently in debug mode! - pdb_complete = _debug.Lock.local_pdb_complete + pdb_complete: trio.Event | None = _debug.Lock.local_pdb_complete if pdb_complete: await pdb_complete.wait() # we immediately start the runtime machinery # shutdown - with trio.CancelScope(shield=True): + with CancelScope(shield=True): # actor.cancel() was called so kill this # msg loop and break out into # ``async_main()`` @@ -1630,7 +1703,7 @@ async def process_messages( # we immediately start the runtime machinery # shutdown - # with trio.CancelScope(shield=True): + # with CancelScope(shield=True): kwargs['chan'] = chan target_cid = kwargs['cid'] kwargs['requesting_uid'] = chan.uid @@ -1655,7 +1728,7 @@ async def process_messages( else: # normally registry methods, eg. # ``.register_actor()`` etc. - func = getattr(actor, funcname) + func: Callable = getattr(actor, funcname) else: # complain to client about restricted modules @@ -1745,9 +1818,10 @@ async def process_messages( Exception, BaseExceptionGroup, ) as err: + if nursery_cancelled_before_task: - sn = actor._service_n - assert sn and sn.cancel_scope.cancel_called + sn: Nursery = actor._service_n + assert sn and sn.cancel_scope.cancel_called # sanity log.cancel( f'Service nursery cancelled before it handled {funcname}' ) From 44e386dd9957b05b6e4d2497d0c2228b577c200d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 2 Jan 2024 10:24:39 -0500 Subject: [PATCH 061/146] ._child: remove some unused imports.. --- tractor/_child.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tractor/_child.py b/tractor/_child.py index 91aaec4f..bd1e830e 100644 --- a/tractor/_child.py +++ b/tractor/_child.py @@ -18,8 +18,6 @@ This is the "bootloader" for actors started using the native trio backend. """ -import sys -import trio import argparse from ast import literal_eval @@ -37,8 +35,6 @@ def parse_ipaddr(arg): return (str(host), int(port)) -from ._entry import _trio_main - if __name__ == "__main__": parser = argparse.ArgumentParser() From 74aa5aa9cd89b259ef1fbb92cb55d3fc372da4f1 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 2 Jan 2024 10:38:04 -0500 Subject: [PATCH 062/146] `StackLevelAdapter._log(stacklevel: int)` for custom levels.. Apparently (and i don't know if this was always broken [i feel like no?] or is a recent change to stdlib's `logging` stuff) we need increment the `stacklevel` input by one for our custom level methods now? Without this you're going to see the path to the method's-callstack-frame on every emission instead of to the caller's. I first noticed this when debugging the workspace layer spawning in `modden.bigd` and then verified it in other depended projects.. I guess we should add some tests for this as well XD --- tractor/log.py | 53 ++++++++++++++++++++++++++++++++++++++++---------- 1 file changed, 43 insertions(+), 10 deletions(-) diff --git a/tractor/log.py b/tractor/log.py index 5710e83e..590779a5 100644 --- a/tractor/log.py +++ b/tractor/log.py @@ -48,12 +48,15 @@ LOG_FORMAT = ( DATE_FORMAT = '%b %d %H:%M:%S' -LEVELS = { +LEVELS: dict[str, int] = { 'TRANSPORT': 5, 'RUNTIME': 15, 'CANCEL': 16, 'PDB': 500, } +# _custom_levels: set[str] = { +# lvlname.lower for lvlname in LEVELS.keys() +# } STD_PALETTE = { 'CRITICAL': 'red', @@ -102,7 +105,11 @@ class StackLevelAdapter(logging.LoggerAdapter): Cancellation logging, mostly for runtime reporting. ''' - return self.log(16, msg) + return self.log( + level=16, + msg=msg, + # stacklevel=4, + ) def pdb( self, @@ -114,14 +121,37 @@ class StackLevelAdapter(logging.LoggerAdapter): ''' return self.log(500, msg) - def log(self, level, msg, *args, **kwargs): - """ + def log( + self, + level, + msg, + *args, + **kwargs, + ): + ''' Delegate a log call to the underlying logger, after adding contextual information from this adapter instance. - """ + + ''' if self.isEnabledFor(level): + stacklevel: int = 3 + if ( + level in LEVELS.values() + # or level in _custom_levels + ): + stacklevel: int = 4 + # msg, kwargs = self.process(msg, kwargs) - self._log(level, msg, args, **kwargs) + self._log( + level=level, + msg=msg, + args=args, + # NOTE: not sure how this worked before but, it + # seems with our custom level methods defined above + # we do indeed (now) require another stack level?? + stacklevel=stacklevel, + **kwargs, + ) # LOL, the stdlib doesn't allow passing through ``stacklevel``.. def _log( @@ -134,12 +164,15 @@ class StackLevelAdapter(logging.LoggerAdapter): stack_info=False, # XXX: bit we added to show fileinfo from actual caller. - # this level then ``.log()`` then finally the caller's level.. - stacklevel=3, + # - this level + # - then ``.log()`` + # - then finally the caller's level.. + stacklevel=4, ): - """ + ''' Low-level log implementation, proxied to allow nested logger adapters. - """ + + ''' return self.logger._log( level, msg, From 2d2d1ca1c4a27c4bc5af57b65e8eb9f3284414cb Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 2 Jan 2024 11:21:20 -0500 Subject: [PATCH 063/146] Drop unused walrus assign of `re` --- tractor/_context.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index fb1fe585..c9e797b9 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -62,6 +62,7 @@ if TYPE_CHECKING: log = get_logger(__name__) +# TODO: make this a msgspec.Struct! @dataclass class Context: ''' @@ -491,15 +492,15 @@ class Context: if self._cancel_called: # XXX NOTE: ALWAYS RAISE any remote error here even if - # it's an expected `ContextCancelled` (after some local - # task having called `.cancel()` ! + # it's an expected `ContextCancelled` due to a local + # task having called `.cancel()`! # # WHY: we expect the error to always bubble up to the # surrounding `Portal.open_context()` call and be # absorbed there (silently) and we DO NOT want to # actually try to stream - a cancel msg was already # sent to the other side! - if re := self._remote_error: + if self._remote_error: raise self._remote_error # XXX NOTE: if no `ContextCancelled` has been responded From 914efd80eb599f77026397d6b07361fe9919e266 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 2 Jan 2024 11:28:55 -0500 Subject: [PATCH 064/146] Fmt repr as multi-line style call --- tractor/_ipc.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tractor/_ipc.py b/tractor/_ipc.py index 4d3f291f..f3579143 100644 --- a/tractor/_ipc.py +++ b/tractor/_ipc.py @@ -329,8 +329,11 @@ class Channel: def __repr__(self) -> str: if self.msgstream: return repr( - self.msgstream.stream.socket._sock).replace( # type: ignore - "socket.socket", "Channel") + self.msgstream.stream.socket._sock + ).replace( # type: ignore + "socket.socket", + "Channel", + ) return object.__repr__(self) @property From 27c5ffe5a79db07a0fe77b9a05e3c05633c85571 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 2 Jan 2024 18:34:15 -0500 Subject: [PATCH 065/146] Move missing-key-in-msg raiser to `._exceptions` Since we use basically the exact same set of logic in `Portal.open_context()` when expecting the first `'started'` msg factor and generalize `._streaming._raise_from_no_yield_msg()` into a new `._exceptions._raise_from_no_key_in_msg()` (as per the lingering todo) which obvi requires a more generalized / optional signature including a caller specific `log` obj. Obvi call the new func from all the other modules X) --- tractor/_context.py | 2 + tractor/_exceptions.py | 96 +++++++++++++++++++++++++++++++++++++++++- tractor/_portal.py | 38 +++++++---------- tractor/_streaming.py | 83 +++++------------------------------- 4 files changed, 122 insertions(+), 97 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index c9e797b9..2b539a6f 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -44,9 +44,11 @@ import warnings import trio from ._exceptions import ( + # _raise_from_no_key_in_msg, unpack_error, pack_error, ContextCancelled, + # MessagingError, StreamOverrun, ) from .log import get_logger diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 214dc88a..7e148586 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -14,16 +14,18 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . -""" +''' Our classy exception set. -""" +''' +from __future__ import annotations import builtins import importlib from pprint import pformat from typing import ( Any, Type, + TYPE_CHECKING, ) import traceback @@ -32,6 +34,11 @@ import trio from ._state import current_actor +if TYPE_CHECKING: + from ._context import Context + from ._stream import MsgStream + from .log import StackLevelAdapter + _this_mod = importlib.import_module(__name__) @@ -246,3 +253,88 @@ def is_multi_cancelled(exc: BaseException) -> bool: ) is not None return False + + +def _raise_from_no_key_in_msg( + ctx: Context, + msg: dict, + src_err: KeyError, + log: StackLevelAdapter, # caller specific `log` obj + expect_key: str = 'yield', + stream: MsgStream | None = None, + +) -> bool: + ''' + Raise an appopriate local error when a `MsgStream` msg arrives + which does not contain the expected (under normal operation) + `'yield'` field. + + ''' + __tracebackhide__: bool = True + + # internal error should never get here + try: + cid: str = msg['cid'] + except KeyError as src_err: + raise MessagingError( + f'IPC `Context` rx-ed msg without a ctx-id (cid)!?\n' + f'cid: {cid}\n' + 'received msg:\n' + f'{pformat(msg)}\n' + ) from src_err + + # TODO: test that shows stream raising an expected error!!! + if msg.get('error'): + # raise the error message + raise unpack_error( + msg, + ctx.chan, + ) from None + + elif ( + msg.get('stop') + or ( + stream + and stream._eoc + ) + ): + log.debug( + f'Context[{cid}] stream was stopped by remote side\n' + f'cid: {cid}\n' + ) + + # XXX: important to set so that a new ``.receive()`` + # call (likely by another task using a broadcast receiver) + # doesn't accidentally pull the ``return`` message + # value out of the underlying feed mem chan! + stream._eoc: bool = True + + # # when the send is closed we assume the stream has + # # terminated and signal this local iterator to stop + # await stream.aclose() + + # XXX: this causes ``ReceiveChannel.__anext__()`` to + # raise a ``StopAsyncIteration`` **and** in our catch + # block below it will trigger ``.aclose()``. + raise trio.EndOfChannel( + 'Context[{cid}] stream ended due to msg:\n' + f'{pformat(msg)}' + ) from src_err + + + if ( + stream + and stream._closed + ): + raise trio.ClosedResourceError('This stream was closed') + + + # always re-raise the source error if no translation error case + # is activated above. + _type: str = 'Stream' if stream else 'Context' + raise MessagingError( + f'{_type} was expecting a `{expect_key}` message' + ' BUT received a non-`error` msg:\n' + f'cid: {cid}\n' + '{pformat(msg)}' + ) from src_err diff --git a/tractor/_portal.py b/tractor/_portal.py index 2e29bcd7..15c8cb50 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -33,7 +33,6 @@ from typing import ( ) from functools import partial from dataclasses import dataclass -from pprint import pformat import warnings import trio @@ -45,13 +44,17 @@ from ._ipc import Channel from .log import get_logger from .msg import NamespacePath from ._exceptions import ( + _raise_from_no_key_in_msg, unpack_error, NoResult, ContextCancelled, - MessagingError, ) -from ._context import Context -from ._streaming import MsgStream +from ._context import ( + Context, +) +from ._streaming import ( + MsgStream, +) log = get_logger(__name__) @@ -464,26 +467,15 @@ class Portal: first: Any = msg['started'] ctx._started_called: bool = True - except KeyError: + except KeyError as src_error: - # TODO: can we maybe factor this into the new raiser - # `_streaming._raise_from_no_yield_msg()` and make that - # helper more generic, say with a `_no__msg()`? - if not (cid := msg.get('cid')): - raise MessagingError( - 'Received internal error at context?\n' - 'No call-id (cid) in startup msg?' - ) - - if msg.get('error'): - # NOTE: mask the key error with the remote one - raise unpack_error(msg, self.channel) from None - else: - raise MessagingError( - f'Context for {cid} was expecting a `started` message' - ' but received a non-error msg:\n' - f'{pformat(msg)}' - ) + _raise_from_no_key_in_msg( + ctx=ctx, + msg=msg, + src_err=src_error, + log=log, + expect_key='started', + ) ctx._portal: Portal = self uid: tuple = self.channel.uid diff --git a/tractor/_streaming.py b/tractor/_streaming.py index f02197b8..4530e144 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -23,7 +23,6 @@ The machinery and types behind ``Context.open_stream()`` from __future__ import annotations import inspect from contextlib import asynccontextmanager as acm -from pprint import pformat from typing import ( Any, Callable, @@ -35,8 +34,7 @@ import warnings import trio from ._exceptions import ( - unpack_error, - MessagingError, + _raise_from_no_key_in_msg, ) from .log import get_logger from .trionics import ( @@ -56,71 +54,6 @@ log = get_logger(__name__) # messages? class ReceiveChannel(AsyncResource, Generic[ReceiveType]): # - use __slots__ on ``Context``? -def _raise_from_no_yield_msg( - stream: MsgStream, - msg: dict, - src_err: KeyError, - -) -> bool: - ''' - Raise an appopriate local error when a `MsgStream` msg arrives - which does not contain the expected (under normal operation) - `'yield'` field. - - ''' - __tracebackhide__: bool = True - - # internal error should never get here - assert msg.get('cid'), ("Received internal error at portal?") - - # TODO: handle 2 cases with 3.10+ match syntax - # - 'stop' - # - 'error' - # possibly just handle msg['stop'] here! - # breakpoint() - - if stream._closed: - raise trio.ClosedResourceError('This stream was closed') - - if ( - msg.get('stop') - or stream._eoc - ): - log.debug(f'{stream} was stopped at remote end') - - # XXX: important to set so that a new ``.receive()`` - # call (likely by another task using a broadcast receiver) - # doesn't accidentally pull the ``return`` message - # value out of the underlying feed mem chan! - stream._eoc: bool = True - - # # when the send is closed we assume the stream has - # # terminated and signal this local iterator to stop - # await stream.aclose() - - # XXX: this causes ``ReceiveChannel.__anext__()`` to - # raise a ``StopAsyncIteration`` **and** in our catch - # block below it will trigger ``.aclose()``. - raise trio.EndOfChannel( - 'Stream ended due to msg:\n' - f'{pformat(msg)}' - ) from src_err - - # TODO: test that shows stream raising an expected error!!! - elif msg.get('error'): - # raise the error message - raise unpack_error(msg, stream._ctx.chan) - - # always re-raise the source error if no translation error case - # is activated above. - raise MessagingError( - f'Context received unexpected non-error msg!?\n' - f'cid: {cid}\n' - 'received msg:\n' - f'{pformat(msg)}' - ) from src_err - - class MsgStream(trio.abc.Channel): ''' A bidirectional message stream for receiving logically sequenced @@ -160,10 +93,13 @@ class MsgStream(trio.abc.Channel): try: return msg['yield'] except KeyError as kerr: - _raise_from_no_yield_msg( - stream=self, + _raise_from_no_key_in_msg( + ctx=self._ctx, msg=msg, src_err=kerr, + log=log, + expect_key='yield', + stream=self, ) async def receive(self): @@ -196,10 +132,13 @@ class MsgStream(trio.abc.Channel): return msg['yield'] except KeyError as kerr: - _raise_from_no_yield_msg( - stream=self, + _raise_from_no_key_in_msg( + ctx=self._ctx, msg=msg, src_err=kerr, + log=log, + expect_key='yield', + stream=self, ) except ( From 45e9cb4d09f407cda316388fbc673119936be391 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 2 Jan 2024 18:43:43 -0500 Subject: [PATCH 066/146] `_root`: drop unused `typing` import --- tractor/_root.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tractor/_root.py b/tractor/_root.py index a2d31586..853eaf0b 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -25,7 +25,6 @@ import logging import signal import sys import os -import typing import warnings From 4aa89bf391e48547b6020fdc691a5058b96aee56 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 3 Jan 2024 22:27:05 -0500 Subject: [PATCH 067/146] Bump timeout on resource cache test a bitty bit. --- tests/test_resource_cache.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/test_resource_cache.py b/tests/test_resource_cache.py index 0e7ad74d..d3859814 100644 --- a/tests/test_resource_cache.py +++ b/tests/test_resource_cache.py @@ -34,7 +34,6 @@ def test_resource_only_entered_once(key_on): global _resource _resource = 0 - kwargs = {} key = None if key_on == 'key_value': key = 'some_common_key' @@ -139,7 +138,7 @@ def test_open_local_sub_to_stream(): N local tasks using ``trionics.maybe_open_context():``. ''' - timeout = 3 if platform.system() != "Windows" else 10 + timeout: float = 3.6 if platform.system() != "Windows" else 10 async def main(): From a2f1bcc23f1d550b8b4410aed5be8875d6a998a2 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 23 Jan 2024 11:09:38 -0500 Subject: [PATCH 068/146] Use `import as blah` over `__all__` in `.trionics` --- tractor/trionics/__init__.py | 23 +++++++---------------- 1 file changed, 7 insertions(+), 16 deletions(-) diff --git a/tractor/trionics/__init__.py b/tractor/trionics/__init__.py index 31e49a9a..c51b7c51 100644 --- a/tractor/trionics/__init__.py +++ b/tractor/trionics/__init__.py @@ -19,22 +19,13 @@ Sugary patterns for trio + tractor designs. ''' from ._mngrs import ( - gather_contexts, - maybe_open_context, - maybe_open_nursery, + gather_contexts as gather_contexts, + maybe_open_context as maybe_open_context, + maybe_open_nursery as maybe_open_nursery, ) from ._broadcast import ( - broadcast_receiver, - BroadcastReceiver, - Lagged, + AsyncReceiver as AsyncReceiver, + broadcast_receiver as broadcast_receiver, + BroadcastReceiver as BroadcastReceiver, + Lagged as Lagged, ) - - -__all__ = [ - 'gather_contexts', - 'broadcast_receiver', - 'BroadcastReceiver', - 'Lagged', - 'maybe_open_context', - 'maybe_open_nursery', -] From cdee6f93547984099801e3964b4287ee17cac4e2 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 23 Jan 2024 11:13:07 -0500 Subject: [PATCH 069/146] Try allowing multi-pops of `_Cache.locks` for now? --- tractor/trionics/_mngrs.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tractor/trionics/_mngrs.py b/tractor/trionics/_mngrs.py index 801b138b..1c079cdb 100644 --- a/tractor/trionics/_mngrs.py +++ b/tractor/trionics/_mngrs.py @@ -225,6 +225,7 @@ async def maybe_open_context( # yielded output yielded: Any = None + lock_registered: bool = False # Lock resource acquisition around task racing / ``trio``'s # scheduler protocol. @@ -232,6 +233,7 @@ async def maybe_open_context( # to allow re-entrant use cases where one `maybe_open_context()` # wrapped factor may want to call into another. lock = _Cache.locks.setdefault(fid, trio.Lock()) + lock_registered: bool = True await lock.acquire() # XXX: one singleton nursery per actor and we want to @@ -291,4 +293,9 @@ async def maybe_open_context( _, no_more_users = entry no_more_users.set() - _Cache.locks.pop(fid) + if lock_registered: + maybe_lock = _Cache.locks.pop(fid, None) + if maybe_lock is None: + log.error( + f'Resource lock for {fid} ALREADY POPPED?' + ) From b5431c0343894795f69ffbfa29695b0d7e3c09d3 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 23 Jan 2024 11:14:10 -0500 Subject: [PATCH 070/146] Never mask original `KeyError` in portal-error unwrapper, for now? --- tractor/_portal.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tractor/_portal.py b/tractor/_portal.py index 15c8cb50..fc094593 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -68,10 +68,10 @@ def _unwrap_msg( __tracebackhide__ = True try: return msg['return'] - except KeyError: + except KeyError as ke: # internal error should never get here assert msg.get('cid'), "Received internal error at portal?" - raise unpack_error(msg, channel) from None + raise unpack_error(msg, channel) from ke class Portal: From a86275996ca551b9bd110901e46fd7bbc0a86a03 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 14 Feb 2024 16:13:32 -0500 Subject: [PATCH 071/146] Offer `unpack_error(hid_tb: bool)` for `pdbp` REPL config --- tractor/_exceptions.py | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 7e148586..c75b7855 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -185,7 +185,8 @@ def unpack_error( msg: dict[str, Any], chan=None, - err_type=RemoteActorError + err_type=RemoteActorError, + hide_tb: bool = True, ) -> None | Exception: ''' @@ -196,7 +197,7 @@ def unpack_error( which is the responsibilitiy of the caller. ''' - __tracebackhide__: bool = True + __tracebackhide__: bool = hide_tb error_dict: dict[str, dict] | None if ( @@ -309,6 +310,11 @@ def _raise_from_no_key_in_msg( # value out of the underlying feed mem chan! stream._eoc: bool = True + # TODO: if the a local task is already blocking on + # a `Context.result()` and thus a `.receive()` on the + # rx-chan, we close the chan and set state ensuring that + # an eoc is raised! + # # when the send is closed we assume the stream has # # terminated and signal this local iterator to stop # await stream.aclose() @@ -317,8 +323,8 @@ def _raise_from_no_key_in_msg( # raise a ``StopAsyncIteration`` **and** in our catch # block below it will trigger ``.aclose()``. raise trio.EndOfChannel( - 'Context[{cid}] stream ended due to msg:\n' - f'{pformat(msg)}' + f'Context stream ended due to msg:\n' + f'{pformat(msg)}' ) from src_err From 9fc9b10b5370f9384955e62655bd731b2aa7c6d1 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 16 Feb 2024 15:23:00 -0500 Subject: [PATCH 072/146] Add `StreamOverrun.sender: tuple` for better handling Since it's generally useful to know who is the cause of an overrun (say bc you want your system to then adjust the writer side to slow tf down) might as well pack an extra `.sender: tuple[str, str]` actor uid field which can be relayed through `RemoteActorError` boxing. Add an extra case for the exc-type to `unpack_error()` to match B) --- tractor/_exceptions.py | 25 ++++++++++++++++++++++--- 1 file changed, 22 insertions(+), 3 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index c75b7855..dcabf402 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -134,8 +134,19 @@ class NoRuntime(RuntimeError): "The root actor has not been initialized yet" -class StreamOverrun(trio.TooSlowError): - "This stream was overrun by sender" +class StreamOverrun( + RemoteActorError, + trio.TooSlowError, +): + ''' + This stream was overrun by sender + + ''' + @property + def sender(self) -> tuple[str, str] | None: + value = self.msgdata.get('sender') + if value: + return tuple(value) class AsyncioCancelled(Exception): @@ -175,7 +186,15 @@ def pack_error( 'src_actor_uid': current_actor().uid, } - if isinstance(exc, ContextCancelled): + # TODO: ?just wholesale proxy `.msgdata: dict`? + # XXX WARNING, when i swapped these ctx-semantics + # tests started hanging..???!!!??? + # if msgdata := exc.getattr('msgdata', {}): + # error_msg.update(msgdata) + if ( + isinstance(exc, ContextCancelled) + or isinstance(exc, StreamOverrun) + ): error_msg.update(exc.msgdata) return {'error': error_msg} From c85757aee110123339de2eaa482036a313819812 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 18 Feb 2024 17:17:31 -0500 Subject: [PATCH 073/146] Let `pack_error()` take a msg injected `cid: str|None` --- tractor/_exceptions.py | 26 ++++++++++++++++---------- 1 file changed, 16 insertions(+), 10 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index dcabf402..d63cf6d4 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -163,13 +163,15 @@ class MessagingError(Exception): def pack_error( exc: BaseException, - tb: str | None = None, + tb: str|None = None, + cid: str|None = None, ) -> dict[str, dict]: ''' - Create an "error message" encoded for wire transport via an IPC - `Channel`; expected to be unpacked on the receiver side using - `unpack_error()` below. + Create an "error message" which boxes a locally caught + exception's meta-data and encodes it for wire transport via an + IPC `Channel`; expected to be unpacked (and thus unboxed) on + the receiver side using `unpack_error()` below. ''' if tb: @@ -197,7 +199,12 @@ def pack_error( ): error_msg.update(exc.msgdata) - return {'error': error_msg} + + pkt: dict = {'error': error_msg} + if cid: + pkt['cid'] = cid + + return pkt def unpack_error( @@ -207,7 +214,7 @@ def unpack_error( err_type=RemoteActorError, hide_tb: bool = True, -) -> None | Exception: +) -> None|Exception: ''' Unpack an 'error' message from the wire into a local `RemoteActorError` (subtype). @@ -358,8 +365,7 @@ def _raise_from_no_key_in_msg( # is activated above. _type: str = 'Stream' if stream else 'Context' raise MessagingError( - f'{_type} was expecting a `{expect_key}` message' - ' BUT received a non-`error` msg:\n' - f'cid: {cid}\n' - '{pformat(msg)}' + f"{_type} was expecting a '{expect_key}' message" + " BUT received a non-error msg:\n" + f'{pformat(msg)}' ) from src_err From 6a303358df7e2b66154fac14e70193df3f831ad3 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 19 Feb 2024 12:25:08 -0500 Subject: [PATCH 074/146] Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `': \n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified. --- tractor/_ipc.py | 10 +- tractor/_runtime.py | 337 ++++++++++++++++++++++++++++++++++---------- 2 files changed, 266 insertions(+), 81 deletions(-) diff --git a/tractor/_ipc.py b/tractor/_ipc.py index f3579143..e5e4d223 100644 --- a/tractor/_ipc.py +++ b/tractor/_ipc.py @@ -19,13 +19,14 @@ Inter-process comms abstractions """ from __future__ import annotations -import platform import struct -import typing +import platform +from pprint import pformat from collections.abc import ( AsyncGenerator, AsyncIterator, ) +import typing from typing import ( Any, runtime_checkable, @@ -370,7 +371,10 @@ class Channel: async def send(self, item: Any) -> None: - log.transport(f"send `{item}`") # type: ignore + log.transport( + '=> send IPC msg:\n\n' + f'{pformat(item)}\n' + ) # type: ignore assert self.msgstream await self.msgstream.send(item) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 46bdd2b5..58eacf4c 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -28,6 +28,7 @@ from itertools import chain import importlib import importlib.util import inspect +from pprint import pformat import signal import sys from typing import ( @@ -48,6 +49,10 @@ import trio from trio import ( CancelScope, ) +from trio.lowlevel import ( + current_task, + Task, +) from trio_typing import ( Nursery, TaskStatus, @@ -80,6 +85,26 @@ if TYPE_CHECKING: log = get_logger('tractor') +_gb_mod: ModuleType|None|False = None + + +async def maybe_import_gb(): + global _gb_mod + if _gb_mod is False: + return + + try: + import greenback + _gb_mod = greenback + await greenback.ensure_portal() + + except ModuleNotFoundError: + log.warning( + '`greenback` is not installed.\n' + 'No sync debug support!' + ) + _gb_mod = False + async def _invoke( @@ -227,15 +252,27 @@ async def _invoke( # wrapper that calls `Context.started()` and then does # the `await coro()`? elif context: - # context func with support for bi-dir streaming - await chan.send({'functype': 'context', 'cid': cid}) + + # a "context" endpoint type is the most general and + # "least sugary" type of RPC ep with support for + # bi-dir streaming B) + await chan.send({ + 'functype': 'context', + 'cid': cid + }) try: async with trio.open_nursery() as nurse: ctx._scope_nursery = nurse ctx._scope = nurse.cancel_scope task_status.started(ctx) + + # TODO: should would be nice to have our + # `TaskMngr` nursery here! + # res: Any = await coro res = await coro + + # deliver final result to caller side. await chan.send({ 'return': res, 'cid': cid @@ -271,9 +308,10 @@ async def _invoke( # associated child isn't in debug any more await _debug.maybe_wait_for_debugger() ctx: Context = actor._contexts.pop((chan.uid, cid)) - log.runtime( - f'Context entrypoint {func} was terminated:\n' - f'{ctx}' + log.cancel( + f'Context task was terminated:\n' + f'func: {func}\n' + f'ctx: {pformat(ctx)}' ) if ctx.cancelled_caught: @@ -285,13 +323,14 @@ async def _invoke( if re := ctx._remote_error: ctx._maybe_raise_remote_err(re) - fname: str = func.__name__ + # fname: str = func.__name__ + task: Task = current_task() cs: CancelScope = ctx._scope if cs.cancel_called: our_uid: tuple = actor.uid canceller: tuple = ctx.canceller msg: str = ( - f'`{fname}()`@{our_uid} cancelled by ' + 'actor was cancelled by ' ) # NOTE / TODO: if we end up having @@ -310,16 +349,37 @@ async def _invoke( # some actor who calls `Portal.cancel_actor()` # and by side-effect cancels this ctx. elif canceller == ctx.chan.uid: - msg += f'its caller {canceller} ' + msg += 'its caller' else: - msg += f'remote actor {canceller}' + msg += 'a remote peer' + + div_chars: str = '------ - ------' + div_offset: int = ( + round(len(msg)/2)+1 + + + round(len(div_chars)/2)+1 + ) + div_str: str = ( + '\n' + + + ' '*div_offset + + + f'{div_chars}\n' + ) + msg += ( + div_str + + f'<= canceller: {canceller}\n' + f'=> uid: {our_uid}\n' + f' |_ task: `{task.name}()`' + ) # TODO: does this ever get set any more or can # we remove it? if ctx._cancel_msg: msg += ( - ' with msg:\n' + '------ - ------\n' + 'IPC msg:\n' f'{ctx._cancel_msg}' ) @@ -439,9 +499,9 @@ async def _invoke( # always ship errors back to caller err_msg: dict[str, dict] = pack_error( err, - tb=tb, + # tb=tb, # TODO: special tb fmting? + cid=cid, ) - err_msg['cid'] = cid if is_rpc: try: @@ -508,19 +568,28 @@ async def try_ship_error_to_parent( err: Union[Exception, BaseExceptionGroup], ) -> None: + ''' + Box, pack and encode a local runtime(-internal) exception for + an IPC channel `.send()` with transport/network failures and + local cancellation ignored but logged as critical(ly bad). + + ''' with CancelScope(shield=True): try: - # internal error so ship to parent without cid - await channel.send(pack_error(err)) + await channel.send( + # NOTE: normally only used for internal runtime errors + # so ship to peer actor without a cid. + pack_error(err) + ) except ( trio.ClosedResourceError, trio.BrokenResourceError, ): # in SC terms this is one of the worst things that can - # happen and creates the 2-general's dilemma. + # happen and provides for a 2-general's dilemma.. log.critical( - f"Failed to ship error to parent " - f"{channel.uid}, channel was closed" + f'Failed to ship error to parent ' + f'{channel.uid}, IPC transport failure!' ) @@ -573,6 +642,11 @@ class Actor: # if started on ``asycio`` running ``trio`` in guest mode _infected_aio: bool = False + # _ans: dict[ + # tuple[str, str], + # list[ActorNursery], + # ] = {} + # Process-global stack closed at end on actor runtime teardown. # NOTE: this is currently an undocumented public api. lifetime_stack: ExitStack = ExitStack() @@ -593,7 +667,10 @@ class Actor: ''' self.name = name - self.uid = (name, uid or str(uuid.uuid4())) + self.uid = ( + name, + uid or str(uuid.uuid4()) + ) self._cancel_complete = trio.Event() self._cancel_called_by_remote: tuple[str, tuple] | None = None @@ -762,7 +839,10 @@ class Actor: return # channel tracking - event = self._peer_connected.pop(uid, None) + event: trio.Event|None = self._peer_connected.pop( + uid, + None, + ) if event: # Instructing connection: this is likely a new channel to # a recently spawned actor which we'd like to control via @@ -771,46 +851,43 @@ class Actor: # Alert any task waiting on this connection to come up event.set() - chans = self._peers[uid] - - # TODO: re-use channels for new connections instead - # of always new ones; will require changing all the - # discovery funcs + chans: list[Channel] = self._peers[uid] if chans: + # TODO: re-use channels for new connections instead + # of always new ones? + # => will require changing all the discovery funcs.. log.runtime( f"already have channel(s) for {uid}:{chans}?" ) - log.runtime(f"Registered {chan} for {uid}") # type: ignore # append new channel + log.runtime(f"Registered {chan} for {uid}") # type: ignore + # TODO: can we just use list-ref directly? + # chans.append(chan) self._peers[uid].append(chan) - local_nursery: ActorNursery | None = None # noqa - disconnected: bool = False - # Begin channel management - respond to remote requests and # process received reponses. + disconnected: bool = False try: - disconnected = await process_messages(self, chan) - - except ( - trio.Cancelled, - ): - log.cancel(f"Msg loop was cancelled for {chan}") + disconnected: bool = await process_messages(self, chan) + except trio.Cancelled: + log.cancel(f'Msg loop was cancelled for {chan}') raise finally: - local_nursery = self._actoruid2nursery.get(uid, local_nursery) + local_nursery: ( + ActorNursery|None + ) = self._actoruid2nursery.get(uid) # This is set in ``Portal.cancel_actor()``. So if # the peer was cancelled we try to wait for them # to tear down their side of the connection before # moving on with closing our own side. - if ( - local_nursery - ): + if local_nursery: if chan._cancel_called: - log.cancel(f"Waiting on cancel request to peer {chan.uid}") + log.cancel(f'Waiting on cancel request to peer {chan.uid}') + # XXX: this is a soft wait on the channel (and its # underlying transport protocol) to close from the # remote peer side since we presume that any channel @@ -853,26 +930,48 @@ class Actor: # the cause of other downstream errors. entry = local_nursery._children.get(uid) if entry: + proc: trio.Process _, proc, _ = entry poll = getattr(proc, 'poll', None) if poll and poll() is None: log.cancel( - f'Actor {uid} IPC broke but proc is alive?' + f'Peer actor IPC broke but proc is alive?\n' + f'uid: {uid}\n' + f'|_{proc}\n' ) # ``Channel`` teardown and closure sequence # Drop ref to channel so it can be gc-ed and disconnected - log.runtime(f"Releasing channel {chan} from {chan.uid}") + log.runtime( + f'Disconnected IPC channel:\n' + f'uid: {chan.uid}\n' + f'|_{pformat(chan)}\n' + ) chans = self._peers.get(chan.uid) chans.remove(chan) if not chans: - log.runtime(f"No more channels for {chan.uid}") + log.runtime( + f'No more channels with {chan.uid}' + ) self._peers.pop(uid, None) - log.runtime(f"Peers is {self._peers}") + peers_str: str = '' + for uid, chans in self._peers.items(): + peers_str += ( + f'- uid: {uid}\n' + ) + for i, chan in enumerate(chans): + peers_str += ( + f' |_[{i}] {pformat(chan)}\n' + ) + + log.runtime( + f'Remaining IPC {len(self._peers)} peers:\n' + + peers_str + ) # No more channels to other actors (at all) registered # as connected. @@ -888,15 +987,58 @@ class Actor: if _state.is_root_process(): pdb_lock = _debug.Lock pdb_lock._blocked.add(uid) - log.runtime(f"{uid} blocked from pdb locking") + # TODO: NEEEDS TO BE TESTED! + # actually, no idea if this ever even enters.. XD + pdb_user_uid: tuple = pdb_lock.global_actor_in_debug + if ( + pdb_user_uid + and local_nursery + ): + entry: tuple|None = local_nursery._children.get(pdb_user_uid) + if entry: + proc: trio.Process + _, proc, _ = entry + + if ( + (poll := getattr(proc, 'poll', None)) + and poll() is None + ): + log.cancel( + 'Root actor reports no-more-peers, BUT ' + 'a DISCONNECTED child still has the debug ' + 'lock!\n' + f'root uid: {self.uid}\n' + f'last disconnected child uid: {uid}\n' + f'locking child uid: {pdb_user_uid}\n' + ) + await _debug.maybe_wait_for_debugger( + child_in_debug=True + ) + + # TODO: just bc a child's transport dropped + # doesn't mean it's not still using the pdb + # REPL! so, + # -[ ] ideally we can check out child proc + # tree to ensure that its alive (and + # actually using the REPL) before we cancel + # it's lock acquire by doing the below! + # -[ ] create a way to read the tree of each actor's + # grandchildren such that when an + # intermediary parent is cancelled but their + # child has locked the tty, the grandparent + # will not allow the parent to cancel or + # zombie reap the child! see open issue: + # - https://github.com/goodboy/tractor/issues/320 + # ------ - ------ # if a now stale local task has the TTY lock still # we cancel it to allow servicing other requests for # the lock. - db_cs = pdb_lock._root_local_task_cs_in_debug + db_cs: trio.CancelScope|None = pdb_lock._root_local_task_cs_in_debug if ( db_cs and not db_cs.cancel_called + and uid == pdb_user_uid ): log.warning( f'STALE DEBUG LOCK DETECTED FOR {uid}' @@ -928,15 +1070,16 @@ class Actor: chan: Channel, cid: str, msg: dict[str, Any], - ) -> None: + + ) -> None|bool: ''' Push an RPC result to the local consumer's queue. ''' - uid = chan.uid + uid: tuple[str, str] = chan.uid assert uid, f"`chan.uid` can't be {uid}" try: - ctx = self._contexts[(uid, cid)] + ctx: Context = self._contexts[(uid, cid)] except KeyError: log.warning( f'Ignoring msg from [no-longer/un]known context {uid}:' @@ -1066,6 +1209,16 @@ class Actor: parent_data.pop('bind_port'), ) rvs = parent_data.pop('_runtime_vars') + + if rvs['_debug_mode']: + try: + from .devx import enable_stack_on_sig + enable_stack_on_sig() + except ImportError: + log.warning( + '`stackscope` not installed for use in debug mode!' + ) + log.runtime(f"Runtime vars are: {rvs}") rvs['_is_root'] = False _state._runtime_vars.update(rvs) @@ -1284,9 +1437,15 @@ class Actor: ''' tasks: dict = self._rpc_tasks if tasks: + tasks_str: str = '' + for (ctx, func, _) in tasks.values(): + tasks_str += ( + f' |_{func.__name__}() [cid={ctx.cid[-6:]}..]\n' + ) + log.cancel( f'Cancelling all {len(tasks)} rpc tasks:\n' - f'{tasks}' + f'{tasks_str}' ) for ( (chan, cid), @@ -1511,7 +1670,10 @@ async def async_main( ) if actor._parent_chan: - await try_ship_error_to_parent(actor._parent_chan, err) + await try_ship_error_to_parent( + actor._parent_chan, + err, + ) # always! match err: @@ -1595,43 +1757,53 @@ async def process_messages( or boxed errors back to the remote caller (task). ''' - # TODO: once https://github.com/python-trio/trio/issues/467 gets - # worked out we'll likely want to use that! - msg: dict | None = None + # TODO: once `trio` get's an "obvious way" for req/resp we + # should use it? + # https://github.com/python-trio/trio/issues/467 + log.runtime( + 'Entering IPC msg loop:\n' + f'peer: {chan.uid}\n' + f'|_{chan}' + ) nursery_cancelled_before_task: bool = False - - log.runtime(f"Entering msg loop for {chan} from {chan.uid}") + msg: dict | None = None try: + # NOTE: this internal scope allows for keeping this + # message loop running despite the current task having + # been cancelled (eg. `open_portal()` may call this method + # from a locally spawned task) and recieve this scope + # using ``scope = Nursery.start()`` with CancelScope(shield=shield) as loop_cs: - # this internal scope allows for keeping this message - # loop running despite the current task having been - # cancelled (eg. `open_portal()` may call this method from - # a locally spawned task) and recieve this scope using - # ``scope = Nursery.start()`` task_status.started(loop_cs) async for msg in chan: - if msg is None: # loop terminate sentinel + # dedicated loop terminate sentinel + if msg is None: + tasks: dict[ + tuple[Channel, str], + tuple[Context, Callable, trio.Event] + ] = actor._rpc_tasks.copy() log.cancel( - f"Channel to {chan.uid} terminated?\n" - "Cancelling all associated tasks..") - - for (channel, cid) in actor._rpc_tasks.copy(): + f'Peer IPC channel terminated via `None` setinel msg?\n' + f'=> Cancelling all {len(tasks)} local RPC tasks..\n' + f'peer: {chan.uid}\n' + f'|_{chan}\n' + ) + for (channel, cid) in tasks: if channel is chan: await actor._cancel_task( cid, channel, ) - - log.runtime( - f"Msg loop signalled to terminate for" - f" {chan} from {chan.uid}") - break log.transport( # type: ignore - f"Received msg {msg} from {chan.uid}") + f'<= IPC msg from peer: {chan.uid}\n\n' + # TODO: conditionally avoid fmting depending + # on log level (for perf)? + f'{pformat(msg)}\n' + ) cid = msg.get('cid') if cid: @@ -1640,7 +1812,10 @@ async def process_messages( await actor._push_result(chan, cid, msg) log.runtime( - f"Waiting on next msg for {chan} from {chan.uid}") + f'Waiting on next IPC msg from {chan.uid}:\n' + # f'last msg: {msg}\n' + f'|_{chan}' + ) continue # TODO: implement with ``match:`` syntax? @@ -1693,7 +1868,7 @@ async def process_messages( ) log.cancel( - f'Cancelling msg loop for {chan.uid}' + f'Cancelling IPC msg-loop with {chan.uid}' ) loop_cs.cancel() break @@ -1735,8 +1910,10 @@ async def process_messages( try: func = actor._get_rpc_func(ns, funcname) except (ModuleNotExposed, AttributeError) as err: - err_msg = pack_error(err) - err_msg['cid'] = cid + err_msg: dict[str, dict] = pack_error( + err, + cid=cid, + ) await chan.send(err_msg) continue @@ -1838,7 +2015,10 @@ async def process_messages( log.exception("Actor errored:") if actor._parent_chan: - await try_ship_error_to_parent(actor._parent_chan, err) + await try_ship_error_to_parent( + actor._parent_chan, + err, + ) # if this is the `MainProcess` we expect the error broadcasting # above to trigger an error at consuming portal "checkpoints" @@ -1847,8 +2027,9 @@ async def process_messages( finally: # msg debugging for when he machinery is brokey log.runtime( - f"Exiting msg loop for {chan} from {chan.uid} " - f"with last msg:\n{msg}" + f'Exiting IPC msg loop with {chan.uid} ' + f'final msg: {msg}\n' + f'|_{chan}' ) # transport **was not** disconnected From 4b0aa5e3799c7c5caae5ccdb225cfd8bea162e4f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 19 Feb 2024 14:41:03 -0500 Subject: [PATCH 075/146] Baboso! fix `chan.send(None)` indent.. --- tractor/_runtime.py | 39 ++++++++++++++++++++++++--------------- 1 file changed, 24 insertions(+), 15 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 58eacf4c..e34e1a5e 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -1047,23 +1047,32 @@ class Actor: db_cs.cancel() # XXX: is this necessary (GC should do it)? - if chan.connected(): - # if the channel is still connected it may mean the far - # end has not closed and we may have gotten here due to - # an error and so we should at least try to terminate - # the channel from this end gracefully. + # XXX WARNING XXX + # Be AWARE OF THE INDENT LEVEL HERE + # -> ONLY ENTER THIS BLOCK WHEN ._peers IS + # EMPTY!!!! + if ( + not self._peers + and chan.connected() + ): + # if the channel is still connected it may mean the far + # end has not closed and we may have gotten here due to + # an error and so we should at least try to terminate + # the channel from this end gracefully. + log.runtime( + 'Terminating channel with `None` setinel msg\n' + f'|_{chan}\n' + ) + try: + # send a msg loop terminate sentinel + await chan.send(None) - log.runtime(f"Disconnecting channel {chan}") - try: - # send a msg loop terminate sentinel - await chan.send(None) + # XXX: do we want this? + # causes "[104] connection reset by peer" on other end + # await chan.aclose() - # XXX: do we want this? - # causes "[104] connection reset by peer" on other end - # await chan.aclose() - - except trio.BrokenResourceError: - log.runtime(f"Channel {chan.uid} was already closed") + except trio.BrokenResourceError: + log.runtime(f"Channel {chan.uid} was already closed") async def _push_result( self, From ddc2e5f0f82d96542d34724ac7f4c0aa0849158f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 19 Feb 2024 17:00:46 -0500 Subject: [PATCH 076/146] WIP: solved the modden client hang.. --- tractor/_context.py | 602 ++++++++++++++++++++++++++++++++++-------- tractor/_portal.py | 167 ++++++++++-- tractor/_streaming.py | 188 ++++++++++--- 3 files changed, 793 insertions(+), 164 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index 2b539a6f..54e309e1 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -43,12 +43,17 @@ import warnings import trio +# from .devx import ( +# maybe_wait_for_debugger, +# pause, +# ) from ._exceptions import ( # _raise_from_no_key_in_msg, unpack_error, pack_error, ContextCancelled, # MessagingError, + RemoteActorError, StreamOverrun, ) from .log import get_logger @@ -64,6 +69,164 @@ if TYPE_CHECKING: log = get_logger(__name__) +async def _drain_to_final_msg( + ctx: Context, +) -> list[dict]: + +# ) -> tuple[ +# Any|Exception, +# list[dict], +# ]: + raise_overrun: bool = not ctx._allow_overruns + + # wait for a final context result by collecting (but + # basically ignoring) any bi-dir-stream msgs still in transit + # from the far end. + pre_result_drained: list[dict] = [] + while not ctx._remote_error: + try: + # NOTE: this REPL usage actually works here dawg! Bo + # from .devx._debug import pause + # await pause() + # if re := ctx._remote_error: + # ctx._maybe_raise_remote_err( + # re, + # # NOTE: obvi we don't care if we + # # overran the far end if we're already + # # waiting on a final result (msg). + # raise_overrun_from_self=raise_overrun, + # ) + + # TODO: bad idea? + # with trio.CancelScope() as res_cs: + # ctx._res_scope = res_cs + # msg: dict = await ctx._recv_chan.receive() + # if res_cs.cancelled_caught: + + # from .devx._debug import pause + # await pause() + msg: dict = await ctx._recv_chan.receive() + ctx._result: Any = msg['return'] + log.runtime( + 'Context delivered final result msg:\n' + f'{pformat(msg)}' + ) + pre_result_drained.append(msg) + # NOTE: we don't need to do this right? + # XXX: only close the rx mem chan AFTER + # a final result is retreived. + # if ctx._recv_chan: + # await ctx._recv_chan.aclose() + break + + # NOTE: we get here if the far end was + # `ContextCancelled` in 2 cases: + # 1. we requested the cancellation and thus + # SHOULD NOT raise that far end error, + # 2. WE DID NOT REQUEST that cancel and thus + # SHOULD RAISE HERE! + except trio.Cancelled: + + # 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. + if re := ctx._remote_error: + ctx._maybe_raise_remote_err(re) + + # CASE 1: we DID request the cancel we simply + # continue to bubble up as normal. + raise + + except KeyError: + + if 'yield' in msg: + # far end task is still streaming to us so discard + log.warning(f'Discarding std "yield"\n{msg}') + pre_result_drained.append(msg) + continue + + # TODO: work out edge cases here where + # a stream is open but the task also calls + # this? + # -[ ] should be a runtime error if a stream is open + # right? + elif 'stop' in msg: + log.cancel( + 'Remote stream terminated due to "stop" msg:\n' + f'{msg}' + ) + pre_result_drained.append(msg) + continue + + # internal error should never get here + assert msg.get('cid'), ( + "Received internal error at portal?" + ) + + # XXX fallthrough to handle expected error XXX + re: Exception|None = ctx._remote_error + if re: + log.critical( + 'Remote ctx terminated due to "error" msg:\n' + f'{re}' + ) + assert msg is ctx._cancel_msg + # NOTE: this solved a super dupe edge case XD + # this was THE super duper edge case of: + # - local task opens a remote task, + # - requests remote cancellation of far end + # ctx/tasks, + # - needs to wait for the cancel ack msg + # (ctxc) or some result in the race case + # where the other side's task returns + # before the cancel request msg is ever + # rxed and processed, + # - here this surrounding drain loop (which + # iterates all ipc msgs until the ack or + # an early result arrives) was NOT exiting + # since we are the edge case: local task + # does not re-raise any ctxc it receives + # IFF **it** was the cancellation + # requester.. + # will raise if necessary, ow break from + # loop presuming any error terminates the + # context! + ctx._maybe_raise_remote_err( + re, + # NOTE: obvi we don't care if we + # overran the far end if we're already + # waiting on a final result (msg). + # raise_overrun_from_self=False, + raise_overrun_from_self=raise_overrun, + ) + + break # OOOOOF, yeah obvi we need this.. + + # XXX we should never really get here + # right! since `._deliver_msg()` should + # always have detected an {'error': ..} + # msg and already called this right!?! + elif error := unpack_error( + msg=msg, + chan=ctx._portal.channel, + hide_tb=False, + ): + log.critical('SHOULD NEVER GET HERE!?') + assert msg is ctx._cancel_msg + assert error.msgdata == ctx._remote_error.msgdata + from .devx._debug import pause + await pause() + ctx._maybe_cancel_and_set_remote_error(error) + ctx._maybe_raise_remote_err(error) + + else: + # bubble the original src key error + raise + + return pre_result_drained + + # TODO: make this a msgspec.Struct! @dataclass class Context: @@ -118,6 +281,7 @@ class Context: # which is exactly the primitive that allows for # cross-actor-task-supervision and thus SC. _scope: trio.CancelScope | None = None + # _res_scope: trio.CancelScope|None = None # on a clean exit there should be a final value # delivered from the far end "callee" task, so @@ -205,6 +369,10 @@ class Context: ) ) + # @property + # def is_waiting_result(self) -> bool: + # return bool(self._res_scope) + @property def side(self) -> str: ''' @@ -247,7 +415,11 @@ class Context: await self.chan.send({'yield': data, 'cid': self.cid}) async def send_stop(self) -> None: - await self.chan.send({'stop': True, 'cid': self.cid}) + # await pause() + await self.chan.send({ + 'stop': True, + 'cid': self.cid + }) def _maybe_cancel_and_set_remote_error( self, @@ -320,27 +492,37 @@ class Context: # XXX: set the remote side's error so that after we cancel # whatever task is the opener of this context it can raise # that error as the reason. + # if self._remote_error: + # return + + # breakpoint() + log.cancel( + 'Setting remote error for ctx \n' + f'<= remote ctx uid: {self.chan.uid}\n' + f'=>\n{error}' + ) self._remote_error: BaseException = error if ( isinstance(error, ContextCancelled) ): - # always record the cancelling actor's uid since its cancellation - # state is linked and we want to know which process was - # the cause / requester of the cancellation. - self._canceller = error.canceller - log.cancel( 'Remote task-context was cancelled for ' f'actor: {self.chan.uid}\n' f'task: {self.cid}\n' f'canceller: {error.canceller}\n' ) + # always record the cancelling actor's uid since its cancellation + # state is linked and we want to know which process was + # the cause / requester of the cancellation. + # if error.canceller is None: + # import pdbp; pdbp.set_trace() + + # breakpoint() + self._canceller = error.canceller + if self._cancel_called: - # from ._debug import breakpoint - # await breakpoint() - # this is an expected cancel request response message # and we **don't need to raise it** in local cancel # scope since it will potentially override a real error. @@ -348,10 +530,11 @@ class Context: else: log.error( - f'Remote context error,\n' - f'remote actor: {self.chan.uid}\n' - f'task: {self.cid}\n' - f'{error}' + f'Remote context error:\n' + f'{error}\n' + f'{pformat(self)}\n' + # f'remote actor: {self.chan.uid}\n' + # f'cid: {self.cid}\n' ) self._canceller = self.chan.uid @@ -376,9 +559,11 @@ class Context: self._scope.cancel() # NOTE: this REPL usage actually works here dawg! Bo - # from .devx._debug import pause # await pause() + # TODO: maybe we have to use `._res_scope.cancel()` if it + # exists? + async def cancel( self, timeout: float = 0.616, @@ -395,6 +580,8 @@ class Context: log.cancel( f'Cancelling {side} side of context to {self.chan.uid}' ) + + # await pause() self._cancel_called: bool = True # caller side who entered `Portal.open_context()` @@ -484,13 +671,11 @@ class Context: ''' actor: Actor = current_actor() - # here we create a mem chan that corresponds to the - # far end caller / callee. - - # Likewise if the surrounding context has been cancelled we error here - # since it likely means the surrounding block was exited or - # killed - + # If the surrounding context has been cancelled by some + # task with a handle to THIS, we error here immediately + # since it likely means the surrounding lexical-scope has + # errored, been `trio.Cancelled` or at the least + # `Context.cancel()` was called by some task. if self._cancel_called: # XXX NOTE: ALWAYS RAISE any remote error here even if @@ -503,6 +688,11 @@ class Context: # actually try to stream - a cancel msg was already # sent to the other side! if self._remote_error: + # NOTE: this is diff then calling + # `._maybe_raise_from_remote_msg()` specifically + # because any task entering this `.open_stream()` + # AFTER cancellation has already been requested, + # we DO NOT want to absorb any ctxc ACK silently! raise self._remote_error # XXX NOTE: if no `ContextCancelled` has been responded @@ -529,7 +719,7 @@ class Context: # to send a stop from the caller to the callee in the # single-direction-stream case you'll get a lookup error # currently. - ctx = actor.get_context( + ctx: Context = actor.get_context( self.chan, self.cid, msg_buffer_size=msg_buffer_size, @@ -548,6 +738,19 @@ class Context: 'The underlying channel for this stream was already closed!?' ) + # NOTE: implicitly this will call `MsgStream.aclose()` on + # `.__aexit__()` due to stream's parent `Channel` type! + # + # XXX NOTE XXX: ensures the stream is "one-shot use", + # which specifically means that on exit, + # - signal ``trio.EndOfChannel``/``StopAsyncIteration`` to + # the far end indicating that the caller exited + # the streaming context purposefully by letting + # the exit block exec. + # - this is diff from the cancel/error case where + # a cancel request from this side or an error + # should be sent to the far end indicating the + # stream WAS NOT just closed normally/gracefully. async with MsgStream( ctx=self, rx_chan=ctx._recv_chan, @@ -567,11 +770,37 @@ class Context: # await trio.lowlevel.checkpoint() yield stream - # NOTE: Make the stream "one-shot use". On exit, - # signal - # ``trio.EndOfChannel``/``StopAsyncIteration`` to - # the far end. - await stream.aclose() + + # XXX: (MEGA IMPORTANT) if this is a root opened process we + # wait for any immediate child in debug before popping the + # context from the runtime msg loop otherwise inside + # ``Actor._push_result()`` the msg will be discarded and in + # the case where that msg is global debugger unlock (via + # a "stop" msg for a stream), this can result in a deadlock + # where the root is waiting on the lock to clear but the + # child has already cleared it and clobbered IPC. + # + # await maybe_wait_for_debugger() + + # XXX TODO: pretty sure this isn't needed (see + # note above this block) AND will result in + # a double `.send_stop()` call. The only reason to + # put it here would be to due with "order" in + # terms of raising any remote error (as per + # directly below) or bc the stream's + # `.__aexit__()` block might not get run + # (doubtful)? Either way if we did put this back + # in we also need a state var to avoid the double + # stop-msg send.. + # + # await stream.aclose() + + # if re := ctx._remote_error: + # ctx._maybe_raise_remote_err( + # re, + # raise_ctxc_from_self_call=True, + # ) + # await trio.lowlevel.checkpoint() finally: if self._portal: @@ -587,7 +816,10 @@ class Context: def _maybe_raise_remote_err( self, err: Exception, - ) -> None: + raise_ctxc_from_self_call: bool = False, + raise_overrun_from_self: bool = True, + + ) -> ContextCancelled|None: ''' Maybe raise a remote error depending on who (which task from which actor) requested a cancellation (if any). @@ -603,13 +835,21 @@ class Context: # "error"-msg. our_uid: tuple[str, str] = current_actor().uid if ( - isinstance(err, ContextCancelled) - and ( + (not raise_ctxc_from_self_call + and isinstance(err, ContextCancelled) + and ( self._cancel_called or self.chan._cancel_called or self.canceller == our_uid - or tuple(err.canceller) == our_uid + or tuple(err.canceller) == our_uid) ) + or + (not raise_overrun_from_self + and isinstance(err, RemoteActorError) + and err.msgdata['type_str'] == 'StreamOverrun' + and tuple(err.msgdata['sender']) == our_uid + ) + ): # NOTE: we set the local scope error to any "self # cancellation" error-response thus "absorbing" @@ -661,77 +901,196 @@ class Context: assert self._portal, "Context.result() can not be called from callee!" assert self._recv_chan - if re := self._remote_error: - return self._maybe_raise_remote_err(re) + raise_overrun: bool = not self._allow_overruns + # if re := self._remote_error: + # return self._maybe_raise_remote_err( + # re, + # # NOTE: obvi we don't care if we + # # overran the far end if we're already + # # waiting on a final result (msg). + # raise_overrun_from_self=raise_overrun, + # ) + res_placeholder: int = id(self) if ( - self._result == id(self) + self._result == res_placeholder and not self._remote_error and not self._recv_chan._closed # type: ignore ): - # wait for a final context result consuming - # and discarding any bi dir stream msgs still - # in transit from the far end. - while True: - try: - msg = await self._recv_chan.receive() - self._result: Any = msg['return'] - # NOTE: we don't need to do this right? - # XXX: only close the rx mem chan AFTER - # a final result is retreived. - # if self._recv_chan: - # await self._recv_chan.aclose() + # wait for a final context result by collecting (but + # basically ignoring) any bi-dir-stream msgs still in transit + # from the far end. + drained_msgs: list[dict] = await _drain_to_final_msg(ctx=self) + log.runtime( + 'Ctx drained pre-result msgs:\n' + f'{drained_msgs}' + ) - break + # TODO: implement via helper func ^^^^ + # pre_result_drained: list[dict] = [] + # while not self._remote_error: + # try: + # # NOTE: this REPL usage actually works here dawg! Bo + # # from .devx._debug import pause + # # await pause() + # # if re := self._remote_error: + # # self._maybe_raise_remote_err( + # # re, + # # # NOTE: obvi we don't care if we + # # # overran the far end if we're already + # # # waiting on a final result (msg). + # # raise_overrun_from_self=raise_overrun, + # # ) - # NOTE: we get here if the far end was - # `ContextCancelled` in 2 cases: - # 1. we requested the cancellation and thus - # SHOULD NOT raise that far end error, - # 2. WE DID NOT REQUEST that cancel and thus - # SHOULD RAISE HERE! - except trio.Cancelled: + # # TODO: bad idea? + # # with trio.CancelScope() as res_cs: + # # self._res_scope = res_cs + # # msg: dict = await self._recv_chan.receive() + # # if res_cs.cancelled_caught: - # CASE 2: mask the local cancelled-error(s) - # only when we are sure the remote error is the - # (likely) source cause of this local runtime - # task's cancellation. - if re := self._remote_error: - self._maybe_raise_remote_err(re) + # # from .devx._debug import pause + # # await pause() + # msg: dict = await self._recv_chan.receive() + # self._result: Any = msg['return'] + # log.runtime( + # 'Context delivered final result msg:\n' + # f'{pformat(msg)}' + # ) + # # NOTE: we don't need to do this right? + # # XXX: only close the rx mem chan AFTER + # # a final result is retreived. + # # if self._recv_chan: + # # await self._recv_chan.aclose() + # break - # CASE 1: we DID request the cancel we simply - # continue to bubble up as normal. - raise + # # NOTE: we get here if the far end was + # # `ContextCancelled` in 2 cases: + # # 1. we requested the cancellation and thus + # # SHOULD NOT raise that far end error, + # # 2. WE DID NOT REQUEST that cancel and thus + # # SHOULD RAISE HERE! + # except trio.Cancelled: - except KeyError: # as msgerr: + # # 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. + # if re := self._remote_error: + # self._maybe_raise_remote_err(re) - if 'yield' in msg: - # far end task is still streaming to us so discard - log.warning(f'Discarding stream delivered {msg}') - continue + # # CASE 1: we DID request the cancel we simply + # # continue to bubble up as normal. + # raise - elif 'stop' in msg: - log.debug('Remote stream terminated') - continue + # except KeyError: - # internal error should never get here - assert msg.get('cid'), ( - "Received internal error at portal?" - ) + # if 'yield' in msg: + # # far end task is still streaming to us so discard + # log.warning(f'Discarding std "yield"\n{msg}') + # pre_result_drained.append(msg) + # continue - if err:= unpack_error( - msg, - self._portal.channel - ): # from msgerr - self._maybe_cancel_and_set_remote_error(err) - self._maybe_raise_remote_err(err) + # # TODO: work out edge cases here where + # # a stream is open but the task also calls + # # this? + # # -[ ] should be a runtime error if a stream is open + # # right? + # elif 'stop' in msg: + # log.cancel( + # 'Remote stream terminated due to "stop" msg:\n' + # f'{msg}' + # ) + # pre_result_drained.append(msg) + # continue - else: - raise + # # internal error should never get here + # assert msg.get('cid'), ( + # "Received internal error at portal?" + # ) - if re := self._remote_error: - return self._maybe_raise_remote_err(re) + # # XXX fallthrough to handle expected error XXX + # re: Exception|None = self._remote_error + # if re: + # log.critical( + # 'Remote ctx terminated due to "error" msg:\n' + # f'{re}' + # ) + # assert msg is self._cancel_msg + # # NOTE: this solved a super dupe edge case XD + # # this was THE super duper edge case of: + # # - local task opens a remote task, + # # - requests remote cancellation of far end + # # ctx/tasks, + # # - needs to wait for the cancel ack msg + # # (ctxc) or some result in the race case + # # where the other side's task returns + # # before the cancel request msg is ever + # # rxed and processed, + # # - here this surrounding drain loop (which + # # iterates all ipc msgs until the ack or + # # an early result arrives) was NOT exiting + # # since we are the edge case: local task + # # does not re-raise any ctxc it receives + # # IFF **it** was the cancellation + # # requester.. + # # will raise if necessary, ow break from + # # loop presuming any error terminates the + # # context! + # self._maybe_raise_remote_err( + # re, + # # NOTE: obvi we don't care if we + # # overran the far end if we're already + # # waiting on a final result (msg). + # # raise_overrun_from_self=False, + # raise_overrun_from_self=raise_overrun, + # ) + + # break # OOOOOF, yeah obvi we need this.. + + # # XXX we should never really get here + # # right! since `._deliver_msg()` should + # # always have detected an {'error': ..} + # # msg and already called this right!?! + # elif error := unpack_error( + # msg=msg, + # chan=self._portal.channel, + # hide_tb=False, + # ): + # log.critical('SHOULD NEVER GET HERE!?') + # assert msg is self._cancel_msg + # assert error.msgdata == self._remote_error.msgdata + # from .devx._debug import pause + # await pause() + # self._maybe_cancel_and_set_remote_error(error) + # self._maybe_raise_remote_err(error) + + # else: + # # bubble the original src key error + # raise + + if ( + (re := self._remote_error) + and self._result == res_placeholder + ): + maybe_err: Exception|None = self._maybe_raise_remote_err( + re, + # NOTE: obvi we don't care if we + # overran the far end if we're already + # waiting on a final result (msg). + # raise_overrun_from_self=False, + raise_overrun_from_self=( + raise_overrun + and + # only when we ARE NOT the canceller + # should we raise overruns, bc ow we're + # raising something we know might happen + # during cancellation ;) + (not self._cancel_called) + ), + ) + if maybe_err: + self._result = maybe_err return self._result @@ -779,7 +1138,7 @@ class Context: while self._overflow_q: # NOTE: these msgs should never be errors since we always do # the check prior to checking if we're in an overrun state - # inside ``.deliver_msg()``. + # inside ``._deliver_msg()``. msg = self._overflow_q.popleft() try: await self._send_chan.send(msg) @@ -830,34 +1189,50 @@ class Context: messages are eventually sent if possible. ''' - cid = self.cid - chan = self.chan - uid = chan.uid + cid: str = self.cid + chan: Channel = self.chan + from_uid: tuple[str, str] = chan.uid send_chan: trio.MemorySendChannel = self._send_chan - log.runtime( - f"Delivering {msg} from {uid} to caller {cid}" - ) - - if ( - msg.get('error') # check for field - and ( - error := unpack_error( - msg, - self.chan, - ) + if re := unpack_error( + msg, + self.chan, + ): + log.error( + f'Delivering error-msg from {from_uid} to caller {cid}' + f'{re}' ) - ): self._cancel_msg = msg - self._maybe_cancel_and_set_remote_error(error) + self._maybe_cancel_and_set_remote_error(re) - if ( - self._in_overrun - ): + # XXX NEVER do this XXX..!! + # bc if the error is a ctxc and there is a task + # waiting on `.result()` we need the msg to be sent + # over the `send_chan`/`._recv_chan` so that the error + # is relayed to that waiter task.. + # return True + # + # XXX ALSO NO!! XXX + # if self._remote_error: + # self._maybe_raise_remote_err(error) + + if self._in_overrun: + log.warning( + f'Capturing overrun-msg from {from_uid} to caller {cid}' + f'{msg}' + ) self._overflow_q.append(msg) return False try: + log.runtime( + f'Delivering IPC `Context` msg:\n' + f'<= {from_uid}\n' + f'=> caller: {cid}\n' + f'{msg}' + ) + # from .devx._debug import pause + # await pause() send_chan.send_nowait(msg) return True # if an error is deteced we should always @@ -890,7 +1265,8 @@ class Context: lines = [ f'OVERRUN on actor-task context {cid}@{local_uid}!\n' # TODO: put remote task name here if possible? - f'remote sender actor: {uid}', + f'sender: {from_uid}', + f'msg: {msg}', # TODO: put task func name here and maybe an arrow # from sender to overrunner? # f'local task {self.func_name}' @@ -926,11 +1302,19 @@ class Context: # anything different. return False else: + # raise local overrun and immediately pack as IPC + # msg for far end. try: - raise StreamOverrun(text) + raise StreamOverrun( + text, + sender=from_uid, + ) except StreamOverrun as err: - err_msg = pack_error(err) - err_msg['cid'] = cid + err_msg: dict[str, dict] = pack_error( + err, + cid=cid, + ) + # err_msg['cid']: str = cid try: await chan.send(err_msg) except trio.BrokenResourceError: diff --git a/tractor/_portal.py b/tractor/_portal.py index fc094593..97b89b3c 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -39,7 +39,15 @@ import trio from async_generator import asynccontextmanager from .trionics import maybe_open_nursery -from ._state import current_actor +from .devx import ( + # acquire_debug_lock, + # pause, + maybe_wait_for_debugger, +) +from ._state import ( + current_actor, + debug_mode, +) from ._ipc import Channel from .log import get_logger from .msg import NamespacePath @@ -48,6 +56,7 @@ from ._exceptions import ( unpack_error, NoResult, ContextCancelled, + RemoteActorError, ) from ._context import ( Context, @@ -468,7 +477,6 @@ class Portal: ctx._started_called: bool = True except KeyError as src_error: - _raise_from_no_key_in_msg( ctx=ctx, msg=msg, @@ -493,6 +501,33 @@ class Portal: # in enter tuple. yield ctx, first + # between the caller exiting and arriving here the + # far end may have sent a ctxc-msg or other error, + # so check for it here immediately and maybe raise + # so as to engage the ctxc handling block below! + # if re := ctx._remote_error: + # maybe_ctxc: ContextCancelled|None = ctx._maybe_raise_remote_err( + # re, + + # # TODO: do we want this to always raise? + # # - means that on self-ctxc, if/when the + # # block is exited before the msg arrives + # # but then the msg during __exit__ + # # calling we may not activate the + # # ctxc-handler block below? should we + # # be? + # # - if there's a remote error that arrives + # # after the child has exited, we won't + # # handle until the `finally:` block + # # where `.result()` is always called, + # # again in which case we handle it + # # differently then in the handler block + # # that would normally engage from THIS + # # block? + # raise_ctxc_from_self_call=True, + # ) + # assert maybe_ctxc + # when in allow_overruns mode there may be # lingering overflow sender tasks remaining? if nurse.child_tasks: @@ -538,7 +573,7 @@ class Portal: # `.canceller: tuple[str, str]` to be same value as # caught here in a `ContextCancelled.canceller`. # - # Again, there are 2 cases: + # AGAIN to restate the above, there are 2 cases: # # 1-some other context opened in this `.open_context()` # block cancelled due to a self or peer cancellation @@ -554,6 +589,16 @@ class Portal: except ContextCancelled as ctxc: scope_err = ctxc + # XXX TODO XXX: FIX THIS debug_mode BUGGGG!!! + # using this code and then resuming the REPL will + # cause a SIGINT-ignoring HANG! + # -> prolly due to a stale debug lock entry.. + # -[ ] USE `.stackscope` to demonstrate that (possibly + # documenting it as a definittive example of + # debugging the tractor-runtime itself using it's + # own `.devx.` tooling! + # await pause() + # CASE 2: context was cancelled by local task calling # `.cancel()`, we don't raise and the exit block should # exit silently. @@ -561,18 +606,23 @@ class Portal: ctx._cancel_called and ( ctxc is ctx._remote_error - or - ctxc.canceller is self.canceller + # ctxc.msgdata == ctx._remote_error.msgdata + + # TODO: uhh `Portal.canceller` ain't a thangg + # dawg? (was `self.canceller` before?!?) + and + ctxc.canceller == self.actor.uid ) ): - log.debug( - f'Context {ctx} cancelled gracefully with:\n' + log.cancel( + f'Context (cid=[{ctx.cid[-6:]}..] cancelled gracefully with:\n' f'{ctxc}' ) # CASE 1: this context was never cancelled via a local # task (tree) having called `Context.cancel()`, raise # the error since it was caused by someone else! else: + # await pause() raise # the above `._scope` can be cancelled due to: @@ -601,8 +651,8 @@ class Portal: trio.Cancelled, # NOTE: NOT from inside the ctx._scope KeyboardInterrupt, - ) as err: - scope_err = err + ) as caller_err: + scope_err = caller_err # XXX: ALWAYS request the context to CANCEL ON any ERROR. # NOTE: `Context.cancel()` is conversely NEVER CALLED in @@ -610,11 +660,26 @@ class Portal: # handled in the block above! log.cancel( 'Context cancelled for task due to\n' - f'{err}\n' + f'{caller_err}\n' 'Sending cancel request..\n' f'task:{cid}\n' f'actor:{uid}' ) + + if debug_mode(): + log.pdb( + 'Delaying `ctx.cancel()` until debug lock ' + 'acquired..' + ) + # async with acquire_debug_lock(self.actor.uid): + # pass + # TODO: factor ^ into below for non-root cases? + await maybe_wait_for_debugger() + log.pdb( + 'Acquired debug lock! ' + 'Calling `ctx.cancel()`!' + ) + try: await ctx.cancel() except trio.BrokenResourceError: @@ -628,6 +693,33 @@ class Portal: # no local scope error, the "clean exit with a result" case. else: + # between the caller exiting and arriving here the + # far end may have sent a ctxc-msg or other error, + # so check for it here immediately and maybe raise + # so as to engage the ctxc handling block below! + # if re := ctx._remote_error: + # maybe_ctxc: ContextCancelled|None = ctx._maybe_raise_remote_err( + # re, + + # # TODO: do we want this to always raise? + # # - means that on self-ctxc, if/when the + # # block is exited before the msg arrives + # # but then the msg during __exit__ + # # calling we may not activate the + # # ctxc-handler block below? should we + # # be? + # # - if there's a remote error that arrives + # # after the child has exited, we won't + # # handle until the `finally:` block + # # where `.result()` is always called, + # # again in which case we handle it + # # differently then in the handler block + # # that would normally engage from THIS + # # block? + # raise_ctxc_from_self_call=True, + # ) + # assert maybe_ctxc + if ctx.chan.connected(): log.info( 'Waiting on final context-task result for\n' @@ -644,13 +736,8 @@ class Portal: # As per `Context._deliver_msg()`, that error IS # ALWAYS SET any time "callee" side fails and causes "caller # side" cancellation via a `ContextCancelled` here. - # result = await ctx.result() try: - result = await ctx.result() - log.runtime( - f'Context {fn_name} returned value from callee:\n' - f'`{result}`' - ) + result_or_err: Exception|Any = await ctx.result() except BaseException as berr: # on normal teardown, if we get some error # raised in `Context.result()` we still want to @@ -662,7 +749,48 @@ class Portal: scope_err = berr raise + # an exception type boxed in a `RemoteActorError` + # is returned (meaning it was obvi not raised). + msgdata: str|None = getattr( + result_or_err, + 'msgdata', + None + ) + # yes! this worx Bp + # from .devx import _debug + # await _debug.pause() + match (msgdata, result_or_err): + case ( + {'tb_str': tbstr}, + ContextCancelled(), + ): + log.cancel(tbstr) + + case ( + {'tb_str': tbstr}, + RemoteActorError(), + ): + log.exception( + f'Context `{fn_name}` remotely errored:\n' + f'`{tbstr}`' + ) + case (None, _): + log.runtime( + f'Context {fn_name} returned value from callee:\n' + f'`{result_or_err}`' + ) + finally: + # XXX: (MEGA IMPORTANT) if this is a root opened process we + # wait for any immediate child in debug before popping the + # context from the runtime msg loop otherwise inside + # ``Actor._push_result()`` the msg will be discarded and in + # the case where that msg is global debugger unlock (via + # a "stop" msg for a stream), this can result in a deadlock + # where the root is waiting on the lock to clear but the + # child has already cleared it and clobbered IPC. + await maybe_wait_for_debugger() + # though it should be impossible for any tasks # operating *in* this scope to have survived # we tear down the runtime feeder chan last @@ -707,6 +835,10 @@ class Portal: # out any exception group or legit (remote) ctx # error that sourced from the remote task or its # runtime. + # + # NOTE: further, this should be the only place the + # underlying feeder channel is + # once-and-only-CLOSED! with trio.CancelScope(shield=True): await ctx._recv_chan.aclose() @@ -747,6 +879,9 @@ class Portal: # FINALLY, remove the context from runtime tracking and # exit! + log.runtime( + f'Exiting context opened with {ctx.chan.uid}' + ) self.actor._contexts.pop( (self.channel.uid, ctx.cid), None, diff --git a/tractor/_streaming.py b/tractor/_streaming.py index 4530e144..e8f735ec 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -21,8 +21,9 @@ The machinery and types behind ``Context.open_stream()`` ''' from __future__ import annotations -import inspect from contextlib import asynccontextmanager as acm +import inspect +from pprint import pformat from typing import ( Any, Callable, @@ -35,6 +36,7 @@ import trio from ._exceptions import ( _raise_from_no_key_in_msg, + ContextCancelled, ) from .log import get_logger from .trionics import ( @@ -84,8 +86,8 @@ class MsgStream(trio.abc.Channel): self._broadcaster = _broadcaster # flag to denote end of stream - self._eoc: bool = False - self._closed: bool = False + self._eoc: bool|trio.EndOfChannel = False + self._closed: bool|trio.ClosedResourceError = False # delegate directly to underlying mem channel def receive_nowait(self): @@ -93,6 +95,9 @@ class MsgStream(trio.abc.Channel): try: return msg['yield'] except KeyError as kerr: + # if 'return' in msg: + # return msg + _raise_from_no_key_in_msg( ctx=self._ctx, msg=msg, @@ -122,30 +127,43 @@ class MsgStream(trio.abc.Channel): # see ``.aclose()`` for notes on the old behaviour prior to # introducing this if self._eoc: - raise trio.EndOfChannel + raise self._eoc + # raise trio.EndOfChannel if self._closed: - raise trio.ClosedResourceError('This stream was closed') + raise self._closed + # raise trio.ClosedResourceError( + # 'This stream was already closed' + # ) + src_err: Exception|None = None try: - msg = await self._rx_chan.receive() - return msg['yield'] + try: + msg = await self._rx_chan.receive() + return msg['yield'] - except KeyError as kerr: - _raise_from_no_key_in_msg( - ctx=self._ctx, - msg=msg, - src_err=kerr, - log=log, - expect_key='yield', - stream=self, - ) + except KeyError as kerr: + src_err = kerr + # NOTE: may raise any of the below error types + # includg EoC when a 'stop' msg is found. + _raise_from_no_key_in_msg( + ctx=self._ctx, + msg=msg, + src_err=kerr, + log=log, + expect_key='yield', + stream=self, + ) + + # XXX: we close the stream on any of these error conditions: except ( - trio.ClosedResourceError, # by self._rx_chan + # trio.ClosedResourceError, # by self._rx_chan trio.EndOfChannel, # by self._rx_chan or `stop` msg from far end - ): - # XXX: we close the stream on any of these error conditions: + ) as eoc: + src_err = eoc + self._eoc = eoc + # await trio.sleep(1) # a ``ClosedResourceError`` indicates that the internal # feeder memory receive channel was closed likely by the @@ -168,14 +186,53 @@ class MsgStream(trio.abc.Channel): # 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 - # when the send is closed we assume the stream has - # terminated and signal this local iterator to stop - await self.aclose() + # raise eoc - raise # propagate + except trio.ClosedResourceError as cre: # by self._rx_chan + src_err = cre + log.warning( + '`Context._rx_chan` was already closed?' + ) + self._closed = cre - async def aclose(self): + # when the send is closed we assume the stream has + # terminated and signal this local iterator to stop + drained: list[Exception|dict] = await self.aclose() + if drained: + log.warning( + 'Drained context msgs during closure:\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 + # probably want to instead raise the remote error + # over the end-of-stream connection error since likely + # the remote error was the source cause? + ctx: Context = self._ctx + if re := ctx._remote_error: + ctx._maybe_raise_remote_err( + re, + raise_ctxc_from_self_call=True, + ) + + raise src_err # propagate + + async def aclose(self) -> list[Exception|dict]: ''' Cancel associated remote actor task and local memory channel on close. @@ -185,15 +242,55 @@ class MsgStream(trio.abc.Channel): # https://trio.readthedocs.io/en/stable/reference-io.html#trio.abc.AsyncResource.aclose rx_chan = self._rx_chan - if rx_chan._closed: - log.cancel(f"{self} is already closed") + if ( + rx_chan._closed + or + self._closed + ): + log.cancel( + f'`MsgStream` is already closed\n' + f'.cid: {self._ctx.cid}\n' + f'._rx_chan`: {rx_chan}\n' + f'._eoc: {self._eoc}\n' + f'._closed: {self._eoc}\n' + ) # this stream has already been closed so silently succeed as # per ``trio.AsyncResource`` semantics. # https://trio.readthedocs.io/en/stable/reference-io.html#trio.abc.AsyncResource.aclose - return + return [] - self._eoc = True + ctx: Context = self._ctx + # caught_eoc: bool = False + drained: list[Exception|dict] = [] + while not drained: + try: + maybe_final_msg = self.receive_nowait() + if maybe_final_msg: + log.cancel( + 'Drained un-processed stream msg:\n' + f'{pformat(maybe_final_msg)}' + ) + # TODO: inject into parent `Context` buf? + drained.append(maybe_final_msg) + + except trio.WouldBlock as be: + drained.append(be) + break + + except trio.EndOfChannel as eoc: + drained.append(eoc) + # caught_eoc = True + self._eoc: bool = eoc + break + + except ContextCancelled as ctxc: + log.cancel( + 'Context was cancelled during stream closure:\n' + f'canceller: {ctxc.canceller}\n' + f'{pformat(ctxc.msgdata)}' + ) + break # NOTE: this is super subtle IPC messaging stuff: # Relay stop iteration to far end **iff** we're @@ -224,26 +321,33 @@ class MsgStream(trio.abc.Channel): except ( trio.BrokenResourceError, trio.ClosedResourceError - ): + ) as re: # the underlying channel may already have been pulled # in which case our stop message is meaningless since # it can't traverse the transport. - ctx = self._ctx log.warning( f'Stream was already destroyed?\n' f'actor: {ctx.chan.uid}\n' f'ctx id: {ctx.cid}' ) + drained.append(re) + self._closed = re - self._closed = True + # if caught_eoc: + # # from .devx import _debug + # # await _debug.pause() + # with trio.CancelScope(shield=True): + # await rx_chan.aclose() - # Do we close the local mem chan ``self._rx_chan`` ??!? + # self._eoc: bool = caught_eoc - # NO, DEFINITELY NOT if we're a bi-dir ``MsgStream``! - # BECAUSE this same core-msg-loop mem recv-chan is used to deliver - # the potential final result from the surrounding inter-actor - # `Context` so we don't want to close it until that context has - # run to completion. + # ?XXX WAIT, why do we not close the local mem chan `._rx_chan` XXX? + # => NO, DEFINITELY NOT! <= + # if we're a bi-dir ``MsgStream`` BECAUSE this same + # core-msg-loop mem recv-chan is used to deliver the + # potential final result from the surrounding inter-actor + # `Context` so we don't want to close it until that + # context has run to completion. # XXX: Notes on old behaviour: # await rx_chan.aclose() @@ -272,6 +376,8 @@ class MsgStream(trio.abc.Channel): # runtime's closure of ``rx_chan`` in the case where we may # still need to consume msgs that are "in transit" from the far # end (eg. for ``Context.result()``). + # self._closed = True + return drained @acm async def subscribe( @@ -337,9 +443,13 @@ class MsgStream(trio.abc.Channel): raise self._ctx._remote_error # from None if self._closed: - raise trio.ClosedResourceError('This stream was already closed') + raise self._closed + # raise trio.ClosedResourceError('This stream was already closed') - await self._ctx.chan.send({'yield': data, 'cid': self._ctx.cid}) + await self._ctx.chan.send({ + 'yield': data, + 'cid': self._ctx.cid, + }) def stream(func: Callable) -> Callable: From fd200047573def4be5c6a801c98cd20f345ffacc Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 20 Feb 2024 09:18:22 -0500 Subject: [PATCH 077/146] _supervise: iter nice expanded multi-line `._children` tups with typing --- tractor/_supervise.py | 41 ++++++++++++++++++++++++++++++++++------- 1 file changed, 34 insertions(+), 7 deletions(-) diff --git a/tractor/_supervise.py b/tractor/_supervise.py index 7f77784b..8f4737a3 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -157,7 +157,7 @@ class ActorNursery: # start a task to spawn a process # blocks until process has been started and a portal setup - nursery = nursery or self._da_nursery + nursery: trio.Nursery = nursery or self._da_nursery # XXX: the type ignore is actually due to a `mypy` bug return await nursery.start( # type: ignore @@ -233,12 +233,14 @@ class ActorNursery: return portal async def cancel(self, hard_kill: bool = False) -> None: - """Cancel this nursery by instructing each subactor to cancel + ''' + Cancel this nursery by instructing each subactor to cancel itself and wait for all subactors to terminate. If ``hard_killl`` is set to ``True`` then kill the processes directly without any far end graceful ``trio`` cancellation. - """ + + ''' self.cancelled = True log.cancel(f"Cancelling nursery in {self._actor.uid}") @@ -246,7 +248,14 @@ class ActorNursery: async with trio.open_nursery() as nursery: - for subactor, proc, portal in self._children.values(): + subactor: Actor + proc: trio.Process + portal: Portal + for ( + subactor, + proc, + portal, + ) in self._children.values(): # TODO: are we ever even going to use this or # is the spawning backend responsible for such @@ -286,8 +295,16 @@ class ActorNursery: # then hard kill all sub-processes if cs.cancelled_caught: log.error( - f"Failed to cancel {self}\nHard killing process tree!") - for subactor, proc, portal in self._children.values(): + f'Failed to cancel {self}\nHard killing process tree!' + ) + subactor: Actor + proc: trio.Process + portal: Portal + for ( + subactor, + proc, + portal, + ) in self._children.values(): log.warning(f"Hard killing process {proc}") proc.terminate() @@ -384,7 +401,17 @@ async def _open_and_supervise_one_cancels_all_nursery( else: log.exception( f"Nursery for {current_actor().uid} " - f"errored with") + "errored with\n" + + # TODO: same thing as in + # `._invoke()` to compute how to + # place this div-line in the + # middle of the above msg + # content.. + # -[ ] prolly helper-func it too + # in our `.log` module.. + # '------ - ------' + ) # cancel all subactors await anursery.cancel() From 6bc67338cfbef875a62f26eec6f3ed645adf3a92 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 20 Feb 2024 13:12:51 -0500 Subject: [PATCH 078/146] Better subproc supervisor logging, todo for #320 Given i just similarly revamped a buncha `._runtime` log msg formatting, might as well do something similar inside the spawning machinery such that groking teardown sequences of each supervising task is much more sane XD Mostly this includes doing similar `': \n'` multi-line formatting when reporting various subproc supervision steps as well as showing a detailed `trio.Process.__repr__()` as appropriate. Also adds a detailed #TODO according to the needs of #320 for which we're going to need some internal mechanism for intermediary parent actors to determine if a given debug tty locker (sub-actor) is one of *their* (transitive) children and thus stall the normal cancellation/teardown sequence until that locker is complete. --- tractor/_spawn.py | 116 ++++++++++++++++++++++++++++++++-------------- 1 file changed, 82 insertions(+), 34 deletions(-) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index 09e98bb7..7ce750be 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -144,7 +144,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 = await portal.result() + final: Any = await portal.result() except ( Exception, @@ -152,13 +152,23 @@ async def exhaust_portal( ) as err: # we reraise in the parent task via a ``BaseExceptionGroup`` return err + except trio.Cancelled as err: # lol, of course we need this too ;P # TODO: merge with above? - log.warning(f"Cancelled result waiter for {portal.actor.uid}") + log.warning( + 'Cancelled portal result waiter task:\n' + f'uid: {portal.channel.uid}\n' + f'error: {err}\n' + ) return err + else: - log.debug(f"Returning final result: {final}") + log.debug( + f'Returning final result from portal:\n' + f'uid: {portal.channel.uid}\n' + f'result: {final}\n' + ) return final @@ -170,26 +180,34 @@ async def cancel_on_completion( ) -> None: ''' - Cancel actor gracefully once it's "main" portal's + Cancel actor gracefully once its "main" portal's result arrives. - Should only be called for actors spawned with `run_in_actor()`. + Should only be called for actors spawned via the + `Portal.run_in_actor()` API. + + => and really this API will be deprecated and should be + re-implemented as a `.hilevel.one_shot_task_nursery()`..) ''' # if this call errors we store the exception for later # in ``errors`` which will be reraised inside # an exception group and we still send out a cancel request - result = await exhaust_portal(portal, actor) + result: Any|Exception = await exhaust_portal(portal, actor) if isinstance(result, Exception): - errors[actor.uid] = result + errors[actor.uid]: Exception = result log.warning( - f"Cancelling {portal.channel.uid} after error {result}" + 'Cancelling subactor due to error:\n' + f'uid: {portal.channel.uid}\n' + f'error: {result}\n' ) else: log.runtime( - f"Cancelling {portal.channel.uid} gracefully " - f"after result {result}") + 'Cancelling subactor gracefully:\n' + f'uid: {portal.channel.uid}\n' + f'result: {result}\n' + ) # cancel the process now that we have a final result await portal.cancel_actor() @@ -219,11 +237,14 @@ async def do_hard_kill( to be handled. ''' + log.cancel( + 'Terminating sub-proc:\n' + f'|_{proc}\n' + ) # NOTE: this timeout used to do nothing since we were shielding # the ``.wait()`` inside ``new_proc()`` which will pretty much # never release until the process exits, now it acts as # a hard-kill time ultimatum. - log.debug(f"Terminating {proc}") with trio.move_on_after(terminate_after) as cs: # NOTE: code below was copied verbatim from the now deprecated @@ -260,7 +281,10 @@ async def do_hard_kill( # zombies (as a feature) we ask the OS to do send in the # removal swad as the last resort. if cs.cancelled_caught: - log.critical(f"#ZOMBIE_LORD_IS_HERE: {proc}") + log.critical( + 'Well, the #ZOMBIE_LORD_IS_HERE# to collect\n' + f'|_{proc}\n' + ) proc.kill() @@ -281,10 +305,16 @@ async def soft_wait( join/reap on an actor-runtime-in-process. ''' - uid = portal.channel.uid + uid: tuple[str, str] = portal.channel.uid try: - log.cancel(f'Soft waiting on actor:\n{uid}') + log.cancel( + 'Soft waiting on sub-actor proc:\n' + f'uid: {uid}\n' + f'|_{proc}\n' + ) + # wait on sub-proc to signal termination await wait_func(proc) + except trio.Cancelled: # if cancelled during a soft wait, cancel the child # actor before entering the hard reap sequence @@ -296,8 +326,8 @@ async def soft_wait( async def cancel_on_proc_deth(): ''' - Cancel the actor cancel request if we detect that - that the process terminated. + "Cancel the (actor) cancel" request if we detect + that that the underlying sub-process terminated. ''' await wait_func(proc) @@ -314,10 +344,10 @@ async def soft_wait( if proc.poll() is None: # type: ignore log.warning( - 'Actor still alive after cancel request:\n' - f'{uid}' + 'Subactor still alive after cancel request?\n\n' + f'uid: {uid}\n' + f'|_{proc}\n' ) - n.cancel_scope.cancel() raise @@ -341,7 +371,7 @@ async def new_proc( ) -> None: # lookup backend spawning target - target = _methods[_spawn_method] + target: Callable = _methods[_spawn_method] # mark the new actor with the global spawn method subactor._spawn_method = _spawn_method @@ -492,8 +522,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 existing result waiter task for ' + f'{subactor.uid}' + ) nursery.cancel_scope.cancel() finally: @@ -511,18 +542,35 @@ async def trio_proc( with trio.move_on_after(0.5): await proc.wait() - if is_root_process(): - # TODO: solve the following issue where we need - # to do a similar wait like this but in an - # "intermediary" parent actor that itself isn't - # in debug but has a child that is, and we need - # to hold off on relaying SIGINT until that child - # is complete. - # https://github.com/goodboy/tractor/issues/320 - await maybe_wait_for_debugger( - child_in_debug=_runtime_vars.get( - '_debug_mode', False), - ) + log.pdb( + 'Delaying subproc reaper while debugger locked..' + ) + await maybe_wait_for_debugger( + child_in_debug=_runtime_vars.get( + '_debug_mode', False + ), + # TODO: need a diff value then default? + # poll_steps=9999999, + ) + # TODO: solve the following issue where we need + # to do a similar wait like this but in an + # "intermediary" parent actor that itself isn't + # in debug but has a child that is, and we need + # to hold off on relaying SIGINT until that child + # is complete. + # https://github.com/goodboy/tractor/issues/320 + # -[ ] we need to handle non-root parent-actors specially + # by somehow determining if a child is in debug and then + # avoiding cancel/kill of said child by this + # (intermediary) parent until such a time as the root says + # the pdb lock is released and we are good to tear down + # (our children).. + # + # -[ ] so maybe something like this where we try to + # acquire the lock and get notified of who has it, + # check that uid against our known children? + # this_uid: tuple[str, str] = current_actor().uid + # await acquire_debug_lock(this_uid) if proc.poll() is None: log.cancel(f"Attempting to hard kill {proc}") From 338ea5529cf98c7dbf8801a65a45cbff9e464699 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 20 Feb 2024 13:22:44 -0500 Subject: [PATCH 079/146] .log: more multi-line styling --- tractor/log.py | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/tractor/log.py b/tractor/log.py index 590779a5..6c040209 100644 --- a/tractor/log.py +++ b/tractor/log.py @@ -289,11 +289,19 @@ def get_console_log( if not level: return log - log.setLevel(level.upper() if not isinstance(level, int) else level) + log.setLevel( + level.upper() + if not isinstance(level, int) + else level + ) if not any( handler.stream == sys.stderr # type: ignore - for handler in logger.handlers if getattr(handler, 'stream', None) + for handler in logger.handlers if getattr( + handler, + 'stream', + None, + ) ): handler = logging.StreamHandler() formatter = colorlog.ColoredFormatter( From 96150600fb05d5b3aa848c59a5a58a11e8577138 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 20 Feb 2024 15:14:58 -0500 Subject: [PATCH 080/146] Make ctx tests support `debug_mode: bool` fixture Such that with `--tpdb` passed (sub)actors will engage the `pdbp` REPL automatically and so that we can use the new `stackscope` support when complex cases hang Bo Also, - simplified some type-annots (ns paths), - doc-ed an inter-peer test func with some ascii msg flows, - added a bottom #TODO for replicating the scenario i hit in `modden` where a separate client actor-tree was hanging on cancelling a `bigd` sub-workspace.. --- tests/test_context_stream_semantics.py | 142 ++++++++++++++++--------- tests/test_inter_peer_cancellation.py | 75 +++++++++---- 2 files changed, 150 insertions(+), 67 deletions(-) diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index dda096ce..4eb06e8a 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -8,7 +8,9 @@ sync-opening a ``tractor.Context`` beforehand. # from contextlib import asynccontextmanager as acm from itertools import count import platform -from typing import Optional +from typing import ( + Callable, +) import pytest import trio @@ -69,7 +71,7 @@ _state: bool = False @tractor.context async def too_many_starteds( - ctx: tractor.Context, + ctx: Context, ) -> None: ''' Call ``Context.started()`` more then once (an error). @@ -84,7 +86,7 @@ async def too_many_starteds( @tractor.context async def not_started_but_stream_opened( - ctx: tractor.Context, + ctx: Context, ) -> None: ''' Enter ``Context.open_stream()`` without calling ``.started()``. @@ -105,11 +107,15 @@ async def not_started_but_stream_opened( ], ids='misuse_type={}'.format, ) -def test_started_misuse(target): - +def test_started_misuse( + target: Callable, + debug_mode: bool, +): async def main(): - async with tractor.open_nursery() as n: - portal = await n.start_actor( + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.start_actor( target.__name__, enable_modules=[__name__], ) @@ -124,7 +130,7 @@ def test_started_misuse(target): @tractor.context async def simple_setup_teardown( - ctx: tractor.Context, + ctx: Context, data: int, block_forever: bool = False, @@ -170,6 +176,7 @@ def test_simple_context( error_parent, callee_blocks_forever, pointlessly_open_stream, + debug_mode: bool, ): timeout = 1.5 if not platform.system() == 'Windows' else 4 @@ -177,9 +184,10 @@ def test_simple_context( async def main(): with trio.fail_after(timeout): - async with tractor.open_nursery() as nursery: - - portal = await nursery.start_actor( + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.start_actor( 'simple_context', enable_modules=[__name__], ) @@ -260,6 +268,7 @@ def test_caller_cancels( cancel_method: str, chk_ctx_result_before_exit: bool, callee_returns_early: bool, + debug_mode: bool, ): ''' Verify that when the opening side of a context (aka the caller) @@ -268,7 +277,7 @@ def test_caller_cancels( ''' async def check_canceller( - ctx: tractor.Context, + ctx: Context, ) -> None: # should not raise yet return the remote # context cancelled error. @@ -287,8 +296,10 @@ def test_caller_cancels( ) async def main(): - async with tractor.open_nursery() as nursery: - portal = await nursery.start_actor( + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.start_actor( 'simple_context', enable_modules=[__name__], ) @@ -338,7 +349,7 @@ def test_caller_cancels( @tractor.context async def close_ctx_immediately( - ctx: tractor.Context, + ctx: Context, ) -> None: @@ -350,17 +361,33 @@ async def close_ctx_immediately( @tractor_test -async def test_callee_closes_ctx_after_stream_open(): - 'callee context closes without using stream' +async def test_callee_closes_ctx_after_stream_open( + debug_mode: bool, +): + ''' + callee context closes without using stream. - async with tractor.open_nursery() as n: + This should result in a msg sequence + |__ + |_ - portal = await n.start_actor( + <= {'started': , 'cid': } + <= {'stop': True, 'cid': } + <= {'result': Any, ..} + + (ignored by child) + => {'stop': True, 'cid': } + + ''' + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.start_actor( 'fast_stream_closer', enable_modules=[__name__], ) - with trio.fail_after(2): + with trio.fail_after(0.5): async with portal.open_context( close_ctx_immediately, @@ -368,10 +395,9 @@ async def test_callee_closes_ctx_after_stream_open(): # cancel_on_exit=True, ) as (ctx, sent): - assert sent is None - with trio.fail_after(0.5): + with trio.fail_after(0.4): async with ctx.open_stream() as stream: # should fall through since ``StopAsyncIteration`` @@ -379,12 +405,15 @@ async def test_callee_closes_ctx_after_stream_open(): # a ``trio.EndOfChannel`` by # ``trio.abc.ReceiveChannel.__anext__()`` async for _ in stream: + # trigger failure if we DO NOT + # get an EOC! assert 0 else: # verify stream is now closed try: - await stream.receive() + with trio.fail_after(0.3): + await stream.receive() except trio.EndOfChannel: pass @@ -405,7 +434,7 @@ async def test_callee_closes_ctx_after_stream_open(): @tractor.context async def expect_cancelled( - ctx: tractor.Context, + ctx: Context, ) -> None: global _state @@ -434,11 +463,15 @@ async def expect_cancelled( @tractor_test async def test_caller_closes_ctx_after_callee_opens_stream( use_ctx_cancel_method: bool, + debug_mode: bool, ): - 'caller context closes without using stream' - - async with tractor.open_nursery() as an: + ''' + caller context closes without using/opening stream + ''' + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: root: Actor = current_actor() portal = await an.start_actor( @@ -522,11 +555,13 @@ async def test_caller_closes_ctx_after_callee_opens_stream( @tractor_test -async def test_multitask_caller_cancels_from_nonroot_task(): - - async with tractor.open_nursery() as n: - - portal = await n.start_actor( +async def test_multitask_caller_cancels_from_nonroot_task( + debug_mode: bool, +): + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.start_actor( 'ctx_cancelled', enable_modules=[__name__], ) @@ -573,7 +608,7 @@ async def test_multitask_caller_cancels_from_nonroot_task(): @tractor.context async def cancel_self( - ctx: tractor.Context, + ctx: Context, ) -> None: global _state @@ -610,16 +645,20 @@ async def cancel_self( raise RuntimeError('Context didnt cancel itself?!') + @tractor_test -async def test_callee_cancels_before_started(): +async def test_callee_cancels_before_started( + debug_mode: bool, +): ''' Callee calls `Context.cancel()` while streaming and caller sees stream terminated in `ContextCancelled`. ''' - async with tractor.open_nursery() as n: - - portal = await n.start_actor( + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.start_actor( 'cancels_self', enable_modules=[__name__], ) @@ -645,7 +684,7 @@ async def test_callee_cancels_before_started(): @tractor.context async def never_open_stream( - ctx: tractor.Context, + ctx: Context, ) -> None: ''' @@ -659,8 +698,8 @@ async def never_open_stream( @tractor.context async def keep_sending_from_callee( - ctx: tractor.Context, - msg_buffer_size: Optional[int] = None, + ctx: Context, + msg_buffer_size: int|None = None, ) -> None: ''' @@ -685,7 +724,10 @@ async def keep_sending_from_callee( ], ids='overrun_condition={}'.format, ) -def test_one_end_stream_not_opened(overrun_by): +def test_one_end_stream_not_opened( + overrun_by: tuple[str, int, Callable], + debug_mode: bool, +): ''' This should exemplify the bug from: https://github.com/goodboy/tractor/issues/265 @@ -696,8 +738,10 @@ def test_one_end_stream_not_opened(overrun_by): buf_size = buf_size_increase + Actor.msg_buffer_size async def main(): - async with tractor.open_nursery() as n: - portal = await n.start_actor( + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.start_actor( entrypoint.__name__, enable_modules=[__name__], ) @@ -754,7 +798,7 @@ def test_one_end_stream_not_opened(overrun_by): @tractor.context async def echo_back_sequence( - ctx: tractor.Context, + ctx: Context, seq: list[int], wait_for_cancel: bool, allow_overruns_side: str, @@ -837,6 +881,7 @@ def test_maybe_allow_overruns_stream( slow_side: str, allow_overruns_side: str, loglevel: str, + debug_mode: bool, ): ''' Demonstrate small overruns of each task back and forth @@ -855,13 +900,14 @@ def test_maybe_allow_overruns_stream( ''' async def main(): - async with tractor.open_nursery() as n: - portal = await n.start_actor( + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.start_actor( 'callee_sends_forever', enable_modules=[__name__], loglevel=loglevel, - - # debug_mode=True, + debug_mode=debug_mode, ) seq = list(range(10)) async with portal.open_context( diff --git a/tests/test_inter_peer_cancellation.py b/tests/test_inter_peer_cancellation.py index 5e1a4cad..1ead6172 100644 --- a/tests/test_inter_peer_cancellation.py +++ b/tests/test_inter_peer_cancellation.py @@ -123,7 +123,9 @@ async def error_before_started( await peer_ctx.cancel() -def test_do_not_swallow_error_before_started_by_remote_contextcancelled(): +def test_do_not_swallow_error_before_started_by_remote_contextcancelled( + debug_mode: bool, +): ''' Verify that an error raised in a remote context which itself opens YET ANOTHER remote context, which it then cancels, does not @@ -132,7 +134,9 @@ def test_do_not_swallow_error_before_started_by_remote_contextcancelled(): ''' async def main(): - async with tractor.open_nursery() as n: + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as n: portal = await n.start_actor( 'errorer', enable_modules=[__name__], @@ -225,13 +229,16 @@ async def stream_from_peer( # NOTE: cancellation of the (sleeper) peer should always # cause a `ContextCancelled` raise in this streaming # actor. - except ContextCancelled as ctxerr: - err = ctxerr + except ContextCancelled as ctxc: + ctxerr = ctxc + assert peer_ctx._remote_error is ctxerr + assert peer_ctx._remote_error.msgdata == ctxerr.msgdata assert peer_ctx.canceller == ctxerr.canceller # caller peer should not be the cancel requester assert not ctx.cancel_called + # XXX can never be true since `._invoke` only # sets this AFTER the nursery block this task # was started in, exits. @@ -269,9 +276,7 @@ async def stream_from_peer( # assert ctx.canceller[0] == 'root' # assert peer_ctx.canceller[0] == 'sleeper' - raise RuntimeError( - 'peer never triggered local `ContextCancelled`?' - ) + raise RuntimeError('Never triggered local `ContextCancelled` ?!?') @pytest.mark.parametrize( @@ -280,6 +285,7 @@ async def stream_from_peer( ) def test_peer_canceller( error_during_ctxerr_handling: bool, + debug_mode: bool, ): ''' Verify that a cancellation triggered by an in-actor-tree peer @@ -336,7 +342,7 @@ def test_peer_canceller( async def main(): async with tractor.open_nursery( # NOTE: to halt the peer tasks on ctxc, uncomment this. - # debug_mode=True + debug_mode=debug_mode, ) as an: canceller: Portal = await an.start_actor( 'canceller', @@ -377,7 +383,8 @@ def test_peer_canceller( try: print('PRE CONTEXT RESULT') - await sleeper_ctx.result() + res = await sleeper_ctx.result() + assert res # should never get here pytest.fail( @@ -387,7 +394,10 @@ def test_peer_canceller( # should always raise since this root task does # not request the sleeper cancellation ;) except ContextCancelled as ctxerr: - print(f'CAUGHT REMOTE CONTEXT CANCEL {ctxerr}') + print( + 'CAUGHT REMOTE CONTEXT CANCEL FOM\n' + f'{ctxerr}' + ) # canceller and caller peers should not # have been remotely cancelled. @@ -410,16 +420,31 @@ def test_peer_canceller( # XXX SHOULD NEVER EVER GET HERE XXX except BaseException as berr: - err = berr - pytest.fail('did not rx ctx-cancelled error?') + raise + + # XXX if needed to debug failure + # _err = berr + # await tractor.pause() + # await trio.sleep_forever() + + pytest.fail( + 'did not rx ctxc ?!?\n\n' + + f'{berr}\n' + ) + else: - pytest.fail('did not rx ctx-cancelled error?') + pytest.fail( + 'did not rx ctxc ?!?\n\n' + + f'{ctxs}\n' + ) except ( ContextCancelled, RuntimeError, - )as ctxerr: - _err = ctxerr + )as loc_err: + _loc_err = loc_err # NOTE: the main state to check on `Context` is: # - `.cancelled_caught` (maps to nursery cs) @@ -436,7 +461,7 @@ def test_peer_canceller( # `ContextCancelled` inside `.open_context()` # block if error_during_ctxerr_handling: - assert isinstance(ctxerr, RuntimeError) + assert isinstance(loc_err, RuntimeError) # NOTE: this root actor task should have # called `Context.cancel()` on the @@ -472,9 +497,10 @@ def test_peer_canceller( # CASE: standard teardown inside in `.open_context()` block else: - assert ctxerr.canceller == sleeper_ctx.canceller + assert isinstance(loc_err, ContextCancelled) + assert loc_err.canceller == sleeper_ctx.canceller assert ( - ctxerr.canceller[0] + loc_err.canceller[0] == sleeper_ctx.canceller[0] == @@ -484,7 +510,7 @@ def test_peer_canceller( # the sleeper's remote error is the error bubbled # out of the context-stack above! re = sleeper_ctx._remote_error - assert re is ctxerr + assert re is loc_err for ctx in ctxs: re: BaseException | None = ctx._remote_error @@ -554,3 +580,14 @@ def test_peer_canceller( assert excinfo.value.type == ContextCancelled assert excinfo.value.canceller[0] == 'canceller' + + +def test_client_tree_spawns_and_cancels_service_subactor(): + ... +# TODO: test for the modden `mod wks open piker` bug! +# -> start actor-tree (server) that offers sub-actor spawns via +# context API +# -> start another full actor-tree (client) which requests to the first to +# spawn over its `@context` ep / api. +# -> client actor cancels the context and should exit gracefully +# and the server's spawned child should cancel and terminate! From 27b750e907a2a948a9fc4e20f0be98080e77beb0 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 20 Feb 2024 15:26:14 -0500 Subject: [PATCH 081/146] Emit warning on any `ContextCancelled.canceller == None` --- tractor/_exceptions.py | 35 +++++++++++++++++++++++++++++++---- 1 file changed, 31 insertions(+), 4 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index d63cf6d4..a6d10de7 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -33,12 +33,15 @@ import exceptiongroup as eg import trio from ._state import current_actor +from .log import get_logger if TYPE_CHECKING: from ._context import Context from ._stream import MsgStream from .log import StackLevelAdapter +log = get_logger('tractor') + _this_mod = importlib.import_module(__name__) @@ -112,11 +115,36 @@ class ContextCancelled(RemoteActorError): ''' @property - def canceller(self) -> tuple[str, str] | None: + def canceller(self) -> tuple[str, str]|None: + ''' + Return the (maybe) `Actor.uid` for the requesting-author + of this ctxc. + + Emit a warning msg when `.canceller` has not been set, + which usually idicates that a `None` msg-loop setinel was + sent before expected in the runtime. This can happen in + a few situations: + + - (simulating) an IPC transport network outage + - a (malicious) pkt sent specifically to cancel an actor's + runtime non-gracefully without ensuring ongoing RPC tasks are + incrementally cancelled as is done with: + `Actor` + |_`.cancel()` + |_`.cancel_soon()` + |_`._cancel_task()` + + ''' value = self.msgdata.get('canceller') if value: return tuple(value) + log.warning( + 'IPC Context cancelled without a requesting actor?\n' + 'Maybe the IPC transport ended abruptly?\n\n' + f'{self}' + ) + class TransportClosed(trio.ClosedResourceError): "Underlying channel transport was closed prior to use" @@ -199,7 +227,6 @@ def pack_error( ): error_msg.update(exc.msgdata) - pkt: dict = {'error': error_msg} if cid: pkt['cid'] = cid @@ -349,8 +376,8 @@ def _raise_from_no_key_in_msg( # raise a ``StopAsyncIteration`` **and** in our catch # block below it will trigger ``.aclose()``. raise trio.EndOfChannel( - f'Context stream ended due to msg:\n' - f'{pformat(msg)}' + f'Context stream ended due to msg:\n\n' + f'{pformat(msg)}\n' ) from src_err From 5fa040c7db31a1ffd8ac968f93f3ebd90aa3aa86 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 20 Feb 2024 15:28:11 -0500 Subject: [PATCH 082/146] Add `NamespacePath._ns` todo for `self:` support --- tractor/msg.py | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/tractor/msg.py b/tractor/msg.py index ca34dba8..d8760411 100644 --- a/tractor/msg.py +++ b/tractor/msg.py @@ -58,6 +58,11 @@ class NamespacePath(str): ''' _ref: object | type | None = None + # TODO: support providing the ns instance in + # order to support 'self.` style to make + # `Portal.run_from_ns()` work! + # _ns: ModuleType|type|None = None + def load_ref(self) -> object | type: if self._ref is None: self._ref = resolve_name(self) @@ -100,5 +105,13 @@ class NamespacePath(str): fqnp: tuple[str, str] = cls._mk_fqnp(ref) return cls(':'.join(fqnp)) - def to_tuple(self) -> tuple[str, str]: - return self._mk_fqnp(self.load_ref()) + def to_tuple( + self, + + # TODO: could this work re `self:` case from above? + # load_ref: bool = True, + + ) -> tuple[str, str]: + return self._mk_fqnp( + self.load_ref() + ) From b28df738fe8cc94d5f0ca8189037a0ecbcbc1b5f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 20 Feb 2024 15:58:11 -0500 Subject: [PATCH 083/146] Drop extra " " when logging actor nursery errors --- tractor/_supervise.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tractor/_supervise.py b/tractor/_supervise.py index 8f4737a3..7684a7c2 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -401,7 +401,7 @@ async def _open_and_supervise_one_cancels_all_nursery( else: log.exception( f"Nursery for {current_actor().uid} " - "errored with\n" + "errored with:" # TODO: same thing as in # `._invoke()` to compute how to From 3c385c6949674ccd19cddf56c78b8b240447c24f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 20 Feb 2024 15:59:55 -0500 Subject: [PATCH 084/146] Use `NamespacePath` in `Context` mgmt internals The only case where we can't is in `Portal.run_from_ns()` usage (since we pass a path with `self:`) and because `.to_tuple()` internally uses `.load_ref()` which will of course fail on such a path.. So or now impl as, - mk `Actor.start_remote_task()` take a `nsf: NamespacePath` but also offer a `load_nsf: bool = False` such that by default we bypass ref loading (maybe this is fine for perf long run as well?) for the `Actor`/'self:'` case mentioned above. - mk `.get_context()` take an instance `nsf` obvi. More logging msg format tweaks: - change msg-flow related content to show the `Context._nsf`, which, right, is coming follow up commit.. - bunch more `.runtime()` format updates to show `msg: dict` contents and internal primitives with trailing `'\n'` for easier reading. - report import loading `stackscope` in subactors. --- tractor/_runtime.py | 175 +++++++++++++++++++++++++++++++------------- 1 file changed, 125 insertions(+), 50 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index e34e1a5e..fe171827 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -49,15 +49,12 @@ import trio from trio import ( CancelScope, ) -from trio.lowlevel import ( - current_task, - Task, -) from trio_typing import ( Nursery, TaskStatus, ) +from .msg import NamespacePath from ._ipc import Channel from ._context import ( mk_context, @@ -138,8 +135,9 @@ async def _invoke( cs: CancelScope | None = None ctx = actor.get_context( - chan, - cid, + chan=chan, + cid=cid, + nsf=NamespacePath.from_ref(func), # We shouldn't ever need to pass this through right? # it's up to the soon-to-be called rpc task to # open the stream with this option. @@ -269,8 +267,8 @@ async def _invoke( # TODO: should would be nice to have our # `TaskMngr` nursery here! - # res: Any = await coro - res = await coro + res: Any = await coro + ctx._result = res # deliver final result to caller side. await chan.send({ @@ -308,11 +306,13 @@ async def _invoke( # associated child isn't in debug any more await _debug.maybe_wait_for_debugger() ctx: Context = actor._contexts.pop((chan.uid, cid)) - log.cancel( - f'Context task was terminated:\n' - f'func: {func}\n' - f'ctx: {pformat(ctx)}' + res_msg: str = ( + 'IPC context terminated with result:\n' + f'result={ctx._result}\n' + f'error={ctx._local_error}\n' + f'|_{pformat(ctx)}\n\n' ) + log.cancel(res_msg) if ctx.cancelled_caught: @@ -324,7 +324,6 @@ async def _invoke( ctx._maybe_raise_remote_err(re) # fname: str = func.__name__ - task: Task = current_task() cs: CancelScope = ctx._scope if cs.cancel_called: our_uid: tuple = actor.uid @@ -371,16 +370,16 @@ async def _invoke( div_str + f'<= canceller: {canceller}\n' f'=> uid: {our_uid}\n' - f' |_ task: `{task.name}()`' + f' |_{ctx._task}()\n' ) # TODO: does this ever get set any more or can # we remove it? if ctx._cancel_msg: msg += ( - '------ - ------\n' - 'IPC msg:\n' - f'{ctx._cancel_msg}' + # '------ - ------\n' + # 'IPC msg:\n' + f'\n{ctx._cancel_msg}' ) # task-contex was either cancelled by request using @@ -428,7 +427,12 @@ async def _invoke( task_status.started(ctx) result = await coro fname: str = func.__name__ - log.runtime(f'{fname}() result: {result}') + log.runtime( + 'RPC complete:\n' + f'task: {ctx._task}\n' + f'|_cid={ctx.cid}\n' + f'|_{fname}() -> {pformat(result)}\n' + ) # NOTE: only send result if we know IPC isn't down if ( @@ -903,7 +907,7 @@ class Actor: # and bail after timeout (2-generals on closure). assert chan.msgstream - log.runtime( + log.warning( f'Draining lingering msgs from stream {chan.msgstream}' ) @@ -915,12 +919,24 @@ class Actor: # making sure any RPC response to that call is # delivered the local calling task. # TODO: factor this into a helper? - log.runtime(f'drained {msg} for {chan.uid}') + log.warning( + 'Draining msg from disconnected\n' + f'peer: {chan.uid}]\n\n' + f'{pformat(msg)}\n' + ) cid = msg.get('cid') if cid: # deliver response to local caller/waiter - await self._push_result(chan, cid, msg) + await self._push_result( + chan, + cid, + msg, + ) + log.runtime( + 'Waiting on local actor nursery to exit..\n' + f'|_{local_nursery}\n' + ) await local_nursery.exited.wait() if disconnected: @@ -1101,6 +1117,7 @@ class Actor: self, chan: Channel, cid: str, + nsf: NamespacePath, msg_buffer_size: int | None = None, allow_overruns: bool = False, @@ -1114,11 +1131,15 @@ class Actor: task-as-function invocation. ''' - log.runtime(f"Getting result queue for {chan.uid} cid {cid}") actor_uid = chan.uid assert actor_uid try: ctx = self._contexts[(actor_uid, cid)] + log.runtime( + f'Retreived cached IPC ctx for\n' + f'peer: {chan.uid}\n' + f'cid:{cid}\n' + ) ctx._allow_overruns = allow_overruns # adjust buffer size if specified @@ -1127,9 +1148,15 @@ class Actor: state.max_buffer_size = msg_buffer_size except KeyError: + log.runtime( + f'Creating NEW IPC ctx for\n' + f'peer: {chan.uid}\n' + f'cid: {cid}\n' + ) ctx = mk_context( chan, cid, + nsf=nsf, msg_buffer_size=msg_buffer_size or self.msg_buffer_size, _allow_overruns=allow_overruns, ) @@ -1140,11 +1167,13 @@ class Actor: async def start_remote_task( self, chan: Channel, - ns: str, - func: str, + nsf: NamespacePath, kwargs: dict, + + # IPC channel config msg_buffer_size: int | None = None, allow_overruns: bool = False, + load_nsf: bool = False, ) -> Context: ''' @@ -1159,20 +1188,43 @@ class Actor: cid = str(uuid.uuid4()) assert chan.uid ctx = self.get_context( - chan, - cid, + chan=chan, + cid=cid, + nsf=nsf, msg_buffer_size=msg_buffer_size, allow_overruns=allow_overruns, ) - log.runtime(f"Sending cmd to {chan.uid}: {ns}.{func}({kwargs})") + + if ( + 'self' in nsf + or not load_nsf + ): + ns, _, func = nsf.partition(':') + else: + # TODO: pass nsf directly over wire! + # -[ ] but, how to do `self:`?? + ns, func = nsf.to_tuple() + + log.runtime( + 'Sending cmd to\n' + f'peer: {chan.uid} => \n' + '\n' + f'=> {ns}.{func}({kwargs})\n' + ) await chan.send( - {'cmd': (ns, func, kwargs, self.uid, cid)} + {'cmd': ( + ns, + func, + kwargs, + self.uid, + cid, + )} ) # Wait on first response msg and validate; this should be # immediate. - first_msg = await ctx._recv_chan.receive() - functype = first_msg.get('functype') + first_msg: dict = await ctx._recv_chan.receive() + functype: str = first_msg.get('functype') if 'error' in first_msg: raise unpack_error(first_msg, chan) @@ -1210,8 +1262,12 @@ class Actor: parent_data: dict[str, Any] parent_data = await chan.recv() log.runtime( - "Received state from parent:\n" - f"{parent_data}" + 'Received state from parent:\n\n' + # TODO: eventually all these msgs as + # `msgspec.Struct` with a special mode that + # pformats them in multi-line mode, BUT only + # if "trace"/"util" mode is enabled? + f'{pformat(parent_data)}\n' ) accept_addr = ( parent_data.pop('bind_host'), @@ -1221,6 +1277,7 @@ class Actor: if rvs['_debug_mode']: try: + log.info('Enabling `stackscope` traces on SIGUSR1') from .devx import enable_stack_on_sig enable_stack_on_sig() except ImportError: @@ -1288,7 +1345,9 @@ class Actor: for listener in listeners ] log.runtime( - f'Started tcp server(s) on {sockets}') + 'Started TCP server(s)\n' + f'|_{sockets}\n' + ) self._listeners.extend(listeners) task_status.started(server_n) finally: @@ -1772,7 +1831,7 @@ async def process_messages( log.runtime( 'Entering IPC msg loop:\n' f'peer: {chan.uid}\n' - f'|_{chan}' + f'|_{chan}\n' ) nursery_cancelled_before_task: bool = False msg: dict | None = None @@ -1818,12 +1877,17 @@ async def process_messages( if cid: # deliver response to local caller/waiter # via its per-remote-context memory channel. - await actor._push_result(chan, cid, msg) + await actor._push_result( + chan, + cid, + msg, + ) log.runtime( - f'Waiting on next IPC msg from {chan.uid}:\n' + 'Waiting on next IPC msg from\n' + f'peer: {chan.uid}:\n' + f'|_{chan}\n' # f'last msg: {msg}\n' - f'|_{chan}' ) continue @@ -1843,9 +1907,11 @@ async def process_messages( raise exc log.runtime( - f"Processing request from {actorid}\n" - f"{ns}.{funcname}({kwargs})") - + 'Handling RPC cmd from\n' + f'peer: {actorid}\n' + '\n' + f'=> {ns}.{funcname}({kwargs})\n' + ) if ns == 'self': if funcname == 'cancel': func: Callable = actor.cancel @@ -1954,17 +2020,18 @@ async def process_messages( # in the lone case where a ``Context`` is not # delivered, it's likely going to be a locally # scoped exception from ``_invoke()`` itself. - if isinstance(ctx, Exception): + if isinstance(err := ctx, Exception): log.warning( - f"Task for RPC func {func} failed with" - f"{ctx}" + 'Task for RPC failed?' + f'|_ {func}()\n\n' + + f'{err}' ) continue else: # mark that we have ongoing rpc tasks actor._ongoing_rpc_tasks = trio.Event() - log.runtime(f"RPC func is {func}") # store cancel scope such that the rpc task can be # cancelled gracefully if requested @@ -1975,7 +2042,10 @@ async def process_messages( ) log.runtime( - f"Waiting on next msg for {chan} from {chan.uid}") + 'Waiting on next IPC msg from\n' + f'peer: {chan.uid}\n' + f'|_{chan}\n' + ) # end of async for, channel disconnect vis # ``trio.EndOfChannel`` @@ -1992,9 +2062,12 @@ async def process_messages( # handshake for them (yet) and instead we simply bail out of # the message loop and expect the teardown sequence to clean # up. + # TODO: don't show this msg if it's an emphemeral + # discovery ep call? log.runtime( - f'channel from {chan.uid} closed abruptly:\n' - f'-> {chan.raddr}\n' + f'channel closed abruptly with\n' + f'peer: {chan.uid}\n' + f'|_{chan.raddr}\n' ) # transport **was** disconnected @@ -2036,9 +2109,11 @@ async def process_messages( finally: # msg debugging for when he machinery is brokey log.runtime( - f'Exiting IPC msg loop with {chan.uid} ' - f'final msg: {msg}\n' - f'|_{chan}' + 'Exiting IPC msg loop with\n' + f'peer: {chan.uid}\n' + f'|_{chan}\n\n' + 'final msg:\n' + f'{pformat(msg)}\n' ) # transport **was not** disconnected From e403d63eb732d748f080bdca79a333d2be554b2a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 21 Feb 2024 13:05:22 -0500 Subject: [PATCH 085/146] Better logging for cancel requests in IPC msg loop As similarly improved in other parts of the runtime, adds much more pedantic (`.cancel()`) logging content to indicate the src of remote cancellation request particularly for `Actor.cancel()` and `._cancel_task()` cases prior to `._invoke()` task scheduling. Also add detailed case comments and much more info to the "request-to-cancel-already-terminated-RPC-task" log emission to include the `Channel` and `Context.cid` deats. This helped me find the src of a race condition causing a test to fail where a callee ctx task was returning a result *before* an expected `ctx.cancel()` request arrived B). Adding much more pedantic `.cancel()` msg contents around the requester's deats should ensure these cases are much easier to detect going forward! Also, simplify the `._invoke()` final result/error log msg to only put *one of either* the final error or returned result above the `Context` pprint. --- tractor/_runtime.py | 79 ++++++++++++++++++++++++++++++++------------- 1 file changed, 56 insertions(+), 23 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index fe171827..a9bbc0d6 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -305,14 +305,19 @@ async def _invoke( # don't pop the local context until we know the # associated child isn't in debug any more await _debug.maybe_wait_for_debugger() - ctx: Context = actor._contexts.pop((chan.uid, cid)) - res_msg: str = ( - 'IPC context terminated with result:\n' - f'result={ctx._result}\n' - f'error={ctx._local_error}\n' - f'|_{pformat(ctx)}\n\n' + ctx: Context = actor._contexts.pop( + (chan.uid, cid) + ) + + res_str: str = ( + 'error: {ctx._local_error}' + if ctx._local_error + else f'result: {ctx._result}' + ) + log.cancel( + f'IPC context terminated with final {res_str}\n' + f'|_{pformat(ctx)}\n' ) - log.cancel(res_msg) if ctx.cancelled_caught: @@ -1453,8 +1458,20 @@ class Actor: # be cancelled was indeed spawned by a request from this channel ctx, func, is_complete = self._rpc_tasks[(chan, cid)] scope: CancelScope = ctx._scope + except KeyError: - log.cancel(f"{cid} has already completed/terminated?") + # NOTE: during msging race conditions this will often + # emit, some examples: + # - callee returns a result before cancel-msg/ctxc-raised + # - callee self raises ctxc before caller send request, + # - callee errors prior to cancel req. + log.cancel( + 'Cancel request invalid, RPC task already completed?\n' + f'<= canceller: {requesting_uid}\n' + f' |_{chan}\n\n' + + f'=> ctx id: {cid}\n' + ) return True log.cancel( @@ -1868,8 +1885,10 @@ async def process_messages( log.transport( # type: ignore f'<= IPC msg from peer: {chan.uid}\n\n' + # TODO: conditionally avoid fmting depending # on log level (for perf)? + # => specifically `pformat()` sub-call..? f'{pformat(msg)}\n' ) @@ -1887,14 +1906,25 @@ async def process_messages( 'Waiting on next IPC msg from\n' f'peer: {chan.uid}:\n' f'|_{chan}\n' + # f'last msg: {msg}\n' ) continue - # TODO: implement with ``match:`` syntax? - # process command request + # process a 'cmd' request-msg upack + # TODO: impl with native `msgspec.Struct` support !! + # -[ ] implement with ``match:`` syntax? + # -[ ] discard un-authed msgs as per, + # try: - ns, funcname, kwargs, actorid, cid = msg['cmd'] + ( + ns, + funcname, + kwargs, + actorid, + cid, + ) = msg['cmd'] + except KeyError: # This is the non-rpc error case, that is, an # error **not** raised inside a call to ``_invoke()`` @@ -1913,25 +1943,27 @@ async def process_messages( f'=> {ns}.{funcname}({kwargs})\n' ) if ns == 'self': + uid: tuple = chan.uid if funcname == 'cancel': func: Callable = actor.cancel - kwargs['requesting_uid'] = chan.uid + kwargs['requesting_uid'] = uid # don't start entire actor runtime cancellation # if this actor is currently in debug mode! - pdb_complete: trio.Event | None = _debug.Lock.local_pdb_complete + pdb_complete: trio.Event|None = _debug.Lock.local_pdb_complete if pdb_complete: await pdb_complete.wait() - # we immediately start the runtime machinery - # shutdown + # Either of `Actor.cancel()`/`.cancel_soon()` + # was called, so terminate this IPC msg + # loop, exit back out into `async_main()`, + # and immediately start the core runtime + # machinery shutdown! with CancelScope(shield=True): - # actor.cancel() was called so kill this - # msg loop and break out into - # ``async_main()`` log.cancel( - "Actor runtime for was remotely cancelled " - f"by {chan.uid}" + f'Cancel request for `Actor` runtime\n' + f'<= canceller: {uid}\n' + # f'=> uid: {actor.uid}\n' ) await _invoke( actor, @@ -1958,9 +1990,10 @@ async def process_messages( target_cid = kwargs['cid'] kwargs['requesting_uid'] = chan.uid log.cancel( - f'Remote request to cancel task\n' - f'remote actor: {chan.uid}\n' - f'task: {target_cid}' + f'Rx task cancel request\n' + f'<= canceller: {chan.uid}\n' + f'=> uid: {actor.uid}\n' + f' |_cid: {target_cid}\n' ) try: await _invoke( From ca1b8e0224deefd3c29561ead2410bc1bfa0fa4d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 21 Feb 2024 13:17:37 -0500 Subject: [PATCH 086/146] Set any `._eoc` to the err in `_raise_from_no_key_in_msg()` Since that's what we're now doing in `MsgStream._eoc` internal assignments (coming in future patch), do the same in this exception re-raise-helper and include more extensive doc string detailing all the msg-type-to-raised-error cases. Also expose a `hide_tb: bool` like we have already in `unpack_error()`. --- tractor/_exceptions.py | 63 ++++++++++++++++++++++++++++-------------- 1 file changed, 43 insertions(+), 20 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index a6d10de7..bdd8d411 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -237,8 +237,10 @@ def pack_error( def unpack_error( msg: dict[str, Any], + chan=None, err_type=RemoteActorError, + hide_tb: bool = True, ) -> None|Exception: @@ -314,37 +316,61 @@ def _raise_from_no_key_in_msg( msg: dict, src_err: KeyError, log: StackLevelAdapter, # caller specific `log` obj + expect_key: str = 'yield', stream: MsgStream | None = None, + # allow "deeper" tbs when debugging B^o + hide_tb: bool = True, + ) -> bool: ''' - Raise an appopriate local error when a `MsgStream` msg arrives - which does not contain the expected (under normal operation) - `'yield'` field. + Raise an appopriate local error when a + `MsgStream` msg arrives which does not + contain the expected (at least under normal + operation) `'yield'` field. + + `Context` and any embedded `MsgStream` termination, + as well as remote task errors are handled in order + of priority as: + + - any 'error' msg is re-boxed and raised locally as + -> `RemoteActorError`|`ContextCancelled` + + - a `MsgStream` 'stop' msg is constructed, assigned + and raised locally as -> `trio.EndOfChannel` + + - All other mis-keyed msgss (like say a "final result" + 'return' msg, normally delivered from `Context.result()`) + are re-boxed inside a `MessagingError` with an explicit + exc content describing the missing IPC-msg-key. ''' - __tracebackhide__: bool = True + __tracebackhide__: bool = hide_tb - # internal error should never get here + # an internal error should never get here try: cid: str = msg['cid'] except KeyError as src_err: raise MessagingError( f'IPC `Context` rx-ed msg without a ctx-id (cid)!?\n' - f'cid: {cid}\n' - 'received msg:\n' + f'cid: {cid}\n\n' + f'{pformat(msg)}\n' ) from src_err # TODO: test that shows stream raising an expected error!!! + + # raise the error message in a boxed exception type! if msg.get('error'): - # raise the error message raise unpack_error( msg, ctx.chan, + hide_tb=hide_tb, + ) from None + # `MsgStream` termination msg. elif ( msg.get('stop') or ( @@ -357,29 +383,26 @@ def _raise_from_no_key_in_msg( f'cid: {cid}\n' ) - # XXX: important to set so that a new ``.receive()`` - # call (likely by another task using a broadcast receiver) - # doesn't accidentally pull the ``return`` message - # value out of the underlying feed mem chan! - stream._eoc: bool = True - # TODO: if the a local task is already blocking on # a `Context.result()` and thus a `.receive()` on the # rx-chan, we close the chan and set state ensuring that # an eoc is raised! - # # when the send is closed we assume the stream has - # # terminated and signal this local iterator to stop - # await stream.aclose() - # XXX: this causes ``ReceiveChannel.__anext__()`` to # raise a ``StopAsyncIteration`` **and** in our catch # block below it will trigger ``.aclose()``. - raise trio.EndOfChannel( + eoc = trio.EndOfChannel( f'Context stream ended due to msg:\n\n' f'{pformat(msg)}\n' - ) from src_err + ) + # XXX: important to set so that a new `.receive()` + # call (likely by another task using a broadcast receiver) + # doesn't accidentally pull the `return` message + # value out of the underlying feed mem chan which is + # destined for the `Context.result()` call during ctx-exit! + stream._eoc: Exception = eoc + raise eoc from src_err if ( stream From 51a3f1bef491f6b092af24210818e3b149b37a2c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 21 Feb 2024 13:21:28 -0500 Subject: [PATCH 087/146] Add `pformat()` of `ActorNursery._children` to logging Such that you see the children entries prior to exit instead of the prior somewhat detail/use-less logging. Also, rename all `anursery` vars to just `an` as is the convention in most examples. --- tractor/_supervise.py | 63 ++++++++++++++++++++++++------------------- 1 file changed, 36 insertions(+), 27 deletions(-) diff --git a/tractor/_supervise.py b/tractor/_supervise.py index 7684a7c2..9d927b19 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -21,6 +21,7 @@ from contextlib import asynccontextmanager as acm from functools import partial import inspect +from pprint import pformat from typing import ( Optional, TYPE_CHECKING, @@ -190,14 +191,16 @@ class ActorNursery: **kwargs, # explicit args to ``fn`` ) -> Portal: - """Spawn a new actor, run a lone task, then terminate the actor and + ''' + Spawn a new actor, run a lone task, then terminate the actor and return its result. Actors spawned using this method are kept alive at nursery teardown until the task spawned by executing ``fn`` completes at which point the actor is terminated. - """ - mod_path = fn.__module__ + + ''' + mod_path: str = fn.__module__ if name is None: # use the explicit function name if not provided @@ -232,7 +235,11 @@ class ActorNursery: ) return portal - async def cancel(self, hard_kill: bool = False) -> None: + async def cancel( + self, + hard_kill: bool = False, + + ) -> None: ''' Cancel this nursery by instructing each subactor to cancel itself and wait for all subactors to terminate. @@ -243,10 +250,12 @@ class ActorNursery: ''' self.cancelled = True - log.cancel(f"Cancelling nursery in {self._actor.uid}") + log.cancel( + 'Cancelling actor nursery\n' + f'|_{self._children}\n' + ) with trio.move_on_after(3) as cs: - - async with trio.open_nursery() as nursery: + async with trio.open_nursery() as tn: subactor: Actor proc: trio.Process @@ -289,7 +298,7 @@ class ActorNursery: # spawn cancel tasks for each sub-actor assert portal if portal.channel.connected(): - nursery.start_soon(portal.cancel_actor) + tn.start_soon(portal.cancel_actor) # if we cancelled the cancel (we hung cancelling remote actors) # then hard kill all sub-processes @@ -344,7 +353,7 @@ async def _open_and_supervise_one_cancels_all_nursery( # the above "daemon actor" nursery will be notified. async with trio.open_nursery() as ria_nursery: - anursery = ActorNursery( + an = ActorNursery( actor, ria_nursery, da_nursery, @@ -353,16 +362,16 @@ async def _open_and_supervise_one_cancels_all_nursery( try: # spawning of actors happens in the caller's scope # after we yield upwards - yield anursery + yield an # When we didn't error in the caller's scope, # signal all process-monitor-tasks to conduct # the "hard join phase". log.runtime( - f"Waiting on subactors {anursery._children} " - "to complete" + 'Waiting on subactors to complete:\n' + f'{pformat(an._children)}\n' ) - anursery._join_procs.set() + an._join_procs.set() except BaseException as inner_err: errors[actor.uid] = inner_err @@ -374,13 +383,13 @@ async def _open_and_supervise_one_cancels_all_nursery( # Instead try to wait for pdb to be released before # tearing down. await maybe_wait_for_debugger( - child_in_debug=anursery._at_least_one_child_in_debug + child_in_debug=an._at_least_one_child_in_debug ) # if the caller's scope errored then we activate our # one-cancels-all supervisor strategy (don't # worry more are coming). - anursery._join_procs.set() + an._join_procs.set() # XXX: hypothetically an error could be # raised and then a cancel signal shows up @@ -414,7 +423,7 @@ async def _open_and_supervise_one_cancels_all_nursery( ) # cancel all subactors - await anursery.cancel() + await an.cancel() # ria_nursery scope end @@ -435,7 +444,7 @@ async def _open_and_supervise_one_cancels_all_nursery( # XXX: yet another guard before allowing the cancel # sequence in case a (single) child is in debug. await maybe_wait_for_debugger( - child_in_debug=anursery._at_least_one_child_in_debug + child_in_debug=an._at_least_one_child_in_debug ) # If actor-local error was raised while waiting on @@ -443,9 +452,9 @@ async def _open_and_supervise_one_cancels_all_nursery( # remaining sub-actors (due to our lone strategy: # one-cancels-all). log.cancel(f"Nursery cancelling due to {err}") - if anursery._children: + if an._children: with trio.CancelScope(shield=True): - await anursery.cancel() + await an.cancel() raise finally: # No errors were raised while awaiting ".run_in_actor()" @@ -455,9 +464,9 @@ async def _open_and_supervise_one_cancels_all_nursery( # collected in ``errors`` so cancel all actors, summarize # all errors and re-raise. if errors: - if anursery._children: + if an._children: with trio.CancelScope(shield=True): - await anursery.cancel() + await an.cancel() # use `BaseExceptionGroup` as needed if len(errors) > 1: @@ -512,20 +521,20 @@ async def open_nursery( try: async with _open_and_supervise_one_cancels_all_nursery( actor - ) as anursery: - yield anursery + ) as an: + yield an finally: - anursery.exited.set() + an.exited.set() else: # sub-nursery case try: async with _open_and_supervise_one_cancels_all_nursery( actor - ) as anursery: - yield anursery + ) as an: + yield an finally: - anursery.exited.set() + an.exited.set() finally: log.debug("Nursery teardown complete") From 9194e5774b81f0fe9bc9168afb1d129228a7379f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 21 Feb 2024 13:24:33 -0500 Subject: [PATCH 088/146] Fix overruns test to avoid return-beats-ctxc race Turns out that py3.11 might be so fast that iterating a EoC-ed `MsgStream` 1k times is faster then a `Context.cancel()` msg transmission from a parent actor to it's child (which i guess makes sense). So tweak the test to delay 5ms between stream async-for iteration attempts when the stream is detected to be `.closed: bool` (coming in patch) or `ctx.cancel_called == true`. --- tests/test_context_stream_semantics.py | 44 +++++++++++++++++++------- 1 file changed, 33 insertions(+), 11 deletions(-) diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index 4eb06e8a..e0ffa874 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -8,6 +8,7 @@ sync-opening a ``tractor.Context`` beforehand. # from contextlib import asynccontextmanager as acm from itertools import count import platform +from pprint import pformat from typing import ( Callable, ) @@ -815,7 +816,10 @@ async def echo_back_sequence( # NOTE: ensure that if the caller is expecting to cancel this task # that we stay echoing much longer then they are so we don't # return early instead of receive the cancel msg. - total_batches: int = 1000 if wait_for_cancel else 6 + total_batches: int = ( + 1000 if wait_for_cancel + else 6 + ) await ctx.started() # await tractor.breakpoint() @@ -834,8 +838,23 @@ async def echo_back_sequence( ) seq = list(seq) # bleh, msgpack sometimes ain't decoded right - for _ in range(total_batches): + for i in range(total_batches): + print(f'starting new stream batch {i} iter in child') batch = [] + + # EoC case, delay a little instead of hot + # iter-stopping (since apparently py3.11+ can do that + # faster then a ctxc can be sent) on the async for + # loop when child was requested to ctxc. + if ( + stream.closed + or + ctx.cancel_called + ): + print('child stream already closed!?!') + await trio.sleep(0.05) + continue + async for msg in stream: batch.append(msg) if batch == seq: @@ -846,15 +865,18 @@ async def echo_back_sequence( print('callee waiting on next') + print(f'callee echoing back latest batch\n{batch}') for msg in batch: - print(f'callee sending {msg}') + print(f'callee sending msg\n{msg}') await stream.send(msg) - print( - 'EXITING CALLEEE:\n' - f'{ctx.canceller}' - ) - return 'yo' + try: + return 'yo' + finally: + print( + 'exiting callee with context:\n' + f'{pformat(ctx)}\n' + ) @pytest.mark.parametrize( @@ -916,8 +938,8 @@ def test_maybe_allow_overruns_stream( wait_for_cancel=cancel_ctx, be_slow=(slow_side == 'child'), allow_overruns_side=allow_overruns_side, - ) as (ctx, sent): + ) as (ctx, sent): assert sent is None async with ctx.open_stream( @@ -945,10 +967,10 @@ def test_maybe_allow_overruns_stream( if cancel_ctx: # cancel the remote task - print('sending root side cancel') + print('Requesting `ctx.cancel()` in parent!') await ctx.cancel() - res = await ctx.result() + res: str|ContextCancelled = await ctx.result() if cancel_ctx: assert isinstance(res, ContextCancelled) From 9420ea0c14d3ea626e8c36b01e39651790d4241b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 22 Feb 2024 13:42:48 -0500 Subject: [PATCH 089/146] Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`. --- tractor/_runtime.py | 331 +++++++++++++++++++++++++++++++------------- 1 file changed, 234 insertions(+), 97 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index a9bbc0d6..165fc40a 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -315,7 +315,7 @@ async def _invoke( else f'result: {ctx._result}' ) log.cancel( - f'IPC context terminated with final {res_str}\n' + f'IPC context terminated with final {res_str}\n\n' f'|_{pformat(ctx)}\n' ) @@ -957,14 +957,14 @@ class Actor: poll = getattr(proc, 'poll', None) if poll and poll() is None: log.cancel( - f'Peer actor IPC broke but proc is alive?\n' - f'uid: {uid}\n' - f'|_{proc}\n' + f'Peer IPC broke but subproc is alive?\n\n' + + f'<=x @{chan.raddr}\n' + f' |_{proc}\n' ) # ``Channel`` teardown and closure sequence - - # Drop ref to channel so it can be gc-ed and disconnected + # drop ref to channel so it can be gc-ed and disconnected log.runtime( f'Disconnected IPC channel:\n' f'uid: {chan.uid}\n' @@ -1112,8 +1112,12 @@ class Actor: ctx: Context = self._contexts[(uid, cid)] except KeyError: log.warning( - f'Ignoring msg from [no-longer/un]known context {uid}:' - f'\n{msg}') + 'Ignoring invalid IPC ctx msg!\n\n' + f'<= sender: {uid}\n' + f'=> cid: {cid}\n\n' + + f'{msg}\n' + ) return return await ctx._deliver_msg(msg) @@ -1310,9 +1314,12 @@ class Actor: except OSError: # failed to connect log.warning( - f"Failed to connect to parent @ {parent_addr}," - " closing server") - await self.cancel(requesting_uid=self.uid) + f'Failed to connect to parent!?\n\n' + 'Closing IPC [TCP] transport server to\n' + f'{parent_addr}\n' + f'|_{self}\n\n' + ) + await self.cancel(chan=None) # self cancel raise async def _serve_forever( @@ -1368,28 +1375,55 @@ class Actor: ''' assert self._service_n - self._service_n.start_soon(self.cancel) + self._service_n.start_soon( + self.cancel, + None, # self cancel all rpc tasks + ) async def cancel( self, - requesting_uid: tuple[str, str], + + # chan whose lifetime limits the lifetime of its remotely + # requested and locally spawned RPC tasks - similar to the + # supervision semantics of a nursery wherein the actual + # implementation does start all such tasks in + # a sub-nursery. + req_chan: Channel|None, ) -> bool: ''' - Cancel this actor's runtime. + Cancel this actor's runtime, eventually resulting in + the exit its containing process. - The "deterministic" teardown sequence in order is: - - cancel all ongoing rpc tasks by cancel scope - - cancel the channel server to prevent new inbound - connections - - cancel the "service" nursery reponsible for - spawning new rpc tasks - - return control the parent channel message loop + The ideal "deterministic" teardown sequence in order is: + - cancel all ongoing rpc tasks by cancel scope + - cancel the channel server to prevent new inbound + connections + - cancel the "service" nursery reponsible for + spawning new rpc tasks + - return control the parent channel message loop ''' - log.cancel( - f'{self.uid} requested to cancel by:\n' - f'{requesting_uid}' + ( + requesting_uid, + requester_type, + req_chan, + + ) = ( + req_chan.uid, + 'peer', + req_chan, + + ) if req_chan else ( + + # a self cancel of ALL rpc tasks + self.uid, + 'self', + self + ) + msg: str = ( + f'`Actor.cancel()` request from {requester_type}:\n' + f'<= {requesting_uid}\n' ) # TODO: what happens here when we self-cancel tho? @@ -1403,12 +1437,16 @@ class Actor: # with the root actor in this tree dbcs = _debug.Lock._debugger_request_cs if dbcs is not None: - log.cancel("Cancelling active debugger request") + msg += ( + '>> Cancelling active debugger request..\n' + f'|_{_debug.Lock}\n' + ) dbcs.cancel() - # kill all ongoing tasks + # self-cancel **all** ongoing RPC tasks await self.cancel_rpc_tasks( - requesting_uid=requesting_uid, + req_uid=requesting_uid, + parent_chan=None, ) # stop channel server @@ -1417,13 +1455,14 @@ class Actor: await self._server_down.wait() else: log.warning( - f'{self.uid} was likely cancelled before it started') + 'Transport[TCP] server was cancelled start?' + ) # cancel all rpc tasks permanently if self._service_n: self._service_n.cancel_scope.cancel() - log.cancel(f"{self.uid} called `Actor.cancel()`") + log.cancel(msg) self._cancel_complete.set() return True @@ -1438,7 +1477,7 @@ class Actor: async def _cancel_task( self, cid: str, - chan: Channel, + parent_chan: Channel, requesting_uid: tuple[str, str] | None = None, ) -> bool: @@ -1450,13 +1489,25 @@ class Actor: in the signature (for now). ''' - # right now this is only implicitly called by + # this ctx based lookup ensures the requested task to + # be cancelled was indeed spawned by a request from + # this channel + ctx: Context + func: Callable + is_complete: trio.Event + + # NOTE: right now this is only implicitly called by # streaming IPC but it should be called # to cancel any remotely spawned task try: - # this ctx based lookup ensures the requested task to - # be cancelled was indeed spawned by a request from this channel - ctx, func, is_complete = self._rpc_tasks[(chan, cid)] + ( + ctx, + func, + is_complete, + ) = self._rpc_tasks[( + parent_chan, + cid, + )] scope: CancelScope = ctx._scope except KeyError: @@ -1467,17 +1518,28 @@ class Actor: # - callee errors prior to cancel req. log.cancel( 'Cancel request invalid, RPC task already completed?\n' - f'<= canceller: {requesting_uid}\n' - f' |_{chan}\n\n' - - f'=> ctx id: {cid}\n' + f'<= canceller: {requesting_uid}\n\n' + f'=>{parent_chan}\n' + f' |_ctx-id: {cid}\n' ) return True log.cancel( - f"Cancelling task:\ncid: {cid}\nfunc: {func}\n" - f"peer: {chan.uid}\n") + 'Cancel request for RPC task\n' + f'<= canceller: {requesting_uid}\n\n' + # TODO: better ascii repr for "supervisor" like + # a nursery or context scope? + f'=> ipc-parent: {parent_chan}\n' + # TODO: simplified `Context.__repr__()` fields output + # shows only application state-related stuff like, + # - ._stream + # - .closed + # - .started_called + # - .. etc. + f' |_ctx: {cid}\n' + f' >> {ctx._nsf}()\n' + ) if ( ctx._canceller is None and requesting_uid @@ -1487,6 +1549,7 @@ class Actor: # don't allow cancelling this function mid-execution # (is this necessary?) if func is self._cancel_task: + log.error('Do not cancel a cancel!?') return True # TODO: shouldn't we eventually be calling ``Context.cancel()`` @@ -1496,23 +1559,29 @@ class Actor: scope.cancel() # wait for _invoke to mark the task complete + flow_info: str = ( + f'<= canceller: {requesting_uid}\n' + f'=> ipc-parent: {parent_chan}\n' + f' |_{ctx}\n' + ) log.runtime( - 'Waiting on task to cancel:\n' - f'cid: {cid}\nfunc: {func}\n' - f'peer: {chan.uid}\n' + 'Waiting on RPC task to cancel\n' + f'{flow_info}' ) await is_complete.wait() - log.runtime( - f"Sucessfully cancelled task:\ncid: {cid}\nfunc: {func}\n" - f"peer: {chan.uid}\n") - + f'Sucessfully cancelled RPC task\n' + f'{flow_info}' + ) return True async def cancel_rpc_tasks( self, - only_chan: Channel | None = None, - requesting_uid: tuple[str, str] | None = None, + req_uid: tuple[str, str], + + # NOTE: when None is passed we cancel **all** rpc + # tasks running in this actor! + parent_chan: Channel|None, ) -> None: ''' @@ -1521,38 +1590,76 @@ class Actor: ''' tasks: dict = self._rpc_tasks - if tasks: - tasks_str: str = '' - for (ctx, func, _) in tasks.values(): - tasks_str += ( - f' |_{func.__name__}() [cid={ctx.cid[-6:]}..]\n' - ) - - log.cancel( - f'Cancelling all {len(tasks)} rpc tasks:\n' - f'{tasks_str}' + if not tasks: + log.warning( + 'Actor has no cancellable RPC tasks?\n' + f'<= cancel requester: {req_uid}\n' + f'=> {self}\n\n' ) - for ( - (chan, cid), - (ctx, func, is_complete), - ) in tasks.copy().items(): - if only_chan is not None: - if only_chan != chan: - continue + return - # TODO: this should really done in a nursery batch - if func != self._cancel_task: - await self._cancel_task( - cid, - chan, - requesting_uid=requesting_uid, - ) + # TODO: seriously factor this into some helper funcs XD + tasks_str: str = '' + for (ctx, func, _) in tasks.values(): - log.cancel( - 'Waiting for remaining rpc tasks to complete:\n' - f'{tasks}' + # TODO: std repr of all primitives in + # a hierarchical tree format, since we can!! + # like => repr for funcs/addrs/msg-typing: + # + # -[ ] use a proper utf8 "arm" like + # `stackscope` has! + # -[ ] for typed msging, show the + # py-type-annot style? + # - maybe auto-gen via `inspect` / `typing` type-sig: + # https://stackoverflow.com/a/57110117 + # => see ex. code pasted into `.msg.types` + # + # -[ ] proper .maddr() for IPC primitives? + # - `Channel.maddr() -> str:` obvi! + # - `Context.maddr() -> str:` + tasks_str += ( + f' |_@ /ipv4/tcp/cid="{ctx.cid[-16:]} .."\n' + f' |>> {ctx._nsf}() -> dict:\n' ) - await self._ongoing_rpc_tasks.wait() + + log.cancel( + f'Cancelling all {len(tasks)} rpc tasks:\n\n' + f'<= .cancel() from {req_uid}\n' + f'{self}\n' + f'{tasks_str}' + ) + for ( + (task_caller_chan, cid), + (ctx, func, is_complete), + ) in tasks.copy().items(): + + if ( + # maybe filter to specific IPC channel? + (parent_chan + and + task_caller_chan != parent_chan) + + # never "cancel-a-cancel" XD + or (func == self._cancel_task) + ): + continue + + # if func == self._cancel_task: + # continue + + # TODO: this maybe block on the task cancellation + # and so should really done in a nursery batch? + await self._cancel_task( + cid, + task_caller_chan, + requesting_uid=req_uid, + ) + + log.cancel( + 'Waiting for remaining rpc tasks to complete\n' + f'|_{tasks}' + ) + await self._ongoing_rpc_tasks.wait() def cancel_server(self) -> None: ''' @@ -1943,10 +2050,11 @@ async def process_messages( f'=> {ns}.{funcname}({kwargs})\n' ) if ns == 'self': - uid: tuple = chan.uid if funcname == 'cancel': func: Callable = actor.cancel - kwargs['requesting_uid'] = uid + kwargs |= { + 'req_chan': chan, + } # don't start entire actor runtime cancellation # if this actor is currently in debug mode! @@ -1960,11 +2068,6 @@ async def process_messages( # and immediately start the core runtime # machinery shutdown! with CancelScope(shield=True): - log.cancel( - f'Cancel request for `Actor` runtime\n' - f'<= canceller: {uid}\n' - # f'=> uid: {actor.uid}\n' - ) await _invoke( actor, cid, @@ -1974,25 +2077,32 @@ async def process_messages( is_rpc=False, ) - log.cancel( - f'Cancelling IPC msg-loop with {chan.uid}' + log.runtime( + 'Cancelling IPC transport msg-loop with peer:\n' + f'|_{chan}\n' ) loop_cs.cancel() break if funcname == '_cancel_task': - func = actor._cancel_task + func: Callable = actor._cancel_task # we immediately start the runtime machinery # shutdown # with CancelScope(shield=True): - kwargs['chan'] = chan - target_cid = kwargs['cid'] - kwargs['requesting_uid'] = chan.uid + target_cid: str = kwargs['cid'] + kwargs |= { + # NOTE: ONLY the rpc-task-owning + # parent IPC channel should be able to + # cancel it! + 'parent_chan': chan, + 'requesting_uid': chan.uid, + } log.cancel( f'Rx task cancel request\n' f'<= canceller: {chan.uid}\n' - f'=> uid: {actor.uid}\n' + f' |_{chan}\n\n' + f'=> {actor}\n' f' |_cid: {target_cid}\n' ) try: @@ -2005,8 +2115,13 @@ async def process_messages( is_rpc=False, ) except BaseException: - log.exception("failed to cancel task?") - + log.exception( + 'Failed to cancel task?\n' + f'<= canceller: {chan.uid}\n' + f' |_{chan}\n\n' + f'=> {actor}\n' + f' |_cid: {target_cid}\n' + ) continue else: # normally registry methods, eg. @@ -2025,9 +2140,25 @@ async def process_messages( await chan.send(err_msg) continue - # spin up a task for the requested function - log.runtime(f"Spawning task for {func}") - assert actor._service_n + # schedule a task for the requested RPC function + # in the actor's main "service nursery". + # TODO: possibly a service-tn per IPC channel for + # supervision isolation? would avoid having to + # manage RPC tasks individually in `._rpc_tasks` + # table? + log.runtime( + f'Spawning task for RPC request\n' + f'<= caller: {chan.uid}\n' + f' |_{chan}\n\n' + # TODO: maddr style repr? + # f' |_@ /ipv4/{chan.raddr}/tcp/{chan.rport}/' + # f'cid="{cid[-16:]} .."\n\n' + + f'=> {actor}\n' + f' |_cid: {cid}\n' + f' |>> {func}()\n' + ) + assert actor._service_n # wait why? do it at top? try: ctx: Context = await actor._service_n.start( partial( @@ -2085,7 +2216,13 @@ async def process_messages( log.runtime( f"{chan} for {chan.uid} disconnected, cancelling tasks" ) - await actor.cancel_rpc_tasks(chan) + await actor.cancel_rpc_tasks( + req_uid=actor.uid, + # a "self cancel" in terms of the lifetime of the + # IPC connection which is presumed to be the + # source of any requests for spawned tasks. + parent_chan=chan, + ) except ( TransportClosed, From 1be296c7259d0c33483a555209f1a5e7797f798b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 22 Feb 2024 14:22:45 -0500 Subject: [PATCH 090/146] Add note that maybe `Context._eoc` should be set by caller? --- tractor/_exceptions.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index bdd8d411..fe3f2706 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -371,6 +371,8 @@ def _raise_from_no_key_in_msg( ) from None # `MsgStream` termination msg. + # TODO: does it make more sense to pack + # the stream._eoc outside this in the calleer always? elif ( msg.get('stop') or ( From 5eb62b3e9bd716ada7ec7a407beb4e87ab9337a5 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 22 Feb 2024 14:41:28 -0500 Subject: [PATCH 091/146] Tweak broadcast fanout test to never inf loop Since a bug in the new `MsgStream.aclose()` impl's drain block logic was triggering an actual inf loop (by not ever canceller the streamer child actor), make sure we put a loop limit on the `inf_streamer`()` XD Also add a bit more deats to the test `print()`s in each actor and toss in `debug_mode` fixture support. --- tests/test_advanced_streaming.py | 59 ++++++++++++++++++++++++-------- 1 file changed, 44 insertions(+), 15 deletions(-) diff --git a/tests/test_advanced_streaming.py b/tests/test_advanced_streaming.py index 799a0897..82cc53a0 100644 --- a/tests/test_advanced_streaming.py +++ b/tests/test_advanced_streaming.py @@ -298,52 +298,77 @@ async def inf_streamer( async with ( ctx.open_stream() as stream, - trio.open_nursery() as n, + trio.open_nursery() as tn, ): - async def bail_on_sentinel(): + async def close_stream_on_sentinel(): async for msg in stream: if msg == 'done': + print( + 'streamer RXed "done" sentinel msg!\n' + 'CLOSING `MsgStream`!' + ) await stream.aclose() else: print(f'streamer received {msg}') + else: + print('streamer exited recv loop') # start termination detector - n.start_soon(bail_on_sentinel) + tn.start_soon(close_stream_on_sentinel) - for val in itertools.count(): + cap: int = 10000 # so that we don't spin forever when bug.. + for val in range(cap): try: + print(f'streamer sending {val}') await stream.send(val) + if val > cap: + raise RuntimeError( + 'Streamer never cancelled by setinel?' + ) + await trio.sleep(0.001) + + # close out the stream gracefully except trio.ClosedResourceError: - # close out the stream gracefully + print('msgstream closed on streamer side!') + assert stream.closed break + else: + raise RuntimeError( + 'Streamer not cancelled before finished sending?' + ) - print('terminating streamer') + print('streamer exited .open_streamer() block') -def test_local_task_fanout_from_stream(): +def test_local_task_fanout_from_stream( + debug_mode: bool, +): ''' Single stream with multiple local consumer tasks using the ``MsgStream.subscribe()` api. - Ensure all tasks receive all values after stream completes sending. + Ensure all tasks receive all values after stream completes + sending. ''' - consumers = 22 + consumers: int = 22 async def main(): counts = Counter() - async with tractor.open_nursery() as tn: - p = await tn.start_actor( + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as tn: + p: tractor.Portal = await tn.start_actor( 'inf_streamer', enable_modules=[__name__], ) + # with trio.fail_after(3): async with ( p.open_context(inf_streamer) as (ctx, _), ctx.open_stream() as stream, ): - async def pull_and_count(name: str): # name = trio.lowlevel.current_task().name async with stream.subscribe() as recver: @@ -352,7 +377,7 @@ def test_local_task_fanout_from_stream(): tractor.trionics.BroadcastReceiver ) async for val in recver: - # print(f'{name}: {val}') + print(f'bx {name} rx: {val}') counts[name] += 1 print(f'{name} bcaster ended') @@ -362,10 +387,14 @@ def test_local_task_fanout_from_stream(): with trio.fail_after(3): async with trio.open_nursery() as nurse: for i in range(consumers): - nurse.start_soon(pull_and_count, i) + nurse.start_soon( + pull_and_count, + i, + ) + # delay to let bcast consumers pull msgs await trio.sleep(0.5) - print('\nterminating') + print('terminating nursery of bcast rxer consumers!') await stream.send('done') print('closed stream connection') From 4d0df1bb4a92594ed7f882ae5707f62efcac4685 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 22 Feb 2024 14:45:08 -0500 Subject: [PATCH 092/146] Call `actor.cancel(None)` from root to avoid mismatch with (any future) meth sig changes --- tests/test_advanced_streaming.py | 1 - tractor/_root.py | 4 +--- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/test_advanced_streaming.py b/tests/test_advanced_streaming.py index 82cc53a0..8061c3b9 100644 --- a/tests/test_advanced_streaming.py +++ b/tests/test_advanced_streaming.py @@ -364,7 +364,6 @@ def test_local_task_fanout_from_stream( 'inf_streamer', enable_modules=[__name__], ) - # with trio.fail_after(3): async with ( p.open_context(inf_streamer) as (ctx, _), ctx.open_stream() as stream, diff --git a/tractor/_root.py b/tractor/_root.py index 853eaf0b..c17249d1 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -253,9 +253,7 @@ async def open_root_actor( # tempn.start_soon(an.exited.wait) logger.cancel("Shutting down root actor") - await actor.cancel( - requesting_uid=actor.uid, - ) + await actor.cancel(None) # self cancel finally: _state._current_actor = None From a388d3185bcf574ca33d696d7ffa8bafe77e27ef Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 22 Feb 2024 15:06:39 -0500 Subject: [PATCH 093/146] Few more log msg tweaks in runtime --- tractor/_runtime.py | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 165fc40a..270599ae 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -911,11 +911,6 @@ class Actor: # Attempt to wait for the far end to close the channel # and bail after timeout (2-generals on closure). assert chan.msgstream - - log.warning( - f'Draining lingering msgs from stream {chan.msgstream}' - ) - async for msg in chan.msgstream.drain(): # try to deliver any lingering msgs # before we destroy the channel. @@ -925,8 +920,11 @@ class Actor: # delivered the local calling task. # TODO: factor this into a helper? log.warning( - 'Draining msg from disconnected\n' - f'peer: {chan.uid}]\n\n' + 'Draining msg from disconnected peer\n' + f'{chan.uid}\n' + f'|_{chan}\n' + f' |_{chan.msgstream}\n\n' + f'{pformat(msg)}\n' ) cid = msg.get('cid') @@ -1593,8 +1591,7 @@ class Actor: if not tasks: log.warning( 'Actor has no cancellable RPC tasks?\n' - f'<= cancel requester: {req_uid}\n' - f'=> {self}\n\n' + f'<= canceller: {req_uid}\n' ) return From a97b45d90b7f21da7068b47f9ffd31ec1d805a3d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 22 Feb 2024 18:33:18 -0500 Subject: [PATCH 094/146] WIP final impl of ctx-cancellation-semantics --- tractor/_context.py | 280 ++++++++++++++++++++++++++++++------------ tractor/_portal.py | 184 +++++++++++++++++---------- tractor/_streaming.py | 86 ++++++++----- 3 files changed, 378 insertions(+), 172 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index 54e309e1..ee05a2ba 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -47,6 +47,7 @@ import trio # maybe_wait_for_debugger, # pause, # ) +from .msg import NamespacePath from ._exceptions import ( # _raise_from_no_key_in_msg, unpack_error, @@ -71,12 +72,23 @@ log = get_logger(__name__) async def _drain_to_final_msg( ctx: Context, -) -> list[dict]: -# ) -> tuple[ -# Any|Exception, -# list[dict], -# ]: + msg_limit: int = 6, + +) -> list[dict]: + ''' + Drain IPC msgs delivered to the underlying rx-mem-chan + `Context._recv_chan` from the runtime in search for a final + result 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. + + ''' raise_overrun: bool = not ctx._allow_overruns # wait for a final context result by collecting (but @@ -88,14 +100,14 @@ async def _drain_to_final_msg( # NOTE: this REPL usage actually works here dawg! Bo # from .devx._debug import pause # await pause() - # if re := ctx._remote_error: - # ctx._maybe_raise_remote_err( - # re, - # # NOTE: obvi we don't care if we - # # overran the far end if we're already - # # waiting on a final result (msg). - # raise_overrun_from_self=raise_overrun, - # ) + if re := ctx._remote_error: + ctx._maybe_raise_remote_err( + re, + # NOTE: obvi we don't care if we + # overran the far end if we're already + # waiting on a final result (msg). + raise_overrun_from_self=raise_overrun, + ) # TODO: bad idea? # with trio.CancelScope() as res_cs: @@ -108,7 +120,7 @@ async def _drain_to_final_msg( msg: dict = await ctx._recv_chan.receive() ctx._result: Any = msg['return'] log.runtime( - 'Context delivered final result msg:\n' + 'Context delivered final draining msg:\n' f'{pformat(msg)}' ) pre_result_drained.append(msg) @@ -142,9 +154,47 @@ async def _drain_to_final_msg( if 'yield' in msg: # far end task is still streaming to us so discard - log.warning(f'Discarding std "yield"\n{msg}') - pre_result_drained.append(msg) - continue + # and report per local context state. + if ( + (ctx._stream.closed + and (reason := 'stream was already closed') + ) + or (ctx._cancel_called + and (reason := 'ctx called `.cancel()`') + ) + or (ctx._cancelled_caught + and (reason := 'ctx caught a cancel') + ) + or (len(pre_result_drained) > msg_limit + and (reason := f'"yield" limit={msg_limit}') + ) + ): + log.cancel( + 'Cancelling `MsgStream` drain since ' + f'{reason}\n\n' + f'<= {ctx.chan.uid}\n' + f' |_{ctx._nsf}()\n\n' + f'=> {ctx._task}\n' + f' |_{ctx._stream}\n\n' + + f'{pformat(msg)}\n' + ) + return pre_result_drained + + # drain up to the `msg_limit` hoping to get + # a final result or error/ctxc. + else: + log.warning( + 'Ignoring "yield" msg during `ctx.result()` drain..\n' + f'<= {ctx.chan.uid}\n' + f' |_{ctx._nsf}()\n\n' + f'=> {ctx._task}\n' + f' |_{ctx._stream}\n\n' + + f'{pformat(msg)}\n' + ) + pre_result_drained.append(msg) + continue # TODO: work out edge cases here where # a stream is open but the task also calls @@ -153,8 +203,8 @@ async def _drain_to_final_msg( # right? elif 'stop' in msg: log.cancel( - 'Remote stream terminated due to "stop" msg:\n' - f'{msg}' + 'Remote stream terminated due to "stop" msg:\n\n' + f'{pformat(msg)}\n' ) pre_result_drained.append(msg) continue @@ -260,12 +310,14 @@ class Context: ''' chan: Channel cid: str # "context id", more or less a unique linked-task-pair id - # the "feeder" channels for delivering message values to the # local task from the runtime's msg processing loop. _recv_chan: trio.MemoryReceiveChannel _send_chan: trio.MemorySendChannel + # full "namespace-path" to target RPC function + _nsf: NamespacePath + # the "invocation type" of the far end task-entry-point # function, normally matching a logic block inside # `._runtime.invoke()`. @@ -281,6 +333,7 @@ class Context: # which is exactly the primitive that allows for # cross-actor-task-supervision and thus SC. _scope: trio.CancelScope | None = None + _task: trio.lowlevel.Task|None = None # _res_scope: trio.CancelScope|None = None # on a clean exit there should be a final value @@ -384,6 +437,7 @@ class Context: # init and streaming state _started_called: bool = False _stream_opened: bool = False + _stream: MsgStream|None = None # overrun handling machinery # NOTE: none of this provides "backpressure" to the remote @@ -577,13 +631,14 @@ class Context: ''' side: str = self.side - log.cancel( - f'Cancelling {side} side of context to {self.chan.uid}' - ) - - # await pause() self._cancel_called: bool = True + header: str = f'Cancelling "{side.upper()}"-side of ctx with peer\n' + reminfo: str = ( + f'uid: {self.chan.uid}\n' + f' |_ {self._nsf}()\n' + ) + # caller side who entered `Portal.open_context()` # NOTE: on the call side we never manually call # `._scope.cancel()` since we expect the eventual @@ -601,8 +656,9 @@ class Context: with trio.move_on_after(timeout) as cs: cs.shield = True log.cancel( - f'Cancelling stream {cid} to ' - f'{self._portal.channel.uid}' + header + + + reminfo ) # NOTE: we're telling the far end actor to cancel a task @@ -621,13 +677,13 @@ class Context: # if not self._portal.channel.connected(): if not self.chan.connected(): log.cancel( - 'May have failed to cancel remote task ' - f'{cid} for {self._portal.channel.uid}' + 'May have failed to cancel remote task?\n' + f'{reminfo}' ) else: log.cancel( - 'Timed out on cancel request of remote task ' - f'{cid} for {self._portal.channel.uid}' + 'Timed out on cancel request of remote task?\n' + f'{reminfo}' ) # callee side remote task @@ -635,6 +691,11 @@ class Context: # the caller expects a `ContextCancelled` to be sent from # `._runtime._invoke()` back to the other side. else: + log.cancel( + header + + + reminfo + ) # TODO: should we have an explicit cancel message # or is relaying the local `trio.Cancelled` as an # {'error': trio.Cancelled, cid: "blah"} enough? @@ -720,8 +781,9 @@ class Context: # single-direction-stream case you'll get a lookup error # currently. ctx: Context = actor.get_context( - self.chan, - self.cid, + chan=self.chan, + cid=self.cid, + nsf=self._nsf, msg_buffer_size=msg_buffer_size, allow_overruns=allow_overruns, ) @@ -735,7 +797,7 @@ class Context: if ctx._recv_chan._closed: raise trio.ClosedResourceError( - 'The underlying channel for this stream was already closed!?' + 'The underlying channel for this stream was already closed!\n' ) # NOTE: implicitly this will call `MsgStream.aclose()` on @@ -764,6 +826,7 @@ class Context: try: self._stream_opened: bool = True + self._stream = stream # XXX: do we need this? # ensure we aren't cancelled before yielding the stream @@ -1174,35 +1237,47 @@ class Context: self, msg: dict, - # draining: bool = False, - ) -> bool: ''' Deliver an IPC msg received from a transport-channel to - this context's underlying mem chan for handling by - user operating tasks; deliver a bool indicating whether the - msg was immediately sent. + this context's underlying mem chan for handling by local + user application tasks; deliver `bool` indicating whether + the msg was able to be delivered. If `._allow_overruns == True` (maybe) append the msg to an "overflow queue" and start a "drainer task" (inside the `._scope_nursery: trio.Nursery`) which ensures that such - messages are eventually sent if possible. + messages are queued up and eventually sent if possible. ''' cid: str = self.cid chan: Channel = self.chan from_uid: tuple[str, str] = chan.uid send_chan: trio.MemorySendChannel = self._send_chan + nsf: NamespacePath = self._nsf + re: Exception|None if re := unpack_error( msg, self.chan, ): log.error( - f'Delivering error-msg from {from_uid} to caller {cid}' - f'{re}' + f'Delivering error-msg to caller\n' + f'<= peer: {from_uid}\n' + f' |_ {nsf}()\n\n' + + f'=> cid: {cid}\n' + f' |_{self._task}\n\n' + + f'{pformat(re)}\n' ) - self._cancel_msg = msg + self._cancel_msg: dict = msg + + # NOTE: this will not raise an error, merely set + # `._remote_error` and maybe cancel any task currently + # entered in `Portal.open_context()` presuming the + # error is "cancel causing" (i.e. `ContextCancelled` + # or `RemoteActorError`). self._maybe_cancel_and_set_remote_error(re) # XXX NEVER do this XXX..!! @@ -1218,26 +1293,44 @@ class Context: if self._in_overrun: log.warning( - f'Capturing overrun-msg from {from_uid} to caller {cid}' - f'{msg}' + f'Queueing OVERRUN msg on caller task:\n' + f'<= peer: {from_uid}\n' + f' |_ {nsf}()\n\n' + + f'=> cid: {cid}\n' + f' |_{self._task}\n\n' + + f'{pformat(msg)}\n' ) self._overflow_q.append(msg) return False try: log.runtime( - f'Delivering IPC `Context` msg:\n' + f'Delivering msg from IPC ctx:\n' f'<= {from_uid}\n' - f'=> caller: {cid}\n' - f'{msg}' + f' |_ {nsf}()\n\n' + + f'=> {self._task}\n' + f' |_cid={self.cid}\n\n' + + f'{pformat(msg)}\n' ) # from .devx._debug import pause # await pause() + + # NOTE: if an error is deteced we should always still + # send it through the feeder-mem-chan and expect + # it to be raised by any context (stream) consumer + # task via the consumer APIs on both the `Context` and + # `MsgStream`! + # + # XXX the reason is that this method is always called + # by the IPC msg handling runtime task and that is not + # normally the task that should get cancelled/error + # from some remote fault! send_chan.send_nowait(msg) return True - # if an error is deteced we should always - # expect it to be raised by any context (stream) - # consumer task except trio.BrokenResourceError: # TODO: what is the right way to handle the case where the @@ -1248,7 +1341,13 @@ class Context: # XXX: local consumer has closed their side # so cancel the far end streaming task - log.warning(f"{send_chan} consumer is already closed") + log.warning( + 'Rx chan for `Context` alfready closed?\n' + f'cid: {self.cid}\n' + 'Failed to deliver msg:\n' + f'send_chan: {send_chan}\n\n' + f'{pformat(msg)}\n' + ) return False # NOTE XXX: by default we do **not** maintain context-stream @@ -1257,44 +1356,54 @@ class Context: # msg handling loop which calls into this method! except trio.WouldBlock: - # XXX: always push an error even if the local - # receiver is in overrun state. - # self._maybe_cancel_and_set_remote_error(msg) + # XXX: always push an error even if the local receiver + # is in overrun state - i.e. if an 'error' msg is + # delivered then + # `._maybe_cancel_and_set_remote_error(msg)` should + # have already been called above! + # + # XXX QUESTION XXX: if we rx an error while in an + # overrun state and that msg isn't stuck in an + # overflow queue what happens?!? local_uid = current_actor().uid - lines = [ - f'OVERRUN on actor-task context {cid}@{local_uid}!\n' - # TODO: put remote task name here if possible? - f'sender: {from_uid}', - f'msg: {msg}', - # TODO: put task func name here and maybe an arrow - # from sender to overrunner? - # f'local task {self.func_name}' - ] - if not self._stream_opened: - lines.insert( - 1, - f'\n*** No stream open on `{local_uid[0]}` side! ***\n' - ) + txt: str = ( + 'on IPC context:\n' - text = '\n'.join(lines) + f'<= sender: {from_uid}\n' + f' |_ {self._nsf}()\n\n' + + f'=> overrun: {local_uid}\n' + f' |_cid: {cid}\n' + f' |_task: {self._task}\n' + ) + if not self._stream_opened: + txt += ( + f'\n*** No stream open on `{local_uid[0]}` side! ***\n\n' + f'{msg}\n' + ) # XXX: lul, this really can't be backpressure since any # blocking here will block the entire msg loop rpc sched for # a whole channel.. maybe we should rename it? if self._allow_overruns: - text += f'\nStarting overflow queuing task on msg: {msg}' - log.warning(text) + txt += ( + '\n*** Starting overflow queuing task on msg ***\n\n' + f'{msg}\n' + ) + log.warning(txt) if ( not self._in_overrun ): self._overflow_q.append(msg) - n = self._scope_nursery - assert not n.child_tasks + tn: trio.Nursery = self._scope_nursery + assert not tn.child_tasks try: - n.start_soon( + tn.start_soon( self._drain_overflows, ) + return True + except RuntimeError: # if the nursery is already cancelled due to # this context exiting or in error, we ignore @@ -1302,11 +1411,12 @@ class Context: # anything different. return False else: + txt += f'\n{msg}\n' # raise local overrun and immediately pack as IPC # msg for far end. try: raise StreamOverrun( - text, + txt, sender=from_uid, ) except StreamOverrun as err: @@ -1314,20 +1424,28 @@ class Context: err, cid=cid, ) - # err_msg['cid']: str = cid try: + # relay condition to sender side remote task await chan.send(err_msg) + return True + except trio.BrokenResourceError: # XXX: local consumer has closed their side # so cancel the far end streaming task - log.warning(f"{chan} is already closed") + log.warning( + 'Channel for ctx is already closed?\n' + f'|_{chan}\n' + ) + # ow, indicate unable to deliver by default return False def mk_context( chan: Channel, cid: str, + nsf: NamespacePath, + msg_buffer_size: int = 2**6, **kwargs, @@ -1345,10 +1463,12 @@ def mk_context( send_chan, recv_chan = trio.open_memory_channel(msg_buffer_size) ctx = Context( - chan, - cid, + chan=chan, + cid=cid, _send_chan=send_chan, _recv_chan=recv_chan, + _nsf=nsf, + _task=trio.lowlevel.current_task(), **kwargs, ) ctx._result: int | Any = id(ctx) diff --git a/tractor/_portal.py b/tractor/_portal.py index 97b89b3c..8235c92f 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -69,18 +69,35 @@ from ._streaming import ( log = get_logger(__name__) +# TODO: rename to `unwrap_result()` and use +# `._raise_from_no_key_in_msg()` (after tweak to +# accept a `chan: Channel` arg) in key block! def _unwrap_msg( msg: dict[str, Any], - channel: Channel + channel: Channel, + + hide_tb: bool = True, ) -> Any: - __tracebackhide__ = True + ''' + Unwrap a final result from a `{return: }` IPC msg. + + ''' + __tracebackhide__: bool = hide_tb + try: return msg['return'] except KeyError as ke: + # internal error should never get here - assert msg.get('cid'), "Received internal error at portal?" - raise unpack_error(msg, channel) from ke + assert msg.get('cid'), ( + "Received internal error at portal?" + ) + + raise unpack_error( + msg, + channel + ) from ke class Portal: @@ -107,7 +124,7 @@ class Portal: cancel_timeout: float = 0.5 def __init__(self, channel: Channel) -> None: - self.channel = channel + self.chan = channel # during the portal's lifetime self._result_msg: Optional[dict] = None @@ -118,6 +135,18 @@ class Portal: self._streams: set[MsgStream] = set() self.actor = current_actor() + @property + def channel(self) -> Channel: + ''' + Proxy to legacy attr name.. + + Consider the shorter `Portal.chan` instead of `.channel` ;) + ''' + log.debug( + 'Consider the shorter `Portal.chan` instead of `.channel` ;)' + ) + return self.chan + async def _submit_for_result( self, ns: str, @@ -125,14 +154,14 @@ class Portal: **kwargs ) -> None: - assert self._expect_result is None, \ - "A pending main result has already been submitted" + assert self._expect_result is None, ( + "A pending main result has already been submitted" + ) self._expect_result = await self.actor.start_remote_task( self.channel, - ns, - func, - kwargs + nsf=NamespacePath(f'{ns}:{func}'), + kwargs=kwargs ) async def _return_once( @@ -173,7 +202,10 @@ class Portal: self._expect_result ) - return _unwrap_msg(self._result_msg, self.channel) + return _unwrap_msg( + self._result_msg, + self.channel, + ) async def _cancel_streams(self): # terminate all locally running async generator @@ -215,26 +247,33 @@ class Portal: purpose. ''' - if not self.channel.connected(): - log.cancel("This channel is already closed can't cancel") + chan: Channel = self.channel + if not chan.connected(): + log.runtime( + 'This channel is already closed, skipping cancel request..' + ) return False + reminfo: str = ( + f'uid: {self.channel.uid}\n' + f' |_{chan}\n' + ) log.cancel( - f"Sending actor cancel request to {self.channel.uid} on " - f"{self.channel}") - - self.channel._cancel_called = True + f'Sending actor cancel request to peer\n' + f'{reminfo}' + ) + self.channel._cancel_called: bool = True try: # send cancel cmd - might not get response # XXX: sure would be nice to make this work with # a proper shield with trio.move_on_after( timeout - or self.cancel_timeout + or + self.cancel_timeout ) as cs: - cs.shield = True - + cs.shield: bool = True await self.run_from_ns( 'self', 'cancel', @@ -242,7 +281,10 @@ class Portal: return True if cs.cancelled_caught: - log.cancel(f"May have failed to cancel {self.channel.uid}") + log.cancel( + 'May have failed to cancel peer?\n' + f'{reminfo}' + ) # if we get here some weird cancellation case happened return False @@ -272,27 +314,33 @@ class Portal: Note:: - A special namespace `self` can be used to invoke `Actor` - instance methods in the remote runtime. Currently this - should only be used solely for ``tractor`` runtime - internals. + A special namespace `self` can be used to invoke `Actor` + instance methods in the remote runtime. Currently this + should only ever be used for `Actor` (method) runtime + internals! ''' + nsf = NamespacePath( + f'{namespace_path}:{function_name}' + ) ctx = await self.actor.start_remote_task( - self.channel, - namespace_path, - function_name, - kwargs, + chan=self.channel, + nsf=nsf, + kwargs=kwargs, ) ctx._portal = self msg = await self._return_once(ctx) - return _unwrap_msg(msg, self.channel) + return _unwrap_msg( + msg, + self.channel, + ) async def run( self, func: str, - fn_name: Optional[str] = None, + fn_name: str|None = None, **kwargs + ) -> Any: ''' Submit a remote function to be scheduled and run by actor, in @@ -311,8 +359,9 @@ class Portal: DeprecationWarning, stacklevel=2, ) - fn_mod_path = func + fn_mod_path: str = func assert isinstance(fn_name, str) + nsf = NamespacePath(f'{fn_mod_path}:{fn_name}') else: # function reference was passed directly if ( @@ -325,13 +374,12 @@ class Portal: raise TypeError( f'{func} must be a non-streaming async function!') - fn_mod_path, fn_name = NamespacePath.from_ref(func).to_tuple() + nsf = NamespacePath.from_ref(func) ctx = await self.actor.start_remote_task( self.channel, - fn_mod_path, - fn_name, - kwargs, + nsf=nsf, + kwargs=kwargs, ) ctx._portal = self return _unwrap_msg( @@ -355,15 +403,10 @@ class Portal: raise TypeError( f'{async_gen_func} must be an async generator function!') - fn_mod_path, fn_name = NamespacePath.from_ref( - async_gen_func - ).to_tuple() - - ctx = await self.actor.start_remote_task( + ctx: Context = await self.actor.start_remote_task( self.channel, - fn_mod_path, - fn_name, - kwargs + nsf=NamespacePath.from_ref(async_gen_func), + kwargs=kwargs, ) ctx._portal = self @@ -405,7 +448,10 @@ class Portal: self, func: Callable, + allow_overruns: bool = False, + + # proxied to RPC **kwargs, ) -> AsyncGenerator[tuple[Context, Any], None]: @@ -448,13 +494,12 @@ class Portal: # TODO: i think from here onward should probably # just be factored into an `@acm` inside a new # a new `_context.py` mod. - fn_mod_path, fn_name = NamespacePath.from_ref(func).to_tuple() + nsf = NamespacePath.from_ref(func) - ctx = await self.actor.start_remote_task( + ctx: Context = await self.actor.start_remote_task( self.channel, - fn_mod_path, - fn_name, - kwargs, + nsf=nsf, + kwargs=kwargs, # NOTE: it's imporant to expose this since you might # get the case where the parent who opened the context does @@ -721,10 +766,10 @@ class Portal: # assert maybe_ctxc if ctx.chan.connected(): - log.info( - 'Waiting on final context-task result for\n' - f'task: {cid}\n' - f'actor: {uid}' + log.runtime( + 'Waiting on final context result for\n' + f'peer: {uid}\n' + f'|_{ctx._task}\n' ) # XXX NOTE XXX: the below call to # `Context.result()` will ALWAYS raise @@ -771,13 +816,19 @@ class Portal: RemoteActorError(), ): log.exception( - f'Context `{fn_name}` remotely errored:\n' - f'`{tbstr}`' + 'Context remotely errored!\n' + f'<= peer: {uid}\n' + f' |_ {nsf}()\n\n' + + f'{tbstr}' ) case (None, _): log.runtime( - f'Context {fn_name} returned value from callee:\n' - f'`{result_or_err}`' + 'Context returned final result from callee task:\n' + f'<= peer: {uid}\n' + f' |_ {nsf}()\n\n' + + f'`{result_or_err}`\n' ) finally: @@ -855,17 +906,20 @@ class Portal: # CASE 2 if ctx._cancel_called: log.cancel( - f'Context {fn_name} cancelled by caller with\n' + 'Context cancelled by caller task\n' + f'|_{ctx._task}\n\n' + f'{etype}' ) # CASE 1 else: log.cancel( - f'Context cancelled by callee with {etype}\n' - f'target: `{fn_name}`\n' - f'task:{cid}\n' - f'actor:{uid}' + f'Context cancelled by remote callee task\n' + f'peer: {uid}\n' + f'|_ {nsf}()\n\n' + + f'{etype}\n' ) # XXX: (MEGA IMPORTANT) if this is a root opened process we @@ -880,10 +934,12 @@ class Portal: # FINALLY, remove the context from runtime tracking and # exit! log.runtime( - f'Exiting context opened with {ctx.chan.uid}' + 'Removing IPC ctx opened with peer\n' + f'{uid}\n' + f'|_{ctx}\n' ) self.actor._contexts.pop( - (self.channel.uid, ctx.cid), + (uid, cid), None, ) diff --git a/tractor/_streaming.py b/tractor/_streaming.py index e8f735ec..64b5dd6d 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -95,9 +95,6 @@ class MsgStream(trio.abc.Channel): try: return msg['yield'] except KeyError as kerr: - # if 'return' in msg: - # return msg - _raise_from_no_key_in_msg( ctx=self._ctx, msg=msg, @@ -128,13 +125,9 @@ class MsgStream(trio.abc.Channel): # introducing this if self._eoc: raise self._eoc - # raise trio.EndOfChannel if self._closed: raise self._closed - # raise trio.ClosedResourceError( - # 'This stream was already closed' - # ) src_err: Exception|None = None try: @@ -143,6 +136,7 @@ class MsgStream(trio.abc.Channel): return msg['yield'] except KeyError as kerr: + # log.exception('GOT KEYERROR') src_err = kerr # NOTE: may raise any of the below error types @@ -161,9 +155,9 @@ class MsgStream(trio.abc.Channel): # trio.ClosedResourceError, # by self._rx_chan trio.EndOfChannel, # by self._rx_chan or `stop` msg from far end ) as eoc: + # log.exception('GOT EOC') src_err = eoc self._eoc = eoc - # await trio.sleep(1) # a ``ClosedResourceError`` indicates that the internal # feeder memory receive channel was closed likely by the @@ -201,6 +195,7 @@ class MsgStream(trio.abc.Channel): # raise eoc except trio.ClosedResourceError as cre: # by self._rx_chan + # log.exception('GOT CRE') src_err = cre log.warning( '`Context._rx_chan` was already closed?' @@ -211,6 +206,8 @@ class MsgStream(trio.abc.Channel): # terminated and signal this local iterator to stop drained: list[Exception|dict] = await self.aclose() if drained: + # from .devx import pause + # await pause() log.warning( 'Drained context msgs during closure:\n' f'{drained}' @@ -237,31 +234,32 @@ class MsgStream(trio.abc.Channel): Cancel associated remote actor task and local memory channel on close. + Notes: + - REMEMBER that this is also called by `.__aexit__()` so + careful consideration must be made to handle whatever + internal stsate is mutated, particuarly in terms of + draining IPC msgs! + + - more or less we try to maintain adherance to trio's `.aclose()` semantics: + https://trio.readthedocs.io/en/stable/reference-io.html#trio.abc.AsyncResource.aclose ''' - # XXX: keep proper adherance to trio's `.aclose()` semantics: - # https://trio.readthedocs.io/en/stable/reference-io.html#trio.abc.AsyncResource.aclose - rx_chan = self._rx_chan - if ( - rx_chan._closed - or - self._closed - ): - log.cancel( - f'`MsgStream` is already closed\n' - f'.cid: {self._ctx.cid}\n' - f'._rx_chan`: {rx_chan}\n' - f'._eoc: {self._eoc}\n' - f'._closed: {self._eoc}\n' - ) + # rx_chan = self._rx_chan + # XXX NOTE XXX + # it's SUPER IMPORTANT that we ensure we don't DOUBLE + # DRAIN msgs on closure so avoid getting stuck handing on + # the `._rx_chan` since we call this method on + # `.__aexit__()` as well!!! + # => SO ENSURE WE CATCH ALL TERMINATION STATES in this + # block including the EoC.. + if self.closed: # this stream has already been closed so silently succeed as # per ``trio.AsyncResource`` semantics. # https://trio.readthedocs.io/en/stable/reference-io.html#trio.abc.AsyncResource.aclose return [] ctx: Context = self._ctx - # caught_eoc: bool = False drained: list[Exception|dict] = [] while not drained: try: @@ -274,17 +272,26 @@ class MsgStream(trio.abc.Channel): # TODO: inject into parent `Context` buf? drained.append(maybe_final_msg) + # NOTE: we only need these handlers due to the + # `.receive_nowait()` call above which may re-raise + # one of these errors on a msg key error! + except trio.WouldBlock as be: drained.append(be) break except trio.EndOfChannel as eoc: + self._eoc: Exception = eoc drained.append(eoc) - # caught_eoc = True - self._eoc: bool = eoc + break + + except trio.ClosedResourceError as cre: + self._closed = cre + drained.append(cre) break except ContextCancelled as ctxc: + # log.exception('GOT CTXC') log.cancel( 'Context was cancelled during stream closure:\n' f'canceller: {ctxc.canceller}\n' @@ -339,8 +346,11 @@ class MsgStream(trio.abc.Channel): # with trio.CancelScope(shield=True): # await rx_chan.aclose() - # self._eoc: bool = caught_eoc - + if not self._eoc: + self._eoc: bool = trio.EndOfChannel( + f'Context stream closed by {self._ctx.side}\n' + f'|_{self}\n' + ) # ?XXX WAIT, why do we not close the local mem chan `._rx_chan` XXX? # => NO, DEFINITELY NOT! <= # if we're a bi-dir ``MsgStream`` BECAUSE this same @@ -379,6 +389,26 @@ class MsgStream(trio.abc.Channel): # self._closed = True return drained + @property + def closed(self) -> bool: + if ( + (rxc := self._rx_chan._closed) + or + (_closed := self._closed) + or + (_eoc := self._eoc) + ): + log.runtime( + f'`MsgStream` is already closed\n' + f'{self}\n' + f' |_cid: {self._ctx.cid}\n' + f' |_rx_chan._closed: {type(rxc)} = {rxc}\n' + f' |_closed: {type(_closed)} = {_closed}\n' + f' |_eoc: {type(_eoc)} = {_eoc}' + ) + return True + return False + @acm async def subscribe( self, From e1d7004aec6bc50d46f4060cd89f67482441c606 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 22 Feb 2024 20:37:12 -0500 Subject: [PATCH 095/146] Add a `pytest.ini` config --- pytest.ini | 8 ++++++++ 1 file changed, 8 insertions(+) create mode 100644 pytest.ini diff --git a/pytest.ini b/pytest.ini new file mode 100644 index 00000000..6a7e51fb --- /dev/null +++ b/pytest.ini @@ -0,0 +1,8 @@ +# vim: ft=ini +# pytest.ini for tractor + +[pytest] +# don't show frickin captured logs AGAIN in the report.. +addopts = --show-capture='no' +log_cli = false +; minversion = 6.0 From 664ae87588ee7625bcc7ac883f677facf8d1684f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 28 Feb 2024 17:13:01 -0500 Subject: [PATCH 096/146] Make `@context`-cancelled tests more pedantic In order to match a very significant and coming-soon patch set to the IPC `Context` and `Channel` cancellation semantics with significant but subtle changes to the primitives and runtime logic: - a new set of `Context` state pub meth APIs for checking exact inter-actor-linked-task outcomes such as `.outcome`, `.maybe_error`, and `.cancel_acked`. - trying to move away from `Context.cancelled_caught` usage since the semantics from `trio` don't really map well (in terms of cancel requests and how they result in cancel-scope graceful closure) and `.cancel_acked: bool` is a better approach for IPC req-resp msging. - change test usage to access `._scope.cancelled_caught` directly. - more pedantic ctxc-raising expects around the "type of self cancellation" and final outcome in ctxc cases: - `ContextCancelled` is raised by ctx (`Context.result()`) consumer methods when `Portal.cancel_actor()` is called (since it's an out-of-band request) despite `Channel._cancel_called` being set. - also raised by `.open_context().__aexit__()` on close. - `.outcome` is always `.maybe_error` is always one of `._local/remote_error`. --- tests/test_cancellation.py | 10 ++- tests/test_context_stream_semantics.py | 119 ++++++++++++++++++++----- 2 files changed, 106 insertions(+), 23 deletions(-) diff --git a/tests/test_cancellation.py b/tests/test_cancellation.py index ce396ace..9a729f3d 100644 --- a/tests/test_cancellation.py +++ b/tests/test_cancellation.py @@ -48,11 +48,13 @@ async def do_nuthin(): ids=['no_args', 'unexpected_args'], ) def test_remote_error(reg_addr, args_err): - """Verify an error raised in a subactor that is propagated + ''' + Verify an error raised in a subactor that is propagated to the parent nursery, contains the underlying boxed builtin error type info and causes cancellation and reraising all the way up the stack. - """ + + ''' args, errtype = args_err async def main(): @@ -65,7 +67,9 @@ def test_remote_error(reg_addr, args_err): # an exception group outside the nursery since the error # here and the far end task error are one in the same? portal = await nursery.run_in_actor( - assert_err, name='errorer', **args + assert_err, + name='errorer', + **args ) # get result(s) from main task diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index e0ffa874..19a87453 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -5,7 +5,7 @@ Verify the we raise errors when streams are opened prior to sync-opening a ``tractor.Context`` beforehand. ''' -# from contextlib import asynccontextmanager as acm +from contextlib import asynccontextmanager as acm from itertools import count import platform from pprint import pformat @@ -250,6 +250,17 @@ def test_simple_context( trio.run(main) +@acm +async def expect_ctxc(yay: bool) -> None: + if yay: + try: + yield + except ContextCancelled: + return + else: + yield + + @pytest.mark.parametrize( 'callee_returns_early', [True, False], @@ -280,23 +291,60 @@ def test_caller_cancels( async def check_canceller( ctx: Context, ) -> None: - # should not raise yet return the remote - # context cancelled error. - res = await ctx.result() + actor: Actor = current_actor() + uid: tuple = actor.uid + if ( + cancel_method == 'portal' + and not callee_returns_early + ): + try: + res = await ctx.result() + assert 0, 'Portal cancel should raise!' + + except ContextCancelled as ctxc: + assert ctx.chan._cancel_called + assert ctxc.canceller == uid + assert ctxc is ctx.maybe_error + + # NOTE: should not ever raise even in the `ctx` + # case since self-cancellation should swallow the ctxc + # silently! + else: + res = await ctx.result() + + # we actually get a result if callee_returns_early: assert res == 'yo' + assert ctx.outcome is res + assert ctx.maybe_error is None else: - err = res + err: Exception = ctx.outcome assert isinstance(err, ContextCancelled) assert ( tuple(err.canceller) == - current_actor().uid + uid ) + assert ( + err + is ctx.maybe_error + is ctx._remote_error + ) + if le := ctx._local_error: + assert err is le + + # else: + # TODO: what should this be then? + # not defined until block closes right? + # + # await tractor.pause() + # assert ctx._local_error is None + async def main(): + async with tractor.open_nursery( debug_mode=debug_mode, ) as an: @@ -306,11 +354,16 @@ def test_caller_cancels( ) timeout = 0.5 if not callee_returns_early else 2 with trio.fail_after(timeout): - async with portal.open_context( - simple_setup_teardown, - data=10, - block_forever=not callee_returns_early, - ) as (ctx, sent): + async with ( + + expect_ctxc(yay=cancel_method == 'portal'), + + portal.open_context( + simple_setup_teardown, + data=10, + block_forever=not callee_returns_early, + ) as (ctx, sent), + ): if callee_returns_early: # ensure we block long enough before sending @@ -332,6 +385,16 @@ def test_caller_cancels( if cancel_method != 'portal': await portal.cancel_actor() + # since the `.cancel_actor()` call just above + # will cause the `.open_context().__aexit__()` raise + # a ctxc which should in turn cause `ctx._scope` to + # catch any cancellation? + if ( + not callee_returns_early + and cancel_method == 'portal' + ): + assert ctx._scope.cancelled_caught + trio.run(main) @@ -434,7 +497,6 @@ async def test_callee_closes_ctx_after_stream_open( @tractor.context async def expect_cancelled( - ctx: Context, ) -> None: @@ -454,7 +516,7 @@ async def expect_cancelled( raise else: - assert 0, "Wasn't cancelled!?" + assert 0, "callee wasn't cancelled !?" @pytest.mark.parametrize( @@ -473,8 +535,8 @@ async def test_caller_closes_ctx_after_callee_opens_stream( async with tractor.open_nursery( debug_mode=debug_mode, ) as an: - root: Actor = current_actor() + root: Actor = current_actor() portal = await an.start_actor( 'ctx_cancelled', enable_modules=[__name__], @@ -487,11 +549,13 @@ async def test_caller_closes_ctx_after_callee_opens_stream( await portal.run(assert_state, value=True) - # call cancel explicitly + # call `ctx.cancel()` explicitly if use_ctx_cancel_method: - await ctx.cancel() + # NOTE: means the local side `ctx._scope` will + # have been cancelled by an ctxc ack and thus + # `._scope.cancelled_caught` should be set. try: async with ctx.open_stream() as stream: async for msg in stream: @@ -520,20 +584,35 @@ async def test_caller_closes_ctx_after_callee_opens_stream( assert portal.channel.connected() # ctx is closed here - await portal.run(assert_state, value=False) + await portal.run( + assert_state, + value=False, + ) else: try: with trio.fail_after(0.2): await ctx.result() assert 0, "Callee should have blocked!?" + except trio.TooSlowError: # NO-OP -> since already called above await ctx.cancel() - # local scope should have absorbed the cancellation - assert ctx.cancelled_caught - assert ctx._remote_error is ctx._local_error + # NOTE: local scope should have absorbed the cancellation since + # in this case we call `ctx.cancel()` and the local + # `._scope` gets `.cancel_called` on the ctxc ack. + if use_ctx_cancel_method: + assert ctx._scope.cancelled_caught + + # rxed ctxc response from far end + assert ctx.cancel_acked + assert ( + ctx._remote_error + is ctx._local_error + is ctx.maybe_error + is ctx.outcome + ) try: async with ctx.open_stream() as stream: From cfcbc4da01974d93672e23b33fac2b2c95cb1bec Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 29 Feb 2024 14:21:45 -0500 Subject: [PATCH 097/146] Add test for `modden` sub-spawner-server hangs on cancel As per a lot of the recent refinements to `Context` cancellation, add a new test case to replicate the original hang-on-cancel found with `modden` when using a client actor to spawn a subactor in some other tree where despite `Context.cancel()` being called the requesting client would hang on the opened context with the server. The specific scenario added here is to have, - root actor spawns 2 children: a client and a spawn server. - the spawn server opens with a spawn-request serve loop and begins to wait for the client. - client spawns and connects to the sibling spawn server, requests to spawn a sub-actor, the "little bro", connects to it then does some echo streaming, cancels the request with it's sibling (the spawn server) which should in turn cancel the root's-grandchild and result in a cancel-ack back to the client's `.open_context()`. - root ensures that it can also connect to the grandchild (little bro), do the same echo streaming, then ensure everything tears down correctly after cancelling all the children. More refinements to come here obvi in the specific cancellation semantics and possibly causes. Also tweaks the other tests in suite to use the new `Context` properties recently introduced and similarly updated in the previous patch to the ctx-semantics suite. --- tests/test_inter_peer_cancellation.py | 484 +++++++++++++++++++++++--- 1 file changed, 445 insertions(+), 39 deletions(-) diff --git a/tests/test_inter_peer_cancellation.py b/tests/test_inter_peer_cancellation.py index 1ead6172..082c5e65 100644 --- a/tests/test_inter_peer_cancellation.py +++ b/tests/test_inter_peer_cancellation.py @@ -10,6 +10,9 @@ import pytest import trio import tractor from tractor import ( # typing + Actor, + current_actor, + open_nursery, Portal, Context, ContextCancelled, @@ -238,19 +241,23 @@ async def stream_from_peer( # caller peer should not be the cancel requester assert not ctx.cancel_called + assert not ctx.cancel_acked - # XXX can never be true since `._invoke` only + # XXX can NEVER BE TRUE since `._invoke` only # sets this AFTER the nursery block this task # was started in, exits. - assert not ctx.cancelled_caught + assert not ctx._scope.cancelled_caught - # we never requested cancellation + # we never requested cancellation, it was the 'canceller' + # peer. assert not peer_ctx.cancel_called + assert not peer_ctx.cancel_acked + # the `.open_context()` exit definitely caught # a cancellation in the internal `Context._scope` since # likely the runtime called `_deliver_msg()` after # receiving the remote error from the streaming task. - assert peer_ctx.cancelled_caught + assert not peer_ctx._scope.cancelled_caught # TODO / NOTE `.canceller` won't have been set yet # here because that machinery is inside @@ -259,6 +266,8 @@ async def stream_from_peer( # checkpoint) that cancellation was due to # a remote, we COULD assert this here..see, # https://github.com/goodboy/tractor/issues/368 + # + # assert 'canceller' in ctx.canceller # root/parent actor task should NEVER HAVE cancelled us! assert not ctx.canceller @@ -356,8 +365,7 @@ def test_peer_canceller( 'just_caller', # but i just met her? enable_modules=[__name__], ) - - root = tractor.current_actor() + root: Actor = current_actor() try: async with ( @@ -395,8 +403,8 @@ def test_peer_canceller( # not request the sleeper cancellation ;) except ContextCancelled as ctxerr: print( - 'CAUGHT REMOTE CONTEXT CANCEL FOM\n' - f'{ctxerr}' + 'CAUGHT REMOTE CONTEXT CANCEL\n\n' + f'{ctxerr}\n' ) # canceller and caller peers should not @@ -404,6 +412,9 @@ def test_peer_canceller( assert canceller_ctx.canceller is None assert caller_ctx.canceller is None + # we were not the actor, our peer was + assert not sleeper_ctx.cancel_acked + assert ctxerr.canceller[0] == 'canceller' # XXX NOTE XXX: since THIS `ContextCancelled` @@ -411,11 +422,13 @@ def test_peer_canceller( # `sleeper.open_context().__aexit__()` this # value is not yet set, however outside this # block it should be. - assert not sleeper_ctx.cancelled_caught + assert not sleeper_ctx._scope.cancelled_caught + # CASE_1: error-during-ctxc-handling, if error_during_ctxerr_handling: raise RuntimeError('Simulated error during teardown') + # CASE_2: standard teardown inside in `.open_context()` block raise # XXX SHOULD NEVER EVER GET HERE XXX @@ -436,7 +449,6 @@ def test_peer_canceller( else: pytest.fail( 'did not rx ctxc ?!?\n\n' - f'{ctxs}\n' ) @@ -447,21 +459,48 @@ def test_peer_canceller( _loc_err = loc_err # NOTE: the main state to check on `Context` is: - # - `.cancelled_caught` (maps to nursery cs) # - `.cancel_called` (bool of whether this side # requested) + # - `.cancel_acked` (bool of whether a ctxc + # response was received due to cancel req). + # - `.maybe_error` (highest prio error to raise + # locally) + # - `.outcome` (final error or result value) # - `.canceller` (uid of cancel-causing actor-task) # - `._remote_error` (any `RemoteActorError` # instance from other side of context) + # - `._local_error` (any error caught inside the + # `.open_context()` block). + # + # XXX: Deprecated and internal only + # - `.cancelled_caught` (maps to nursery cs) + # - now just use `._scope.cancelled_caught` + # since it maps to the internal (maps to nursery cs) + # # TODO: are we really planning to use this tho? # - `._cancel_msg` (any msg that caused the # cancel) - # CASE: error raised during handling of - # `ContextCancelled` inside `.open_context()` - # block + # CASE_1: error-during-ctxc-handling, + # - far end cancels due to peer 'canceller', + # - `ContextCancelled` relayed to this scope, + # - inside `.open_context()` ctxc is caught and + # a rte raised instead + # + # => block should raise the rte but all peers + # should be cancelled by US. + # if error_during_ctxerr_handling: assert isinstance(loc_err, RuntimeError) + print(f'_loc_err: {_loc_err}\n') + # assert sleeper_ctx._local_error is _loc_err + # assert sleeper_ctx._local_error is _loc_err + assert not ( + loc_err + is sleeper_ctx.maybe_error + is sleeper_ctx.outcome + is sleeper_ctx._remote_error + ) # NOTE: this root actor task should have # called `Context.cancel()` on the @@ -495,7 +534,25 @@ def test_peer_canceller( root.uid ) - # CASE: standard teardown inside in `.open_context()` block + # since the sleeper errors while handling a + # peer-cancelled (by ctxc) scenario, we expect + # that the `.open_context()` block DOES call + # `.cancel() (despite in this test case it + # being unecessary). + assert ( + sleeper_ctx.cancel_called + and + not sleeper_ctx.cancel_acked + ) + + # CASE_2: standard teardown inside in `.open_context()` block + # - far end cancels due to peer 'canceller', + # - `ContextCancelled` relayed to this scope and + # raised locally without any raise-during-handle, + # + # => inside `.open_context()` ctxc is raised and + # propagated + # else: assert isinstance(loc_err, ContextCancelled) assert loc_err.canceller == sleeper_ctx.canceller @@ -509,24 +566,42 @@ def test_peer_canceller( # the sleeper's remote error is the error bubbled # out of the context-stack above! - re = sleeper_ctx._remote_error - assert re is loc_err + re = sleeper_ctx.outcome + assert ( + re is loc_err + is sleeper_ctx.maybe_error + is sleeper_ctx._remote_error + ) for ctx in ctxs: - re: BaseException | None = ctx._remote_error - assert re + re: BaseException|None = ctx._remote_error + re: BaseException|None = ctx.outcome + assert ( + re and + ( + re is ctx.maybe_error + is ctx._remote_error + ) + ) + le: trio.MultiError = ctx._local_error + assert ( + le + and ctx._local_error + ) # root doesn't cancel sleeper since it's # cancelled by its peer. if ctx is sleeper_ctx: assert not ctx.cancel_called + assert not ctx.cancel_acked + # since sleeper_ctx.result() IS called # above we should have (silently) # absorbed the corresponding # `ContextCancelled` for it and thus # the logic inside `.cancelled_caught` # should trigger! - assert ctx.cancelled_caught + assert ctx._scope.cancelled_caught elif ctx is caller_ctx: # since its context was remotely @@ -535,15 +610,33 @@ def test_peer_canceller( # done by the peer and also we never assert ctx.cancel_called - # TODO: figure out the details of - # this.. + # TODO: figure out the details of this..? # if you look the `._local_error` here # is a multi of ctxc + 2 Cancelleds? # assert not ctx.cancelled_caught + elif ctx is canceller_ctx: + + # XXX NOTE XXX: ONLY the canceller + # will get a self-cancelled outcome + # whilst everyone else gets + # a peer-caused cancellation! + # + # TODO: really we should avoid calling + # .cancel() whenever an interpeer + # cancel takes place since each + # reception of a ctxc + assert ( + ctx.cancel_called + and ctx.cancel_acked + ) + assert not ctx._scope.cancelled_caught + else: - assert ctx.cancel_called - assert not ctx.cancelled_caught + pytest.fail( + 'Uhh wut ctx is this?\n' + f'{ctx}\n' + ) # TODO: do we even need this flag? # -> each context should have received @@ -559,14 +652,24 @@ def test_peer_canceller( # `Context.cancel()` SHOULD NOT have been # called inside # `Portal.open_context().__aexit__()`. - assert not sleeper_ctx.cancel_called + assert not ( + sleeper_ctx.cancel_called + or + sleeper_ctx.cancel_acked + ) # XXX NOTE XXX: and see matching comment above but, - # this flag is set only AFTER the `.open_context()` - # has exited and should be set in both outcomes - # including the case where ctx-cancel handling - # itself errors. - assert sleeper_ctx.cancelled_caught + # the `._scope` is only set by `trio` AFTER the + # `.open_context()` block has exited and should be + # set in both outcomes including the case where + # ctx-cancel handling itself errors. + assert sleeper_ctx._scope.cancelled_caught + assert _loc_err is sleeper_ctx._local_error + assert ( + sleeper_ctx.outcome + is sleeper_ctx.maybe_error + is sleeper_ctx._remote_error + ) raise # always to ensure teardown @@ -582,12 +685,315 @@ def test_peer_canceller( assert excinfo.value.canceller[0] == 'canceller' -def test_client_tree_spawns_and_cancels_service_subactor(): - ... -# TODO: test for the modden `mod wks open piker` bug! -# -> start actor-tree (server) that offers sub-actor spawns via -# context API -# -> start another full actor-tree (client) which requests to the first to -# spawn over its `@context` ep / api. -# -> client actor cancels the context and should exit gracefully -# and the server's spawned child should cancel and terminate! +@tractor.context +async def basic_echo_server( + ctx: Context, + peer_name: str = 'stepbro', + +) -> None: + ''' + Just the simplest `MsgStream` echo server which resays what + you told it but with its uid in front ;) + + ''' + actor: Actor = tractor.current_actor() + uid: tuple = actor.uid + await ctx.started(uid) + async with ctx.open_stream() as ipc: + async for msg in ipc: + + # repack msg pair with our uid + # as first element. + ( + client_uid, + i, + ) = msg + resp: tuple = ( + uid, + i, + ) + # OOF! looks like my runtime-error is causing a lockup + # assert 0 + await ipc.send(resp) + + +@tractor.context +async def serve_subactors( + ctx: Context, + peer_name: str, + +) -> None: + async with open_nursery() as an: + await ctx.started(peer_name) + async with ctx.open_stream() as reqs: + async for msg in reqs: + peer_name: str = msg + peer: Portal = await an.start_actor( + name=peer_name, + enable_modules=[__name__], + ) + print( + 'Spawning new subactor\n' + f'{peer_name}\n' + f'|_{peer}\n' + ) + await reqs.send(( + peer.chan.uid, + peer.chan.raddr, + )) + + print('Spawner exiting spawn serve loop!') + + +@tractor.context +async def client_req_subactor( + ctx: Context, + peer_name: str, + + # used to simulate a user causing an error to be raised + # directly in thread (like a KBI) to better replicate the + # case where a `modden` CLI client would hang afer requesting + # a `Context.cancel()` to `bigd`'s wks spawner. + reraise_on_cancel: str|None = None, + +) -> None: + # TODO: other cases to do with sub lifetimes: + # -[ ] test that we can have the server spawn a sub + # that lives longer then ctx with this client. + # -[ ] test that + + # open ctx with peer spawn server and ask it to spawn a little + # bro which we'll then connect and stream with. + async with ( + tractor.find_actor( + name='spawn_server', + raise_on_none=True, + + # TODO: we should be isolating this from other runs! + # => ideally so we can eventually use something like + # `pytest-xdist` Bo + # registry_addrs=bigd._reg_addrs, + ) as spawner, + + spawner.open_context( + serve_subactors, + peer_name=peer_name, + ) as (spawner_ctx, first), + ): + assert first == peer_name + await ctx.started( + 'yup i had brudder', + ) + + async with spawner_ctx.open_stream() as reqs: + + # send single spawn request to the server + await reqs.send(peer_name) + with trio.fail_after(3): + ( + sub_uid, + sub_raddr, + ) = await reqs.receive() + + + await tell_little_bro( + actor_name=sub_uid[0], + caller='client', + ) + + # TODO: test different scope-layers of + # cancellation? + # with trio.CancelScope() as cs: + try: + await trio.sleep_forever() + + # TODO: would be super nice to have a special injected + # cancel type here (maybe just our ctxc) but using + # some native mechanism in `trio` :p + except ( + trio.Cancelled + ) as err: + _err = err + if reraise_on_cancel: + errtype = globals()['__builtins__'][reraise_on_cancel] + assert errtype + to_reraise: BaseException = errtype() + print(f'client re-raising on cancel: {repr(to_reraise)}') + raise err + + raise + + # if cs.cancelled_caught: + # print('client handling expected KBI!') + # await ctx. + # await trio.sleep( + # await tractor.pause() + # await spawner_ctx.cancel() + + # cancel spawned sub-actor directly? + # await sub_ctx.cancel() + + # maybe cancel runtime? + # await sub.cancel_actor() + + +async def tell_little_bro( + actor_name: str, + caller: str = '' +): + # contact target actor, do a stream dialog. + async with ( + tractor.wait_for_actor( + name=actor_name + ) as lb, + lb.open_context( + basic_echo_server, + ) as (sub_ctx, first), + sub_ctx.open_stream( + basic_echo_server, + ) as echo_ipc, + ): + actor: Actor = current_actor() + uid: tuple = actor.uid + for i in range(100): + msg: tuple = ( + uid, + i, + ) + await echo_ipc.send(msg) + resp = await echo_ipc.receive() + print( + f'{caller} => {actor_name}: {msg}\n' + f'{caller} <= {actor_name}: {resp}\n' + ) + ( + sub_uid, + _i, + ) = resp + assert sub_uid != uid + assert _i == i + + +@pytest.mark.parametrize( + 'raise_client_error', + [None, 'KeyboardInterrupt'], +) +def test_peer_spawns_and_cancels_service_subactor( + debug_mode: bool, + raise_client_error: str, +): + # NOTE: this tests for the modden `mod wks open piker` bug + # discovered as part of implementing workspace ctx + # open-.pause()-ctx.cancel() as part of the CLI.. + + # -> start actor-tree (server) that offers sub-actor spawns via + # context API + # -> start another full actor-tree (client) which requests to the first to + # spawn over its `@context` ep / api. + # -> client actor cancels the context and should exit gracefully + # and the server's spawned child should cancel and terminate! + peer_name: str = 'little_bro' + + async def main(): + async with tractor.open_nursery( + # NOTE: to halt the peer tasks on ctxc, uncomment this. + debug_mode=debug_mode, + ) as an: + server: Portal = await an.start_actor( + (server_name := 'spawn_server'), + enable_modules=[__name__], + ) + print(f'Spawned `{server_name}`') + + client: Portal = await an.start_actor( + client_name := 'client', + enable_modules=[__name__], + ) + print(f'Spawned `{client_name}`') + + try: + async with ( + server.open_context( + serve_subactors, + peer_name=peer_name, + ) as (spawn_ctx, first), + + client.open_context( + client_req_subactor, + peer_name=peer_name, + reraise_on_cancel=raise_client_error, + ) as (client_ctx, client_says), + ): + print( + f'Server says: {first}\n' + f'Client says: {client_says}\n' + ) + + # attach to client-requested-to-spawn + # (grandchild of this root actor) "little_bro" + # and ensure we can also use it as an echo + # server. + async with tractor.wait_for_actor( + name=peer_name, + ) as sub: + assert sub + + print( + 'Sub-spawn came online\n' + f'portal: {sub}\n' + f'.uid: {sub.actor.uid}\n' + f'chan.raddr: {sub.chan.raddr}\n' + ) + await tell_little_bro( + actor_name=peer_name, + caller='root', + ) + + # signal client to raise a KBI + await client_ctx.cancel() + print('root cancelled client, checking that sub-spawn is down') + + async with tractor.find_actor( + name=peer_name, + ) as sub: + assert not sub + + print('root cancelling server/client sub-actors') + + # await tractor.pause() + res = await client_ctx.result(hide_tb=False) + assert isinstance(res, ContextCancelled) + assert client_ctx.cancel_acked + assert res.canceller == current_actor().uid + + await spawn_ctx.cancel() + # await server.cancel_actor() + + # since we called `.cancel_actor()`, `.cancel_ack` + # will not be set on the ctx bc `ctx.cancel()` was not + # called directly fot this confext. + except ContextCancelled as ctxc: + print('caught ctxc from contexts!') + assert ctxc.canceller == current_actor().uid + assert ctxc is spawn_ctx.outcome + assert ctxc is spawn_ctx.maybe_error + raise + + # assert spawn_ctx.cancel_acked + assert spawn_ctx.cancel_acked + assert client_ctx.cancel_acked + + await client.cancel_actor() + await server.cancel_actor() + + # WOA WOA WOA! we need this to close..!!!?? + # that's super bad XD + + # TODO: why isn't this working!?!? + # we're now outside the `.open_context()` block so + # the internal `Context._scope: CancelScope` should be + # gracefully "closed" ;) + + # assert spawn_ctx.cancelled_caught + + trio.run(main) From d7f2f51f7f05956ed4fb5fc5921d784292d6c365 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 28 Jan 2024 16:33:10 -0500 Subject: [PATCH 098/146] Bring in pretty-ified `msgspec.Struct` extension Originally designed and used throughout `piker`, the subtype adds some handy pprinting and field diffing extras often handy when viewing struct types in logging or REPL console interfaces B) Obvi this rejigs the `tractor.msg` mod into a sub-pkg and moves the existing namespace obj-pointer stuff into a new `.msg.ptr` sub mod. --- tractor/msg/__init__.py | 26 ++++ tractor/{msg.py => msg/ptr.py} | 2 +- tractor/msg/types.py | 251 +++++++++++++++++++++++++++++++++ 3 files changed, 278 insertions(+), 1 deletion(-) create mode 100644 tractor/msg/__init__.py rename tractor/{msg.py => msg/ptr.py} (98%) create mode 100644 tractor/msg/types.py diff --git a/tractor/msg/__init__.py b/tractor/msg/__init__.py new file mode 100644 index 00000000..906627cf --- /dev/null +++ b/tractor/msg/__init__.py @@ -0,0 +1,26 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +''' +Built-in messaging patterns, types, APIs and helpers. + +''' +from .ptr import ( + NamespacePath as NamespacePath, +) +from .types import ( + Struct as Struct, +) diff --git a/tractor/msg.py b/tractor/msg/ptr.py similarity index 98% rename from tractor/msg.py rename to tractor/msg/ptr.py index d8760411..87d7bf2b 100644 --- a/tractor/msg.py +++ b/tractor/msg/ptr.py @@ -15,7 +15,7 @@ # along with this program. If not, see . ''' -Built-in messaging patterns, types, APIs and helpers. +IPC-compat cross-mem-boundary object pointer. ''' diff --git a/tractor/msg/types.py b/tractor/msg/types.py new file mode 100644 index 00000000..25e7b39b --- /dev/null +++ b/tractor/msg/types.py @@ -0,0 +1,251 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +''' +Extensions to built-in or (heavily used but 3rd party) friend-lib +types. + +''' +from __future__ import annotations +from collections import UserList +from pprint import ( + saferepr, +) +from typing import ( + Any, + Iterator, +) + +from msgspec import ( + msgpack, + Struct as _Struct, + structs, +) + + +class DiffDump(UserList): + ''' + Very simple list delegator that repr() dumps (presumed) tuple + elements of the form `tuple[str, Any, Any]` in a nice + multi-line readable form for analyzing `Struct` diffs. + + ''' + def __repr__(self) -> str: + if not len(self): + return super().__repr__() + + # format by displaying item pair's ``repr()`` on multiple, + # indented lines such that they are more easily visually + # comparable when printed to console when printed to + # console. + repstr: str = '[\n' + for k, left, right in self: + repstr += ( + f'({k},\n' + f'\t{repr(left)},\n' + f'\t{repr(right)},\n' + ')\n' + ) + repstr += ']\n' + return repstr + + +class Struct( + _Struct, + + # https://jcristharif.com/msgspec/structs.html#tagged-unions + # tag='pikerstruct', + # tag=True, +): + ''' + A "human friendlier" (aka repl buddy) struct subtype. + + ''' + def _sin_props(self) -> Iterator[ + tuple[ + structs.FieldIinfo, + str, + Any, + ] + ]: + ''' + Iterate over all non-@property fields of this struct. + + ''' + fi: structs.FieldInfo + for fi in structs.fields(self): + key: str = fi.name + val: Any = getattr(self, key) + yield fi, key, val + + def to_dict( + self, + include_non_members: bool = True, + + ) -> dict: + ''' + Like it sounds.. direct delegation to: + https://jcristharif.com/msgspec/api.html#msgspec.structs.asdict + + BUT, by default we pop all non-member (aka not defined as + struct fields) fields by default. + + ''' + asdict: dict = structs.asdict(self) + if include_non_members: + return asdict + + # only return a dict of the struct members + # which were provided as input, NOT anything + # added as type-defined `@property` methods! + sin_props: dict = {} + fi: structs.FieldInfo + for fi, k, v in self._sin_props(): + sin_props[k] = asdict[k] + + return sin_props + + def pformat( + self, + field_indent: int = 2, + indent: int = 0, + + ) -> str: + ''' + Recursion-safe `pprint.pformat()` style formatting of + a `msgspec.Struct` for sane reading by a human using a REPL. + + ''' + # global whitespace indent + ws: str = ' '*indent + + # field whitespace indent + field_ws: str = ' '*(field_indent + indent) + + # qtn: str = ws + self.__class__.__qualname__ + qtn: str = self.__class__.__qualname__ + + obj_str: str = '' # accumulator + fi: structs.FieldInfo + k: str + v: Any + for fi, k, v in self._sin_props(): + + # TODO: how can we prefer `Literal['option1', 'option2, + # ..]` over .__name__ == `Literal` but still get only the + # latter for simple types like `str | int | None` etc..? + ft: type = fi.type + typ_name: str = getattr(ft, '__name__', str(ft)) + + # recurse to get sub-struct's `.pformat()` output Bo + if isinstance(v, Struct): + val_str: str = v.pformat( + indent=field_indent + indent, + field_indent=indent + field_indent, + ) + + else: # the `pprint` recursion-safe format: + # https://docs.python.org/3.11/library/pprint.html#pprint.saferepr + val_str: str = saferepr(v) + + obj_str += (field_ws + f'{k}: {typ_name} = {val_str},\n') + + return ( + f'{qtn}(\n' + f'{obj_str}' + f'{ws})' + ) + + # TODO: use a pprint.PrettyPrinter instance around ONLY rendering + # inside a known tty? + # def __repr__(self) -> str: + # ... + + # __str__ = __repr__ = pformat + __repr__ = pformat + + def copy( + self, + update: dict | None = None, + + ) -> Struct: + ''' + Validate-typecast all self defined fields, return a copy of + us with all such fields. + + NOTE: This is kinda like the default behaviour in + `pydantic.BaseModel` except a copy of the object is + returned making it compat with `frozen=True`. + + ''' + if update: + for k, v in update.items(): + setattr(self, k, v) + + # NOTE: roundtrip serialize to validate + # - enode to msgpack binary format, + # - decode that back to a struct. + return msgpack.Decoder(type=type(self)).decode( + msgpack.Encoder().encode(self) + ) + + def typecast( + self, + + # TODO: allow only casting a named subset? + # fields: set[str] | None = None, + + ) -> None: + ''' + Cast all fields using their declared type annotations + (kinda like what `pydantic` does by default). + + NOTE: this of course won't work on frozen types, use + ``.copy()`` above in such cases. + + ''' + # https://jcristharif.com/msgspec/api.html#msgspec.structs.fields + fi: structs.FieldInfo + for fi in structs.fields(self): + setattr( + self, + fi.name, + fi.type(getattr(self, fi.name)), + ) + + def __sub__( + self, + other: Struct, + + ) -> DiffDump[tuple[str, Any, Any]]: + ''' + Compare fields/items key-wise and return a ``DiffDump`` + for easy visual REPL comparison B) + + ''' + diffs: DiffDump[tuple[str, Any, Any]] = DiffDump() + for fi in structs.fields(self): + attr_name: str = fi.name + ours: Any = getattr(self, attr_name) + theirs: Any = getattr(other, attr_name) + if ours != theirs: + diffs.append(( + attr_name, + ours, + theirs, + )) + + return diffs From 1c217ef36fa5a54e68315c428b7289362f002d23 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 29 Feb 2024 17:21:43 -0500 Subject: [PATCH 099/146] Add #TODO for generating func-sig type-annots as `str` for pprinting --- tractor/msg/types.py | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/tractor/msg/types.py b/tractor/msg/types.py index 25e7b39b..3ceff845 100644 --- a/tractor/msg/types.py +++ b/tractor/msg/types.py @@ -35,6 +35,24 @@ from msgspec import ( structs, ) +# TODO: auto-gen type sig for input func both for +# type-msgs and logging of RPC tasks? +# taken and modified from: +# https://stackoverflow.com/a/57110117 +# import inspect +# from typing import List + +# def my_function(input_1: str, input_2: int) -> list[int]: +# pass + +# def types_of(func): +# specs = inspect.getfullargspec(func) +# return_type = specs.annotations['return'] +# input_types = [t.__name__ for s, t in specs.annotations.items() if s != 'return'] +# return f'{func.__name__}({": ".join(input_types)}) -> {return_type}' + +# types_of(my_function) + class DiffDump(UserList): ''' @@ -161,6 +179,7 @@ class Struct( # https://docs.python.org/3.11/library/pprint.html#pprint.saferepr val_str: str = saferepr(v) + # TODO: LOLOL use `textwrap.indent()` instead dawwwwwg! obj_str += (field_ws + f'{k}: {typ_name} = {val_str},\n') return ( From c421f7e722567174a3bcf120eb33c8c7189fc72c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 29 Feb 2024 17:37:02 -0500 Subject: [PATCH 100/146] Make `NamespacePath` kinda support methods.. Obviously we can't deterministic-ally call `.load_ref()` (since you'd have to point to an `id()` or something and presume a particular py-runtime + virt-mem space for it to exist?) but it at least helps with the `str` formatting for logging purposes (like `._cancel_rpc_tasks()`) when `repr`-ing ctxs and their specific "rpc signatures". Maybe in the future getting this working at least for singleton types per process (like `Actor` XD ) will be a thing we can support and make some sense of.. Bo --- tractor/msg/ptr.py | 37 +++++++++++++++++++++++++++---------- 1 file changed, 27 insertions(+), 10 deletions(-) diff --git a/tractor/msg/ptr.py b/tractor/msg/ptr.py index 87d7bf2b..4d089c3e 100644 --- a/tractor/msg/ptr.py +++ b/tractor/msg/ptr.py @@ -43,17 +43,24 @@ IPC-compat cross-mem-boundary object pointer. # - https://github.com/msgpack/msgpack-python#packingunpacking-of-custom-data-type from __future__ import annotations -from inspect import isfunction +from inspect import ( + isfunction, + ismethod, +) from pkgutil import resolve_name class NamespacePath(str): ''' - A serializeable description of a (function) Python object - location described by the target's module path and namespace - key meant as a message-native "packet" to allows actors to - point-and-load objects by an absolute ``str`` (and thus - serializable) reference. + A serializeable `str`-subtype implementing a "namespace + pointer" to any Python object reference (like a function) + using the same format as the built-in `pkgutil.resolve_name()` + system. + + A value describes a target's module-path and namespace-key + separated by a ':' and thus can be easily used as + a IPC-message-native reference-type allowing memory isolated + actors to point-and-load objects via a minimal `str` value. ''' _ref: object | type | None = None @@ -81,13 +88,23 @@ class NamespacePath(str): ''' if ( - isinstance(ref, object) - and not isfunction(ref) + isfunction(ref) ): - name: str = type(ref).__name__ - else: name: str = getattr(ref, '__name__') + elif ismethod(ref): + # build out the path manually i guess..? + # TODO: better way? + name: str = '.'.join([ + type(ref.__self__).__name__, + ref.__func__.__name__, + ]) + + else: # object or other? + # isinstance(ref, object) + # and not isfunction(ref) + name: str = type(ref).__name__ + # fully qualified namespace path, tuple. fqnp: tuple[str, str] = ( ref.__module__, From 2cc712cd811ceae265d12134a0e8973ac027eb39 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 29 Feb 2024 18:20:41 -0500 Subject: [PATCH 101/146] Fix `Channel.__repr__()` safety, renames to `._transport` Hit a reallly weird bug in the `._runtime` IPC msg handling loop where it seems that by `str.format()`-ing a `Channel` before initializing it would put the `._MsgTransport._agen()` in an already started state causing an irrecoverable core startup failure.. I presume it's something to do with delegating to the `MsgpackTCPStream.__repr__()` and, something something.. the `.set_msg_transport(stream)` getting called to too early such that `.msgstream.__init__()` is called thus init-ing the `._agen()` before necessary? I'm sure there's a design lesson to be learned in here somewhere XD This was discovered while trying to add more "fancy" logging throughout said core for the purposes of cobbling together an init attempt at libp2p style multi-address representations for our IPC primitives. Thus I also tinker here with adding some new fields to `MsgpackTCPStream`: - `layer_key`: int = 4 - `name_key`: str = 'tcp' - `codec_key`: str = 'msgpack' Anyway, just changed it so that if `.msgstream` ain't set then we just return a little "null repr" `str` value thinger. Also renames `Channel.msgstream` internally to `._transport` with appropriate pub `@property`s added such that everything else won't break ;p Also drops `Optional` typing vis-a-vi modern union syntax B) --- tractor/_ipc.py | 93 +++++++++++++++++++++++++++++-------------------- 1 file changed, 56 insertions(+), 37 deletions(-) diff --git a/tractor/_ipc.py b/tractor/_ipc.py index e5e4d223..33110c04 100644 --- a/tractor/_ipc.py +++ b/tractor/_ipc.py @@ -30,7 +30,6 @@ import typing from typing import ( Any, runtime_checkable, - Optional, Protocol, Type, TypeVar, @@ -113,6 +112,13 @@ class MsgpackTCPStream(MsgTransport): using the ``msgspec`` codec lib. ''' + layer_key: int = 4 + name_key: str = 'tcp' + + # TODO: better naming for this? + # -[ ] check how libp2p does naming for such things? + codec_key: str = 'msgpack' + def __init__( self, stream: trio.SocketStream, @@ -268,7 +274,7 @@ class Channel: def __init__( self, - destaddr: Optional[tuple[str, int]], + destaddr: tuple[str, int]|None, msg_transport_type_key: tuple[str, str] = ('msgpack', 'tcp'), @@ -286,14 +292,14 @@ class Channel: # Either created in ``.connect()`` or passed in by # user in ``.from_stream()``. - self._stream: Optional[trio.SocketStream] = None - self.msgstream: Optional[MsgTransport] = None + self._stream: trio.SocketStream|None = None + self._transport: MsgTransport|None = None # set after handshake - always uid of far end - self.uid: Optional[tuple[str, str]] = None + self.uid: tuple[str, str]|None = None self._agen = self._aiter_recv() - self._exc: Optional[Exception] = None # set if far end actor errors + self._exc: Exception|None = None # set if far end actor errors self._closed: bool = False # flag set by ``Portal.cancel_actor()`` indicating remote @@ -301,6 +307,15 @@ class Channel: # runtime. self._cancel_called: bool = False + @property + def msgstream(self) -> MsgTransport: + log.info('`Channel.msgstream` is an old name, use `._transport`') + return self._transport + + @property + def transport(self) -> MsgTransport: + return self._transport + @classmethod def from_stream( cls, @@ -310,40 +325,44 @@ class Channel: ) -> Channel: src, dst = get_stream_addrs(stream) - chan = Channel(destaddr=dst, **kwargs) + chan = Channel( + destaddr=dst, + **kwargs, + ) # set immediately here from provided instance - chan._stream = stream + chan._stream: trio.SocketStream = stream chan.set_msg_transport(stream) return chan def set_msg_transport( self, stream: trio.SocketStream, - type_key: Optional[tuple[str, str]] = None, + type_key: tuple[str, str]|None = None, ) -> MsgTransport: type_key = type_key or self._transport_key - self.msgstream = get_msg_transport(type_key)(stream) - return self.msgstream + self._transport = get_msg_transport(type_key)(stream) + return self._transport def __repr__(self) -> str: - if self.msgstream: - return repr( - self.msgstream.stream.socket._sock - ).replace( # type: ignore - "socket.socket", - "Channel", - ) - return object.__repr__(self) + if not self._transport: + return '' + + return repr( + self._transport.stream.socket._sock + ).replace( # type: ignore + "socket.socket", + "Channel", + ) @property - def laddr(self) -> Optional[tuple[str, int]]: - return self.msgstream.laddr if self.msgstream else None + def laddr(self) -> tuple[str, int]|None: + return self._transport.laddr if self._transport else None @property - def raddr(self) -> Optional[tuple[str, int]]: - return self.msgstream.raddr if self.msgstream else None + def raddr(self) -> tuple[str, int]|None: + return self._transport.raddr if self._transport else None async def connect( self, @@ -362,12 +381,12 @@ class Channel: *destaddr, **kwargs ) - msgstream = self.set_msg_transport(stream) + transport = self.set_msg_transport(stream) log.transport( - f'Opened channel[{type(msgstream)}]: {self.laddr} -> {self.raddr}' + f'Opened channel[{type(transport)}]: {self.laddr} -> {self.raddr}' ) - return msgstream + return transport async def send(self, item: Any) -> None: @@ -375,16 +394,16 @@ class Channel: '=> send IPC msg:\n\n' f'{pformat(item)}\n' ) # type: ignore - assert self.msgstream + assert self._transport - await self.msgstream.send(item) + await self._transport.send(item) async def recv(self) -> Any: - assert self.msgstream - return await self.msgstream.recv() + assert self._transport + return await self._transport.recv() # try: - # return await self.msgstream.recv() + # return await self._transport.recv() # except trio.BrokenResourceError: # if self._autorecon: # await self._reconnect() @@ -397,8 +416,8 @@ class Channel: f'Closing channel to {self.uid} ' f'{self.laddr} -> {self.raddr}' ) - assert self.msgstream - await self.msgstream.stream.aclose() + assert self._transport + await self._transport.stream.aclose() self._closed = True async def __aenter__(self): @@ -449,16 +468,16 @@ class Channel: Async iterate items from underlying stream. ''' - assert self.msgstream + assert self._transport while True: try: - async for item in self.msgstream: + async for item in self._transport: yield item # sent = yield item # if sent is not None: # # optimization, passing None through all the # # time is pointless - # await self.msgstream.send(sent) + # await self._transport.send(sent) except trio.BrokenResourceError: # if not self._autorecon: @@ -471,7 +490,7 @@ class Channel: # continue def connected(self) -> bool: - return self.msgstream.connected() if self.msgstream else False + return self._transport.connected() if self._transport else False @asynccontextmanager From 7bb44e69307da810927242ebf33b92a9f9f246d2 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 29 Feb 2024 18:56:31 -0500 Subject: [PATCH 102/146] Add "fancier" remote-error `.__repr__()`-ing Our remote error box types `RemoteActorError`, `ContextCancelled` and `StreamOverrun` needed a console display makeover particularly for logging content and `repr()` in higher level primitives like `Context`. This adds a more "dramatic" str-representation to showcase the underlying boxed traceback content more sensationally (via ascii-art emphasis) as well as support a more terse `.reprol()` (representation for one-line) format that can be used for types that track remote errors/cancels like with `Context._remote_error`. Impl deats: - change `RemoteActorError.__repr__()` formatting to show (sub-type specific) `.msgdata` fields in a multi-line format (similar to our new `.msg.types.Struct` style) followed by some ascii accented delimiter lines to emphasize any `.msgdata["tb_str"]` packed by the remote - for rme and subtypes allow picking the specifically relevant fields via a type defined `.reprol_fields: list[str]` and pick for each subtype: |_ `RemoteActorError.src_actor_uid` |_ `ContextCancelled.canceller` |_ `StreamOverrun.sender` - add `.reprol()` to show a `repr()`-on-one-line formatted string that can be used by other multi-line-field-`repr()` styled composite types as needed in (high level) logging info. - toss in some mod level `_body_fields: list[str]` for summary of such fields (if needed). - add some new rae (remote-actor-error) props: - `.type` around a newly named `.boxed_type` - `.type_str: str` - `.tb_str: str` --- tractor/_exceptions.py | 134 +++++++++++++++++++++++++++++++++++------ 1 file changed, 116 insertions(+), 18 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index fe3f2706..259a28a7 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -27,6 +27,7 @@ from typing import ( Type, TYPE_CHECKING, ) +import textwrap import traceback import exceptiongroup as eg @@ -37,8 +38,9 @@ from .log import get_logger if TYPE_CHECKING: from ._context import Context - from ._stream import MsgStream from .log import StackLevelAdapter + from ._stream import MsgStream + from ._ipc import Channel log = get_logger('tractor') @@ -49,6 +51,25 @@ class ActorFailure(Exception): "General actor failure" +class InternalError(RuntimeError): + ''' + Entirely unexpected internal machinery error indicating + a completely invalid state or interface. + + ''' + +_body_fields: list[str] = [ + 'src_actor_uid', + 'canceller', + 'sender', +] + +_msgdata_keys: list[str] = [ + 'type_str', +] + _body_fields + + + # TODO: rename to just `RemoteError`? class RemoteActorError(Exception): ''' @@ -60,6 +81,10 @@ class RemoteActorError(Exception): a special "error" IPC msg sent by some remote actor-runtime. ''' + reprol_fields: list[str] = [ + 'src_actor_uid', + ] + def __init__( self, message: str, @@ -77,23 +102,82 @@ class RemoteActorError(Exception): # - .remote_type # also pertains to our long long oustanding issue XD # https://github.com/goodboy/tractor/issues/5 - self.type: str = suberror_type + self.boxed_type: str = suberror_type self.msgdata: dict[str, Any] = msgdata @property - def src_actor_uid(self) -> tuple[str, str] | None: + def type(self) -> str: + return self.boxed_type + + @property + def type_str(self) -> str: + return str(type(self.boxed_type).__name__) + + @property + def src_actor_uid(self) -> tuple[str, str]|None: return self.msgdata.get('src_actor_uid') - def __repr__(self) -> str: + @property + def tb_str( + self, + indent: str = ' '*3, + ) -> str: if remote_tb := self.msgdata.get('tb_str'): - pformat(remote_tb) - return ( - f'{type(self).__name__}(\n' - f'msgdata={pformat(self.msgdata)}\n' - ')' + return textwrap.indent( + remote_tb, + prefix=indent, ) - return super().__repr__() + return '' + + def reprol(self) -> str: + ''' + Represent this error for "one line" display, like in + a field of our `Context.__repr__()` output. + + ''' + _repr: str = f'{type(self).__name__}(' + for key in self.reprol_fields: + val: Any|None = self.msgdata.get(key) + if val: + _repr += f'{key}={repr(val)} ' + + return _repr + + def __repr__(self) -> str: + + fields: str = '' + for key in _body_fields: + val: str|None = self.msgdata.get(key) + if val: + fields += f'{key}={val}\n' + + fields: str = textwrap.indent( + fields, + # prefix=' '*2, + prefix=' |_', + ) + indent: str = ''*1 + body: str = ( + f'{fields}' + f' |\n' + f' ------ - ------\n\n' + f'{self.tb_str}\n' + f' ------ - ------\n' + f' _|\n' + ) + # f'|\n' + # f' |\n' + if indent: + body: str = textwrap.indent( + body, + prefix=indent, + ) + return ( + f'<{type(self).__name__}(\n' + f'{body}' + ')>' + ) # TODO: local recontruction of remote exception deats # def unbox(self) -> BaseException: @@ -102,8 +186,9 @@ class RemoteActorError(Exception): class InternalActorError(RemoteActorError): ''' - Remote internal ``tractor`` error indicating - failure of some primitive or machinery. + (Remote) internal `tractor` error indicating failure of some + primitive, machinery state or lowlevel task that should never + occur. ''' @@ -114,6 +199,9 @@ class ContextCancelled(RemoteActorError): ``Portal.cancel_actor()`` or ``Context.cancel()``. ''' + reprol_fields: list[str] = [ + 'canceller', + ] @property def canceller(self) -> tuple[str, str]|None: ''' @@ -145,6 +233,9 @@ class ContextCancelled(RemoteActorError): f'{self}' ) + # to make `.__repr__()` work uniformly + # src_actor_uid = canceller + class TransportClosed(trio.ClosedResourceError): "Underlying channel transport was closed prior to use" @@ -166,6 +257,9 @@ class StreamOverrun( RemoteActorError, trio.TooSlowError, ): + reprol_fields: list[str] = [ + 'sender', + ] ''' This stream was overrun by sender @@ -213,6 +307,7 @@ def pack_error( ] = { 'tb_str': tb_str, 'type_str': type(exc).__name__, + 'boxed_type': type(exc).__name__, 'src_actor_uid': current_actor().uid, } @@ -238,8 +333,8 @@ def unpack_error( msg: dict[str, Any], - chan=None, - err_type=RemoteActorError, + chan: Channel|None = None, + box_type: RemoteActorError = RemoteActorError, hide_tb: bool = True, @@ -264,12 +359,15 @@ def unpack_error( # retrieve the remote error's msg encoded details tb_str: str = error_dict.get('tb_str', '') message: str = f'{chan.uid}\n' + tb_str - type_name: str = error_dict['type_str'] + type_name: str = ( + error_dict.get('type_str') + or error_dict['boxed_type'] + ) suberror_type: Type[BaseException] = Exception if type_name == 'ContextCancelled': - err_type = ContextCancelled - suberror_type = err_type + box_type = ContextCancelled + suberror_type = box_type else: # try to lookup a suitable local error type for ns in [ @@ -285,7 +383,7 @@ def unpack_error( ): break - exc = err_type( + exc = box_type( message, suberror_type=suberror_type, From 04aea5c4db64bd5431aee407a2c1dcec8930651e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 29 Feb 2024 20:01:39 -0500 Subject: [PATCH 103/146] Add field-first subproca `.info()` to `._entry` --- tractor/_entry.py | 28 +++++++++++++++------------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/tractor/_entry.py b/tractor/_entry.py index e8fb56db..23dd05b6 100644 --- a/tractor/_entry.py +++ b/tractor/_entry.py @@ -106,25 +106,25 @@ def _trio_main( Entry point for a `trio_run_in_process` subactor. ''' - log.info(f"Started new trio process for {actor.uid}") - - if actor.loglevel is not None: - log.info( - f"Setting loglevel for {actor.uid} to {actor.loglevel}") - get_console_log(actor.loglevel) - - log.info( - f"Started {actor.uid}") - _state._current_actor = actor - - log.debug(f"parent_addr is {parent_addr}") trio_main = partial( async_main, actor, parent_addr=parent_addr ) + if actor.loglevel is not None: + get_console_log(actor.loglevel) + import os + log.info( + 'Started new trio process:\n' + f'|_{actor}\n' + f' uid: {actor.uid}\n' + f' pid: {os.getpid()}\n' + f' parent_addr: {parent_addr}\n' + f' loglevel: {actor.loglevel}\n' + ) + try: if infect_asyncio: actor._infected_aio = True @@ -132,7 +132,9 @@ def _trio_main( else: trio.run(trio_main) except KeyboardInterrupt: - log.cancel(f"Actor {actor.uid} received KBI") + log.cancel( + f'@{actor.uid} received KBI' + ) finally: log.info(f"Actor {actor.uid} terminated") From 1bc858cd004d42a38148f994dd96347b3fe10044 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 1 Mar 2024 10:47:42 -0500 Subject: [PATCH 104/146] Spawn naming and log format tweaks - rename `.soft_wait()` -> `.soft_kill()` - rename `.do_hard_kill()` -> `.hard_kill()` - adjust any `trio.Process.__repr__()` log msg contents to have the little tree branch prefix: `'|_'` --- tractor/_spawn.py | 51 +++++++++++++++++++++++++++-------------------- 1 file changed, 29 insertions(+), 22 deletions(-) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index 7ce750be..58c187f0 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -196,16 +196,16 @@ async def cancel_on_completion( result: Any|Exception = await exhaust_portal(portal, actor) if isinstance(result, Exception): errors[actor.uid]: Exception = result - log.warning( - 'Cancelling subactor due to error:\n' - f'uid: {portal.channel.uid}\n' + log.cancel( + 'Cancelling subactor runtime due to error:\n\n' + f'Portal.cancel_actor() => {portal.channel.uid}\n\n' f'error: {result}\n' ) else: log.runtime( - 'Cancelling subactor gracefully:\n' - f'uid: {portal.channel.uid}\n' + 'Cancelling subactor gracefully:\n\n' + f'Portal.cancel_actor() => {portal.channel.uid}\n\n' f'result: {result}\n' ) @@ -213,7 +213,7 @@ async def cancel_on_completion( await portal.cancel_actor() -async def do_hard_kill( +async def hard_kill( proc: trio.Process, terminate_after: int = 3, @@ -288,7 +288,7 @@ async def do_hard_kill( proc.kill() -async def soft_wait( +async def soft_kill( proc: ProcessType, wait_func: Callable[ @@ -299,17 +299,20 @@ async def soft_wait( ) -> None: ''' - Wait for proc termination but **dont' yet** teardown - std-streams (since it will clobber any ongoing pdb REPL - session). This is our "soft" (and thus itself cancellable) - join/reap on an actor-runtime-in-process. + Wait for proc termination but **don't yet** teardown + std-streams since it will clobber any ongoing pdb REPL + session. + + This is our "soft"/graceful, and thus itself also cancellable, + join/reap on an actor-runtime-in-process shutdown; it is + **not** the same as a "hard kill" via an OS signal (for that + see `.hard_kill()`). ''' uid: tuple[str, str] = portal.channel.uid try: log.cancel( - 'Soft waiting on sub-actor proc:\n' - f'uid: {uid}\n' + 'Soft killing sub-actor via `Portal.cancel_actor()`\n' f'|_{proc}\n' ) # wait on sub-proc to signal termination @@ -326,8 +329,9 @@ async def soft_wait( async def cancel_on_proc_deth(): ''' - "Cancel the (actor) cancel" request if we detect - that that the underlying sub-process terminated. + "Cancel-the-cancel" request: if we detect that the + underlying sub-process exited prior to + a `Portal.cancel_actor()` call completing . ''' await wait_func(proc) @@ -439,19 +443,22 @@ async def trio_proc( spawn_cmd.append("--asyncio") cancelled_during_spawn: bool = False - proc: trio.Process | None = None + proc: trio.Process|None = None try: try: # TODO: needs ``trio_typing`` patch? proc = await trio.lowlevel.open_process(spawn_cmd) - - log.runtime(f"Started {proc}") + log.runtime( + 'Started new sub-proc\n' + f'|_{proc}\n' + ) # wait for actor to spawn and connect back to us # channel should have handshake completed by the # local actor by the time we get a ref to it event, chan = await actor_nursery._actor.wait_for_peer( - subactor.uid) + subactor.uid + ) except trio.Cancelled: cancelled_during_spawn = True @@ -513,7 +520,7 @@ async def trio_proc( # This is a "soft" (cancellable) join/reap which # will remote cancel the actor on a ``trio.Cancelled`` # condition. - await soft_wait( + await soft_kill( proc, trio.Process.wait, portal @@ -574,7 +581,7 @@ async def trio_proc( if proc.poll() is None: log.cancel(f"Attempting to hard kill {proc}") - await do_hard_kill(proc) + await hard_kill(proc) log.debug(f"Joined {proc}") else: @@ -718,7 +725,7 @@ async def mp_proc( # This is a "soft" (cancellable) join/reap which # will remote cancel the actor on a ``trio.Cancelled`` # condition. - await soft_wait( + await soft_kill( proc, proc_waiter, portal From fb55784798ea391b45e315ef8e2a20a5be9fd2b8 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 1 Mar 2024 15:44:01 -0500 Subject: [PATCH 105/146] Add `._implicit_runtime_started` mark, better logs After some deep logging improvements to many parts of `._runtime`, I realized a silly detail where we are always waiting on any opened `local_nursery: ActorNursery` to signal exit from `Actor._stream_handler()` even in the case of being an implicitly opened root actor (`open_root_actor()` wasn't called by user/app code) via `._supervise.open_nursery()`.. So, to address this add a `ActorNursery._implicit_runtime_started: bool` that can be set and then checked to avoid doing the unnecessary `.exited.wait()` (and any subsequent warn logging on an exit timeout) in that special but most common case XD Matching with other subsys log format refinements, improve readability and simplicity of the actor-nursery supervisory log msgs, including: - simplify and/or remove any content that more or less duplicates msg content found in emissions from lower-level primitives and sub-systems (like `._runtime`, `_context`, `_portal` etc.). - add a specific `._open_and_supervise_one_cancels_all_nursery()` handler block for `ContextCancelled` to log with `.cancel()` level noting that the case is a "remote cancellation". - put the nursery-exit and actor-tree shutdown status into a single msg in the `implicit_runtime` case. --- tractor/_supervise.py | 96 ++++++++++++++++++++++++++++++++----------- 1 file changed, 73 insertions(+), 23 deletions(-) diff --git a/tractor/_supervise.py b/tractor/_supervise.py index 9d927b19..a93c3ebc 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -37,7 +37,10 @@ from ._state import current_actor, is_main_process from .log import get_logger, get_loglevel from ._runtime import Actor from ._portal import Portal -from ._exceptions import is_multi_cancelled +from ._exceptions import ( + is_multi_cancelled, + ContextCancelled, +) from ._root import open_root_actor from . import _state from . import _spawn @@ -107,6 +110,14 @@ class ActorNursery: self.errors = errors self.exited = trio.Event() + # NOTE: when no explicit call is made to + # `.open_root_actor()` by application code, + # `.open_nursery()` will implicitly call it to start the + # actor-tree runtime. In this case we mark ourselves as + # such so that runtime components can be aware for logging + # and syncing purposes to any actor opened nurseries. + self._implicit_runtime_started: bool = False + async def start_actor( self, name: str, @@ -250,10 +261,11 @@ class ActorNursery: ''' self.cancelled = True - log.cancel( - 'Cancelling actor nursery\n' - f'|_{self._children}\n' - ) + # TODO: impl a repr for spawn more compact + # then `._children`.. + children: dict = self._children + child_count: int = len(children) + msg: str = f'Cancelling actor nursery with {child_count} children\n' with trio.move_on_after(3) as cs: async with trio.open_nursery() as tn: @@ -264,7 +276,7 @@ class ActorNursery: subactor, proc, portal, - ) in self._children.values(): + ) in children.values(): # TODO: are we ever even going to use this or # is the spawning backend responsible for such @@ -276,12 +288,13 @@ class ActorNursery: if portal is None: # actor hasn't fully spawned yet event = self._actor._peer_connected[subactor.uid] log.warning( - f"{subactor.uid} wasn't finished spawning?") + f"{subactor.uid} never 't finished spawning?" + ) await event.wait() # channel/portal should now be up - _, _, portal = self._children[subactor.uid] + _, _, portal = children[subactor.uid] # XXX should be impossible to get here # unless method was called from within @@ -300,11 +313,13 @@ class ActorNursery: if portal.channel.connected(): tn.start_soon(portal.cancel_actor) + log.cancel(msg) # if we cancelled the cancel (we hung cancelling remote actors) # then hard kill all sub-processes if cs.cancelled_caught: log.error( - f'Failed to cancel {self}\nHard killing process tree!' + f'Failed to cancel {self}?\n' + 'Hard killing underlying subprocess tree!\n' ) subactor: Actor proc: trio.Process @@ -313,7 +328,7 @@ class ActorNursery: subactor, proc, portal, - ) in self._children.values(): + ) in children.values(): log.warning(f"Hard killing process {proc}") proc.terminate() @@ -391,26 +406,39 @@ async def _open_and_supervise_one_cancels_all_nursery( # worry more are coming). an._join_procs.set() - # XXX: hypothetically an error could be - # raised and then a cancel signal shows up + # XXX NOTE XXX: hypothetically an error could + # be raised and then a cancel signal shows up # slightly after in which case the `else:` # block here might not complete? For now, # shield both. with trio.CancelScope(shield=True): - etype = type(inner_err) + etype: type = type(inner_err) if etype in ( trio.Cancelled, - KeyboardInterrupt + KeyboardInterrupt, ) or ( is_multi_cancelled(inner_err) ): log.cancel( - f"Nursery for {current_actor().uid} " - f"was cancelled with {etype}") + f'Actor-nursery cancelled by {etype}\n\n' + + f'{current_actor().uid}\n' + f' |_{an}\n\n' + + # TODO: show tb str? + # f'{tb_str}' + ) + elif etype in { + ContextCancelled, + }: + log.cancel( + 'Actor-nursery caught remote cancellation\n\n' + + f'{inner_err.tb_str}' + ) else: log.exception( - f"Nursery for {current_actor().uid} " - "errored with:" + 'Nursery errored with:\n' # TODO: same thing as in # `._invoke()` to compute how to @@ -451,11 +479,15 @@ async def _open_and_supervise_one_cancels_all_nursery( # ".run_in_actor()" actors then we also want to cancel all # remaining sub-actors (due to our lone strategy: # one-cancels-all). - log.cancel(f"Nursery cancelling due to {err}") if an._children: + log.cancel( + 'Actor-nursery cancelling due error type:\n' + f'{err}\n' + ) with trio.CancelScope(shield=True): await an.cancel() raise + finally: # No errors were raised while awaiting ".run_in_actor()" # actors but those actors may have returned remote errors as @@ -501,7 +533,7 @@ async def open_nursery( which cancellation scopes correspond to each spawned subactor set. ''' - implicit_runtime = False + implicit_runtime: bool = False actor = current_actor(err_on_no_runtime=False) @@ -513,7 +545,7 @@ async def open_nursery( log.info("Starting actor runtime!") # mark us for teardown on exit - implicit_runtime = True + implicit_runtime: bool = True async with open_root_actor(**kwargs) as actor: assert actor is current_actor() @@ -522,8 +554,21 @@ async def open_nursery( async with _open_and_supervise_one_cancels_all_nursery( actor ) as an: + + # NOTE: mark this nursery as having + # implicitly started the root actor so + # that `._runtime` machinery can avoid + # certain teardown synchronization + # blocking/waits and any associated (warn) + # logging when it's known that this + # nursery shouldn't be exited before the + # root actor is. + an._implicit_runtime_started = True yield an finally: + # XXX: this event will be set after the root actor + # runtime is already torn down, so we want to + # avoid any blocking on it. an.exited.set() else: # sub-nursery case @@ -537,8 +582,13 @@ async def open_nursery( an.exited.set() finally: - log.debug("Nursery teardown complete") + msg: str = ( + 'Actor-nursery exited\n' + f'|_{an}\n\n' + ) # shutdown runtime if it was started if implicit_runtime: - log.info("Shutting down actor tree") + msg += '=> Shutting down actor runtime <=\n' + + log.info(msg) From d8d206b93f164fe81b7848d0367d3c878faf6f7e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 1 Mar 2024 19:27:10 -0500 Subject: [PATCH 106/146] Make stream draining status logs `.debug()` level --- tractor/_root.py | 8 ++++++-- tractor/_streaming.py | 2 +- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/tractor/_root.py b/tractor/_root.py index c17249d1..88df774f 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -25,6 +25,7 @@ import logging import signal import sys import os +import typing import warnings @@ -236,8 +237,11 @@ async def open_root_actor( entered = await _debug._maybe_enter_pm(err) - if not entered and not is_multi_cancelled(err): - logger.exception("Root actor crashed:") + if ( + not entered + and not is_multi_cancelled(err) + ): + logger.exception('Root actor crashed:\n') # always re-raise raise diff --git a/tractor/_streaming.py b/tractor/_streaming.py index 64b5dd6d..b2cfe485 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -265,7 +265,7 @@ class MsgStream(trio.abc.Channel): try: maybe_final_msg = self.receive_nowait() if maybe_final_msg: - log.cancel( + log.debug( 'Drained un-processed stream msg:\n' f'{pformat(maybe_final_msg)}' ) From 910c07db06b4d3bd8754c62a8e3822efb1b7491e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 1 Mar 2024 22:37:32 -0500 Subject: [PATCH 107/146] Deep `Context` refinements Spanning from the pub API, to instance `repr()` customization (for logging/REPL content), to the impl details around the notion of a "final outcome" and surrounding IPC msg draining mechanics during teardown. A few API and field updates: - new `.cancel_acked: bool` to replace what we were mostly using `.cancelled_caught: bool` for but, for purposes of better mapping the semantics of remote cancellation of parallel executing tasks; it's set only when `.cancel_called` is set and a ctxc arrives with a `.canceller` field set to the current actor uid indicating we requested and received acknowledgement from the other side's task that is cancelled gracefully. - strongly document and delegate (and prolly eventually remove as a pub attr) the `.cancelled_caught` property entirely to the underlying `._scope: trio.CancelScope`; the `trio` semantics don't really map well to the "parallel with IPC msging" case in the sense that for us it breaks the concept of the ctx/scope closure having "caught" something instead of having "received" a msg that the other side has "acknowledged" (i.e. which for us is the completion of cancellation). - new `.__repr__()`/`.__str__()` format that tries to tersely yet comprehensively as possible display everything you need to know about the 3 main layers of an SC-linked-IPC-context: * ipc: the transport + runtime layers net-addressing and prot info. * rpc: the specific linked caller-callee task signature details including task and msg-stream instances. * state: current execution and final outcome state of the task pair. * a teensie extra `.repr_rpc` for a condensed rpc signature. - new `.dst_maddr` to get a `libp2p` style "multi-address" (though right now it's just showing the transport layers so maybe we should move to to our `Channel`?) - new public instance-var fields supporting more granular remote cancellation/result/error state: * `.maybe_error: Exception|None` for any final (remote) error/ctxc which computes logic on the values of `._remote_error`/`._local_error` to determine the "final error" (if any) on termination. * `.outcome` to the final error or result (or `None` if un-terminated) * `.repr_outcome()` for a console/logging friendly version of the final result or error as needed for the `.__str__()`. - new private interface bits to support all of ^: * a new "no result yet" sentinel value, `Unresolved`, using a module level class singleton that `._result` is set too (instead of `id(self)`) to both determine if and present when no final result from the callee has-yet-been/was delivered (ever). => really we should get rid of `.result()` and change it to `.wait_for_result()` (or something)u * `_final_result_is_set()` predicate to avoid waiting for an already delivered result. * `._maybe_raise()` proto-impl that we should use to replace all the `if re:` blocks it can XD * new `._stream: MsgStream|None` for when a stream is opened to aid with the state repr mentioned above. Tweaks to the termination drain loop `_drain_to_final_msg()`: - obviously (obvi) use all the changes above when determining whether or not a "final outcome" has arrived and thus breaking from the loop ;) * like the `.outcome` `.maybe_error` and `._final_ctx_is_set()` in the `while` pred expression. - drop the `_recv_chan.receive_nowait()` + guard logic since it seems with all the surrounding (and coming soon) changes to `Portal.open_context()` using all the new API stuff (mentioned in first bullet set above) we never hit the case of inf-block? Oh right and obviously a ton of (hopefully improved) logging msg content changes, commented code removal and detailed comment-docs strewn about! --- tractor/_context.py | 1048 ++++++++++++++++++++++++++++--------------- 1 file changed, 675 insertions(+), 373 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index ee05a2ba..f8aaf1c9 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -43,19 +43,14 @@ import warnings import trio -# from .devx import ( -# maybe_wait_for_debugger, -# pause, -# ) from .msg import NamespacePath from ._exceptions import ( - # _raise_from_no_key_in_msg, - unpack_error, - pack_error, ContextCancelled, - # MessagingError, + InternalError, RemoteActorError, StreamOverrun, + pack_error, + unpack_error, ) from .log import get_logger from ._ipc import Channel @@ -65,6 +60,7 @@ from ._state import current_actor if TYPE_CHECKING: from ._portal import Portal from ._runtime import Actor + from ._ipc import MsgTransport log = get_logger(__name__) @@ -73,6 +69,7 @@ log = get_logger(__name__) async def _drain_to_final_msg( ctx: Context, + hide_tb: bool = True, msg_limit: int = 6, ) -> list[dict]: @@ -89,47 +86,72 @@ async def _drain_to_final_msg( `ctx.result()` cleanup and teardown sequence. ''' + __tracebackhide__: bool = hide_tb raise_overrun: bool = not ctx._allow_overruns # wait for a final context result by collecting (but # basically ignoring) any bi-dir-stream msgs still in transit # from the far end. pre_result_drained: list[dict] = [] - while not ctx._remote_error: + while not ( + ctx.maybe_error + and not ctx._final_result_is_set() + ): try: + # TODO: can remove? + # await trio.lowlevel.checkpoint() + # NOTE: this REPL usage actually works here dawg! Bo # from .devx._debug import pause # await pause() - if re := ctx._remote_error: - ctx._maybe_raise_remote_err( - re, - # NOTE: obvi we don't care if we - # overran the far end if we're already - # waiting on a final result (msg). - raise_overrun_from_self=raise_overrun, - ) # TODO: bad idea? + # -[ ] wrap final outcome channel wait in a scope so + # it can be cancelled out of band if needed? + # # with trio.CancelScope() as res_cs: # ctx._res_scope = res_cs # msg: dict = await ctx._recv_chan.receive() # if res_cs.cancelled_caught: + # TODO: ensure there's no more hangs, debugging the + # runtime pretty preaase! # from .devx._debug import pause # await pause() + + # TODO: can remove this finally? + # we have no more need for the sync draining right + # since we're can kinda guarantee the async + # `.receive()` below will never block yah? + # + # if ( + # ctx._cancel_called and ( + # ctx.cancel_acked + # # or ctx.chan._cancel_called + # ) + # # or not ctx._final_result_is_set() + # # ctx.outcome is not + # # or ctx.chan._closed + # ): + # try: + # msg: dict = await ctx._recv_chan.receive_nowait()() + # except trio.WouldBlock: + # log.warning( + # 'When draining already `.cancel_called` ctx!\n' + # 'No final msg arrived..\n' + # ) + # break + # else: + # msg: dict = await ctx._recv_chan.receive() + + # TODO: don't need it right jefe? + # with trio.move_on_after(1) as cs: + # if cs.cancelled_caught: + # from .devx._debug import pause + # await pause() + + # pray to the `trio` gawds that we're corrent with this msg: dict = await ctx._recv_chan.receive() - ctx._result: Any = msg['return'] - log.runtime( - 'Context delivered final draining msg:\n' - f'{pformat(msg)}' - ) - pre_result_drained.append(msg) - # NOTE: we don't need to do this right? - # XXX: only close the rx mem chan AFTER - # a final result is retreived. - # if ctx._recv_chan: - # await ctx._recv_chan.aclose() - break # NOTE: we get here if the far end was # `ContextCancelled` in 2 cases: @@ -150,7 +172,22 @@ async def _drain_to_final_msg( # continue to bubble up as normal. raise + try: + ctx._result: Any = msg['return'] + log.runtime( + 'Context delivered final draining msg:\n' + f'{pformat(msg)}' + ) + # XXX: only close the rx mem chan AFTER + # a final result is retreived. + # if ctx._recv_chan: + # await ctx._recv_chan.aclose() + # TODO: ^ we don't need it right? + break + except KeyError: + # always capture unexpected/non-result msgs + pre_result_drained.append(msg) if 'yield' in msg: # far end task is still streaming to us so discard @@ -159,12 +196,12 @@ async def _drain_to_final_msg( (ctx._stream.closed and (reason := 'stream was already closed') ) - or (ctx._cancel_called - and (reason := 'ctx called `.cancel()`') - ) or (ctx._cancelled_caught and (reason := 'ctx caught a cancel') ) + or (ctx._cancel_called + and (reason := 'ctx called `.cancel()`') + ) or (len(pre_result_drained) > msg_limit and (reason := f'"yield" limit={msg_limit}') ) @@ -193,7 +230,6 @@ async def _drain_to_final_msg( f'{pformat(msg)}\n' ) - pre_result_drained.append(msg) continue # TODO: work out edge cases here where @@ -206,13 +242,15 @@ async def _drain_to_final_msg( 'Remote stream terminated due to "stop" msg:\n\n' f'{pformat(msg)}\n' ) - pre_result_drained.append(msg) continue - # internal error should never get here - assert msg.get('cid'), ( - "Received internal error at portal?" - ) + # It's an internal error if any other msg type without + # a`'cid'` field arrives here! + if not msg.get('cid'): + raise InternalError( + 'Unexpected cid-missing msg?\n\n' + f'{msg}\n' + ) # XXX fallthrough to handle expected error XXX re: Exception|None = ctx._remote_error @@ -273,11 +311,27 @@ async def _drain_to_final_msg( else: # bubble the original src key error raise + else: + log.cancel( + 'Skipping `MsgStream` drain since final outcome is set\n\n' + f'{ctx.outcome}\n' + ) return pre_result_drained -# TODO: make this a msgspec.Struct! +class Unresolved: + ''' + Placeholder value for `Context._result` until + a final return value or raised error is resolved. + + ''' + ... + + +# TODO: make this a .msg.types.Struct! +# -[ ] ideally we can freeze it +# -[ ] let's us do field diffing nicely in tests Bo @dataclass class Context: ''' @@ -332,28 +386,38 @@ class Context: # 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. - _scope: trio.CancelScope | None = None + _scope: trio.CancelScope|None = None _task: trio.lowlevel.Task|None = None + + # TODO: cs around result waiting so we can cancel any + # permanently blocking `._recv_chan.receive()` call in + # a drain loop? # _res_scope: trio.CancelScope|None = None # on a clean exit there should be a final value # delivered from the far end "callee" task, so # this value is only set on one side. - _result: Any | int = None + # _result: Any | int = None + _result: Any|Unresolved = Unresolved - # if the local "caller" task errors this - # value is always set to the error that was - # captured in the `Portal.open_context().__aexit__()` - # teardown. - _local_error: BaseException | None = None + # if the local "caller" task errors this value is always set + # to the error that was captured in the + # `Portal.open_context().__aexit__()` teardown block OR, in + # 2 special cases when an (maybe) expected remote error + # arrives that we purposely swallow silently: + # - `ContextCancelled` with `.canceller` set to our uid: + # a self-cancel, + # - `RemoteActorError[StreamOverrun]` which was caught during + # a self-cancellation teardown msg drain. + _local_error: BaseException|None = None # if the either side gets an error from the other # this value is set to that error unpacked from an # IPC msg. - _remote_error: BaseException | None = None + _remote_error: BaseException|None = None - # only set if the local task called `.cancel()` - _cancel_called: bool = False # did WE cancel the far end? + # only set if an actor-local task called `.cancel()` + _cancel_called: bool = False # did WE request cancel of the far end? # TODO: do we even need this? we can assume that if we're # cancelled that the other side is as well, so maybe we should @@ -379,61 +443,6 @@ class Context: # actors from being able to acquire the debugger lock. _enter_debugger_on_cancel: bool = True - @property - def cancel_called(self) -> bool: - ''' - Records whether cancellation has been requested for this context - by either an explicit call to ``.cancel()`` or an implicit call - due to an error caught inside the ``Portal.open_context()`` - block. - - ''' - return self._cancel_called - - @property - def canceller(self) -> tuple[str, str] | None: - ''' - ``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. - - ''' - return self._canceller - - @property - def cancelled_caught(self) -> bool: - return ( - # the local scope was cancelled either by - # remote error or self-request - self._scope.cancelled_caught - - # the local scope was never cancelled - # and instead likely we received a remote side - # cancellation that was raised inside `.result()` - or ( - (se := self._local_error) - and - isinstance(se, ContextCancelled) - and ( - se.canceller == self.canceller - or - se is self._remote_error - ) - ) - ) - - # @property - # def is_waiting_result(self) -> bool: - # return bool(self._res_scope) - - @property - def side(self) -> str: - ''' - Return string indicating which task this instance is wrapping. - - ''' - return 'caller' if self._portal else 'callee' - # init and streaming state _started_called: bool = False _stream_opened: bool = False @@ -450,10 +459,196 @@ class Context: maxlen=616, ) ) - _scope_nursery: trio.Nursery | None = None + + # NOTE: this was originally a legacy interface from when we + # were raising remote errors (set to `._remote_error`) by + # starting a task inside this nursery that simply raised the + # boxed exception. NOW, it's used for spawning overrun queuing + # tasks when `.allow_overruns == True` !!! + _scope_nursery: trio.Nursery|None = None + + # streaming overrun state tracking _in_overrun: bool = False _allow_overruns: bool = False + + def __str__(self) -> str: + ds: str = '=' + # ds: str = ': ' + + # only show if opened + maybe_stream_repr: str = '' + if stream := self._stream: + # TODO: a `MsgStream.reprol()` !! + # f' stream{ds}{self._stream}\n' + # f' {self._stream}\n' + maybe_stream_repr: str = ( + f' {stream}\n' + ) + + return ( + f'\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 + # https://docs.python.org/3/reference/datamodel.html#object.__repr__ + # + # XXX: Currently we target **readability** from a (console) + # logging perspective over `eval()`-ability since we do NOT + # target serializing non-struct instances! + # def __repr__(self) -> str: + __repr__ = __str__ + + @property + def cancel_called(self) -> bool: + ''' + Records whether cancellation has been requested for this context + by a call to `.cancel()` either due to, + - either an explicit call by some local task, + - or an implicit call due to an error caught inside + the ``Portal.open_context()`` block. + + ''' + return self._cancel_called + + @property + def canceller(self) -> tuple[str, str] | None: + ''' + ``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. + + ''' + return self._canceller + + @property + def cancel_acked(self) -> bool: + ''' + Records whether the task on the remote side of this IPC + context acknowledged a cancel request via a relayed + `ContextCancelled` with the `.canceller` attr set to the + `Actor.uid` of the local actor who's task entered + `Portal.open_context()`. + + This will only be `True` when `.cancel()` is called and + the ctxc response contains a `.canceller: tuple` field + equal to the uid of the calling task's actor. + + ''' + portal: Portal|None = self._portal + if portal: + our_uid: tuple = portal.actor.uid + + return bool( + self._cancel_called + and (re := self._remote_error) + and isinstance(re, ContextCancelled) + and ( + re.canceller + == + self.canceller + == + our_uid + ) + ) + + @property + def cancelled_caught(self) -> bool: + ''' + Exactly the value of `self._scope.cancelled_caught` + (delegation) and should only be (able to be read as) + `True` for a `.side == "caller"` ctx wherein the + `Portal.open_context()` block was exited due to a call to + `._scope.cancel()` - which should only ocurr in 2 cases: + + - a caller side calls `.cancel()`, the far side cancels + and delivers back a `ContextCancelled` (making + `.cancel_acked == True`) and `._scope.cancel()` is + called by `._maybe_cancel_and_set_remote_error()` which + in turn cancels all `.open_context()` started tasks + (including any overrun queuing ones). + => `._scope.cancelled_caught == True` by normal `trio` + cs semantics. + + - a caller side is delivered a `._remote_error: + RemoteActorError` via `._deliver_msg()` and a transitive + call to `_maybe_cancel_and_set_remote_error()` calls + `._scope.cancel()` and that cancellation eventually + results in `trio.Cancelled`(s) caught in the + `.open_context()` handling around the @acm's `yield`. + + Only as an FYI, in the "callee" side case it can also be + set but never is readable by any task outside the RPC + machinery in `._invoke()` since,: + - when a callee side calls `.cancel()`, `._scope.cancel()` + is called immediately and handled specially inside + `._invoke()` to raise a `ContextCancelled` which is then + sent to the caller side. + + However, `._scope.cancelled_caught` can NEVER be + accessed/read as `True` by any RPC invoked task since it + will have terminated before the cs block exit. + + ''' + return bool( + # the local scope was cancelled either by + # remote error or self-request + (self._scope and self._scope.cancelled_caught) + + # the local scope was never cancelled + # and instead likely we received a remote side + # # cancellation that was raised inside `.result()` + # or ( + # (se := self._local_error) + # and se is re + # ) + ) + + # @property + # def is_waiting_result(self) -> bool: + # return bool(self._res_scope) + + @property + def side(self) -> str: + ''' + Return string indicating which task this instance is wrapping. + + ''' + return 'caller' if self._portal else 'callee' + async def send_yield( self, data: Any, @@ -501,17 +696,20 @@ class Context: when called/closed by actor local task(s). - NOTEs & TODOs: + NOTEs: - It is expected that the caller has previously unwrapped the remote error using a call to `unpack_error()` and provides that output exception value as the input - `error` argument here. + `error` argument *here*. + + TODOs: - If this is an error message from a context opened by - `Portal.open_context()` we want to interrupt any - ongoing local tasks operating within that `Context`'s - cancel-scope so as to be notified ASAP of the remote - error and engage any caller handling (eg. for - cross-process task supervision). + `Portal.open_context()` (ideally) we want to interrupt + any ongoing local tasks operating within that + `Context`'s cancel-scope so as to be notified ASAP of + the remote error and engage any caller handling (eg. + for cross-process task supervision). + - In some cases we may want to raise the remote error immediately since there is no guarantee the locally operating task(s) will attempt to execute a checkpoint @@ -519,10 +717,13 @@ class Context: approaches depending on the current task's work and wrapping "thread" type: - - `trio`-native-and-graceful: only ever wait for tasks - to exec a next `trio.lowlevel.checkpoint()` assuming - that any such task must do so to interact with the - actor runtime and IPC interfaces. + - Currently we only support + a `trio`-native-and-graceful approach: we only ever + wait for local tasks to exec a next + `trio.lowlevel.checkpoint()` assuming that any such + task must do so to interact with the actor runtime + and IPC interfaces and will then be cancelled by + the internal `._scope` block. - (NOT IMPLEMENTED) system-level-aggressive: maybe we could eventually interrupt sync code (invoked using @@ -543,80 +744,106 @@ class Context: # do their own error checking at their own call points and # result processing. - # XXX: set the remote side's error so that after we cancel - # whatever task is the opener of this context it can raise - # that error as the reason. + # TODO: never do this right? # if self._remote_error: # return - # breakpoint() - log.cancel( - 'Setting remote error for ctx \n' + # XXX: denote and set the remote side's error so that + # after we cancel whatever task is the opener of this + # context, it can raise or swallow that error + # appropriately. + log.runtime( + 'Setting remote error for ctx\n\n' f'<= remote ctx uid: {self.chan.uid}\n' - f'=>\n{error}' + f'=>{error}' ) self._remote_error: BaseException = error - if ( - isinstance(error, ContextCancelled) - ): - log.cancel( - 'Remote task-context was cancelled for ' - f'actor: {self.chan.uid}\n' - f'task: {self.cid}\n' - f'canceller: {error.canceller}\n' - ) - # always record the cancelling actor's uid since its cancellation - # state is linked and we want to know which process was - # the cause / requester of the cancellation. - # if error.canceller is None: - # import pdbp; pdbp.set_trace() + # self-cancel (ack) or, + # peer propagated remote cancellation. + if isinstance(error, ContextCancelled): + ctxc_src: tuple = error.canceller - # breakpoint() - self._canceller = error.canceller + whom: str = ( + 'us' if ctxc_src == current_actor().uid + else 'peer' + ) + log.cancel( + f'IPC context cancelled by {whom}!\n\n' + f'{error}' + ) + # always record the cancelling actor's uid since its + # cancellation state is linked and we want to know + # which process was the cause / requester of the + # cancellation. + self._canceller = ctxc_src if self._cancel_called: - # this is an expected cancel request response message - # and we **don't need to raise it** in local cancel - # scope since it will potentially override a real error. + # this is an expected cancel request response + # message and we **don't need to raise it** in the + # local cancel `._scope` since it will potentially + # override a real error. After this returns + # `.cancel_acked == True`. return else: log.error( - f'Remote context error:\n' + f'Remote context error:\n\n' + f'{error}\n' f'{pformat(self)}\n' - # f'remote actor: {self.chan.uid}\n' - # f'cid: {self.cid}\n' ) self._canceller = self.chan.uid - # TODO: tempted to **not** do this by-reraising in a - # nursery and instead cancel a surrounding scope, detect - # the cancellation, then lookup the error that was set? - # YES! this is way better and simpler! + # Cancel the local `._scope`, catch that + # `._scope.cancelled_caught` and re-raise any remote error + # once exiting (or manually calling `.result()`) the + # `.open_context()` block. cs: trio.CancelScope = self._scope if ( cs and not cs.cancel_called and not cs.cancelled_caught ): - - # TODO: we can for sure drop this right? - # from trio.testing import wait_all_tasks_blocked - # await wait_all_tasks_blocked() - # TODO: it'd sure be handy to inject our own # `trio.Cancelled` subtype here ;) # https://github.com/goodboy/tractor/issues/368 self._scope.cancel() - # NOTE: this REPL usage actually works here dawg! Bo - # await pause() + # TODO: maybe we should also call `._res_scope.cancel()` if it + # exists to support cancelling any drain loop hangs? - # TODO: maybe we have to use `._res_scope.cancel()` if it - # exists? + # TODO: add to `Channel`? + @property + def dst_maddr(self) -> str: + chan: Channel = self.chan + dst_addr, dst_port = chan.raddr + trans: MsgTransport = chan.transport + # cid: str = self.cid + # cid_head, cid_tail = cid[:6], cid[-6:] + return ( + f'/ipv4/{dst_addr}' + f'/{trans.name_key}/{dst_port}' + # f'/{self.chan.uid[0]}' + # f'/{self.cid}' + + # f'/cid={cid_head}..{cid_tail}' + # TODO: ? not use this ^ right ? + ) + + dmaddr = dst_maddr + + @property + def repr_rpc( + self, + ) -> str: + # TODO: how to show the transport interchange fmt? + # codec: str = self.chan.transport.codec_key + return ( + # f'{self._nsf}() -{{{codec}}}-> {repr(self.outcome)}:' + f'{self._nsf}() -> {self.repr_outcome()}:' + ) async def cancel( self, @@ -633,13 +860,23 @@ class Context: side: str = self.side self._cancel_called: bool = True - header: str = f'Cancelling "{side.upper()}"-side of ctx with peer\n' + header: str = ( + f'Cancelling ctx with peer from {side.upper()} side\n\n' + ) reminfo: str = ( - f'uid: {self.chan.uid}\n' - f' |_ {self._nsf}()\n' + # ' =>\n' + f'Context.cancel() => {self.chan.uid}\n' + # f'{self.chan.uid}\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 ) - # caller side who entered `Portal.open_context()` + # CALLER side task + # ------ - ------ + # Aka the one that entered `Portal.open_context()` + # # NOTE: on the call side we never manually call # `._scope.cancel()` since we expect the eventual # `ContextCancelled` from the other side to trigger this @@ -648,8 +885,9 @@ class Context: # `Portal.open_context().__aexit__()`) if side == 'caller': if not self._portal: - raise RuntimeError( - "No portal found, this is likely a callee side context" + raise InternalError( + 'No portal found!?\n' + 'Why is this supposed caller context missing it?' ) cid: str = self.cid @@ -686,10 +924,18 @@ class Context: f'{reminfo}' ) - # callee side remote task - # NOTE: on this side we ALWAYS cancel the local scope since - # the caller expects a `ContextCancelled` to be sent from - # `._runtime._invoke()` back to the other side. + # CALLEE side task + # ------ - ------ + # Aka the one that DID NOT EVER enter a `Portal.open_context()` + # and instead was constructed and scheduled as an + # `_invoke()` RPC task. + # + # NOTE: on this side we ALWAYS cancel the local scope + # since the caller expects a `ContextCancelled` to be sent + # from `._runtime._invoke()` back to the other side. The + # logic for catching the result of the below + # `._scope.cancel()` is inside the `._runtime._invoke()` + # context RPC handling block. else: log.cancel( header @@ -750,7 +996,7 @@ class Context: # sent to the other side! if self._remote_error: # NOTE: this is diff then calling - # `._maybe_raise_from_remote_msg()` specifically + # `._maybe_raise_remote_err()` specifically # because any task entering this `.open_stream()` # AFTER cancellation has already been requested, # we DO NOT want to absorb any ctxc ACK silently! @@ -876,53 +1122,105 @@ class Context: f'ctx id: {self.cid}' ) + # TODO: replace all the instances of this!! XD + def maybe_raise( + self, + **kwargs, + ) -> Exception|None: + if re := self._remote_error: + return self._maybe_raise_remote_err( + re, + **kwargs, + ) + def _maybe_raise_remote_err( self, - err: Exception, + remote_error: Exception, raise_ctxc_from_self_call: bool = False, raise_overrun_from_self: bool = True, - ) -> ContextCancelled|None: + ) -> ( + ContextCancelled # `.cancel()` request to far side + |RemoteActorError # stream overrun caused and ignored by us + ): ''' - Maybe raise a remote error depending on who (which task from - which actor) requested a cancellation (if any). + Maybe raise a remote error depending on the type of error + and *who* (i.e. which task from which actor) requested + a cancellation (if any). ''' - # NOTE: whenever the context's "opener" side (task) **is** - # the side which requested the cancellation (likekly via - # ``Context.cancel()``), we don't want to re-raise that - # cancellation signal locally (would be akin to - # a ``trio.Nursery`` nursery raising ``trio.Cancelled`` - # whenever ``CancelScope.cancel()`` was called) and - # instead silently reap the expected cancellation - # "error"-msg. - our_uid: tuple[str, str] = current_actor().uid - if ( - (not raise_ctxc_from_self_call - and isinstance(err, ContextCancelled) - and ( - self._cancel_called - or self.chan._cancel_called - or self.canceller == our_uid - or tuple(err.canceller) == our_uid) - ) - or - (not raise_overrun_from_self - and isinstance(err, RemoteActorError) - and err.msgdata['type_str'] == 'StreamOverrun' - and tuple(err.msgdata['sender']) == our_uid - ) + if (( + # NOTE: whenever the context's "opener" side (task) **is** + # the side which requested the cancellation (likekly via + # ``Context.cancel()``), we don't want to re-raise that + # cancellation signal locally (would be akin to + # a ``trio.Nursery`` nursery raising ``trio.Cancelled`` + # whenever ``CancelScope.cancel()`` was called) and + # instead silently reap the expected cancellation + # "error"-msg-as-ack. In this case the `err: + # ContextCancelled` must have a `.canceller` set to the + # uid of the requesting task's actor and we only do NOT + # raise that error locally if WE ARE THAT ACTOR which + # requested the cancellation. + not raise_ctxc_from_self_call + and isinstance(remote_error, ContextCancelled) + and ( + self._cancel_called + # or self.chan._cancel_called + # TODO: ^ should we have a special separate case + # for this ^ ? + ) + and ( # one of, + + (portal := self._portal) + and (our_uid := portal.actor.uid) + # TODO: ?potentially it is useful to emit certain + # warning/cancel logs for the cases where the + # cancellation is due to a lower level cancel + # request, such as `Portal.cancel_actor()`, since in + # that case it's not actually this specific ctx that + # made a `.cancel()` call, but it is the same + # actor-process? + and tuple(remote_error.canceller) == our_uid + or self.chan._cancel_called + or self.canceller == our_uid + ) + ) or ( + + # NOTE: whenever this context is the cause of an + # overrun on the remote side (aka we sent msgs too + # fast that the remote task was overrun according + # to `MsgStream` buffer settings) AND the caller + # has requested to not raise overruns this side + # caused, we also silently absorb any remotely + # boxed `StreamOverrun`. This is mostly useful for + # supressing such faults during + # cancellation/error/final-result handling inside + # `_drain_to_final_msg()` such that we do not + # raise such errors particularly in the case where + # `._cancel_called == True`. + not raise_overrun_from_self + and isinstance(remote_error, RemoteActorError) + and remote_error.msgdata['type_str'] == 'StreamOverrun' + and tuple(remote_error.msgdata['sender']) == our_uid + ) ): # NOTE: we set the local scope error to any "self # cancellation" error-response thus "absorbing" # the error silently B) if self._local_error is None: - self._local_error = err + self._local_error = remote_error - return err + else: + log.warning( + 'Local error already set for ctx?\n' + f'{self._local_error}\n' + ) - # NOTE: currently we are masking underlying runtime errors + return remote_error + + # NOTE: currently we are hiding underlying runtime errors # which are often superfluous to user handler code. not # sure if this is still needed / desired for all operation? # TODO: maybe we can only NOT mask if: @@ -932,10 +1230,15 @@ class Context: # runtime frames from the tb explicitly? # https://docs.python.org/3/reference/simple_stmts.html#the-raise-statement # https://stackoverflow.com/a/24752607 - # __tracebackhide__: bool = True - raise err from None + __tracebackhide__: bool = True + raise remote_error from None - async def result(self) -> Any | Exception: + # TODO: change to `.wait_for_result()`? + async def result( + self, + hide_tb: bool = True, + + ) -> Any|Exception: ''' From some (caller) side task, wait for and return the final result from the remote (callee) side's task. @@ -961,182 +1264,53 @@ class Context: of the remote cancellation. ''' - assert self._portal, "Context.result() can not be called from callee!" + __tracebackhide__ = hide_tb + assert self._portal, ( + "Context.result() can not be called from callee side!" + ) + if self._final_result_is_set(): + return self._result + assert self._recv_chan - raise_overrun: bool = not self._allow_overruns - # if re := self._remote_error: - # return self._maybe_raise_remote_err( - # re, - # # NOTE: obvi we don't care if we - # # overran the far end if we're already - # # waiting on a final result (msg). - # raise_overrun_from_self=raise_overrun, - # ) - - res_placeholder: int = id(self) + # res_placeholder: int = id(self) if ( - self._result == res_placeholder - and not self._remote_error + # self._result == res_placeholder + # and not self._remote_error + self.maybe_error is None + # not self._remote_error + # and not self._local_error and not self._recv_chan._closed # type: ignore ): - # wait for a final context result by collecting (but - # basically ignoring) any bi-dir-stream msgs still in transit - # from the far end. - drained_msgs: list[dict] = await _drain_to_final_msg(ctx=self) - log.runtime( + # wait for a final context result/error by "draining" + # (by more or less ignoring) any bi-dir-stream "yield" + # msgs still in transit from the far end. + drained_msgs: list[dict] = await _drain_to_final_msg( + ctx=self, + hide_tb=hide_tb, + ) + for msg in drained_msgs: + + # TODO: mask this by default.. + if 'return' in msg: + # from .devx import pause + # await pause() + raise InternalError( + 'Final `return` msg should never be drained !?!?\n\n' + f'{msg}\n' + ) + + log.cancel( 'Ctx drained pre-result msgs:\n' f'{drained_msgs}' ) - # TODO: implement via helper func ^^^^ - # pre_result_drained: list[dict] = [] - # while not self._remote_error: - # try: - # # NOTE: this REPL usage actually works here dawg! Bo - # # from .devx._debug import pause - # # await pause() - # # if re := self._remote_error: - # # self._maybe_raise_remote_err( - # # re, - # # # NOTE: obvi we don't care if we - # # # overran the far end if we're already - # # # waiting on a final result (msg). - # # raise_overrun_from_self=raise_overrun, - # # ) - - # # TODO: bad idea? - # # with trio.CancelScope() as res_cs: - # # self._res_scope = res_cs - # # msg: dict = await self._recv_chan.receive() - # # if res_cs.cancelled_caught: - - # # from .devx._debug import pause - # # await pause() - # msg: dict = await self._recv_chan.receive() - # self._result: Any = msg['return'] - # log.runtime( - # 'Context delivered final result msg:\n' - # f'{pformat(msg)}' - # ) - # # NOTE: we don't need to do this right? - # # XXX: only close the rx mem chan AFTER - # # a final result is retreived. - # # if self._recv_chan: - # # await self._recv_chan.aclose() - # break - - # # NOTE: we get here if the far end was - # # `ContextCancelled` in 2 cases: - # # 1. we requested the cancellation and thus - # # SHOULD NOT raise that far end error, - # # 2. WE DID NOT REQUEST that cancel and thus - # # SHOULD RAISE HERE! - # except trio.Cancelled: - - # # 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. - # if re := self._remote_error: - # self._maybe_raise_remote_err(re) - - # # CASE 1: we DID request the cancel we simply - # # continue to bubble up as normal. - # raise - - # except KeyError: - - # if 'yield' in msg: - # # far end task is still streaming to us so discard - # log.warning(f'Discarding std "yield"\n{msg}') - # pre_result_drained.append(msg) - # continue - - # # TODO: work out edge cases here where - # # a stream is open but the task also calls - # # this? - # # -[ ] should be a runtime error if a stream is open - # # right? - # elif 'stop' in msg: - # log.cancel( - # 'Remote stream terminated due to "stop" msg:\n' - # f'{msg}' - # ) - # pre_result_drained.append(msg) - # continue - - # # internal error should never get here - # assert msg.get('cid'), ( - # "Received internal error at portal?" - # ) - - # # XXX fallthrough to handle expected error XXX - # re: Exception|None = self._remote_error - # if re: - # log.critical( - # 'Remote ctx terminated due to "error" msg:\n' - # f'{re}' - # ) - # assert msg is self._cancel_msg - # # NOTE: this solved a super dupe edge case XD - # # this was THE super duper edge case of: - # # - local task opens a remote task, - # # - requests remote cancellation of far end - # # ctx/tasks, - # # - needs to wait for the cancel ack msg - # # (ctxc) or some result in the race case - # # where the other side's task returns - # # before the cancel request msg is ever - # # rxed and processed, - # # - here this surrounding drain loop (which - # # iterates all ipc msgs until the ack or - # # an early result arrives) was NOT exiting - # # since we are the edge case: local task - # # does not re-raise any ctxc it receives - # # IFF **it** was the cancellation - # # requester.. - # # will raise if necessary, ow break from - # # loop presuming any error terminates the - # # context! - # self._maybe_raise_remote_err( - # re, - # # NOTE: obvi we don't care if we - # # overran the far end if we're already - # # waiting on a final result (msg). - # # raise_overrun_from_self=False, - # raise_overrun_from_self=raise_overrun, - # ) - - # break # OOOOOF, yeah obvi we need this.. - - # # XXX we should never really get here - # # right! since `._deliver_msg()` should - # # always have detected an {'error': ..} - # # msg and already called this right!?! - # elif error := unpack_error( - # msg=msg, - # chan=self._portal.channel, - # hide_tb=False, - # ): - # log.critical('SHOULD NEVER GET HERE!?') - # assert msg is self._cancel_msg - # assert error.msgdata == self._remote_error.msgdata - # from .devx._debug import pause - # await pause() - # self._maybe_cancel_and_set_remote_error(error) - # self._maybe_raise_remote_err(error) - - # else: - # # bubble the original src key error - # raise - if ( (re := self._remote_error) - and self._result == res_placeholder + # and self._result == res_placeholder ): - maybe_err: Exception|None = self._maybe_raise_remote_err( + self._maybe_raise_remote_err( re, # NOTE: obvi we don't care if we # overran the far end if we're already @@ -1152,10 +1326,126 @@ class Context: (not self._cancel_called) ), ) - if maybe_err: - self._result = maybe_err + # if maybe_err: + # self._result = maybe_err - return self._result + return self.outcome + # None if self._result == res_placeholder + # else self._result + # ) + + # TODO: switch this with above which should be named + # `.wait_for_outcome()` and instead do + # a `.outcome.Outcome.unwrap()` ? + # @property + # def result(self) -> Any|None: + # if self._final_result_is_set(): + # return self._result + + # raise RuntimeError('No result is available!') + + @property + def maybe_error(self) -> BaseException|None: + le: Exception|None = self._local_error + re: RemoteActorError|ContextCancelled|None = self._remote_error + + match (le, re): + # NOTE: remote errors always get precedence since even + # in the cases where a local error was the cause, the + # received boxed ctxc should include the src info + # caused by us right? + case ( + _, + RemoteActorError(), + ): + # give precedence to remote error if it's + # NOT a cancel ack (ctxc). + return ( + re or le + ) + + # TODO: extra logic to handle ctxc ack case(s)? + # -[ ] eg. we error, call .cancel(), rx ack but should + # raise the _local_error instead? + # -[ ] are there special error conditions where local vs. + # remote should take precedence? + # case ( + # _, + # ContextCancelled(canceller=), + # ): + + error: Exception|None = le or re + if error: + return error + + assert not self._cancel_msg + return None + + def _final_result_is_set(self) -> bool: + # return not (self._result == id(self)) + return self._result is not Unresolved + + # def get_result_nowait(self) -> Any|None: + # TODO: use `outcome.Outcome` here instead? + @property + def outcome(self) -> ( + Any| + RemoteActorError| + ContextCancelled + ): + ''' + 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. + + Note that if the remote task has not terminated then this + field always resolves to the module defined `Unresolved` handle. + + TODO: implement this using `outcome.Outcome` types? + + ''' + return self.maybe_error or self._result + + # @property + def repr_outcome( + self, + show_error_fields: bool = False, + + ) -> str: + ''' + Deliver a (simplified) `str` representation (as in + `.__repr__()`) of the final `.outcome` + + ''' + merr: Exception|None = self.maybe_error + if merr: + # if the error-type is one of ours and has the custom + # defined "repr-(in)-one-line" method call it, ow + # just deliver the type name. + if ( + (reprol := getattr(merr, 'reprol', False)) + and show_error_fields + ): + return reprol() + + elif isinstance(merr, BaseExceptionGroup): + # TODO: maybe for multis we should just show + # a one-line count per error type, like with + # `collections.Counter`? + # + # just the type name for now to avoid long lines + # when tons of cancels.. + return type(merr).__name__ + + # just the type name + # else: # but wen? + # return type(merr).__name__ + + # for all other errors show their regular output + return str(merr) + + return str(self._result) async def started( self, @@ -1261,8 +1551,14 @@ class Context: msg, self.chan, ): - log.error( - f'Delivering error-msg to caller\n' + if not isinstance(re, ContextCancelled): + log_meth = log.error + else: + log_meth = log.runtime + + log_meth( + f'Delivering error-msg to caller\n\n' + f'<= peer: {from_uid}\n' f' |_ {nsf}()\n\n' @@ -1276,7 +1572,7 @@ class Context: # NOTE: this will not raise an error, merely set # `._remote_error` and maybe cancel any task currently # entered in `Portal.open_context()` presuming the - # error is "cancel causing" (i.e. `ContextCancelled` + # error is "cancel causing" (i.e. a `ContextCancelled` # or `RemoteActorError`). self._maybe_cancel_and_set_remote_error(re) @@ -1288,6 +1584,10 @@ class Context: # return True # # XXX ALSO NO!! XXX + # => NEVER raise remote errors from the calling + # runtime task, they should always be raised by + # consumer side tasks operating on the + # `Portal`/`Context` APIs. # if self._remote_error: # self._maybe_raise_remote_err(error) @@ -1471,7 +1771,9 @@ def mk_context( _task=trio.lowlevel.current_task(), **kwargs, ) - ctx._result: int | Any = id(ctx) + # TODO: we can drop the old placeholder yah? + # ctx._result: int | Any = id(ctx) + ctx._result = Unresolved return ctx From 1c9589cfc44d986ad528e514ee5fd0ac288fea21 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 2 Mar 2024 16:27:46 -0500 Subject: [PATCH 108/146] Avoid `ctx.cancel()` after ctxc rxed in `.open_context()` In the case where the callee side delivers us a ctxc with `.canceller` set we can presume that remote cancellation already has taken place and thus we don't need to do the normal call-`Context.cancel()`-on-error step. Further, in the case where we do call it also handle any `trio.CloseResourceError` gracefully with a `.warning()`. Also, originally I had added a post-`yield`-maybe-raise to attempt handling any remote ctxc the same as for the local case (i.e. raised from `yield` line) wherein if we get a remote ctxc the same handler branch-path would trigger, thus avoiding different behaviour in that case. I ended up masking it out (but can't member why.. ) as it seems the normal `.result()` call and its internal handling gets the same behaviour? I've left in the heavily commented code in case it ends up being the better way to go; likely making the move to having a single code in both cases is better even if it is just a matter of deciding whether to swallow the ctxc or not in the `.cancel_acked` case. Further teensie improvements: - obvi improve/simplify log msg contents as in prior patches. - use the new `maybe_wait_for_debugger(header_msg: str)` if/when waiting to exit in debug mode. - another `hide_tb: bool` frame hider flag. - rando type-annot updates of course :) --- tractor/_portal.py | 248 +++++++++++++++++++++++++-------------------- 1 file changed, 140 insertions(+), 108 deletions(-) diff --git a/tractor/_portal.py b/tractor/_portal.py index 8235c92f..041c8104 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -27,8 +27,9 @@ from __future__ import annotations import importlib import inspect from typing import ( - Any, Optional, - Callable, AsyncGenerator, + Any, + Callable, + AsyncGenerator, Type, ) from functools import partial @@ -52,6 +53,7 @@ from ._ipc import Channel from .log import get_logger from .msg import NamespacePath from ._exceptions import ( + InternalError, _raise_from_no_key_in_msg, unpack_error, NoResult, @@ -126,7 +128,7 @@ class Portal: def __init__(self, channel: Channel) -> None: self.chan = channel # during the portal's lifetime - self._result_msg: Optional[dict] = None + self._result_msg: dict|None = None # When set to a ``Context`` (when _submit_for_result is called) # it is expected that ``result()`` will be awaited at some @@ -171,7 +173,7 @@ class Portal: ) -> dict[str, Any]: assert ctx._remote_func_type == 'asyncfunc' # single response - msg = await ctx._recv_chan.receive() + msg: dict = await ctx._recv_chan.receive() return msg async def result(self) -> Any: @@ -255,11 +257,11 @@ class Portal: return False reminfo: str = ( - f'uid: {self.channel.uid}\n' - f' |_{chan}\n' + f'{self.channel.uid}\n' + f' |_{chan}\n' ) log.cancel( - f'Sending actor cancel request to peer\n' + f'Sending runtime `.cancel()` request to peer\n\n' f'{reminfo}' ) @@ -281,7 +283,9 @@ class Portal: return True if cs.cancelled_caught: - log.cancel( + # may timeout and we never get an ack (obvi racy) + # but that doesn't mean it wasn't cancelled. + log.debug( 'May have failed to cancel peer?\n' f'{reminfo}' ) @@ -293,9 +297,11 @@ class Portal: trio.ClosedResourceError, trio.BrokenResourceError, ): - log.cancel( - f"{self.channel} for {self.channel.uid} was already " - "closed or broken?") + log.debug( + 'IPC chan for actor already closed or broken?\n\n' + f'{self.channel.uid}\n' + f' |_{self.channel}\n' + ) return False async def run_from_ns( @@ -416,7 +422,8 @@ class Portal: try: # deliver receive only stream async with MsgStream( - ctx, ctx._recv_chan, + ctx=ctx, + rx_chan=ctx._recv_chan, ) as rchan: self._streams.add(rchan) yield rchan @@ -443,6 +450,11 @@ class Portal: # await recv_chan.aclose() self._streams.remove(rchan) + # TODO: move this impl to `._context` mod and + # instead just bind it here as a method so that the logic + # for ctx stuff stays all in one place (instead of frickin + # having to open this file in tandem every gd time!!! XD) + # @asynccontextmanager async def open_context( @@ -451,6 +463,11 @@ class Portal: allow_overruns: bool = False, + # TODO: if we set this the wrapping `@acm` body will + # still be shown (awkwardly) on pdb REPL entry. Ideally + # we can similarly annotate that frame to NOT show? + hide_tb: bool = False, + # proxied to RPC **kwargs, @@ -484,6 +501,8 @@ class Portal: collection. See ``tractor.Context`` for more details. ''' + __tracebackhide__: bool = hide_tb + # conduct target func method structural checks if not inspect.iscoroutinefunction(func) and ( getattr(func, '_tractor_contex_function', False) @@ -536,9 +555,12 @@ class Portal: # placeholder for any exception raised in the runtime # or by user tasks which cause this context's closure. - scope_err: BaseException | None = None + scope_err: BaseException|None = None + ctxc_from_callee: ContextCancelled|None = None try: async with trio.open_nursery() as nurse: + + # NOTE: used to start overrun queuing tasks ctx._scope_nursery: trio.Nursery = nurse ctx._scope: trio.CancelScope = nurse.cancel_scope @@ -546,14 +568,26 @@ class Portal: # in enter tuple. yield ctx, first - # between the caller exiting and arriving here the - # far end may have sent a ctxc-msg or other error, - # so check for it here immediately and maybe raise - # so as to engage the ctxc handling block below! + # ??TODO??: do we still want to consider this or is + # the `else:` block handling via a `.result()` + # call below enough?? + # -[ ] pretty sure `.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 + # that we can re-use it around the `yield` ^ here + # or vice versa? + # + # NOTE: between the caller exiting and arriving + # here the far end may have sent a ctxc-msg or + # other error, so check for it here immediately + # and maybe raise so as to engage the ctxc + # handling block below! + # # if re := ctx._remote_error: # maybe_ctxc: ContextCancelled|None = ctx._maybe_raise_remote_err( # re, - # # TODO: do we want this to always raise? # # - means that on self-ctxc, if/when the # # block is exited before the msg arrives @@ -571,7 +605,7 @@ class Portal: # # block? # raise_ctxc_from_self_call=True, # ) - # assert maybe_ctxc + # ctxc_from_callee = maybe_ctxc # when in allow_overruns mode there may be # lingering overflow sender tasks remaining? @@ -583,13 +617,18 @@ class Portal: not ctx._allow_overruns or len(nurse.child_tasks) > 1 ): - raise RuntimeError( + raise InternalError( 'Context has sub-tasks but is ' 'not in `allow_overruns=True` mode!?' ) - # ensure cancel of all overflow sender tasks - # started in the ctx nursery. + # ensure we cancel all overflow sender + # tasks started in the nursery when + # `._allow_overruns == True`. + # + # NOTE: this means `._scope.cancelled_caught` + # will prolly be set! not sure if that's + # non-ideal or not ??? ctx._scope.cancel() # XXX NOTE XXX: maybe shield against @@ -602,14 +641,15 @@ class Portal: # of a `Context`. In both cases any `ContextCancelled` # raised in this scope-block came from a transport msg # relayed from some remote-actor-task which our runtime set - # as a `Context._remote_error` + # as to `Context._remote_error` # # the CASES: # # - if that context IS THE SAME ONE that called # `Context.cancel()`, we want to absorb the error # silently and let this `.open_context()` block to exit - # without raising. + # without raising, ideally eventually receiving the ctxc + # ack msg thus resulting in `ctx.cancel_acked == True`. # # - if it is from some OTHER context (we did NOT call # `.cancel()`), we want to re-RAISE IT whilst also @@ -633,6 +673,7 @@ class Portal: # `Nursery.cancel_scope.cancel()`) except ContextCancelled as ctxc: scope_err = ctxc + ctxc_from_callee = ctxc # XXX TODO XXX: FIX THIS debug_mode BUGGGG!!! # using this code and then resuming the REPL will @@ -642,6 +683,7 @@ class Portal: # documenting it as a definittive example of # debugging the tractor-runtime itself using it's # own `.devx.` tooling! + # # await pause() # CASE 2: context was cancelled by local task calling @@ -649,15 +691,10 @@ class Portal: # exit silently. if ( ctx._cancel_called - and ( - ctxc is ctx._remote_error - # ctxc.msgdata == ctx._remote_error.msgdata - - # TODO: uhh `Portal.canceller` ain't a thangg - # dawg? (was `self.canceller` before?!?) - and - ctxc.canceller == self.actor.uid - ) + and + ctxc is ctx._remote_error + and + ctxc.canceller == self.actor.uid ): log.cancel( f'Context (cid=[{ctx.cid[-6:]}..] cancelled gracefully with:\n' @@ -665,9 +702,9 @@ class Portal: ) # CASE 1: this context was never cancelled via a local # task (tree) having called `Context.cancel()`, raise - # the error since it was caused by someone else! + # the error since it was caused by someone else + # -> probably a remote peer! else: - # await pause() raise # the above `._scope` can be cancelled due to: @@ -680,19 +717,29 @@ class Portal: # CASE 3: standard local error in this caller/yieldee Exception, - # CASES 1 & 2: normally manifested as - # a `Context._scope_nursery` raised + # CASES 1 & 2: can manifest as a `ctx._scope_nursery` # exception-group of, + # # 1.-`trio.Cancelled`s, since - # `._scope.cancel()` will have been called and any - # `ContextCancelled` absorbed and thus NOT RAISED in - # any `Context._maybe_raise_remote_err()`, + # `._scope.cancel()` will have been called + # (transitively by the runtime calling + # `._deliver_msg()`) and any `ContextCancelled` + # eventually absorbed and thus absorbed/supressed in + # any `Context._maybe_raise_remote_err()` call. + # # 2.-`BaseExceptionGroup[ContextCancelled | RemoteActorError]` - # from any error raised in the "callee" side with - # a group only raised if there was any more then one - # task started here in the "caller" in the - # `yield`-ed to task. - BaseExceptionGroup, # since overrun handler tasks may have been spawned + # from any error delivered from the "callee" side + # 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()` + # `._maybe_raise_remote_err()` will be transitively + # called and the remote error raised causing all + # tasks to be cancelled. + # NOTE: ^ this case always can happen if any + # overrun handler tasks were spawned! + BaseExceptionGroup, + trio.Cancelled, # NOTE: NOT from inside the ctx._scope KeyboardInterrupt, @@ -702,69 +749,48 @@ class Portal: # XXX: ALWAYS request the context to CANCEL ON any ERROR. # NOTE: `Context.cancel()` is conversely NEVER CALLED in # the `ContextCancelled` "self cancellation absorbed" case - # handled in the block above! + # handled in the block above ^^^ !! log.cancel( - 'Context cancelled for task due to\n' + 'Context terminated due to\n\n' f'{caller_err}\n' - 'Sending cancel request..\n' - f'task:{cid}\n' - f'actor:{uid}' ) if debug_mode(): - log.pdb( - 'Delaying `ctx.cancel()` until debug lock ' - 'acquired..' - ) # async with acquire_debug_lock(self.actor.uid): # pass # TODO: factor ^ into below for non-root cases? - await maybe_wait_for_debugger() - log.pdb( - 'Acquired debug lock! ' - 'Calling `ctx.cancel()`!' + was_acquired: bool = await maybe_wait_for_debugger( + header_msg=( + 'Delaying `ctx.cancel()` until debug lock ' + 'acquired..\n' + ), ) + if was_acquired: + log.pdb( + 'Acquired debug lock! ' + 'Calling `ctx.cancel()`!\n' + ) - try: - await ctx.cancel() - except trio.BrokenResourceError: - log.warning( - 'IPC connection for context is broken?\n' - f'task:{cid}\n' - f'actor:{uid}' - ) + + # we don't need to cancel the callee if it already + # told us it's cancelled ;p + if ctxc_from_callee is None: + try: + await ctx.cancel() + except ( + trio.BrokenResourceError, + trio.ClosedResourceError, + ): + log.warning( + 'IPC connection for context is broken?\n' + f'task:{cid}\n' + f'actor:{uid}' + ) raise # duh # no local scope error, the "clean exit with a result" case. else: - # between the caller exiting and arriving here the - # far end may have sent a ctxc-msg or other error, - # so check for it here immediately and maybe raise - # so as to engage the ctxc handling block below! - # if re := ctx._remote_error: - # maybe_ctxc: ContextCancelled|None = ctx._maybe_raise_remote_err( - # re, - - # # TODO: do we want this to always raise? - # # - means that on self-ctxc, if/when the - # # block is exited before the msg arrives - # # but then the msg during __exit__ - # # calling we may not activate the - # # ctxc-handler block below? should we - # # be? - # # - if there's a remote error that arrives - # # after the child has exited, we won't - # # handle until the `finally:` block - # # where `.result()` is always called, - # # again in which case we handle it - # # differently then in the handler block - # # that would normally engage from THIS - # # block? - # raise_ctxc_from_self_call=True, - # ) - # assert maybe_ctxc - if ctx.chan.connected(): log.runtime( 'Waiting on final context result for\n' @@ -794,16 +820,18 @@ class Portal: scope_err = berr raise + # yes! this worx Bp + # from .devx import _debug + # await _debug.pause() + # an exception type boxed in a `RemoteActorError` - # is returned (meaning it was obvi not raised). + # is returned (meaning it was obvi not raised) + # that we want to log-report on. msgdata: str|None = getattr( result_or_err, 'msgdata', None ) - # yes! this worx Bp - # from .devx import _debug - # await _debug.pause() match (msgdata, result_or_err): case ( {'tb_str': tbstr}, @@ -846,9 +874,8 @@ class Portal: # operating *in* this scope to have survived # we tear down the runtime feeder chan last # to avoid premature stream clobbers. - rxchan: trio.ReceiveChannel = ctx._recv_chan if ( - rxchan + (rxchan := ctx._recv_chan) # maybe TODO: yes i know the below check is # touching `trio` memchan internals..BUT, there are @@ -904,22 +931,27 @@ class Portal: etype: Type[BaseException] = type(scope_err) # CASE 2 - if ctx._cancel_called: + if ( + ctx._cancel_called + and ctx.cancel_acked + ): log.cancel( 'Context cancelled by caller task\n' f'|_{ctx._task}\n\n' - f'{etype}' + f'{repr(scope_err)}\n' ) + # TODO: should we add a `._cancel_req_received` + # flag to determine if the callee manually called + # `ctx.cancel()`? + # -[ ] going to need a cid check no? + # CASE 1 else: log.cancel( - f'Context cancelled by remote callee task\n' - f'peer: {uid}\n' - f'|_ {nsf}()\n\n' - - f'{etype}\n' + f'Context terminated due to local scope error:\n' + f'{etype.__name__}\n' ) # XXX: (MEGA IMPORTANT) if this is a root opened process we @@ -971,7 +1003,7 @@ class LocalPortal: async def open_portal( channel: Channel, - nursery: Optional[trio.Nursery] = None, + nursery: trio.Nursery|None = None, start_msg_loop: bool = True, shield: bool = False, @@ -996,7 +1028,7 @@ async def open_portal( if channel.uid is None: await actor._do_handshake(channel) - msg_loop_cs: Optional[trio.CancelScope] = None + msg_loop_cs: trio.CancelScope|None = None if start_msg_loop: from ._runtime import process_messages msg_loop_cs = await nursery.start( From 4fb34772e7bc4900ca3757023b8b4f02e9d11f36 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 2 Mar 2024 19:26:40 -0500 Subject: [PATCH 109/146] Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= ` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin! --- tractor/_runtime.py | 944 +++++++++++++++++++++++++++----------------- 1 file changed, 582 insertions(+), 362 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 270599ae..abdaf91a 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -22,6 +22,10 @@ low-level transport msg handling. """ from __future__ import annotations +from contextlib import ( + ExitStack, + asynccontextmanager as acm, +) from collections import defaultdict from functools import partial from itertools import chain @@ -35,12 +39,12 @@ from typing import ( Any, Callable, Union, + Coroutine, TYPE_CHECKING, ) import uuid from types import ModuleType import os -from contextlib import ExitStack import warnings from async_generator import aclosing @@ -96,13 +100,290 @@ async def maybe_import_gb(): await greenback.ensure_portal() except ModuleNotFoundError: - log.warning( + log.debug( '`greenback` is not installed.\n' - 'No sync debug support!' + 'No sync debug support!\n' ) _gb_mod = False +async def _invoke_non_context( + actor: Actor, + cancel_scope: CancelScope, + ctx: Context, + cid: str, + chan: Channel, + func: Callable, + coro: Coroutine, + kwargs: dict[str, Any], + + treat_as_gen: bool, + is_rpc: bool, + + task_status: TaskStatus[ + Context | BaseException + ] = trio.TASK_STATUS_IGNORED, +): + + # TODO: can we unify this with the `context=True` impl below? + if inspect.isasyncgen(coro): + await chan.send({'functype': 'asyncgen', 'cid': cid}) + # XXX: massive gotcha! If the containing scope + # is cancelled and we execute the below line, + # any ``ActorNursery.__aexit__()`` WON'T be + # triggered in the underlying async gen! So we + # have to properly handle the closing (aclosing) + # of the async gen in order to be sure the cancel + # is propagated! + with cancel_scope as cs: + ctx._scope = cs + task_status.started(ctx) + async with aclosing(coro) as agen: + async for item in agen: + # TODO: can we send values back in here? + # it's gonna require a `while True:` and + # some non-blocking way to retrieve new `asend()` + # values from the channel: + # to_send = await chan.recv_nowait() + # if to_send is not None: + # to_yield = await coro.asend(to_send) + await chan.send({'yield': item, 'cid': cid}) + + log.runtime(f"Finished iterating {coro}") + # TODO: we should really support a proper + # `StopAsyncIteration` system here for returning a final + # value if desired + await chan.send({'stop': True, 'cid': cid}) + + # one way @stream func that gets treated like an async gen + # TODO: can we unify this with the `context=True` impl below? + elif treat_as_gen: + await chan.send({'functype': 'asyncgen', 'cid': cid}) + # 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) + await coro + + if not cs.cancelled_caught: + # task was not cancelled so we can instruct the + # far end async gen to tear down + await chan.send({'stop': True, 'cid': cid}) + 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: + await chan.send({ + 'functype': 'asyncfunc', + 'cid': cid + }) + except ( + trio.ClosedResourceError, + trio.BrokenResourceError, + BrokenPipeError, + ) as ipc_err: + failed_resp = True + if is_rpc: + raise + else: + # TODO: should this be an `.exception()` call? + log.warning( + f'Failed to respond to non-rpc request: {func}\n' + f'{ipc_err}' + ) + + with cancel_scope as cs: + ctx._scope: CancelScope = cs + task_status.started(ctx) + result = await coro + fname: str = func.__name__ + log.runtime( + 'RPC complete:\n' + f'task: {ctx._task}\n' + f'|_cid={ctx.cid}\n' + f'|_{fname}() -> {pformat(result)}\n' + ) + + # NOTE: only send result if we know IPC isn't down + if ( + not failed_resp + and chan.connected() + ): + try: + await chan.send( + {'return': result, + 'cid': cid} + ) + except ( + BrokenPipeError, + trio.BrokenResourceError, + ): + log.warning( + 'Failed to return result:\n' + f'{func}@{actor.uid}\n' + f'remote chan: {chan.uid}' + ) + +@acm +async def _errors_relayed_via_ipc( + actor: Actor, + chan: Channel, + ctx: Context, + is_rpc: bool, + + hide_tb: bool = False, + debug_kbis: bool = False, + task_status: TaskStatus[ + Context | BaseException + ] = trio.TASK_STATUS_IGNORED, + +) -> None: + __tracebackhide__: bool = hide_tb # TODO: use hide_tb here? + try: + yield # run RPC invoke body + + # box and ship RPC errors for wire-transit via + # the task's requesting parent IPC-channel. + except ( + Exception, + BaseExceptionGroup, + KeyboardInterrupt, + ) as err: + + # always hide this frame from debug REPL if the crash + # originated from an rpc task and we DID NOT fail due to + # an IPC transport error! + if ( + is_rpc + and chan.connected() + ): + __tracebackhide__: bool = hide_tb + + if not is_multi_cancelled(err): + + # TODO: maybe we'll want different "levels" of debugging + # eventualy such as ('app', 'supervisory', 'runtime') ? + + # if not isinstance(err, trio.ClosedResourceError) and ( + # if not is_multi_cancelled(err) and ( + + entered_debug: bool = False + if ( + ( + not isinstance(err, ContextCancelled) + or ( + isinstance(err, ContextCancelled) + and ctx._cancel_called + + # if the root blocks the debugger lock request from a child + # we will get a remote-cancelled condition. + and ctx._enter_debugger_on_cancel + ) + ) + and + ( + not isinstance(err, KeyboardInterrupt) + or ( + isinstance(err, KeyboardInterrupt) + and debug_kbis + ) + ) + ): + # await pause() + # XXX QUESTION XXX: is there any case where we'll + # want to debug IPC disconnects as a default? + # => I can't think of a reason that inspecting this + # type of failure will be useful for respawns or + # recovery logic - the only case is some kind of + # strange bug in our transport layer itself? Going + # to keep this open ended for now. + entered_debug = await _debug._maybe_enter_pm(err) + + if not entered_debug: + log.exception('Actor crashed:\n') + + # always ship errors back to caller + err_msg: dict[str, dict] = pack_error( + err, + # tb=tb, # TODO: special tb fmting? + cid=ctx.cid, + ) + + if is_rpc: + try: + await chan.send(err_msg) + + # TODO: tests for this scenario: + # - RPC caller closes connection before getting a response + # should **not** crash this actor.. + except ( + trio.ClosedResourceError, + trio.BrokenResourceError, + BrokenPipeError, + ) as ipc_err: + + # if we can't propagate the error that's a big boo boo + log.exception( + f"Failed to ship error to caller @ {chan.uid} !?\n" + f'{ipc_err}' + + ) + + # error is probably from above coro running code *not from + # the target rpc invocation since a scope was never + # allocated around the coroutine await. + if ctx._scope is None: + # we don't ever raise directly here to allow the + # msg-loop-scheduler to continue running for this + # channel. + task_status.started(err) + + # always reraise KBIs so they propagate at the sys-process + # level. + if isinstance(err, KeyboardInterrupt): + raise + + + # RPC task bookeeping + finally: + try: + ctx, func, is_complete = actor._rpc_tasks.pop( + (chan, ctx.cid) + ) + is_complete.set() + + except KeyError: + if is_rpc: + # If we're cancelled before the task returns then the + # cancel scope will not have been inserted yet + log.warning( + 'RPC task likely errored or cancelled before start?' + f'|_{ctx._task}\n' + f' >> {ctx.repr_rpc}\n' + ) + else: + log.cancel( + 'Failed to de-alloc internal runtime cancel task?\n' + f'|_{ctx._task}\n' + f' >> {ctx.repr_rpc}\n' + ) + + finally: + if not actor._rpc_tasks: + log.runtime("All RPC tasks have completed") + actor._ongoing_rpc_tasks.set() + + async def _invoke( actor: 'Actor', @@ -112,6 +393,8 @@ async def _invoke( kwargs: dict[str, Any], is_rpc: bool = True, + hide_tb: bool = True, + task_status: TaskStatus[ Union[Context, BaseException] ] = trio.TASK_STATUS_IGNORED, @@ -124,15 +407,15 @@ async def _invoke( remotely invoked function, normally in `Actor._service_n: Nursery`. ''' + __tracebackhide__: bool = hide_tb treat_as_gen: bool = False - failed_resp: bool = False # possibly a traceback (not sure what typing is for this..) tb = None cancel_scope = CancelScope() # activated cancel scope ref - cs: CancelScope | None = None + cs: CancelScope|None = None ctx = actor.get_context( chan=chan, @@ -153,6 +436,7 @@ async def _invoke( # compat with old api kwargs['ctx'] = ctx + treat_as_gen = True if 'ctx' in params: warnings.warn( @@ -167,7 +451,6 @@ async def _invoke( assert 'stream' in params kwargs['stream'] = ctx - treat_as_gen = True elif getattr(func, '_tractor_context_function', False): # handle decorated ``@tractor.context`` async function @@ -175,65 +458,45 @@ async def _invoke( context = True # errors raised inside this block are propgated back to caller - try: + async with _errors_relayed_via_ipc( + actor, + chan, + ctx, + is_rpc, + hide_tb=hide_tb, + task_status=task_status, + ): if not ( inspect.isasyncgenfunction(func) or inspect.iscoroutinefunction(func) ): raise TypeError(f'{func} must be an async function!') + # init coroutine with `kwargs` to immediately catch any + # type-sig errors. try: coro = func(**kwargs) except TypeError: raise - # TODO: can we unify this with the `context=True` impl below? - if inspect.isasyncgen(coro): - await chan.send({'functype': 'asyncgen', 'cid': cid}) - # XXX: massive gotcha! If the containing scope - # is cancelled and we execute the below line, - # any ``ActorNursery.__aexit__()`` WON'T be - # triggered in the underlying async gen! So we - # have to properly handle the closing (aclosing) - # of the async gen in order to be sure the cancel - # is propagated! - with cancel_scope as cs: - ctx._scope = cs - task_status.started(ctx) - async with aclosing(coro) as agen: - async for item in agen: - # TODO: can we send values back in here? - # it's gonna require a `while True:` and - # some non-blocking way to retrieve new `asend()` - # values from the channel: - # to_send = await chan.recv_nowait() - # if to_send is not None: - # to_yield = await coro.asend(to_send) - await chan.send({'yield': item, 'cid': cid}) - - log.runtime(f"Finished iterating {coro}") - # TODO: we should really support a proper - # `StopAsyncIteration` system here for returning a final - # value if desired - await chan.send({'stop': True, 'cid': cid}) - - # one way @stream func that gets treated like an async gen - # TODO: can we unify this with the `context=True` impl below? - elif treat_as_gen: - await chan.send({'functype': 'asyncgen', 'cid': cid}) - # 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) - await coro - - if not cs.cancelled_caught: - # task was not cancelled so we can instruct the - # far end async gen to tear down - await chan.send({'stop': True, 'cid': cid}) + # TODO: implement all these cases in terms of the + # `Context` one! + if not context: + await _invoke_non_context( + actor, + cancel_scope, + ctx, + cid, + chan, + func, + coro, + kwargs, + treat_as_gen, + is_rpc, + task_status, + ) + # below is only for `@context` funcs + return # our most general case: a remote SC-transitive, # IPC-linked, cross-actor-task "context" @@ -249,77 +512,53 @@ async def _invoke( # here and awaited directly, possibly just with a small # wrapper that calls `Context.started()` and then does # the `await coro()`? - elif context: - # a "context" endpoint type is the most general and - # "least sugary" type of RPC ep with support for - # bi-dir streaming B) - await chan.send({ - 'functype': 'context', - 'cid': cid - }) + # a "context" endpoint type is the most general and + # "least sugary" type of RPC ep with support for + # bi-dir streaming B) + await chan.send({ + 'functype': 'context', + 'cid': cid + }) - try: - async with trio.open_nursery() as nurse: - ctx._scope_nursery = nurse - ctx._scope = nurse.cancel_scope - task_status.started(ctx) + # TODO: should we also use an `.open_context()` equiv + # for this callee side by factoring the impl from + # `Portal.open_context()` into a common helper? + # + # NOTE: there are many different ctx state details + # in a callee side instance according to current impl: + # - `.cancelled_caught` can never be `True`. + # -> the below scope is never exposed to the + # `@context` marked RPC function. + # - `._portal` is never set. + try: + async with trio.open_nursery() as tn: + ctx._scope_nursery = tn + ctx._scope = tn.cancel_scope + task_status.started(ctx) - # TODO: should would be nice to have our - # `TaskMngr` nursery here! - res: Any = await coro - ctx._result = res + # TODO: should would be nice to have our + # `TaskMngr` nursery here! + res: Any = await coro + ctx._result = res - # deliver final result to caller side. - await chan.send({ - 'return': res, - 'cid': cid - }) + # deliver final result to caller side. + await chan.send({ + 'return': res, + 'cid': cid + }) - # XXX: do we ever trigger this block any more? - except ( - BaseExceptionGroup, - trio.Cancelled, - ) as scope_error: - - # always set this (callee) side's exception as the - # local error on the context - ctx._local_error: BaseException = scope_error - - # if a remote error was set then likely the - # exception group was raised due to that, so - # and we instead raise that error immediately! - if re := ctx._remote_error: - ctx._maybe_raise_remote_err(re) - - # maybe TODO: pack in - # ``trio.Cancelled.__traceback__`` here so they can - # be unwrapped and displayed on the caller side? - raise - - finally: - # XXX: only pop the context tracking if - # a ``@tractor.context`` entrypoint was called - assert chan.uid - - # don't pop the local context until we know the - # associated child isn't in debug any more - await _debug.maybe_wait_for_debugger() - ctx: Context = actor._contexts.pop( - (chan.uid, cid) - ) - - res_str: str = ( - 'error: {ctx._local_error}' - if ctx._local_error - else f'result: {ctx._result}' - ) - log.cancel( - f'IPC context terminated with final {res_str}\n\n' - f'|_{pformat(ctx)}\n' - ) - - if ctx.cancelled_caught: + # NOTE: this happens IFF `ctx._scope.cancel()` is + # called by any of, + # - *this* callee task manually calling `ctx.cancel()`. + # - the runtime calling `ctx._deliver_msg()` which + # itself calls `ctx._maybe_cancel_and_set_remote_error()` + # which cancels the scope presuming the input error + # is not a `.cancel_acked` pleaser. + # - currently a never-should-happen-fallthrough case + # inside ._context._drain_to_final_msg()`.. + # # TODO: remove this ^ right? + if ctx._scope.cancelled_caught: # first check for and raise any remote error # before raising any context cancelled case @@ -328,7 +567,6 @@ async def _invoke( if re := ctx._remote_error: ctx._maybe_raise_remote_err(re) - # fname: str = func.__name__ cs: CancelScope = ctx._scope if cs.cancel_called: our_uid: tuple = actor.uid @@ -375,7 +613,16 @@ async def _invoke( div_str + f'<= canceller: {canceller}\n' f'=> uid: {our_uid}\n' - f' |_{ctx._task}()\n' + f' |_{ctx._task}()' + + # TODO: instead just show the + # ctx.__str__() here? + # -[ ] textwrap.indent() it correctly! + # -[ ] BUT we need to wait until + # the state is filled out before emitting + # this msg right ow its kinda empty? bleh.. + # + # f' |_{ctx}' ) # TODO: does this ever get set any more or can @@ -384,7 +631,7 @@ async def _invoke( msg += ( # '------ - ------\n' # 'IPC msg:\n' - f'\n{ctx._cancel_msg}' + f'\n\n{ctx._cancel_msg}' ) # task-contex was either cancelled by request using @@ -392,180 +639,68 @@ async def _invoke( # on the far end, or it was cancelled by the local # (callee) task, so relay this cancel signal to the # other side. - raise ContextCancelled( + ctxc = ContextCancelled( msg, suberror_type=trio.Cancelled, canceller=canceller, ) + # assign local error so that the `.outcome` + # resolves to an error for both reporting and + # state checks. + ctx._local_error = ctxc + raise ctxc - # 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. - else: - try: - await chan.send({ - 'functype': 'asyncfunc', - 'cid': cid - }) - except ( - trio.ClosedResourceError, - trio.BrokenResourceError, - BrokenPipeError, - ) as ipc_err: - failed_resp = True - if is_rpc: - raise - else: - # TODO: should this be an `.exception()` call? - log.warning( - f'Failed to respond to non-rpc request: {func}\n' - f'{ipc_err}' - ) + # XXX: do we ever trigger this block any more? + except ( + BaseExceptionGroup, + trio.Cancelled, + BaseException, - with cancel_scope as cs: - ctx._scope: CancelScope = cs - task_status.started(ctx) - result = await coro - fname: str = func.__name__ - log.runtime( - 'RPC complete:\n' - f'task: {ctx._task}\n' - f'|_cid={ctx.cid}\n' - f'|_{fname}() -> {pformat(result)}\n' - ) + ) as scope_error: - # NOTE: only send result if we know IPC isn't down - if ( - not failed_resp - and chan.connected() - ): - try: - await chan.send( - {'return': result, - 'cid': cid} - ) - except ( - BrokenPipeError, - trio.BrokenResourceError, - ): - log.warning( - 'Failed to return result:\n' - f'{func}@{actor.uid}\n' - f'remote chan: {chan.uid}' - ) + # always set this (callee) side's exception as the + # local error on the context + ctx._local_error: BaseException = scope_error - except ( - Exception, - BaseExceptionGroup, - ) as err: + # if a remote error was set then likely the + # exception group was raised due to that, so + # and we instead raise that error immediately! + ctx.maybe_raise() - # always hide this frame from debug REPL if the crash - # originated from an rpc task and we DID NOT fail - # due to an IPC transport error! - if ( - is_rpc - and chan.connected() - ): - __tracebackhide__: bool = True - - if not is_multi_cancelled(err): - - # TODO: maybe we'll want different "levels" of debugging - # eventualy such as ('app', 'supervisory', 'runtime') ? - - # if not isinstance(err, trio.ClosedResourceError) and ( - # if not is_multi_cancelled(err) and ( - - entered_debug: bool = False - if ( - not isinstance(err, ContextCancelled) - or ( - isinstance(err, ContextCancelled) - and ctx._cancel_called - - # if the root blocks the debugger lock request from a child - # we will get a remote-cancelled condition. - and ctx._enter_debugger_on_cancel - ) - ): - # XXX QUESTION XXX: is there any case where we'll - # want to debug IPC disconnects as a default? - # => I can't think of a reason that inspecting this - # type of failure will be useful for respawns or - # recovery logic - the only case is some kind of - # strange bug in our transport layer itself? Going - # to keep this open ended for now. - entered_debug = await _debug._maybe_enter_pm(err) - - if not entered_debug: - log.exception("Actor crashed:") - - # always ship errors back to caller - err_msg: dict[str, dict] = pack_error( - err, - # tb=tb, # TODO: special tb fmting? - cid=cid, - ) - - if is_rpc: - try: - await chan.send(err_msg) - - # TODO: tests for this scenario: - # - RPC caller closes connection before getting a response - # should **not** crash this actor.. - except ( - trio.ClosedResourceError, - trio.BrokenResourceError, - BrokenPipeError, - ) as ipc_err: - - # if we can't propagate the error that's a big boo boo - log.exception( - f"Failed to ship error to caller @ {chan.uid} !?\n" - f'{ipc_err}' - - ) - - # error is probably from above coro running code *not from the - # underlyingn rpc invocation* since a scope was never allocated - # around actual coroutine await. - if ctx._scope is None: - # we don't ever raise directly here to allow the - # msg-loop-scheduler to continue running for this - # channel. - task_status.started(err) - - finally: - # RPC task bookeeping - try: - ctx, func, is_complete = actor._rpc_tasks.pop( - (chan, cid) - ) - is_complete.set() - - except KeyError: - if is_rpc: - # If we're cancelled before the task returns then the - # cancel scope will not have been inserted yet - log.warning( - f"Task {func} likely errored or cancelled before start") - else: - log.cancel( - 'Failed to de-alloc internal task!?\n' - f'cid: {cid}\n' - f'{func.__name__}({kwargs})' - ) + # maybe TODO: pack in come kinda + # `trio.Cancelled.__traceback__` here so they can be + # unwrapped and displayed on the caller side? no se.. + raise + # `@context` entrypoint task bookeeping. + # i.e. only pop the context tracking if used ;) finally: - if not actor._rpc_tasks: - log.runtime("All RPC tasks have completed") - actor._ongoing_rpc_tasks.set() + assert chan.uid + + # don't pop the local context until we know the + # associated child isn't in debug any more + await _debug.maybe_wait_for_debugger() + ctx: Context = actor._contexts.pop( + (chan.uid, cid) + ) + + merr: Exception|None = ctx.maybe_error + + ( + res_type_str, + res_str, + ) = ( + ('error', f'{type(merr)}',) + if merr + else ( + 'result', + f'`{repr(ctx.outcome)}`', + ) + ) + log.cancel( + f'IPC context terminated with a final {res_type_str}\n\n' + f'{ctx}\n' + ) def _get_mod_abspath(module): @@ -817,19 +952,33 @@ class Actor: stream: trio.SocketStream, ) -> None: - """Entry point for new inbound connections to the channel server. - - """ - self._no_more_peers = trio.Event() # unset + ''' + Entry point for new inbound connections to the channel server. + ''' + self._no_more_peers = trio.Event() # unset by making new chan = Channel.from_stream(stream) - uid: tuple[str, str] | None = chan.uid - log.runtime(f"New connection to us {chan}") + their_uid: tuple[str, str]|None = chan.uid + con_msg: str = '' + if their_uid: + # NOTE: `.uid` is only set after first contact + con_msg = ( + 'IPC Re-connection from already known peer? ' + ) + else: + con_msg = ( + 'New IPC connection to us ' + ) + + con_msg += ( + f'<= @{chan.raddr}\n' + f'|_{chan}\n' + # f' |_@{chan.raddr}\n\n' + ) # send/receive initial handshake response try: - uid = await self._do_handshake(chan) - + uid: tuple|None = await self._do_handshake(chan) except ( # we need this for ``msgspec`` for some reason? # for now, it's been put in the stream backend. @@ -844,44 +993,66 @@ class Actor: # 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.warning(f"Channel {chan} failed to handshake") + log.warning( + con_msg + + + ' -> But failed to handshake? Ignoring..\n' + ) return - # channel tracking + con_msg += ( + f' -> Handshake with actor `{uid[0]}[{uid[1][-6:]}]` complete\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 + # by a call to `.wait_for_peer()`. + # - if a peer is connecting no such event will exit. event: trio.Event|None = self._peer_connected.pop( uid, None, ) if event: - # Instructing connection: this is likely a new channel to - # a recently spawned actor which we'd like to control via - # async-rpc calls. - log.runtime(f"Waking channel waiters {event.statistics()}") - # Alert any task waiting on this connection to come up + con_msg += ( + ' -> Waking subactor spawn waiters: ' + f'{event.statistics().tasks_waiting}\n' + f' -> Registered IPC chan for child actor {uid}@{chan.raddr}\n' + # f' {event}\n' + # f' |{event.statistics()}\n' + ) + # wake tasks waiting on this IPC-transport "connect-back" event.set() + else: + con_msg += ( + f' -> Registered IPC chan for peer actor {uid}@{chan.raddr}\n' + ) # type: ignore + chans: list[Channel] = self._peers[uid] - if chans: - # TODO: re-use channels for new connections instead - # of always new ones? - # => will require changing all the discovery funcs.. - log.runtime( - f"already have channel(s) for {uid}:{chans}?" - ) + # if chans: + # # TODO: re-use channels for new connections instead + # # of always new ones? + # # => will require changing all the discovery funcs.. # append new channel - log.runtime(f"Registered {chan} for {uid}") # type: ignore # TODO: can we just use list-ref directly? - # chans.append(chan) - self._peers[uid].append(chan) + chans.append(chan) + + log.runtime(con_msg) # Begin channel management - respond to remote requests and # process received reponses. disconnected: bool = False try: - disconnected: bool = await process_messages(self, chan) + disconnected: bool = await process_messages( + self, + chan, + ) except trio.Cancelled: - log.cancel(f'Msg loop was cancelled for {chan}') + log.cancel( + 'IPC transport msg loop was cancelled for \n' + f'|_{chan}\n' + ) raise finally: @@ -895,7 +1066,10 @@ class Actor: # moving on with closing our own side. if local_nursery: if chan._cancel_called: - log.cancel(f'Waiting on cancel request to peer {chan.uid}') + log.cancel( + 'Waiting on cancel request to peer\n' + f'`Portal.cancel_actor()` => {chan.uid}\n' + ) # XXX: this is a soft wait on the channel (and its # underlying transport protocol) to close from the @@ -908,10 +1082,13 @@ class Actor: # loop processing. with trio.move_on_after(0.5) as cs: cs.shield = True - # Attempt to wait for the far end to close the channel - # and bail after timeout (2-generals on closure). - assert chan.msgstream - async for msg in chan.msgstream.drain(): + + # attempt to wait for the far end to close the + # channel and bail after timeout (a 2-generals + # problem on closure). + assert chan.transport + async for msg in chan.transport.drain(): + # try to deliver any lingering msgs # before we destroy the channel. # This accomplishes deterministic @@ -923,7 +1100,7 @@ class Actor: 'Draining msg from disconnected peer\n' f'{chan.uid}\n' f'|_{chan}\n' - f' |_{chan.msgstream}\n\n' + f' |_{chan.transport}\n\n' f'{pformat(msg)}\n' ) @@ -936,11 +1113,30 @@ class Actor: msg, ) - log.runtime( - 'Waiting on local actor nursery to exit..\n' + # NOTE: when no call to `open_root_actor()` was + # made, we implicitly make that call inside + # the first `.open_nursery()`, in this case we + # can assume that we are the root actor and do + # not have to wait for the nursery-enterer to + # exit before shutting down the actor runtime. + # + # see matching note inside `._supervise.open_nursery()` + if not local_nursery._implicit_runtime_started: + log.runtime( + 'Waiting on local actor nursery to exit..\n' + f'|_{local_nursery}\n' + ) + await local_nursery.exited.wait() + + if ( + cs.cancelled_caught + and not local_nursery._implicit_runtime_started + ): + log.warning( + 'Failed to exit local actor nursery?\n' f'|_{local_nursery}\n' ) - await local_nursery.exited.wait() + # await _debug.pause() if disconnected: # if the transport died and this actor is still @@ -957,7 +1153,7 @@ class Actor: log.cancel( f'Peer IPC broke but subproc is alive?\n\n' - f'<=x @{chan.raddr}\n' + f'<=x {chan.uid}@{chan.raddr}\n' f' |_{proc}\n' ) @@ -968,9 +1164,9 @@ class Actor: f'uid: {chan.uid}\n' f'|_{pformat(chan)}\n' ) - chans = self._peers.get(chan.uid) chans.remove(chan) + # TODO: do we need to be this pedantic? if not chans: log.runtime( f'No more channels with {chan.uid}' @@ -980,7 +1176,7 @@ class Actor: peers_str: str = '' for uid, chans in self._peers.items(): peers_str += ( - f'- uid: {uid}\n' + f'|_ uid: {uid}\n' ) for i, chan in enumerate(chans): peers_str += ( @@ -1406,22 +1602,27 @@ class Actor: requesting_uid, requester_type, req_chan, + log_meth, ) = ( req_chan.uid, 'peer', req_chan, + log.cancel, ) if req_chan else ( # a self cancel of ALL rpc tasks self.uid, 'self', - self + self, + log.runtime, ) + # TODO: just use the new `Context.repr_rpc: str` (and + # other) repr fields instead of doing this all manual.. msg: str = ( - f'`Actor.cancel()` request from {requester_type}:\n' - f'<= {requesting_uid}\n' + f'Runtime cancel request from {requester_type}:\n\n' + f'<= .cancel(): {requesting_uid}\n' ) # TODO: what happens here when we self-cancel tho? @@ -1460,7 +1661,7 @@ class Actor: if self._service_n: self._service_n.cancel_scope.cancel() - log.cancel(msg) + log_meth(msg) self._cancel_complete.set() return True @@ -1523,20 +1724,23 @@ class Actor: return True log.cancel( - 'Cancel request for RPC task\n' - f'<= canceller: {requesting_uid}\n\n' + 'Cancel request for RPC task\n\n' + f'<= ._cancel_task(): {requesting_uid}\n' + f' |_ @{ctx.dmaddr}\n\n' # TODO: better ascii repr for "supervisor" like # a nursery or context scope? - f'=> ipc-parent: {parent_chan}\n' + # f'=> {parent_chan}\n' + f'=> {ctx._task}\n' # TODO: simplified `Context.__repr__()` fields output # shows only application state-related stuff like, # - ._stream # - .closed # - .started_called # - .. etc. - f' |_ctx: {cid}\n' - f' >> {ctx._nsf}()\n' + f' >> {ctx.repr_rpc}\n' + # f' |_ctx: {cid}\n' + # f' >> {ctx._nsf}()\n' ) if ( ctx._canceller is None @@ -1589,7 +1793,7 @@ class Actor: ''' tasks: dict = self._rpc_tasks if not tasks: - log.warning( + log.runtime( 'Actor has no cancellable RPC tasks?\n' f'<= canceller: {req_uid}\n' ) @@ -1619,11 +1823,17 @@ class Actor: f' |>> {ctx._nsf}() -> dict:\n' ) + descr: str = ( + 'all' if not parent_chan + else + "IPC channel's " + ) + log.cancel( - f'Cancelling all {len(tasks)} rpc tasks:\n\n' - f'<= .cancel() from {req_uid}\n' - f'{self}\n' - f'{tasks_str}' + f'Cancelling {descr} {len(tasks)} rpc tasks\n\n' + f'<= .cancel_rpc_tasks(): {req_uid}\n' + # f'{self}\n' + # f'{tasks_str}' ) for ( (task_caller_chan, cid), @@ -1652,10 +1862,11 @@ class Actor: requesting_uid=req_uid, ) - log.cancel( - 'Waiting for remaining rpc tasks to complete\n' - f'|_{tasks}' - ) + if tasks: + log.cancel( + 'Waiting for remaining rpc tasks to complete\n' + f'|_{tasks}' + ) await self._ongoing_rpc_tasks.wait() def cancel_server(self) -> None: @@ -1698,21 +1909,21 @@ class Actor: ) -> tuple[str, str]: ''' - Exchange (name, UUIDs) identifiers as the first communication step. + Exchange `(name, UUIDs)` identifiers as the first + communication step. - These are essentially the "mailbox addresses" found in actor model - parlance. + These are essentially the "mailbox addresses" found in + actor model parlance. ''' await chan.send(self.uid) - value = await chan.recv() + value: tuple = await chan.recv() uid: tuple[str, str] = (str(value[0]), str(value[1])) if not isinstance(uid, tuple): raise ValueError(f"{uid} is not a valid uid?!") chan.uid = str(uid[0]), str(uid[1]) - log.runtime(f"Handshake with actor {uid}@{chan.raddr} complete") return uid def is_infected_aio(self) -> bool: @@ -1832,7 +2043,10 @@ async def async_main( shield=True, ) ) - log.runtime("Waiting on service nursery to complete") + log.runtime( + 'Actor runtime is up!' + # 'Blocking on service nursery to exit..\n' + ) log.runtime( "Service nursery complete\n" "Waiting on root nursery to complete" @@ -1876,11 +2090,13 @@ async def async_main( raise finally: - log.info("Runtime nursery complete") - + log.runtime( + 'Runtime nursery complete' + '-> Closing all actor lifetime contexts..' + ) # tear down all lifetime contexts if not in guest mode # XXX: should this just be in the entrypoint? - log.info("Closing all actor lifetime contexts") + actor.lifetime_stack.close() # TODO: we can't actually do this bc the debugger # uses the _service_n to spawn the lock task, BUT, @@ -1891,9 +2107,7 @@ async def async_main( # with CancelScope(shield=True): # await _debug.breakpoint() - actor.lifetime_stack.close() - - # Unregister actor from the arbiter + # Unregister actor from the registry-sys / registrar. if ( registered_with_arbiter and not actor.is_arbiter @@ -2095,13 +2309,14 @@ async def process_messages( 'parent_chan': chan, 'requesting_uid': chan.uid, } - log.cancel( - f'Rx task cancel request\n' - f'<= canceller: {chan.uid}\n' - f' |_{chan}\n\n' - f'=> {actor}\n' - f' |_cid: {target_cid}\n' - ) + # TODO: remove? already have emit in meth. + # log.runtime( + # f'Rx RPC task cancel request\n' + # f'<= canceller: {chan.uid}\n' + # f' |_{chan}\n\n' + # f'=> {actor}\n' + # f' |_cid: {target_cid}\n' + # ) try: await _invoke( actor, @@ -2360,6 +2575,11 @@ class Arbiter(Actor): sockaddr: tuple[str, int] for (aname, _), sockaddr in self._registry.items(): + log.info( + f'Actor mailbox info:\n' + f'aname: {aname}\n' + f'sockaddr: {sockaddr}\n' + ) if name == aname: sockaddrs.append(sockaddr) From 18792432578872134b78749c0fc75ac76671c03b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 5 Mar 2024 10:34:32 -0500 Subject: [PATCH 110/146] Flip rpc tests over to use `ExceptionGroup` on new `trio` --- tests/test_advanced_streaming.py | 2 +- tests/test_child_manages_service_nursery.py | 1 - tests/test_rpc.py | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/test_advanced_streaming.py b/tests/test_advanced_streaming.py index 8061c3b9..e8696346 100644 --- a/tests/test_advanced_streaming.py +++ b/tests/test_advanced_streaming.py @@ -329,7 +329,7 @@ async def inf_streamer( # close out the stream gracefully except trio.ClosedResourceError: - print('msgstream closed on streamer side!') + print('transport closed on streamer side!') assert stream.closed break else: diff --git a/tests/test_child_manages_service_nursery.py b/tests/test_child_manages_service_nursery.py index fd1ceb80..228d6ade 100644 --- a/tests/test_child_manages_service_nursery.py +++ b/tests/test_child_manages_service_nursery.py @@ -10,7 +10,6 @@ from contextlib import asynccontextmanager as acm import pytest import trio -from trio_typing import TaskStatus import tractor from tractor import RemoteActorError from async_generator import aclosing diff --git a/tests/test_rpc.py b/tests/test_rpc.py index 7ede231b..972cae17 100644 --- a/tests/test_rpc.py +++ b/tests/test_rpc.py @@ -112,7 +112,7 @@ def test_rpc_errors(reg_addr, to_call, testdir): value = err.value # might get multiple `trio.Cancelled`s as well inside an inception - if isinstance(value, trio.MultiError): + if isinstance(value, ExceptionGroup): value = next(itertools.dropwhile( lambda exc: not isinstance(exc, tractor.RemoteActorError), value.exceptions From 885ba04908b09301fce6e1e55ca205ec92519491 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 5 Mar 2024 12:26:33 -0500 Subject: [PATCH 111/146] Tweak `._portal` log content to use `Context.repr_outcome()` --- tractor/_portal.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/tractor/_portal.py b/tractor/_portal.py index 041c8104..bd23a6e4 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -257,7 +257,7 @@ class Portal: return False reminfo: str = ( - f'{self.channel.uid}\n' + f'`Portal.cancel_actor()` => {self.channel.uid}\n' f' |_{chan}\n' ) log.cancel( @@ -949,9 +949,13 @@ class Portal: # CASE 1 else: + outcome_str: str = ctx.repr_outcome( + show_error_fields=True, + # type_only=True, + ) log.cancel( - f'Context terminated due to local scope error:\n' - f'{etype.__name__}\n' + f'Context terminated due to local scope error:\n\n' + f'{ctx.chan.uid} => {outcome_str}\n' ) # XXX: (MEGA IMPORTANT) if this is a root opened process we From ffed35e263124cc75c17fc0fddc0196f9a8e0e2c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 5 Mar 2024 12:30:09 -0500 Subject: [PATCH 112/146] `._entry`: use same msg info in start/terminate log --- tractor/_entry.py | 18 ++++++++++++++---- tractor/_root.py | 4 +++- 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/tractor/_entry.py b/tractor/_entry.py index 23dd05b6..a59975ce 100644 --- a/tractor/_entry.py +++ b/tractor/_entry.py @@ -116,14 +116,18 @@ def _trio_main( if actor.loglevel is not None: get_console_log(actor.loglevel) import os - log.info( - 'Started new trio process:\n' + actor_info: str = ( f'|_{actor}\n' f' uid: {actor.uid}\n' f' pid: {os.getpid()}\n' f' parent_addr: {parent_addr}\n' f' loglevel: {actor.loglevel}\n' ) + log.info( + 'Started new trio process:\n' + + + actor_info + ) try: if infect_asyncio: @@ -133,8 +137,14 @@ def _trio_main( trio.run(trio_main) except KeyboardInterrupt: log.cancel( - f'@{actor.uid} received KBI' + 'Actor received KBI\n' + + + actor_info ) finally: - log.info(f"Actor {actor.uid} terminated") + log.info( + 'Actor terminated\n' + + + actor_info + ) diff --git a/tractor/_root.py b/tractor/_root.py index 88df774f..7c951df4 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -256,7 +256,9 @@ async def open_root_actor( # for an in nurseries: # tempn.start_soon(an.exited.wait) - logger.cancel("Shutting down root actor") + logger.info( + 'Closing down root actor' + ) await actor.cancel(None) # self cancel finally: _state._current_actor = None From 59d6d0cd7fd5498452a600c62fc9306560eec9a1 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 6 Mar 2024 09:48:46 -0500 Subject: [PATCH 113/146] Woops, add `.msg` sub-pkg to install set --- setup.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/setup.py b/setup.py index d26deb9b..b8f915ff 100755 --- a/setup.py +++ b/setup.py @@ -36,8 +36,9 @@ setup( platforms=['linux', 'windows'], packages=[ 'tractor', - 'tractor.experimental', - 'tractor.trionics', + 'tractor.experimental', # wacky ideas + 'tractor.trionics', # trio extensions + 'tractor.msg', # lowlevel data types ], install_requires=[ From 7fb1c45ac7cc5b11fea9a6e3d747feb37a45394d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 6 Mar 2024 10:13:41 -0500 Subject: [PATCH 114/146] Tweak inter-peer tests for new/refined semantics Buncha subtle details changed mostly to do with when `Context.cancel()` gets called on "real" remote errors vs. (peer requested) cancellation and then local side handling of `ContextCancelled`. Specific changes to make tests pass: - due to raciness with `sleeper_ctx.result()` raising the ctxc locally vs. the child-peers receiving similar ctxcs themselves (and then erroring and propagating back to the root parent), we might not see `._remote_error` set during the sub-ctx loops (except for the sleeper itself obvi). - do not expect `.cancel_called`/`.cancel_caught` to be set on any sub-ctx since currently `Context.cancel()` is only called non-shielded and thus is not in invoked when `._scope.cancel()` is called as part of each root-side ctx ref/block handling the inter-peer ctxc. - do not expect `Context._scope.cancelled_caught` to be set in most cases (even the sleeper) TODO Outstanding adjustments not fixed yet: -[ ] `_scope.cancelled_caught` checks outside the `.open_context()` blocks. --- tests/test_inter_peer_cancellation.py | 97 +++++++++++++++++---------- 1 file changed, 62 insertions(+), 35 deletions(-) diff --git a/tests/test_inter_peer_cancellation.py b/tests/test_inter_peer_cancellation.py index 082c5e65..81e8afa6 100644 --- a/tests/test_inter_peer_cancellation.py +++ b/tests/test_inter_peer_cancellation.py @@ -220,11 +220,12 @@ async def stream_from_peer( # - what about IPC-transport specific errors, should # they bubble from the async for and trigger # other special cases? + # # NOTE: current ctl flow: # - stream raises `trio.EndOfChannel` and # exits the loop - # - `.open_context()` will raise the ctxcanc - # received from the sleeper. + # - `.open_context()` will raise the ctxc received + # from the sleeper. async for msg in stream: assert msg is not None print(msg) @@ -383,11 +384,11 @@ def test_peer_canceller( ) as (canceller_ctx, sent), ): - ctxs: list[Context] = [ - sleeper_ctx, - caller_ctx, - canceller_ctx, - ] + ctxs: dict[str, Context] = { + 'sleeper': sleeper_ctx, + 'caller': caller_ctx, + 'canceller': canceller_ctx, + } try: print('PRE CONTEXT RESULT') @@ -505,14 +506,17 @@ def test_peer_canceller( # NOTE: this root actor task should have # called `Context.cancel()` on the # `.__aexit__()` to every opened ctx. - for ctx in ctxs: - assert ctx.cancel_called + for name, ctx in ctxs.items(): # this root actor task should have # cancelled all opened contexts except the # sleeper which is obvi by the "canceller" # peer. re = ctx._remote_error + le = ctx._local_error + + assert ctx.cancel_called + if ( ctx is sleeper_ctx or ctx is caller_ctx @@ -566,32 +570,43 @@ def test_peer_canceller( # the sleeper's remote error is the error bubbled # out of the context-stack above! - re = sleeper_ctx.outcome + final_err = sleeper_ctx.outcome assert ( - re is loc_err + final_err is loc_err is sleeper_ctx.maybe_error is sleeper_ctx._remote_error ) - for ctx in ctxs: + for name, ctx in ctxs.items(): + re: BaseException|None = ctx._remote_error - re: BaseException|None = ctx.outcome - assert ( - re and - ( - re is ctx.maybe_error - is ctx._remote_error - ) - ) - le: trio.MultiError = ctx._local_error + le: BaseException|None = ctx._local_error + err = ctx.maybe_error + out = ctx.outcome + + # every ctx should error! + assert out is err + + # the recorded local erro should always be + # the same as the one raised by the + # `sleeper_ctx.result()` call assert ( le - and ctx._local_error + and + le is loc_err ) # root doesn't cancel sleeper since it's # cancelled by its peer. if ctx is sleeper_ctx: + assert re + assert ( + ctx._remote_error + is ctx.maybe_error + is ctx.outcome + is ctx._local_error + ) + assert not ctx.cancel_called assert not ctx.cancel_acked @@ -601,21 +616,38 @@ def test_peer_canceller( # `ContextCancelled` for it and thus # the logic inside `.cancelled_caught` # should trigger! - assert ctx._scope.cancelled_caught + assert not ctx._scope.cancelled_caught - elif ctx is caller_ctx: + elif ctx in ( + caller_ctx, + canceller_ctx, + ): + + assert not ctx._remote_error + + # the `canceller_ctx` shouldn't + # have called `ctx.cancel()` either! + # # since its context was remotely - # cancelled, we never needed to - # call `Context.cancel()` bc it was - # done by the peer and also we never - assert ctx.cancel_called + # cancelled, we never needed to call + # `Context.cancel()` bc the far end + # task already done by the peer and + # also we never + assert not ctx.cancel_called # TODO: figure out the details of this..? # if you look the `._local_error` here # is a multi of ctxc + 2 Cancelleds? # assert not ctx.cancelled_caught - elif ctx is canceller_ctx: + assert ( + not ctx.cancel_called + and not ctx.cancel_acked + ) + assert not ctx._scope.cancelled_caught + + # elif ctx is canceller_ctx: + # assert not ctx._remote_error # XXX NOTE XXX: ONLY the canceller # will get a self-cancelled outcome @@ -626,11 +658,6 @@ def test_peer_canceller( # .cancel() whenever an interpeer # cancel takes place since each # reception of a ctxc - assert ( - ctx.cancel_called - and ctx.cancel_acked - ) - assert not ctx._scope.cancelled_caught else: pytest.fail( @@ -663,7 +690,7 @@ def test_peer_canceller( # `.open_context()` block has exited and should be # set in both outcomes including the case where # ctx-cancel handling itself errors. - assert sleeper_ctx._scope.cancelled_caught + assert not sleeper_ctx._scope.cancelled_caught assert _loc_err is sleeper_ctx._local_error assert ( sleeper_ctx.outcome From 9966dbdfc19cb789331364bca7afb3e4dd8d5c05 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 6 Mar 2024 16:07:30 -0500 Subject: [PATCH 115/146] Tweak inter-peer `._scope` state asserts We don't expect `._scope.cancelled_caught` to be set really ever on inter-peer cancellation since no ctx is ever cancelling itself, a peer cancels some other and then bubbles back to all other peers. Also add `ids: lambda` for `error_during_ctxerr_handling` param to `test_peer_canceller()` --- tests/test_inter_peer_cancellation.py | 44 ++++++++++++++++++++++----- 1 file changed, 36 insertions(+), 8 deletions(-) diff --git a/tests/test_inter_peer_cancellation.py b/tests/test_inter_peer_cancellation.py index 81e8afa6..d878b06d 100644 --- a/tests/test_inter_peer_cancellation.py +++ b/tests/test_inter_peer_cancellation.py @@ -292,6 +292,7 @@ async def stream_from_peer( @pytest.mark.parametrize( 'error_during_ctxerr_handling', [False, True], + ids=lambda item: f'rte_during_ctxerr={item}', ) def test_peer_canceller( error_during_ctxerr_handling: bool, @@ -492,6 +493,15 @@ def test_peer_canceller( # should be cancelled by US. # if error_during_ctxerr_handling: + # since we do a rte reraise above, the + # `.open_context()` error handling should have + # raised a local rte, thus the internal + # `.open_context()` enterer task's + # cancel-scope should have raised the RTE, NOT + # a `trio.Cancelled` due to a local + # `._scope.cancel()` call. + assert not sleeper_ctx._scope.cancelled_caught + assert isinstance(loc_err, RuntimeError) print(f'_loc_err: {_loc_err}\n') # assert sleeper_ctx._local_error is _loc_err @@ -558,6 +568,13 @@ def test_peer_canceller( # propagated # else: + # since sleeper_ctx.result() IS called above + # we should have (silently) absorbed the + # corresponding `ContextCancelled` for it and + # `._scope.cancel()` should never have been + # called. + assert not sleeper_ctx._scope.cancelled_caught + assert isinstance(loc_err, ContextCancelled) assert loc_err.canceller == sleeper_ctx.canceller assert ( @@ -625,20 +642,31 @@ def test_peer_canceller( assert not ctx._remote_error - # the `canceller_ctx` shouldn't - # have called `ctx.cancel()` either! + # neither of the `caller/canceller_ctx` should + # have called `ctx.cancel()` bc the + # canceller's task internally issues + # a `Portal.cancel_actor()` to the + # sleeper and thus never should call + # `ctx.cancel()` per say UNLESS the + # sleeper's `.result()` call above + # ctxc exception results in the + # canceller's + # `.open_context().__aexit__()` error + # handling to kick in BEFORE a remote + # error is delivered - which since + # we're asserting what we are above, + # that should normally be the case + # right? # - # since its context was remotely - # cancelled, we never needed to call - # `Context.cancel()` bc the far end - # task already done by the peer and - # also we never assert not ctx.cancel_called + # + # assert ctx.cancel_called + # orig ^ # TODO: figure out the details of this..? # if you look the `._local_error` here # is a multi of ctxc + 2 Cancelleds? - # assert not ctx.cancelled_caught + # assert not ctx._scope.cancelled_caught assert ( not ctx.cancel_called From c5228e7be52fadc2ef06c49a0c501ad85655b05f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 7 Mar 2024 18:24:00 -0500 Subject: [PATCH 116/146] Set `._cancel_msg` to RPC `{cmd: 'self._cancel_task', ..}` msg Like how we set `Context._cancel_msg` in `._deliver_msg()` (in which case normally it's an `{'error': ..}` msg), do the same when any RPC task is remotely cancelled via `Actor._cancel_task` where that task doesn't yet have a cancel msg set yet. This makes is much easier to distinguish between ctx cancellations due to some remote error vs. Explicit remote requests via any of `Actor.cancel()`, `Portal.cancel_actor()` or `Context.cancel()`. --- tractor/_runtime.py | 74 +++++++++++++++++++++++++++------------------ 1 file changed, 45 insertions(+), 29 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index abdaf91a..81bab1d5 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -299,7 +299,7 @@ async def _errors_relayed_via_ipc( ) ) ): - # await pause() + # await _debug.pause() # XXX QUESTION XXX: is there any case where we'll # want to debug IPC disconnects as a default? # => I can't think of a reason that inspecting this @@ -319,6 +319,12 @@ async def _errors_relayed_via_ipc( cid=ctx.cid, ) + # NOTE: the src actor should always be packed into the + # error.. but how should we verify this? + # assert err_msg['src_actor_uid'] + # if not err_msg['error'].get('src_actor_uid'): + # import pdbp; pdbp.set_trace() + if is_rpc: try: await chan.send(err_msg) @@ -559,6 +565,7 @@ async def _invoke( # inside ._context._drain_to_final_msg()`.. # # TODO: remove this ^ right? if ctx._scope.cancelled_caught: + our_uid: tuple = actor.uid # first check for and raise any remote error # before raising any context cancelled case @@ -568,8 +575,9 @@ async def _invoke( ctx._maybe_raise_remote_err(re) cs: CancelScope = ctx._scope + if cs.cancel_called: - our_uid: tuple = actor.uid + canceller: tuple = ctx.canceller msg: str = ( 'actor was cancelled by ' @@ -625,15 +633,6 @@ async def _invoke( # f' |_{ctx}' ) - # TODO: does this ever get set any more or can - # we remove it? - if ctx._cancel_msg: - msg += ( - # '------ - ------\n' - # 'IPC msg:\n' - f'\n\n{ctx._cancel_msg}' - ) - # task-contex was either cancelled by request using # ``Portal.cancel_actor()`` or ``Context.cancel()`` # on the far end, or it was cancelled by the local @@ -1677,7 +1676,9 @@ class Actor: self, cid: str, parent_chan: Channel, - requesting_uid: tuple[str, str] | None = None, + + requesting_uid: tuple[str, str]|None = None, + ipc_msg: dict|None|bool = False, ) -> bool: ''' @@ -1688,16 +1689,13 @@ class Actor: in the signature (for now). ''' - # this ctx based lookup ensures the requested task to - # be cancelled was indeed spawned by a request from - # this channel + + # this ctx based lookup ensures the requested task to be + # cancelled was indeed spawned by a request from its + # parent (or some grandparent's) channel ctx: Context func: Callable is_complete: trio.Event - - # NOTE: right now this is only implicitly called by - # streaming IPC but it should be called - # to cancel any remotely spawned task try: ( ctx, @@ -1725,20 +1723,23 @@ class Actor: log.cancel( 'Cancel request for RPC task\n\n' - f'<= ._cancel_task(): {requesting_uid}\n' - f' |_ @{ctx.dmaddr}\n\n' + f'<= Actor.cancel_task(): {requesting_uid}\n\n' + f'=> {ctx._task}\n' + f' |_ >> {ctx.repr_rpc}\n' + # f' >> Actor._cancel_task() => {ctx._task}\n' + # f' |_ {ctx._task}\n\n' # TODO: better ascii repr for "supervisor" like # a nursery or context scope? # f'=> {parent_chan}\n' - f'=> {ctx._task}\n' + # f' |_{ctx._task}\n' # TODO: simplified `Context.__repr__()` fields output # shows only application state-related stuff like, # - ._stream # - .closed # - .started_called # - .. etc. - f' >> {ctx.repr_rpc}\n' + # f' >> {ctx.repr_rpc}\n' # f' |_ctx: {cid}\n' # f' >> {ctx._nsf}()\n' ) @@ -1748,6 +1749,16 @@ class Actor: ): ctx._canceller: tuple = requesting_uid + # TODO: pack the RPC `{'cmd': }` msg into a ctxc and + # then raise and pack it here? + if ( + ipc_msg + and ctx._cancel_msg is None + ): + # assign RPC msg directly from the loop which usually + # the case with `ctx.cancel()` on the other side. + ctx._cancel_msg = ipc_msg + # don't allow cancelling this function mid-execution # (is this necessary?) if func is self._cancel_task: @@ -1828,10 +1839,15 @@ class Actor: else "IPC channel's " ) - + rent_chan_repr: str = ( + f'|_{parent_chan}' + if parent_chan + else '' + ) log.cancel( f'Cancelling {descr} {len(tasks)} rpc tasks\n\n' - f'<= .cancel_rpc_tasks(): {req_uid}\n' + f'<= `Actor.cancel_rpc_tasks()`: {req_uid}\n' + f' {rent_chan_repr}\n' # f'{self}\n' # f'{tasks_str}' ) @@ -1851,9 +1867,6 @@ class Actor: ): continue - # if func == self._cancel_task: - # continue - # TODO: this maybe block on the task cancellation # and so should really done in a nursery batch? await self._cancel_task( @@ -2198,6 +2211,8 @@ async def process_messages( await actor._cancel_task( cid, channel, + + ipc_msg=msg, ) break @@ -2308,6 +2323,7 @@ async def process_messages( # cancel it! 'parent_chan': chan, 'requesting_uid': chan.uid, + 'ipc_msg': msg, } # TODO: remove? already have emit in meth. # log.runtime( @@ -2575,7 +2591,7 @@ class Arbiter(Actor): sockaddr: tuple[str, int] for (aname, _), sockaddr in self._registry.items(): - log.info( + log.runtime( f'Actor mailbox info:\n' f'aname: {aname}\n' f'sockaddr: {sockaddr}\n' From 7b1528abede00ac8845433ae081870b8bed8f13d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 7 Mar 2024 20:35:43 -0500 Subject: [PATCH 117/146] (Event) more pedantic `.cancel_acked: bool` def Changes the condition logic to be more strict and moves it to a private `._is_self_cancelled() -> bool` predicate which can be used elsewhere (instead of having almost similar duplicate checks all over the place..) and allows taking in a specific `remote_error` just for verification purposes (like for tests). Main strictness distinctions are now: - obvi that `.cancel_called` is set (this filters any `Portal.cancel_actor()` or other out-of-band RPC), - the received `ContextCancelled` **must** have its `.canceller` set to this side's `Actor.uid` (indicating we are the requester). - `.src_actor_uid` **must** be the same as the `.chan.uid` (so the error must have originated from the opposite side's task. - `ContextCancelled.canceller` should be already set to the `.chan.uid` indicating we received the msg via the runtime calling `._deliver_msg()` -> `_maybe_cancel_and_set_remote_error()` which ensures the error is specifically destined for this ctx-task exactly the same as how `Actor._cancel_task()` sets it from an input `requesting_uid` arg. In support of the above adjust some impl deats: - add `Context._actor: Actor` which is set once in `mk_context()` to avoid issues (particularly in testing) where `current_actor()` raises after the root actor / runtime is already exited. Use `._actor.uid` in both `.cancel_acked` (obvi) and '_maybe_cancel_and_set_remote_error()` when deciding whether to call `._scope.cancel()`. - always cast `.canceller` to `tuple` if not null. - delegate `.cancel_acked` directly to new private predicate (obvi). - always set `._canceller` from any `RemoteActorError.src_actor_uid` or failing over to the `.chan.uid` when a non-remote error (tho that shouldn't ever happen right?). - more extensive doc-string for `.cancel()` detailing the new strictness rules about whether an eventual `.cancel_acked` might be set. Also tossed in even more logging format tweaks by adding a `type_only: bool` to `.repr_outcome()` as desired for simpler output in the `state: ` and `.repr_rpc()` sections of the `.__str__()`. --- tractor/_context.py | 285 ++++++++++++++++++++++++++++---------------- 1 file changed, 184 insertions(+), 101 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index f8aaf1c9..9179456b 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -364,6 +364,9 @@ class Context: ''' chan: Channel cid: str # "context id", more or less a unique linked-task-pair id + + _actor: Actor + # the "feeder" channels for delivering message values to the # local task from the runtime's msg processing loop. _recv_chan: trio.MemoryReceiveChannel @@ -429,7 +432,7 @@ class Context: # there's always going to be an "underlying reason" that any # context was closed due to either a remote side error or # a call to `.cancel()` which triggers `ContextCancelled`. - _cancel_msg: str | dict | None = None + _cancel_msg: str|dict|None = None # NOTE: this state var used by the runtime to determine if the # `pdbp` REPL is allowed to engage on contexts terminated via @@ -486,6 +489,13 @@ class Context: f' {stream}\n' ) + outcome_str: str = self.repr_outcome( + show_error_fields=True + ) + outcome_typ_str: str = self.repr_outcome( + type_only=True + ) + return ( f' tuple[str, str] | None: + def canceller(self) -> tuple[str, str]|None: ''' ``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. ''' - return self._canceller + if canc := self._canceller: + return tuple(canc) + + return None + + def _is_self_cancelled( + self, + remote_error: Exception|None = None, + + ) -> bool: + + if not self._cancel_called: + return False + + re: BaseException|None = ( + remote_error + or self._remote_error + ) + if not re: + return False + + if from_uid := re.src_actor_uid: + from_uid: tuple = tuple(from_uid) + + our_uid: tuple = self._actor.uid + our_canceller = self.canceller + + return bool( + isinstance(re, ContextCancelled) + and from_uid == self.chan.uid + and re.canceller == our_uid + and our_canceller == from_uid + ) @property def cancel_acked(self) -> bool: @@ -568,22 +618,7 @@ class Context: equal to the uid of the calling task's actor. ''' - portal: Portal|None = self._portal - if portal: - our_uid: tuple = portal.actor.uid - - return bool( - self._cancel_called - and (re := self._remote_error) - and isinstance(re, ContextCancelled) - and ( - re.canceller - == - self.canceller - == - our_uid - ) - ) + return self._is_self_cancelled() @property def cancelled_caught(self) -> bool: @@ -762,30 +797,15 @@ class Context: # self-cancel (ack) or, # peer propagated remote cancellation. if isinstance(error, ContextCancelled): - ctxc_src: tuple = error.canceller whom: str = ( - 'us' if ctxc_src == current_actor().uid + 'us' if error.canceller == self._actor.uid else 'peer' ) log.cancel( f'IPC context cancelled by {whom}!\n\n' f'{error}' ) - # always record the cancelling actor's uid since its - # cancellation state is linked and we want to know - # which process was the cause / requester of the - # cancellation. - self._canceller = ctxc_src - - - if self._cancel_called: - # this is an expected cancel request response - # message and we **don't need to raise it** in the - # local cancel `._scope` since it will potentially - # override a real error. After this returns - # `.cancel_acked == True`. - return else: log.error( @@ -794,7 +814,23 @@ class Context: f'{error}\n' f'{pformat(self)}\n' ) - self._canceller = self.chan.uid + + # always record the cancelling actor's uid since its + # cancellation state is linked and we want to know + # which process was the cause / requester of the + # cancellation. + maybe_error_src: tuple = getattr( + error, + 'src_actor_uid', + None, + ) + self._canceller = ( + maybe_error_src + or + # XXX: in the case we get a non-boxed error? + # -> wait but this should never happen right? + self.chan.uid + ) # Cancel the local `._scope`, catch that # `._scope.cancelled_caught` and re-raise any remote error @@ -803,6 +839,15 @@ class Context: cs: trio.CancelScope = self._scope if ( cs + + # XXX this is an expected cancel request response + # message and we **don't need to raise it** in the + # local cancel `._scope` since it will potentially + # override a real error. After this method returns + # if `._cancel_called` then `.cancel_acked and .cancel_called` + # always should be set. + and not self._is_self_cancelled() + and not cs.cancel_called and not cs.cancelled_caught ): @@ -840,9 +885,13 @@ class Context: ) -> str: # TODO: how to show the transport interchange fmt? # codec: str = self.chan.transport.codec_key + outcome_str: str = self.repr_outcome( + show_error_fields=True, + type_only=True, + ) return ( # f'{self._nsf}() -{{{codec}}}-> {repr(self.outcome)}:' - f'{self._nsf}() -> {self.repr_outcome()}:' + f'{self._nsf}() -> {outcome_str}:' ) async def cancel( @@ -851,10 +900,32 @@ class Context: ) -> None: ''' - Cancel this inter-actor-task context. + Cancel this inter-actor IPC context by requestng the + remote side's cancel-scope-linked `trio.Task` by calling + `._scope.cancel()` and delivering an `ContextCancelled` + ack msg in reponse. - Request that the far side cancel it's current linked context, - Timeout quickly in an attempt to sidestep 2-generals... + Behaviour: + --------- + - after the far end cancels, the `.cancel()` calling side + should receive a `ContextCancelled` with the + `.canceller: tuple` uid set to the current `Actor.uid`. + + - timeout (quickly) on failure to rx this ACK error-msg in + an attempt to sidestep 2-generals when the transport + layer fails. + + Note, that calling this method DOES NOT also necessarily + result in `Context._scope.cancel()` being called + **locally**! + + => That is, an IPC `Context` (this) **does not** + have the same semantics as a `trio.CancelScope`. + + If the caller (who entered the `Portal.open_context()`) + desires that the internal block's cancel-scope be + cancelled it should open its own `trio.CancelScope` and + manage it as needed. ''' side: str = self.side @@ -976,7 +1047,7 @@ class Context: ``trio``'s cancellation system. ''' - actor: Actor = current_actor() + actor: Actor = self._actor # If the surrounding context has been cancelled by some # task with a handle to THIS, we error here immediately @@ -1149,62 +1220,58 @@ class Context: a cancellation (if any). ''' - if (( - # NOTE: whenever the context's "opener" side (task) **is** - # the side which requested the cancellation (likekly via - # ``Context.cancel()``), we don't want to re-raise that - # cancellation signal locally (would be akin to - # a ``trio.Nursery`` nursery raising ``trio.Cancelled`` - # whenever ``CancelScope.cancel()`` was called) and - # instead silently reap the expected cancellation - # "error"-msg-as-ack. In this case the `err: - # ContextCancelled` must have a `.canceller` set to the - # uid of the requesting task's actor and we only do NOT - # raise that error locally if WE ARE THAT ACTOR which - # requested the cancellation. - not raise_ctxc_from_self_call - and isinstance(remote_error, ContextCancelled) - and ( - self._cancel_called + our_uid: tuple = self.chan.uid - # or self.chan._cancel_called - # TODO: ^ should we have a special separate case - # for this ^ ? - ) - and ( # one of, + # XXX NOTE XXX: `ContextCancelled`/`StreamOverrun` absorption + # for "graceful cancellation" case: + # + # Whenever a "side" of a context (a `trio.Task` running in + # an actor) **is** the side which requested ctx + # cancellation (likekly via ``Context.cancel()``), we + # **don't** want to re-raise any eventually received + # `ContextCancelled` response locally (would be akin to + # a `trio.Nursery` nursery raising `trio.Cancelled` + # whenever `CancelScope.cancel()` was called). + # + # Instead, silently reap the remote delivered ctxc + # (`ContextCancelled`) as an expected + # error-msg-is-cancellation-ack IFF said + # `remote_error: ContextCancelled` has `.canceller` + # 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). + if ( + not raise_ctxc_from_self_call + and self._is_self_cancelled(remote_error) - (portal := self._portal) - and (our_uid := portal.actor.uid) - # TODO: ?potentially it is useful to emit certain - # warning/cancel logs for the cases where the - # cancellation is due to a lower level cancel - # request, such as `Portal.cancel_actor()`, since in - # that case it's not actually this specific ctx that - # made a `.cancel()` call, but it is the same - # actor-process? - and tuple(remote_error.canceller) == our_uid - or self.chan._cancel_called - or self.canceller == our_uid - ) - ) or ( + # TODO: ?potentially it is useful to emit certain + # warning/cancel logs for the cases where the + # cancellation is due to a lower level cancel + # request, such as `Portal.cancel_actor()`, since in + # that case it's not actually this specific ctx that + # made a `.cancel()` call, but it is the same + # actor-process? + # or self.chan._cancel_called + # XXX: ^ should we have a special separate case + # for this ^, NO right? - # NOTE: whenever this context is the cause of an - # overrun on the remote side (aka we sent msgs too - # fast that the remote task was overrun according - # to `MsgStream` buffer settings) AND the caller - # has requested to not raise overruns this side - # caused, we also silently absorb any remotely - # boxed `StreamOverrun`. This is mostly useful for - # supressing such faults during - # cancellation/error/final-result handling inside - # `_drain_to_final_msg()` such that we do not - # raise such errors particularly in the case where - # `._cancel_called == True`. - not raise_overrun_from_self - and isinstance(remote_error, RemoteActorError) - and remote_error.msgdata['type_str'] == 'StreamOverrun' - and tuple(remote_error.msgdata['sender']) == our_uid - ) + ) or ( + # NOTE: whenever this context is the cause of an + # overrun on the remote side (aka we sent msgs too + # fast that the remote task was overrun according + # to `MsgStream` buffer settings) AND the caller + # has requested to not raise overruns this side + # caused, we also silently absorb any remotely + # boxed `StreamOverrun`. This is mostly useful for + # supressing such faults during + # cancellation/error/final-result handling inside + # `_drain_to_final_msg()` such that we do not + # raise such errors particularly in the case where + # `._cancel_called == True`. + not raise_overrun_from_self + and isinstance(remote_error, RemoteActorError) + and remote_error.msgdata['type_str'] == 'StreamOverrun' + and tuple(remote_error.msgdata['sender']) == our_uid ): # NOTE: we set the local scope error to any "self # cancellation" error-response thus "absorbing" @@ -1236,7 +1303,7 @@ class Context: # TODO: change to `.wait_for_result()`? async def result( self, - hide_tb: bool = True, + hide_tb: bool = False, ) -> Any|Exception: ''' @@ -1378,7 +1445,20 @@ class Context: if error: return error - assert not self._cancel_msg + if cancmsg := self._cancel_msg: + # NOTE: means we're prolly in the process of + # processing the cancellation caused by + # this msg (eg. logging from `Actor._cancel_task()` + # method after receiving a `Context.cancel()` RPC) + # though there shouldn't ever be a `._cancel_msg` + # without it eventually resulting in this property + # delivering a value! + log.debug( + '`Context._cancel_msg` is set but has not yet resolved to `.maybe_error`?\n\n' + f'{cancmsg}\n' + ) + + # assert not self._cancel_msg return None def _final_result_is_set(self) -> bool: @@ -1411,6 +1491,7 @@ class Context: def repr_outcome( self, show_error_fields: bool = False, + type_only: bool = False, ) -> str: ''' @@ -1420,6 +1501,9 @@ class Context: ''' merr: Exception|None = self.maybe_error if merr: + if type_only: + return type(merr).__name__ + # if the error-type is one of ours and has the custom # defined "repr-(in)-one-line" method call it, ow # just deliver the type name. @@ -1616,8 +1700,6 @@ class Context: f'{pformat(msg)}\n' ) - # from .devx._debug import pause - # await pause() # NOTE: if an error is deteced we should always still # send it through the feeder-mem-chan and expect @@ -1666,7 +1748,7 @@ class Context: # overrun state and that msg isn't stuck in an # overflow queue what happens?!? - local_uid = current_actor().uid + local_uid = self._actor.uid txt: str = ( 'on IPC context:\n' @@ -1765,6 +1847,7 @@ def mk_context( ctx = Context( chan=chan, cid=cid, + _actor=current_actor(), _send_chan=send_chan, _recv_chan=recv_chan, _nsf=nsf, From c2480c2b9745c746266b28b6bc9418a6158701d5 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 7 Mar 2024 21:26:57 -0500 Subject: [PATCH 118/146] Update ctx test suites to stricter semantics Including mostly tweaking asserts on relayed `ContextCancelled`s and the new pub ctx properties: `.outcome`, `.maybe_error`, etc. as it pertains to graceful (absorbed) remote cancellation vs. loud ctxc cases expected to be raised by any `Portal.cancel_actor()` style teardown. Start checking a variety internals like `._remote/local_error`, `._is_self_cancelled()`, `._is_final_result_set()`, `._cancel_msg` where applicable. Also factor out the new `expect_ctxc()` checker to our `conftest.py` for use in other suites. --- tests/conftest.py | 24 ++++++ tests/test_context_stream_semantics.py | 107 +++++++++++++++++-------- tests/test_inter_peer_cancellation.py | 51 ++++++------ 3 files changed, 126 insertions(+), 56 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index 3363cf56..4a822096 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -1,6 +1,7 @@ """ ``tractor`` testing!! """ +from contextlib import asynccontextmanager as acm import sys import subprocess import os @@ -248,3 +249,26 @@ def daemon( time.sleep(_PROC_SPAWN_WAIT) yield proc sig_prog(proc, _INT_SIGNAL) + + +@acm +async def expect_ctxc( + yay: bool, + reraise: bool = False, +) -> None: + ''' + Small acm to catch `ContextCancelled` errors when expected + below it in a `async with ()` block. + + ''' + if yay: + try: + yield + raise RuntimeError('Never raised ctxc?') + except tractor.ContextCancelled: + if reraise: + raise + else: + return + else: + yield diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index 19a87453..d8e946bf 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -5,7 +5,6 @@ Verify the we raise errors when streams are opened prior to sync-opening a ``tractor.Context`` beforehand. ''' -from contextlib import asynccontextmanager as acm from itertools import count import platform from pprint import pformat @@ -26,7 +25,10 @@ from tractor._exceptions import ( ContextCancelled, ) -from conftest import tractor_test +from conftest import ( + tractor_test, + expect_ctxc, +) # ``Context`` semantics are as follows, # ------------------------------------ @@ -194,12 +196,13 @@ def test_simple_context( ) try: - async with portal.open_context( - simple_setup_teardown, - data=10, - block_forever=callee_blocks_forever, - ) as (ctx, sent): - + async with ( + portal.open_context( + simple_setup_teardown, + data=10, + block_forever=callee_blocks_forever, + ) as (ctx, sent), + ): assert sent == 11 if callee_blocks_forever: @@ -250,17 +253,6 @@ def test_simple_context( trio.run(main) -@acm -async def expect_ctxc(yay: bool) -> None: - if yay: - try: - yield - except ContextCancelled: - return - else: - yield - - @pytest.mark.parametrize( 'callee_returns_early', [True, False], @@ -293,6 +285,7 @@ def test_caller_cancels( ) -> None: actor: Actor = current_actor() uid: tuple = actor.uid + _ctxc: ContextCancelled|None = None if ( cancel_method == 'portal' @@ -303,6 +296,9 @@ def test_caller_cancels( assert 0, 'Portal cancel should raise!' except ContextCancelled as ctxc: + # with trio.CancelScope(shield=True): + # await tractor.pause() + _ctxc = ctxc assert ctx.chan._cancel_called assert ctxc.canceller == uid assert ctxc is ctx.maybe_error @@ -311,7 +307,10 @@ def test_caller_cancels( # case since self-cancellation should swallow the ctxc # silently! else: - res = await ctx.result() + try: + res = await ctx.result() + except ContextCancelled as ctxc: + pytest.fail(f'should not have raised ctxc\n{ctxc}') # we actually get a result if callee_returns_early: @@ -342,6 +341,10 @@ def test_caller_cancels( # await tractor.pause() # assert ctx._local_error is None + # TODO: don't need this right? + # if _ctxc: + # raise _ctxc + async def main(): @@ -352,11 +355,19 @@ def test_caller_cancels( 'simple_context', enable_modules=[__name__], ) - timeout = 0.5 if not callee_returns_early else 2 + timeout: float = ( + 0.5 + if not callee_returns_early + else 2 + ) with trio.fail_after(timeout): async with ( - - expect_ctxc(yay=cancel_method == 'portal'), + expect_ctxc( + yay=( + not callee_returns_early + and cancel_method == 'portal' + ) + ), portal.open_context( simple_setup_teardown, @@ -372,10 +383,18 @@ def test_caller_cancels( await trio.sleep(0.5) if cancel_method == 'ctx': + print('cancelling with `Context.cancel()`') await ctx.cancel() - else: + + elif cancel_method == 'portal': + print('cancelling with `Portal.cancel_actor()`') await portal.cancel_actor() + else: + pytest.fail( + f'Unknown `cancel_method={cancel_method} ?' + ) + if chk_ctx_result_before_exit: await check_canceller(ctx) @@ -385,15 +404,22 @@ def test_caller_cancels( if cancel_method != 'portal': await portal.cancel_actor() - # since the `.cancel_actor()` call just above - # will cause the `.open_context().__aexit__()` raise - # a ctxc which should in turn cause `ctx._scope` to + # XXX NOTE XXX: non-normal yet purposeful + # test-specific ctxc suppression is implemented! + # + # WHY: the `.cancel_actor()` case (cancel_method='portal') + # will cause both: + # * the `ctx.result()` inside `.open_context().__aexit__()` + # * AND the `ctx.result()` inside `check_canceller()` + # to raise ctxc. + # + # which should in turn cause `ctx._scope` to # catch any cancellation? if ( not callee_returns_early - and cancel_method == 'portal' + and cancel_method != 'portal' ): - assert ctx._scope.cancelled_caught + assert not ctx._scope.cancelled_caught trio.run(main) @@ -511,6 +537,23 @@ async def expect_cancelled( await stream.send(msg) # echo server except trio.Cancelled: + + # on ctx.cancel() the internal RPC scope is cancelled but + # never caught until the func exits. + assert ctx._scope.cancel_called + assert not ctx._scope.cancelled_caught + + # should be the RPC cmd request for `._cancel_task()` + assert ctx._cancel_msg + # which, has not yet resolved to an error outcome + # since this rpc func has not yet exited. + assert not ctx.maybe_error + assert not ctx._final_result_is_set() + + # debug REPL if needed + # with trio.CancelScope(shield=True): + # await tractor.pause() + # expected case _state = False raise @@ -594,16 +637,16 @@ async def test_caller_closes_ctx_after_callee_opens_stream( with trio.fail_after(0.2): await ctx.result() assert 0, "Callee should have blocked!?" - except trio.TooSlowError: # NO-OP -> since already called above await ctx.cancel() # NOTE: local scope should have absorbed the cancellation since # in this case we call `ctx.cancel()` and the local - # `._scope` gets `.cancel_called` on the ctxc ack. + # `._scope` does not get `.cancel_called` and thus + # `.cancelled_caught` neither will ever bet set. if use_ctx_cancel_method: - assert ctx._scope.cancelled_caught + assert not ctx._scope.cancelled_caught # rxed ctxc response from far end assert ctx.cancel_acked diff --git a/tests/test_inter_peer_cancellation.py b/tests/test_inter_peer_cancellation.py index d878b06d..c3d9e4fd 100644 --- a/tests/test_inter_peer_cancellation.py +++ b/tests/test_inter_peer_cancellation.py @@ -238,7 +238,12 @@ async def stream_from_peer( assert peer_ctx._remote_error is ctxerr assert peer_ctx._remote_error.msgdata == ctxerr.msgdata - assert peer_ctx.canceller == ctxerr.canceller + + # the peer ctx is the canceller even though it's canceller + # is the "canceller" XD + assert peer_name in peer_ctx.canceller + + assert "canceller" in ctxerr.canceller # caller peer should not be the cancel requester assert not ctx.cancel_called @@ -272,7 +277,6 @@ async def stream_from_peer( # root/parent actor task should NEVER HAVE cancelled us! assert not ctx.canceller - assert 'canceller' in peer_ctx.canceller raise # TODO: IN THEORY we could have other cases depending on @@ -527,27 +531,24 @@ def test_peer_canceller( assert ctx.cancel_called - if ( - ctx is sleeper_ctx - or ctx is caller_ctx - ): - assert ( - re.canceller - == - ctx.canceller - == - canceller.channel.uid - ) + if ctx is sleeper_ctx: + assert 'canceller' in re.canceller + assert 'sleeper' in ctx.canceller - else: + if ctx is canceller_ctx: assert ( re.canceller == - ctx.canceller - == root.uid ) + else: # the other 2 ctxs + assert ( + re.canceller + == + canceller.channel.uid + ) + # since the sleeper errors while handling a # peer-cancelled (by ctxc) scenario, we expect # that the `.open_context()` block DOES call @@ -576,14 +577,16 @@ def test_peer_canceller( assert not sleeper_ctx._scope.cancelled_caught assert isinstance(loc_err, ContextCancelled) - assert loc_err.canceller == sleeper_ctx.canceller - assert ( - loc_err.canceller[0] - == - sleeper_ctx.canceller[0] - == - 'canceller' - ) + + # the received remote error's `.canceller` + # will of course be the "canceller" actor BUT + # the canceller set on the local handle to + # `sleeper_ctx` will be the "sleeper" uid + # since it's the actor that relayed us the + # error which was **caused** by the + # "canceller". + assert 'sleeper' in sleeper_ctx.canceller + assert 'canceller' == loc_err.canceller[0] # the sleeper's remote error is the error bubbled # out of the context-stack above! From 36a81a60cd37938f0d79224a89012ba7808e32da Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 7 Mar 2024 21:33:07 -0500 Subject: [PATCH 119/146] Adjust `asyncio` test for stricter ctx-self-cancels Use `expect_ctx()` around the portal cancellation case, toss in a `'context'` parametrization and return just the `Context.outcome` from `main()` B) --- tests/test_infected_asyncio.py | 35 +++++++++++++++++++++++++++------- 1 file changed, 28 insertions(+), 7 deletions(-) diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 56b5fde5..24bd371f 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -19,6 +19,8 @@ from tractor import ( ) from tractor.trionics import BroadcastReceiver +from conftest import expect_ctxc + async def sleep_and_err( sleep_for: float = 0.1, @@ -190,7 +192,8 @@ async def trio_ctx( @pytest.mark.parametrize( - 'parent_cancels', [False, True], + 'parent_cancels', + ['context', 'actor', False], ids='parent_actor_cancels_child={}'.format ) def test_context_spawns_aio_task_that_errors( @@ -214,18 +217,36 @@ def test_context_spawns_aio_task_that_errors( # debug_mode=True, loglevel='cancel', ) - async with p.open_context( - trio_ctx, - ) as (ctx, first): + async with ( + expect_ctxc( + yay=parent_cancels == 'actor', + ), + p.open_context( + trio_ctx, + ) as (ctx, first), + ): assert first == 'start' - if parent_cancels: + if parent_cancels == 'actor': await p.cancel_actor() - await trio.sleep_forever() + elif parent_cancels == 'context': + await ctx.cancel() - return await ctx.result() + else: + await trio.sleep_forever() + + async with expect_ctxc( + yay=parent_cancels == 'actor', + ): + await ctx.result() + + if parent_cancels == 'context': + # to tear down sub-acor + await p.cancel_actor() + + return ctx.outcome if parent_cancels: # bc the parent made the cancel request, From f90ca0668b6e5d1aef2f49e18c0f9e0188996f16 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 8 Mar 2024 13:48:35 -0500 Subject: [PATCH 120/146] Woops, fix one last `ctx._cancelled_caught` in drain loop --- tractor/_context.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index 9179456b..ee058196 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -196,8 +196,8 @@ async def _drain_to_final_msg( (ctx._stream.closed and (reason := 'stream was already closed') ) - or (ctx._cancelled_caught - and (reason := 'ctx caught a cancel') + or (ctx.cancel_acked + and (reason := 'ctx cancelled other side') ) or (ctx._cancel_called and (reason := 'ctx called `.cancel()`') From 8e3a2a929747f468aca7493c183e27a75a55619b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 8 Mar 2024 14:03:18 -0500 Subject: [PATCH 121/146] Make `Actor._cancel_task(requesting_uid: tuple)` required arg --- tractor/_runtime.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 81bab1d5..0956042e 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -1676,8 +1676,8 @@ class Actor: self, cid: str, parent_chan: Channel, + requesting_uid: tuple[str, str]|None, - requesting_uid: tuple[str, str]|None = None, ipc_msg: dict|None|bool = False, ) -> bool: @@ -2211,6 +2211,7 @@ async def process_messages( await actor._cancel_task( cid, channel, + requesting_uid=channel.uid, ipc_msg=msg, ) From fd2391539ed67e419bde2ffb925f6f59543dfb00 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 8 Mar 2024 14:11:17 -0500 Subject: [PATCH 122/146] Support a `._state.last_actor()` getter Not sure if it's really that useful other then for reporting errors from `current_actor()` but at least it alerts `tractor` devs and/or users when the runtime has already terminated vs. hasn't been started yet/correctly. Set the `._last_actor_terminated: tuple` in the root's final block which allows testing for an already terminated tree which is the case where `._state._current_actor == None` and the last is set. --- tractor/_root.py | 1 + tractor/_state.py | 53 ++++++++++++++++++++++++++++++++++++++----- tractor/_supervise.py | 11 +++++---- 3 files changed, 55 insertions(+), 10 deletions(-) diff --git a/tractor/_root.py b/tractor/_root.py index 7c951df4..1ec745c9 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -262,6 +262,7 @@ async def open_root_actor( await actor.cancel(None) # self cancel finally: _state._current_actor = None + _state._last_actor_terminated = actor # restore breakpoint hook state sys.breakpointhook = builtin_bp_handler diff --git a/tractor/_state.py b/tractor/_state.py index f94c3ebb..f3917436 100644 --- a/tractor/_state.py +++ b/tractor/_state.py @@ -18,12 +18,18 @@ Per process state """ +from __future__ import annotations from typing import ( - Optional, Any, + TYPE_CHECKING, ) -_current_actor: Optional['Actor'] = None # type: ignore # noqa +if TYPE_CHECKING: + from ._runtime import Actor + + +_current_actor: Actor|None = None # type: ignore # noqa +_last_actor_terminated: Actor|None = None _runtime_vars: dict[str, Any] = { '_debug_mode': False, '_is_root': False, @@ -31,14 +37,49 @@ _runtime_vars: dict[str, Any] = { } -def current_actor(err_on_no_runtime: bool = True) -> 'Actor': # type: ignore # noqa +def last_actor() -> Actor|None: + ''' + Try to return last active `Actor` singleton + for this process. + + For case where runtime already exited but someone is asking + about the "last" actor probably to get its `.uid: tuple`. + + ''' + return _last_actor_terminated + + +def current_actor( + err_on_no_runtime: bool = True, +) -> Actor: ''' Get the process-local actor instance. ''' - from ._exceptions import NoRuntime - if _current_actor is None and err_on_no_runtime: - raise NoRuntime("No local actor has been initialized yet") + if ( + err_on_no_runtime + and _current_actor is None + ): + msg: str = 'No local actor has been initialized yet' + from ._exceptions import NoRuntime + + if last := last_actor(): + msg += ( + f'Apparently the lact active actor was\n' + f'|_{last}\n' + f'|_{last.uid}\n' + ) + # no actor runtime has (as of yet) ever been started for + # this process. + else: + msg += ( + 'No last actor found?\n' + 'Did you forget to open one of:\n\n' + '- `tractor.open_root_actor()`\n' + '- `tractor.open_nursery()`\n' + ) + + raise NoRuntime(msg) return _current_actor diff --git a/tractor/_supervise.py b/tractor/_supervise.py index a93c3ebc..8dce5030 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -534,12 +534,15 @@ async def open_nursery( ''' implicit_runtime: bool = False - - actor = current_actor(err_on_no_runtime=False) + actor: Actor = current_actor( + err_on_no_runtime=False + ) try: - if actor is None and is_main_process(): - + if ( + actor is None + and is_main_process() + ): # if we are the parent process start the # actor runtime implicitly log.info("Starting actor runtime!") From c8775dee413d17d6704d7ba210e9968799fcaea1 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 8 Mar 2024 15:46:42 -0500 Subject: [PATCH 123/146] Tweak `Context.repr_outcome()` for KBIs Since apparently `str(KeyboardInterrupt()) == ''`? So instead add little ` or repr(merr)` expressions throughout to avoid blank strings rendering if various `repr()`/`.__str__()` outputs.. --- tractor/_context.py | 24 ++++++++++++++++++++---- 1 file changed, 20 insertions(+), 4 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index ee058196..a7ce5832 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -1485,7 +1485,11 @@ class Context: TODO: implement this using `outcome.Outcome` types? ''' - return self.maybe_error or self._result + return ( + self.maybe_error + or + self._result + ) # @property def repr_outcome( @@ -1520,16 +1524,28 @@ class Context: # # just the type name for now to avoid long lines # when tons of cancels.. - return type(merr).__name__ + return ( + str(type(merr).__name__) + or + repr(merr) + ) # just the type name # else: # but wen? # return type(merr).__name__ # for all other errors show their regular output - return str(merr) + return ( + str(merr) + or + repr(merr) + ) - return str(self._result) + return ( + str(self._result) + or + repr(self._result) + ) async def started( self, From f730749dc972a05084663887c06bece114393283 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 8 Mar 2024 15:48:13 -0500 Subject: [PATCH 124/146] Assign `ctx._local_error` ASAP from `.open_context()` Such that `.outcome` related fields render nicely asap for logging withing `Portal.open_context()` itself. --- tractor/_portal.py | 21 +++++++++++++-------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/tractor/_portal.py b/tractor/_portal.py index bd23a6e4..27fbaec1 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -30,7 +30,7 @@ from typing import ( Any, Callable, AsyncGenerator, - Type, + # Type, ) from functools import partial from dataclasses import dataclass @@ -41,8 +41,7 @@ from async_generator import asynccontextmanager from .trionics import maybe_open_nursery from .devx import ( - # acquire_debug_lock, - # pause, + # _debug, maybe_wait_for_debugger, ) from ._state import ( @@ -673,6 +672,7 @@ class Portal: # `Nursery.cancel_scope.cancel()`) except ContextCancelled as ctxc: scope_err = ctxc + ctx._local_error: BaseException = scope_err ctxc_from_callee = ctxc # XXX TODO XXX: FIX THIS debug_mode BUGGGG!!! @@ -684,7 +684,7 @@ class Portal: # debugging the tractor-runtime itself using it's # own `.devx.` tooling! # - # await pause() + # await _debug.pause() # CASE 2: context was cancelled by local task calling # `.cancel()`, we don't raise and the exit block should @@ -745,18 +745,20 @@ class Portal: ) as caller_err: scope_err = caller_err + ctx._local_error: BaseException = scope_err # XXX: ALWAYS request the context to CANCEL ON any ERROR. # NOTE: `Context.cancel()` is conversely NEVER CALLED in # the `ContextCancelled` "self cancellation absorbed" case # handled in the block above ^^^ !! + # await _debug.pause() log.cancel( 'Context terminated due to\n\n' - f'{caller_err}\n' + f'.outcome => {ctx.repr_outcome()}\n' ) if debug_mode(): - # async with acquire_debug_lock(self.actor.uid): + # async with _debug.acquire_debug_lock(self.actor.uid): # pass # TODO: factor ^ into below for non-root cases? was_acquired: bool = await maybe_wait_for_debugger( @@ -818,6 +820,7 @@ class Portal: # this task didn't know until final teardown # / value collection. scope_err = berr + ctx._local_error: BaseException = scope_err raise # yes! this worx Bp @@ -927,8 +930,10 @@ class Portal: # should be stored as the `Context._local_error` and # used in determining `Context.cancelled_caught: bool`. if scope_err is not None: - ctx._local_error: BaseException = scope_err - etype: Type[BaseException] = type(scope_err) + # sanity, tho can remove? + assert ctx._local_error is scope_err + # ctx._local_error: BaseException = scope_err + # etype: Type[BaseException] = type(scope_err) # CASE 2 if ( From cf68e075c974e510fd6da056dd2716a621a8d756 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 8 Mar 2024 16:07:17 -0500 Subject: [PATCH 125/146] Unify some log msgs in `.to_asyncio` Much like similar recent changes throughout the core, build out `msg: str` depending on error cases and emit with `.cancel()` level as appropes. Also mute (via level) some duplication in the cancel case inside `_run_asyncio_task()` for console noise reduction. --- tractor/to_asyncio.py | 49 ++++++++++++++++++++++++++++++------------- 1 file changed, 35 insertions(+), 14 deletions(-) diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index be3ac8d3..174a99d3 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -217,7 +217,14 @@ def _run_asyncio_task( try: result = await coro except BaseException as aio_err: - log.exception('asyncio task errored') + if isinstance(aio_err, CancelledError): + log.runtime( + '`asyncio` task was cancelled..\n' + ) + else: + log.exception( + '`asyncio` task errored\n' + ) chan._aio_err = aio_err raise @@ -272,12 +279,22 @@ def _run_asyncio_task( except BaseException as terr: task_err = terr + msg: str = ( + 'Infected `asyncio` task {etype_str}\n' + f'|_{task}\n' + ) if isinstance(terr, CancelledError): - log.cancel(f'`asyncio` task cancelled: {task.get_name()}') + log.cancel( + msg.format(etype_str='cancelled') + ) else: - log.exception(f'`asyncio` task: {task.get_name()} errored') + log.exception( + msg.format(etype_str='cancelled') + ) - assert type(terr) is type(aio_err), 'Asyncio task error mismatch?' + assert type(terr) is type(aio_err), ( + '`asyncio` task error mismatch?!?' + ) if aio_err is not None: # XXX: uhh is this true? @@ -290,18 +307,22 @@ def _run_asyncio_task( # We might want to change this in the future though. from_aio.close() - if type(aio_err) is CancelledError: - log.cancel("infected task was cancelled") - - # TODO: show that the cancellation originated - # from the ``trio`` side? right? - # if cancel_scope.cancelled: - # raise aio_err from err - - elif task_err is None: + if task_err is None: assert aio_err aio_err.with_traceback(aio_err.__traceback__) - log.error('infected task errorred') + # log.error( + # 'infected task errorred' + # ) + + # TODO: show that the cancellation originated + # from the ``trio`` side? right? + # elif type(aio_err) is CancelledError: + # log.cancel( + # 'infected task was cancelled' + # ) + + # if cancel_scope.cancelled: + # raise aio_err from err # XXX: alway cancel the scope on error # in case the trio task is blocking From 68a39695855ec83793dfd106c290a83ba37a532c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 11 Mar 2024 10:20:55 -0400 Subject: [PATCH 126/146] Let `MsgStream.receive_nowait()` take in msg key list Call it `allow_msg_keys: list[str] = ['yield']` and set it to accept `['yield', 'return']` from the drain loop in `.aclose()`. Only pass the last key error to `_raise_from_no_key_in_msg()` in the fall-through case. Somehow this seems to prevent all the intermittent test failures i was seeing in local runs including when running the entire suite all in sequence; i ain't complaining B) --- tractor/_streaming.py | 40 ++++++++++++++++++++++++++-------------- 1 file changed, 26 insertions(+), 14 deletions(-) diff --git a/tractor/_streaming.py b/tractor/_streaming.py index b2cfe485..50a32ae9 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -90,19 +90,29 @@ class MsgStream(trio.abc.Channel): self._closed: bool|trio.ClosedResourceError = False # delegate directly to underlying mem channel - def receive_nowait(self): - msg = self._rx_chan.receive_nowait() - try: - return msg['yield'] - except KeyError as kerr: - _raise_from_no_key_in_msg( - ctx=self._ctx, - msg=msg, - src_err=kerr, - log=log, - expect_key='yield', - stream=self, - ) + def receive_nowait( + self, + allow_msg_keys: list[str] = ['yield'], + ): + msg: dict = self._rx_chan.receive_nowait() + for ( + i, + key, + ) in enumerate(allow_msg_keys): + try: + return msg[key] + except KeyError as kerr: + if i < (len(allow_msg_keys) - 1): + continue + + _raise_from_no_key_in_msg( + ctx=self._ctx, + msg=msg, + src_err=kerr, + log=log, + expect_key=key, + stream=self, + ) async def receive(self): ''' @@ -263,7 +273,9 @@ class MsgStream(trio.abc.Channel): drained: list[Exception|dict] = [] while not drained: try: - maybe_final_msg = self.receive_nowait() + maybe_final_msg = self.receive_nowait( + allow_msg_keys=['yield', 'return'], + ) if maybe_final_msg: log.debug( 'Drained un-processed stream msg:\n' From cbaf4fc05b7e47f8bcf633410b7015f16132643a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 11 Mar 2024 10:24:44 -0400 Subject: [PATCH 127/146] Add a open-ctx-with-self test Found exactly why trying this won't work when playing around with opening workspaces in `modden` using a `Portal.open_context()` back to the 'bigd' root actor: the RPC machinery only registers one entry in `Actor._contexts` which will get overwritten by each task's side and then experience race-based IPC msging errors (eg. rxing `{'started': _}` on the callee side..). Instead make opening a ctx back to the self-actor a runtime error describing it as an invalid op. To match: - add a new test `test_ctx_with_self_actor()` to the context semantics suite. - tried out adding a new `side: str` to the `Actor.get_context()` (and callers) but ran into not being able to determine the value from in `._push_result()` where it's needed to figure out which side to push to.. So, just leaving the commented arg (passing) in the runtime core for now in case we can come back to trying to make it work, tho i'm thinking it's not the right hack anyway XD --- tests/test_context_stream_semantics.py | 51 ++++++++++++++++++++++++++ tractor/_context.py | 4 +- tractor/_portal.py | 10 +++++ tractor/_runtime.py | 50 +++++++++++++++++++++---- 4 files changed, 107 insertions(+), 8 deletions(-) diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index d8e946bf..d5767eec 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -1024,6 +1024,8 @@ def test_maybe_allow_overruns_stream( cancel_ctx: bool, slow_side: str, allow_overruns_side: str, + + # conftest wide loglevel: str, debug_mode: bool, ): @@ -1147,3 +1149,52 @@ def test_maybe_allow_overruns_stream( # if this hits the logic blocks from above are not # exhaustive.. pytest.fail('PARAMETRIZED CASE GEN PROBLEM YO') + + +def test_ctx_with_self_actor( + loglevel: str, + debug_mode: bool, +): + ''' + NOTE: for now this is an INVALID OP! + + BUT, eventually presuming we add a "side" key to `Actor.get_context()`, + we might be able to get this working symmetrically, but should we?? + + Open a context back to the same actor and ensure all cancellation + and error semantics hold the same. + + ''' + async def main(): + async with tractor.open_nursery( + debug_mode=debug_mode, + enable_modules=[__name__], + ) as an: + assert an + async with ( + tractor.find_actor('root') as portal, + portal.open_context( + expect_cancelled, + # echo_back_sequence, + # seq=seq, + # wait_for_cancel=cancel_ctx, + # be_slow=(slow_side == 'child'), + # allow_overruns_side=allow_overruns_side, + + ) as (ctx, sent), + ctx.open_stream() as ipc, + ): + assert sent is None + + seq = list(range(10)) + for i in seq: + await ipc.send(i) + rx: int = await ipc.receive() + assert rx == i + + await ctx.cancel() + + with pytest.raises(RuntimeError) as excinfo: + trio.run(main) + + assert 'Invalid Operation' in repr(excinfo.value) diff --git a/tractor/_context.py b/tractor/_context.py index a7ce5832..a31c3b1b 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -1101,6 +1101,8 @@ class Context: chan=self.chan, cid=self.cid, nsf=self._nsf, + # side=self.side, + msg_buffer_size=msg_buffer_size, allow_overruns=allow_overruns, ) @@ -1298,7 +1300,7 @@ class Context: # https://docs.python.org/3/reference/simple_stmts.html#the-raise-statement # https://stackoverflow.com/a/24752607 __tracebackhide__: bool = True - raise remote_error from None + raise remote_error # from None # TODO: change to `.wait_for_result()`? async def result( diff --git a/tractor/_portal.py b/tractor/_portal.py index 27fbaec1..c9f314f3 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -514,6 +514,16 @@ class Portal: # a new `_context.py` mod. nsf = NamespacePath.from_ref(func) + # XXX NOTE XXX: currenly we do NOT allow opening a contex + # with "self" since the local feeder mem-chan processing + # is not built for it. + if self.channel.uid == self.actor.uid: + raise RuntimeError( + '** !! Invalid Operation !! **\n' + 'Can not open an IPC ctx with the local actor!\n' + f'|_{self.actor}\n' + ) + ctx: Context = await self.actor.start_remote_task( self.channel, nsf=nsf, diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 0956042e..a06b5948 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -427,6 +427,10 @@ async def _invoke( chan=chan, cid=cid, nsf=NamespacePath.from_ref(func), + + # TODO: if we wanted to get cray and support it? + # side='callee', + # We shouldn't ever need to pass this through right? # it's up to the soon-to-be called rpc task to # open the stream with this option. @@ -679,9 +683,11 @@ async def _invoke( # don't pop the local context until we know the # associated child isn't in debug any more await _debug.maybe_wait_for_debugger() - ctx: Context = actor._contexts.pop( - (chan.uid, cid) - ) + ctx: Context = actor._contexts.pop(( + chan.uid, + cid, + # ctx.side, + )) merr: Exception|None = ctx.maybe_error @@ -860,7 +866,11 @@ class Actor: # map {actor uids -> Context} self._contexts: dict[ - tuple[tuple[str, str], str], + tuple[ + tuple[str, str], # .uid + str, # .cid + str, # .side + ], Context ] = {} @@ -1302,7 +1312,13 @@ class Actor: uid: tuple[str, str] = chan.uid assert uid, f"`chan.uid` can't be {uid}" try: - ctx: Context = self._contexts[(uid, cid)] + ctx: Context = self._contexts[( + uid, + cid, + + # TODO: how to determine this tho? + # side, + )] except KeyError: log.warning( 'Ignoring invalid IPC ctx msg!\n\n' @@ -1321,6 +1337,16 @@ class Actor: cid: str, nsf: NamespacePath, + # TODO: support lookup by `Context.side: str` ? + # -> would allow making a self-context which might have + # certain special use cases where RPC isolation is wanted + # between 2 tasks running in the same process? + # => prolly needs some deeper though on the real use cases + # and whether or not such things should be better + # implemented using a `TaskManager` style nursery.. + # + # side: str|None = None, + msg_buffer_size: int | None = None, allow_overruns: bool = False, @@ -1336,7 +1362,11 @@ class Actor: actor_uid = chan.uid assert actor_uid try: - ctx = self._contexts[(actor_uid, cid)] + ctx = self._contexts[( + actor_uid, + cid, + # side, + )] log.runtime( f'Retreived cached IPC ctx for\n' f'peer: {chan.uid}\n' @@ -1362,7 +1392,11 @@ class Actor: msg_buffer_size=msg_buffer_size or self.msg_buffer_size, _allow_overruns=allow_overruns, ) - self._contexts[(actor_uid, cid)] = ctx + self._contexts[( + actor_uid, + cid, + # side, + )] = ctx return ctx @@ -1393,6 +1427,8 @@ class Actor: chan=chan, cid=cid, nsf=nsf, + + # side='caller', msg_buffer_size=msg_buffer_size, allow_overruns=allow_overruns, ) From 31ccdd79d7f618711358890b6d298c1a17839755 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 11 Mar 2024 10:37:34 -0400 Subject: [PATCH 128/146] Tweak some tests for spurious failues With the seeming cause that some cases occasionally raise `ExceptionGroup` instead of a (collapsed out) single error which, in those cases at least try to check that `.exceptions` has the original error. --- pytest.ini | 2 +- tests/test_advanced_faults.py | 5 ++++- tests/test_advanced_streaming.py | 13 +++++++++++-- tests/test_infected_asyncio.py | 30 ++++++++++++++++++++++++++---- tests/test_rpc.py | 21 +++++++++++++++------ 5 files changed, 57 insertions(+), 14 deletions(-) diff --git a/pytest.ini b/pytest.ini index 6a7e51fb..b2527228 100644 --- a/pytest.ini +++ b/pytest.ini @@ -1,4 +1,4 @@ -# vim: ft=ini +# vim: ft=conf # pytest.ini for tractor [pytest] diff --git a/tests/test_advanced_faults.py b/tests/test_advanced_faults.py index a48866ea..f34738bd 100644 --- a/tests/test_advanced_faults.py +++ b/tests/test_advanced_faults.py @@ -18,7 +18,10 @@ from conftest import ( @pytest.mark.parametrize( 'debug_mode', [False, True], - ids=['no_debug_mode', 'debug_mode'], + ids=[ + 'no_debug_mode', + 'debug_mode', + ], ) @pytest.mark.parametrize( 'ipc_break', diff --git a/tests/test_advanced_streaming.py b/tests/test_advanced_streaming.py index e8696346..3134b9c2 100644 --- a/tests/test_advanced_streaming.py +++ b/tests/test_advanced_streaming.py @@ -6,6 +6,7 @@ from collections import Counter import itertools import platform +import pytest import trio import tractor @@ -143,8 +144,16 @@ def test_dynamic_pub_sub(): try: trio.run(main) - except trio.TooSlowError: - pass + except ( + trio.TooSlowError, + ExceptionGroup, + ) as err: + if isinstance(err, ExceptionGroup): + for suberr in err.exceptions: + if isinstance(suberr, trio.TooSlowError): + break + else: + pytest.fail('Never got a `TooSlowError` ?') @tractor.context diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 24bd371f..1ac1fba4 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -70,7 +70,7 @@ def test_trio_cancels_aio_on_actor_side(reg_addr): async def asyncio_actor( target: str, - expect_err: Optional[Exception] = None + expect_err: Exception|None = None ) -> None: @@ -114,10 +114,21 @@ def test_aio_simple_error(reg_addr): infect_asyncio=True, ) - with pytest.raises(RemoteActorError) as excinfo: + with pytest.raises( + expected_exception=(RemoteActorError, ExceptionGroup), + ) as excinfo: trio.run(main) err = excinfo.value + + # might get multiple `trio.Cancelled`s as well inside an inception + if isinstance(err, ExceptionGroup): + err = next(itertools.dropwhile( + lambda exc: not isinstance(exc, tractor.RemoteActorError), + err.exceptions + )) + assert err + assert isinstance(err, RemoteActorError) assert err.type == AssertionError @@ -290,11 +301,22 @@ def test_aio_cancelled_from_aio_causes_trio_cancelled(reg_addr): infect_asyncio=True, ) - with pytest.raises(RemoteActorError) as excinfo: + with pytest.raises( + expected_exception=(RemoteActorError, ExceptionGroup), + ) as excinfo: trio.run(main) + # might get multiple `trio.Cancelled`s as well inside an inception + err = excinfo.value + if isinstance(err, ExceptionGroup): + err = next(itertools.dropwhile( + lambda exc: not isinstance(exc, tractor.RemoteActorError), + err.exceptions + )) + assert err + # ensure boxed error is correct - assert excinfo.value.type == to_asyncio.AsyncioCancelled + assert err.type == to_asyncio.AsyncioCancelled # TODO: verify open_channel_from will fail on this.. diff --git a/tests/test_rpc.py b/tests/test_rpc.py index 972cae17..71f3258b 100644 --- a/tests/test_rpc.py +++ b/tests/test_rpc.py @@ -1,6 +1,8 @@ -""" -RPC related -""" +''' +RPC (or maybe better labelled as "RTS: remote task scheduling"?) +related API and error checks. + +''' import itertools import pytest @@ -42,8 +44,13 @@ async def short_sleep(): (['tmp_mod'], 'import doggy', ModuleNotFoundError), (['tmp_mod'], '4doggy', SyntaxError), ], - ids=['no_mods', 'this_mod', 'this_mod_bad_func', 'fail_to_import', - 'fail_on_syntax'], + ids=[ + 'no_mods', + 'this_mod', + 'this_mod_bad_func', + 'fail_to_import', + 'fail_on_syntax', + ], ) def test_rpc_errors(reg_addr, to_call, testdir): """Test errors when making various RPC requests to an actor @@ -105,7 +112,9 @@ def test_rpc_errors(reg_addr, to_call, testdir): run() else: # underlying errors aren't propagated upwards (yet) - with pytest.raises(remote_err) as err: + with pytest.raises( + expected_exception=(remote_err, ExceptionGroup), + ) as err: run() # get raw instance from pytest wrapper From 1975b92dbab04f3ecac3db84bde0d3d95fc48879 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 12 Mar 2024 08:56:17 -0400 Subject: [PATCH 129/146] Add `an: ActorNursery` var placeholder for final log msg --- tractor/_supervise.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tractor/_supervise.py b/tractor/_supervise.py index 8dce5030..507e2544 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -534,10 +534,8 @@ async def open_nursery( ''' implicit_runtime: bool = False - actor: Actor = current_actor( - err_on_no_runtime=False - ) - + actor: Actor = current_actor(err_on_no_runtime=False) + an: ActorNursery|None = None try: if ( actor is None From 389b305d3bd984b82bc8bd13bd416788d2b926d2 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 12 Mar 2024 15:48:20 -0400 Subject: [PATCH 130/146] Add (back) a `tractor._testing` sub-pkg Since importing from our top level `conftest.py` is not scaleable or as "future forward thinking" in terms of: - LoC-wise (it's only one file), - prevents "external" (aka non-test) example scripts from importing content easily, - seemingly(?) can't be used via abs-import if using a `[tool.pytest.ini_options]` in a `pyproject.toml` vs. a `pytest.ini`, see: https://docs.pytest.org/en/8.0.x/reference/customize.html#pyproject-toml) => Go back to having an internal "testing" pkg like `trio` (kinda) does. Deats: - move generic top level helpers into pkg-mod including the new `expect_ctxc()` (which i needed in the advanced faults testing script. - move `@tractor_test` into `._testing.pytest` sub-mod. - adjust all the helper imports to be a `from tractor._testing import <..>` Rework `test_ipc_channel_break_during_stream()` and backing script: - make test(s) pull `debug_mode` from new fixture (which is now controlled manually from `--tpdb` flag) and drop the previous parametrized input. - update logic in ^ test for "which-side-fails" cases to better match recently updated/stricter cancel/failure semantics in terms of `ClosedResouruceError` vs. `EndOfChannel` expectations. - handle `ExceptionGroup`s with expected embedded errors in test. - better pendantics around whether to expect a user simulated KBI. - for `examples/advanced_faults/ipc_failure_during_stream.py` script: - generalize ipc breakage in new `break_ipc()` with support for diff internal `trio` methods and a #TODO for future disti frameworks - only make one sub-actor task break and the other just stream. - use new `._testing.expect_ctxc()` around ctx block. - add a bit of exception handling with `print()`s around ctxc (unused except if 'msg' break method is set) and eoc cases. - don't break parent side ipc in loop any more then once after first break, checked via flag var. - add a `pre_close: bool` flag to control whether `MsgStreama.aclose()` is called *before* any ipc breakage method. Still TODO: - drop `pytest.ini` and add the alt section to `pyproject.py`. -> currently can't get `--rootdir=` opt to work.. not showing in console header. -> ^ also breaks on 'tests' `enable_modules` imports in subactors during discovery tests? --- .../ipc_failure_during_stream.py | 216 ++++++++++++++---- pyproject.toml | 20 ++ pytest.ini | 8 - tests/conftest.py | 148 ++---------- tests/test_advanced_faults.py | 178 ++++++++++----- tests/test_cancellation.py | 6 +- tests/test_clustering.py | 4 +- tests/test_context_stream_semantics.py | 2 +- tests/test_debugger.py | 6 +- tests/test_discovery.py | 3 +- tests/test_docs_examples.py | 3 +- tests/test_infected_asyncio.py | 3 +- tests/test_legacy_one_way_streaming.py | 2 +- tests/test_local.py | 2 +- tests/test_multi_program.py | 4 +- tests/test_pubsub.py | 3 +- tests/test_runtime.py | 2 +- tests/test_spawning.py | 2 +- tractor/_testing/__init__.py | 74 ++++++ tractor/_testing/pytest.py | 113 +++++++++ 20 files changed, 543 insertions(+), 256 deletions(-) delete mode 100644 pytest.ini create mode 100644 tractor/_testing/__init__.py create mode 100644 tractor/_testing/pytest.py diff --git a/examples/advanced_faults/ipc_failure_during_stream.py b/examples/advanced_faults/ipc_failure_during_stream.py index 6728b8d2..c7322a7c 100644 --- a/examples/advanced_faults/ipc_failure_during_stream.py +++ b/examples/advanced_faults/ipc_failure_during_stream.py @@ -6,47 +6,120 @@ been an outage) and we want to ensure that despite being in debug mode actor tree will eventually be cancelled without leaving any zombies. ''' -import trio +from functools import partial + from tractor import ( open_nursery, context, Context, + ContextCancelled, MsgStream, + _testing, ) +import trio -async def break_channel_silently_then_error( +async def break_ipc( stream: MsgStream, + method: str|None = None, + pre_close: bool = False, + + def_method: str = 'eof', + +) -> None: + ''' + XXX: close the channel right after an error is raised + purposely breaking the IPC transport to make sure the parent + doesn't get stuck in debug or hang on the connection join. + this more or less simulates an infinite msg-receive hang on + the other end. + + ''' + # close channel via IPC prot msging before + # any transport breakage + if pre_close: + await stream.aclose() + + method: str = method or def_method + + match method: + case 'trans_aclose': + await stream._ctx.chan.transport.stream.aclose() + + case 'eof': + await stream._ctx.chan.transport.stream.send_eof() + + case 'msg': + await stream._ctx.chan.send(None) + + # TODO: the actual real-world simulated cases like + # transport layer hangs and/or lower layer 2-gens type + # scenarios.. + # + # -[ ] already have some issues for this general testing + # area: + # - https://github.com/goodboy/tractor/issues/97 + # - https://github.com/goodboy/tractor/issues/124 + # - PR from @guille: + # https://github.com/goodboy/tractor/pull/149 + # case 'hang': + # TODO: framework research: + # + # - https://github.com/GuoTengda1993/pynetem + # - https://github.com/shopify/toxiproxy + # - https://manpages.ubuntu.com/manpages/trusty/man1/wirefilter.1.html + + case _: + raise RuntimeError( + f'IPC break method unsupported: {method}' + ) + + +async def break_ipc_then_error( + stream: MsgStream, + break_ipc_with: str|None = None, + pre_close: bool = False, ): async for msg in stream: await stream.send(msg) - - # XXX: close the channel right after an error is raised - # purposely breaking the IPC transport to make sure the parent - # doesn't get stuck in debug or hang on the connection join. - # this more or less simulates an infinite msg-receive hang on - # the other end. - await stream._ctx.chan.send(None) + await break_ipc( + stream=stream, + method=break_ipc_with, + pre_close=pre_close, + ) assert 0 -async def close_stream_and_error( +# async def close_stream_and_error( +async def iter_ipc_stream( stream: MsgStream, + break_ipc_with: str|None = None, + pre_close: bool = False, ): async for msg in stream: await stream.send(msg) # wipe out channel right before raising - await stream._ctx.chan.send(None) - await stream.aclose() - assert 0 + # await break_ipc( + # stream=stream, + # method=break_ipc_with, + # pre_close=pre_close, + # ) + + # send channel close msg at SC-prot level + # + # TODO: what should get raised here if anything? + # await stream.aclose() + + # assert 0 @context async def recv_and_spawn_net_killers( ctx: Context, - break_ipc_after: bool | int = False, + break_ipc_after: bool|int = False, + pre_close: bool = False, ) -> None: ''' @@ -63,27 +136,42 @@ async def recv_and_spawn_net_killers( await stream.send(i) if ( break_ipc_after - and i > break_ipc_after + and + i > break_ipc_after ): '#################################\n' - 'Simulating child-side IPC BREAK!\n' - '#################################' - n.start_soon(break_channel_silently_then_error, stream) - n.start_soon(close_stream_and_error, stream) + 'Simulating CHILD-side IPC BREAK!\n' + '#################################\n' + n.start_soon( + partial( + break_ipc_then_error, + stream=stream, + pre_close=pre_close, + ) + ) + n.start_soon( + iter_ipc_stream, + stream, + ) async def main( debug_mode: bool = False, start_method: str = 'trio', + loglevel: str = 'cancel', # by default we break the parent IPC first (if configured to break # at all), but this can be changed so the child does first (even if # both are set to break). - break_parent_ipc_after: int | bool = False, - break_child_ipc_after: int | bool = False, + break_parent_ipc_after: int|bool = False, + break_child_ipc_after: int|bool = False, + pre_close: bool = False, ) -> None: + # from tractor._state import _runtime_vars as rtv + # rtv['_debug_mode'] = debug_mode + async with ( open_nursery( start_method=start_method, @@ -91,57 +179,107 @@ async def main( # NOTE: even debugger is used we shouldn't get # a hang since it never engages due to broken IPC debug_mode=debug_mode, - loglevel='warning', + loglevel=loglevel, ) as an, ): + sub_name: str = 'chitty_hijo' portal = await an.start_actor( - 'chitty_hijo', + sub_name, enable_modules=[__name__], ) - async with portal.open_context( - recv_and_spawn_net_killers, - break_ipc_after=break_child_ipc_after, - - ) as (ctx, sent): + async with ( + _testing.expect_ctxc( + yay=( + break_parent_ipc_after + or break_child_ipc_after, + ), + # TODO: we CAN'T remove this right? + # since we need the ctxc to bubble up from either + # the stream API after the `None` msg is sent + # (which actually implicitly cancels all remote + # tasks in the hijo) or from simluated + # KBI-mash-from-user + # or should we expect that a KBI triggers the ctxc + # and KBI in an eg? + reraise=True, + ), + portal.open_context( + recv_and_spawn_net_killers, + break_ipc_after=break_child_ipc_after, + pre_close=pre_close, + ) as (ctx, sent), + ): + ipc_break_sent: bool = False async with ctx.open_stream() as stream: for i in range(1000): if ( break_parent_ipc_after - and i > break_parent_ipc_after + and + i > break_parent_ipc_after + and + not ipc_break_sent ): print( '#################################\n' - 'Simulating parent-side IPC BREAK!\n' - '#################################' + 'Simulating PARENT-side IPC BREAK!\n' + '#################################\n' ) - await stream._ctx.chan.send(None) + + # await stream._ctx.chan.send(None) + # await stream._ctx.chan.transport.stream.send_eof() + await stream._ctx.chan.transport.stream.aclose() + + ipc_break_sent = True # it actually breaks right here in the # mp_spawn/forkserver backends and thus the zombie # reaper never even kicks in? print(f'parent sending {i}') - await stream.send(i) + try: + await stream.send(i) + except ContextCancelled as ctxc: + print( + 'parent received ctxc on `stream.send()`\n' + f'{ctxc}\n' + ) + assert 'root' in ctxc.canceller + assert sub_name in ctx.canceller - with trio.move_on_after(2) as cs: + # TODO: is this needed or no? + raise + + timeout: int = 1 + print(f'Entering `stream.receive()` with timeout={timeout}\n') + with trio.move_on_after(timeout) as cs: # NOTE: in the parent side IPC failure case this # will raise an ``EndOfChannel`` after the child # is killed and sends a stop msg back to it's # caller/this-parent. - rx = await stream.receive() - - print(f"I'm a happy user and echoed to me is {rx}") + try: + rx = await stream.receive() + print( + "I'm a happy PARENT user and echoed to me is\n" + f'{rx}\n' + ) + except trio.EndOfChannel: + print('MsgStream got EoC for PARENT') + raise if cs.cancelled_caught: # pretend to be a user seeing no streaming action # thinking it's a hang, and then hitting ctl-c.. - print("YOO i'm a user anddd thingz hangin..") + print( + f"YOO i'm a PARENT user anddd thingz hangin..\n" + f'after timeout={timeout}\n' + ) print( - "YOO i'm mad send side dun but thingz hangin..\n" + "YOO i'm mad!\n" + 'The send side is dun but thingz hangin..\n' 'MASHING CTlR-C Ctl-c..' ) raise KeyboardInterrupt diff --git a/pyproject.toml b/pyproject.toml index e52aa476..84633806 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -26,3 +26,23 @@ all_bullets = true directory = "trivial" name = "Trivial/Internal Changes" showcontent = true + + +[tool.pytest.ini_options] +minversion = '6.0' +testpaths = [ + 'tests' +] +addopts = [ + # TODO: figure out why this isn't working.. + '--rootdir=./tests', + + '--import-mode=importlib', + # don't show frickin captured logs AGAIN in the report.. + '--show-capture=no', +] +log_cli = false + +# TODO: maybe some of these layout choices? +# https://docs.pytest.org/en/8.0.x/explanation/goodpractices.html#choosing-a-test-layout-import-rules +# pythonpath = "src" diff --git a/pytest.ini b/pytest.ini deleted file mode 100644 index b2527228..00000000 --- a/pytest.ini +++ /dev/null @@ -1,8 +0,0 @@ -# vim: ft=conf -# pytest.ini for tractor - -[pytest] -# don't show frickin captured logs AGAIN in the report.. -addopts = --show-capture='no' -log_cli = false -; minversion = 6.0 diff --git a/tests/conftest.py b/tests/conftest.py index 4a822096..4bedc4f1 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -1,101 +1,25 @@ """ ``tractor`` testing!! """ -from contextlib import asynccontextmanager as acm import sys import subprocess import os import random import signal import platform -import pathlib import time -import inspect -from functools import partial, wraps import pytest -import trio import tractor +from tractor._testing import ( + examples_dir as examples_dir, + tractor_test as tractor_test, + expect_ctxc as expect_ctxc, +) +# TODO: include wtv plugin(s) we build in `._testing.pytest`? pytest_plugins = ['pytester'] - -def tractor_test(fn): - """ - Use: - - @tractor_test - async def test_whatever(): - await ... - - If fixtures: - - - ``arb_addr`` (a socket addr tuple where arbiter is listening) - - ``loglevel`` (logging level passed to tractor internals) - - ``start_method`` (subprocess spawning backend) - - are defined in the `pytest` fixture space they will be automatically - injected to tests declaring these funcargs. - """ - @wraps(fn) - def wrapper( - *args, - loglevel=None, - arb_addr=None, - start_method=None, - **kwargs - ): - # __tracebackhide__ = True - - if 'arb_addr' in inspect.signature(fn).parameters: - # injects test suite fixture value to test as well - # as `run()` - kwargs['arb_addr'] = arb_addr - - if 'loglevel' in inspect.signature(fn).parameters: - # allows test suites to define a 'loglevel' fixture - # that activates the internal logging - kwargs['loglevel'] = loglevel - - if start_method is None: - if platform.system() == "Windows": - start_method = 'trio' - - if 'start_method' in inspect.signature(fn).parameters: - # set of subprocess spawning backends - kwargs['start_method'] = start_method - - if kwargs: - - # use explicit root actor start - - async def _main(): - async with tractor.open_root_actor( - # **kwargs, - arbiter_addr=arb_addr, - loglevel=loglevel, - start_method=start_method, - - # TODO: only enable when pytest is passed --pdb - # debug_mode=True, - - ): - await fn(*args, **kwargs) - - main = _main - - else: - # use implicit root actor start - main = partial(fn, *args, **kwargs) - - return trio.run(main) - - return wrapper - - -_arb_addr = '127.0.0.1', random.randint(1000, 9999) - - # Sending signal.SIGINT on subprocess fails on windows. Use CTRL_* alternatives if platform.system() == 'Windows': _KILL_SIGNAL = signal.CTRL_BREAK_EVENT @@ -115,23 +39,6 @@ no_windows = pytest.mark.skipif( ) -def repodir() -> pathlib.Path: - ''' - Return the abspath to the repo directory. - - ''' - # 2 parents up to step up through tests/ - return pathlib.Path(__file__).parent.parent.absolute() - - -def examples_dir() -> pathlib.Path: - ''' - Return the abspath to the examples directory as `pathlib.Path`. - - ''' - return repodir() / 'examples' - - def pytest_addoption(parser): parser.addoption( "--ll", action="store", dest='loglevel', @@ -169,11 +76,26 @@ _ci_env: bool = os.environ.get('CI', False) @pytest.fixture(scope='session') def ci_env() -> bool: - """Detect CI envoirment. - """ + ''' + Detect CI envoirment. + + ''' return _ci_env +# TODO: also move this to `._testing` for now? +# -[ ] possibly generalize and re-use for multi-tree spawning +# along with the new stuff for multi-addrs in distribute_dis +# branch? +# +# choose randomly at import time +_reg_addr: tuple[str, int] = ( + '127.0.0.1', + random.randint(1000, 9999), +) +_arb_addr = _reg_addr + + @pytest.fixture(scope='session') def arb_addr(): return _arb_addr @@ -213,6 +135,7 @@ def sig_prog(proc, sig): assert ret +# TODO: factor into @cm and move to `._testing`? @pytest.fixture def daemon( loglevel: str, @@ -249,26 +172,3 @@ def daemon( time.sleep(_PROC_SPAWN_WAIT) yield proc sig_prog(proc, _INT_SIGNAL) - - -@acm -async def expect_ctxc( - yay: bool, - reraise: bool = False, -) -> None: - ''' - Small acm to catch `ContextCancelled` errors when expected - below it in a `async with ()` block. - - ''' - if yay: - try: - yield - raise RuntimeError('Never raised ctxc?') - except tractor.ContextCancelled: - if reraise: - raise - else: - return - else: - yield diff --git a/tests/test_advanced_faults.py b/tests/test_advanced_faults.py index f34738bd..8b73b4c2 100644 --- a/tests/test_advanced_faults.py +++ b/tests/test_advanced_faults.py @@ -3,24 +3,28 @@ Sketchy network blackoutz, ugly byzantine gens, puedes eschuchar la cancelacion?.. ''' +import itertools from functools import partial +from types import ModuleType import pytest from _pytest.pathlib import import_path import trio import tractor - -from conftest import ( +from tractor._testing import ( examples_dir, ) @pytest.mark.parametrize( - 'debug_mode', - [False, True], + 'pre_aclose_msgstream', + [ + False, + True, + ], ids=[ - 'no_debug_mode', - 'debug_mode', + 'no_msgstream_aclose', + 'pre_aclose_msgstream', ], ) @pytest.mark.parametrize( @@ -66,8 +70,10 @@ from conftest import ( ) def test_ipc_channel_break_during_stream( debug_mode: bool, + loglevel: str, spawn_backend: str, - ipc_break: dict | None, + ipc_break: dict|None, + pre_aclose_msgstream: bool, ): ''' Ensure we can have an IPC channel break its connection during @@ -79,77 +85,123 @@ def test_ipc_channel_break_during_stream( ''' if spawn_backend != 'trio': - if debug_mode: - pytest.skip('`debug_mode` only supported on `trio` spawner') + # if debug_mode: + # pytest.skip('`debug_mode` only supported on `trio` spawner') # 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 - mod = import_path( + mod: ModuleType = import_path( examples_dir() / 'advanced_faults' / 'ipc_failure_during_stream.py', root=examples_dir(), ) - expect_final_exc = KeyboardInterrupt - - # when ONLY the child breaks we expect the parent to get a closed - # resource error on the next `MsgStream.receive()` and then fail out - # and cancel the child from there. + # by def we expect KBI from user after a simulated "hang + # period" wherein the user eventually hits ctl-c to kill the + # root-actor tree. + expect_final_exc: BaseException = KeyboardInterrupt if ( - - # only child breaks - ( - ipc_break['break_child_ipc_after'] - and ipc_break['break_parent_ipc_after'] is False - ) - - # both break but, parent breaks first - or ( - ipc_break['break_child_ipc_after'] is not False - and ( - ipc_break['break_parent_ipc_after'] - > ipc_break['break_child_ipc_after'] - ) - ) - - ): - expect_final_exc = trio.ClosedResourceError - - # when the parent IPC side dies (even if the child's does as well - # but the child fails BEFORE the parent) we expect the channel to be - # sent a stop msg from the child at some point which will signal the - # parent that the stream has been terminated. - # NOTE: when the parent breaks "after" the child you get this same - # case as well, the child breaks the IPC channel with a stop msg - # before any closure takes place. - elif ( - # only parent breaks - ( - ipc_break['break_parent_ipc_after'] - and ipc_break['break_child_ipc_after'] is False - ) - - # both break but, child breaks first - or ( - ipc_break['break_parent_ipc_after'] is not False - and ( - ipc_break['break_child_ipc_after'] - > ipc_break['break_parent_ipc_after'] - ) - ) + # only expect EoC if trans is broken on the child side, + ipc_break['break_child_ipc_after'] is not False + # AND we tell the child to call `MsgStream.aclose()`. + and pre_aclose_msgstream ): expect_final_exc = trio.EndOfChannel - with pytest.raises(expect_final_exc): - trio.run( - partial( - mod.main, - debug_mode=debug_mode, - start_method=spawn_backend, - **ipc_break, + # NOTE when ONLY the child breaks or it breaks BEFORE the + # parent we expect the parent to get a closed resource error + # on the next `MsgStream.receive()` and then fail out and + # cancel the child from there. + # + # ONLY CHILD breaks + if ( + ipc_break['break_child_ipc_after'] + and + ipc_break['break_parent_ipc_after'] is False + ): + expect_final_exc = trio.ClosedResourceError + + # if child calls `MsgStream.aclose()` then expect EoC. + if pre_aclose_msgstream: + expect_final_exc = trio.EndOfChannel + + # BOTH but, CHILD breaks FIRST + elif ( + ipc_break['break_child_ipc_after'] is not False + and ( + ipc_break['break_parent_ipc_after'] + > ipc_break['break_child_ipc_after'] + ) + ): + expect_final_exc = trio.ClosedResourceError + + # child will send a 'stop' msg before it breaks + # the transport channel. + if pre_aclose_msgstream: + expect_final_exc = trio.EndOfChannel + + # NOTE when the parent IPC side dies (even if the child's 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 + # IMMEDIATELY before the child ever sends any 'stop' msg. + # + # ONLY PARENT breaks + elif ( + ipc_break['break_parent_ipc_after'] + and + ipc_break['break_child_ipc_after'] is False + ): + expect_final_exc = trio.ClosedResourceError + + # BOTH but, PARENT breaks FIRST + elif ( + ipc_break['break_parent_ipc_after'] is not False + and ( + ipc_break['break_child_ipc_after'] + > ipc_break['break_parent_ipc_after'] + ) + ): + expect_final_exc = trio.ClosedResourceError + + with pytest.raises( + expected_exception=( + expect_final_exc, + ExceptionGroup, + ), + ) as excinfo: + try: + trio.run( + partial( + mod.main, + debug_mode=debug_mode, + start_method=spawn_backend, + loglevel=loglevel, + pre_close=pre_aclose_msgstream, + **ipc_break, + ) + ) + except KeyboardInterrupt as kbi: + _err = kbi + if expect_final_exc is not KeyboardInterrupt: + pytest.fail( + 'Rxed unexpected KBI !?\n' + f'{repr(kbi)}' + ) + + raise + + # get raw instance from pytest wrapper + value = excinfo.value + if isinstance(value, ExceptionGroup): + value = next( + itertools.dropwhile( + lambda exc: not isinstance(exc, expect_final_exc), + value.exceptions, ) ) + assert value @tractor.context diff --git a/tests/test_cancellation.py b/tests/test_cancellation.py index 9a729f3d..5b589f6a 100644 --- a/tests/test_cancellation.py +++ b/tests/test_cancellation.py @@ -15,8 +15,10 @@ from exceptiongroup import ( import pytest import trio import tractor - -from conftest import tractor_test, no_windows +from tractor._testing import ( + tractor_test, +) +from conftest import no_windows def is_win(): diff --git a/tests/test_clustering.py b/tests/test_clustering.py index 02b1f8fa..92362b58 100644 --- a/tests/test_clustering.py +++ b/tests/test_clustering.py @@ -5,9 +5,7 @@ import trio import tractor from tractor import open_actor_cluster from tractor.trionics import gather_contexts - -from conftest import tractor_test - +from tractor._testing import tractor_test MESSAGE = 'tractoring at full speed' diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index d5767eec..42b1f7d0 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -25,7 +25,7 @@ from tractor._exceptions import ( ContextCancelled, ) -from conftest import ( +from tractor._testing import ( tractor_test, expect_ctxc, ) diff --git a/tests/test_debugger.py b/tests/test_debugger.py index e7bb0d73..f665d850 100644 --- a/tests/test_debugger.py +++ b/tests/test_debugger.py @@ -11,11 +11,9 @@ TODO: """ import itertools -from os import path from typing import Optional import platform import pathlib -import sys import time import pytest @@ -25,8 +23,10 @@ from pexpect.exceptions import ( EOF, ) -from conftest import ( +from tractor._testing import ( examples_dir, +) +from conftest import ( _ci_env, ) diff --git a/tests/test_discovery.py b/tests/test_discovery.py index 8b47700c..cd9dc022 100644 --- a/tests/test_discovery.py +++ b/tests/test_discovery.py @@ -9,10 +9,9 @@ import itertools import pytest import tractor +from tractor._testing import tractor_test import trio -from conftest import tractor_test - @tractor_test async def test_reg_then_unreg(reg_addr): diff --git a/tests/test_docs_examples.py b/tests/test_docs_examples.py index f134c71b..5099aeba 100644 --- a/tests/test_docs_examples.py +++ b/tests/test_docs_examples.py @@ -11,8 +11,7 @@ import platform import shutil import pytest - -from conftest import ( +from tractor._testing import ( examples_dir, ) diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 1ac1fba4..a3f96ee8 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -18,8 +18,7 @@ from tractor import ( ContextCancelled, ) from tractor.trionics import BroadcastReceiver - -from conftest import expect_ctxc +from tractor._testing import expect_ctxc async def sleep_and_err( diff --git a/tests/test_legacy_one_way_streaming.py b/tests/test_legacy_one_way_streaming.py index 0cbda4d8..1e7ec987 100644 --- a/tests/test_legacy_one_way_streaming.py +++ b/tests/test_legacy_one_way_streaming.py @@ -9,7 +9,7 @@ import trio import tractor import pytest -from conftest import tractor_test +from tractor._testing import tractor_test def test_must_define_ctx(): diff --git a/tests/test_local.py b/tests/test_local.py index 009d0d71..a019d771 100644 --- a/tests/test_local.py +++ b/tests/test_local.py @@ -7,7 +7,7 @@ import pytest import trio import tractor -from conftest import tractor_test +from tractor._testing import tractor_test @pytest.mark.trio diff --git a/tests/test_multi_program.py b/tests/test_multi_program.py index d1ee0f5e..0b6b5baf 100644 --- a/tests/test_multi_program.py +++ b/tests/test_multi_program.py @@ -7,8 +7,10 @@ import time import pytest import trio import tractor -from conftest import ( +from tractor._testing import ( tractor_test, +) +from conftest import ( sig_prog, _INT_SIGNAL, _INT_RETURN_CODE, diff --git a/tests/test_pubsub.py b/tests/test_pubsub.py index 20554fa5..6d416f89 100644 --- a/tests/test_pubsub.py +++ b/tests/test_pubsub.py @@ -5,8 +5,7 @@ import pytest import trio import tractor from tractor.experimental import msgpub - -from conftest import tractor_test +from tractor._testing import tractor_test def test_type_checks(): diff --git a/tests/test_runtime.py b/tests/test_runtime.py index 127138c2..3755af1b 100644 --- a/tests/test_runtime.py +++ b/tests/test_runtime.py @@ -8,7 +8,7 @@ import pytest import trio import tractor -from conftest import tractor_test +from tractor._testing import tractor_test _file_path: str = '' diff --git a/tests/test_spawning.py b/tests/test_spawning.py index 0f6a8cfe..6a4b2988 100644 --- a/tests/test_spawning.py +++ b/tests/test_spawning.py @@ -8,7 +8,7 @@ import pytest import trio import tractor -from conftest import tractor_test +from tractor._testing import tractor_test data_to_pass_down = {'doggy': 10, 'kitty': 4} diff --git a/tractor/_testing/__init__.py b/tractor/_testing/__init__.py new file mode 100644 index 00000000..876c87e8 --- /dev/null +++ b/tractor/_testing/__init__.py @@ -0,0 +1,74 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +''' +Various helpers/utils for auditing your `tractor` app and/or the +core runtime. + +''' +from contextlib import asynccontextmanager as acm +import pathlib + +import tractor +from .pytest import ( + tractor_test as tractor_test +) + + +def repodir() -> pathlib.Path: + ''' + Return the abspath to the repo directory. + + ''' + # 2 parents up to step up through tests/ + return pathlib.Path( + __file__ + + # 3 .parents bc: + # <._testing-pkg>.. + # /$HOME/..//tractor/_testing/__init__.py + ).parent.parent.parent.absolute() + + +def examples_dir() -> pathlib.Path: + ''' + Return the abspath to the examples directory as `pathlib.Path`. + + ''' + return repodir() / 'examples' + + +@acm +async def expect_ctxc( + yay: bool, + reraise: bool = False, +) -> None: + ''' + Small acm to catch `ContextCancelled` errors when expected + below it in a `async with ()` block. + + ''' + if yay: + try: + yield + raise RuntimeError('Never raised ctxc?') + except tractor.ContextCancelled: + if reraise: + raise + else: + return + else: + yield diff --git a/tractor/_testing/pytest.py b/tractor/_testing/pytest.py new file mode 100644 index 00000000..93eeaf72 --- /dev/null +++ b/tractor/_testing/pytest.py @@ -0,0 +1,113 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +''' +`pytest` utils helpers and plugins for testing `tractor`'s runtime +and applications. + +''' +from functools import ( + partial, + wraps, +) +import inspect +import platform + +import tractor +import trio + + +def tractor_test(fn): + ''' + Decorator for async test funcs to present them as "native" + looking sync funcs runnable by `pytest` using `trio.run()`. + + Use: + + @tractor_test + async def test_whatever(): + await ... + + If fixtures: + + - ``reg_addr`` (a socket addr tuple where arbiter is listening) + - ``loglevel`` (logging level passed to tractor internals) + - ``start_method`` (subprocess spawning backend) + + are defined in the `pytest` fixture space they will be automatically + injected to tests declaring these funcargs. + ''' + @wraps(fn) + def wrapper( + *args, + loglevel=None, + reg_addr=None, + start_method: str|None = None, + debug_mode: bool = False, + **kwargs + ): + # __tracebackhide__ = True + + # NOTE: inject ant test func declared fixture + # names by manually checking! + if 'reg_addr' in inspect.signature(fn).parameters: + # injects test suite fixture value to test as well + # as `run()` + kwargs['reg_addr'] = reg_addr + + if 'loglevel' in inspect.signature(fn).parameters: + # allows test suites to define a 'loglevel' fixture + # that activates the internal logging + kwargs['loglevel'] = loglevel + + if start_method is None: + if platform.system() == "Windows": + start_method = 'trio' + + if 'start_method' in inspect.signature(fn).parameters: + # set of subprocess spawning backends + kwargs['start_method'] = start_method + + if 'debug_mode' in inspect.signature(fn).parameters: + # set of subprocess spawning backends + kwargs['debug_mode'] = debug_mode + + + if kwargs: + + # use explicit root actor start + async def _main(): + async with tractor.open_root_actor( + # **kwargs, + registry_addrs=[reg_addr] if reg_addr else None, + loglevel=loglevel, + start_method=start_method, + + # TODO: only enable when pytest is passed --pdb + debug_mode=debug_mode, + + ): + await fn(*args, **kwargs) + + main = _main + + else: + # use implicit root actor start + main = partial(fn, *args, **kwargs) + + return trio.run(main) + + return wrapper From 544cb4053384215caf03b39cdeebf134b2488fbf Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 13 Mar 2024 09:55:47 -0400 Subject: [PATCH 131/146] Attempt at better internal traceback hiding Previously i was trying to approach this using lots of `__tracebackhide__`'s in various internal funcs but since it's not exactly straight forward to do this inside core deps like `trio` and the stdlib, it makes a bit more sense to optionally catch and re-raise certain classes of errors from their originals using `raise from` syntax as per: https://docs.python.org/3/library/exceptions.html#exception-context Deats: - litter `._context` methods with `__tracebackhide__`/`hide_tb` which were previously being shown but that don't need to be to application code now that cancel semantics testing is finished up. - i originally did the same but later commented it all out in `._ipc` since error catch and re-raise instead in higher level layers (above the transport) seems to be a much saner approach. - add catch-n-reraise-from in `MsgStream.send()`/.`receive()` to avoid seeing the depths of `trio` and/or our `._ipc` layers on comms errors. Further this patch adds some refactoring to use the same remote-error shipper routine from both the actor-core in the RPC invoker: - rename it as `try_ship_error_to_remote()` and call it from `._invoke()` as well as it's prior usage. - make it optionally accept `cid: str` a `remote_descr: str` and of course a `hide_tb: bool`. Other misc tweaks: - add some todo notes around `Actor.load_modules()` debug hooking. - tweak the zombie reaper log msg and timeout value ;) --- tractor/_context.py | 9 +- tractor/_ipc.py | 50 +++-- tractor/_portal.py | 2 +- tractor/_runtime.py | 502 +++++------------------------------------- tractor/_spawn.py | 7 +- tractor/_streaming.py | 51 ++++- 6 files changed, 150 insertions(+), 471 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index a31c3b1b..7a562155 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -1198,8 +1198,12 @@ class Context: # TODO: replace all the instances of this!! XD def maybe_raise( self, + + hide_tb: bool = True, **kwargs, + ) -> Exception|None: + __tracebackhide__: bool = hide_tb if re := self._remote_error: return self._maybe_raise_remote_err( re, @@ -1209,8 +1213,10 @@ class Context: def _maybe_raise_remote_err( self, remote_error: Exception, + raise_ctxc_from_self_call: bool = False, raise_overrun_from_self: bool = True, + hide_tb: bool = True, ) -> ( ContextCancelled # `.cancel()` request to far side @@ -1222,6 +1228,7 @@ class Context: a cancellation (if any). ''' + __tracebackhide__: bool = hide_tb our_uid: tuple = self.chan.uid # XXX NOTE XXX: `ContextCancelled`/`StreamOverrun` absorption @@ -1305,7 +1312,7 @@ class Context: # TODO: change to `.wait_for_result()`? async def result( self, - hide_tb: bool = False, + hide_tb: bool = True, ) -> Any|Exception: ''' diff --git a/tractor/_ipc.py b/tractor/_ipc.py index 33110c04..e80a1c35 100644 --- a/tractor/_ipc.py +++ b/tractor/_ipc.py @@ -19,13 +19,14 @@ Inter-process comms abstractions """ from __future__ import annotations -import struct -import platform -from pprint import pformat from collections.abc import ( AsyncGenerator, AsyncIterator, ) +from contextlib import asynccontextmanager as acm +import platform +from pprint import pformat +import struct import typing from typing import ( Any, @@ -35,18 +36,16 @@ from typing import ( TypeVar, ) -from tricycle import BufferedReceiveStream import msgspec +from tricycle import BufferedReceiveStream import trio -from async_generator import asynccontextmanager -from .log import get_logger -from ._exceptions import TransportClosed +from tractor.log import get_logger +from tractor._exceptions import TransportClosed + log = get_logger(__name__) - _is_windows = platform.system() == 'Windows' -log = get_logger(__name__) def get_stream_addrs(stream: trio.SocketStream) -> tuple: @@ -206,7 +205,17 @@ class MsgpackTCPStream(MsgTransport): else: raise - async def send(self, msg: Any) -> None: + async def send( + self, + msg: Any, + + # hide_tb: bool = False, + ) -> None: + ''' + Send a msgpack coded blob-as-msg over TCP. + + ''' + # __tracebackhide__: bool = hide_tb async with self._send_lock: bytes_data: bytes = self.encode(msg) @@ -388,15 +397,28 @@ class Channel: ) return transport - async def send(self, item: Any) -> None: + async def send( + self, + payload: Any, + # 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(item)}\n' + f'{pformat(payload)}\n' ) # type: ignore assert self._transport - await self._transport.send(item) + await self._transport.send( + payload, + # hide_tb=hide_tb, + ) async def recv(self) -> Any: assert self._transport @@ -493,7 +515,7 @@ class Channel: return self._transport.connected() if self._transport else False -@asynccontextmanager +@acm async def _connect_chan( host: str, port: int ) -> typing.AsyncGenerator[Channel, None]: diff --git a/tractor/_portal.py b/tractor/_portal.py index c9f314f3..bd6cf860 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -465,7 +465,7 @@ class Portal: # TODO: if we set this the wrapping `@acm` body will # still be shown (awkwardly) on pdb REPL entry. Ideally # we can similarly annotate that frame to NOT show? - hide_tb: bool = False, + hide_tb: bool = True, # proxied to RPC **kwargs, diff --git a/tractor/_runtime.py b/tractor/_runtime.py index a06b5948..c034bd86 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -1,423 +1,4 @@ -# tractor: structured concurrent "actors". -# Copyright 2018-eternity Tyler Goodlet. - -# This program is free software: you can redistribute it and/or modify -# it under the terms of the GNU Affero General Public License as published by -# the Free Software Foundation, either version 3 of the License, or -# (at your option) any later version. - -# This program is distributed in the hope that it will be useful, -# but WITHOUT ANY WARRANTY; without even the implied warranty of -# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -# GNU Affero General Public License for more details. - -# You should have received a copy of the GNU Affero General Public License -# along with this program. If not, see . - -""" -The fundamental core machinery implementing every "actor" including -the process-local (python-interpreter global) `Actor` state-type -primitive(s), RPC-in-task scheduling, and IPC connectivity and -low-level transport msg handling. - -""" -from __future__ import annotations -from contextlib import ( - ExitStack, - asynccontextmanager as acm, -) -from collections import defaultdict -from functools import partial -from itertools import chain -import importlib -import importlib.util -import inspect -from pprint import pformat -import signal -import sys -from typing import ( - Any, - Callable, - Union, - Coroutine, - TYPE_CHECKING, -) -import uuid -from types import ModuleType -import os -import warnings - -from async_generator import aclosing -from exceptiongroup import BaseExceptionGroup -import trio -from trio import ( - CancelScope, -) -from trio_typing import ( - Nursery, - TaskStatus, -) - -from .msg import NamespacePath -from ._ipc import Channel -from ._context import ( - mk_context, - Context, -) -from .log import get_logger -from ._exceptions import ( - pack_error, - unpack_error, - ModuleNotExposed, - is_multi_cancelled, - ContextCancelled, - TransportClosed, -) -from . import _debug -from ._discovery import get_arbiter -from ._portal import Portal -from . import _state -from . import _mp_fixup_main - - -if TYPE_CHECKING: - from ._supervise import ActorNursery - - -log = get_logger('tractor') - -_gb_mod: ModuleType|None|False = None - - -async def maybe_import_gb(): - global _gb_mod - if _gb_mod is False: - return - - try: - import greenback - _gb_mod = greenback - await greenback.ensure_portal() - - except ModuleNotFoundError: - log.debug( - '`greenback` is not installed.\n' - 'No sync debug support!\n' - ) - _gb_mod = False - - -async def _invoke_non_context( - actor: Actor, - cancel_scope: CancelScope, - ctx: Context, - cid: str, - chan: Channel, - func: Callable, - coro: Coroutine, - kwargs: dict[str, Any], - - treat_as_gen: bool, - is_rpc: bool, - - task_status: TaskStatus[ - Context | BaseException - ] = trio.TASK_STATUS_IGNORED, -): - - # TODO: can we unify this with the `context=True` impl below? - if inspect.isasyncgen(coro): - await chan.send({'functype': 'asyncgen', 'cid': cid}) - # XXX: massive gotcha! If the containing scope - # is cancelled and we execute the below line, - # any ``ActorNursery.__aexit__()`` WON'T be - # triggered in the underlying async gen! So we - # have to properly handle the closing (aclosing) - # of the async gen in order to be sure the cancel - # is propagated! - with cancel_scope as cs: - ctx._scope = cs - task_status.started(ctx) - async with aclosing(coro) as agen: - async for item in agen: - # TODO: can we send values back in here? - # it's gonna require a `while True:` and - # some non-blocking way to retrieve new `asend()` - # values from the channel: - # to_send = await chan.recv_nowait() - # if to_send is not None: - # to_yield = await coro.asend(to_send) - await chan.send({'yield': item, 'cid': cid}) - - log.runtime(f"Finished iterating {coro}") - # TODO: we should really support a proper - # `StopAsyncIteration` system here for returning a final - # value if desired - await chan.send({'stop': True, 'cid': cid}) - - # one way @stream func that gets treated like an async gen - # TODO: can we unify this with the `context=True` impl below? - elif treat_as_gen: - await chan.send({'functype': 'asyncgen', 'cid': cid}) - # 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) - await coro - - if not cs.cancelled_caught: - # task was not cancelled so we can instruct the - # far end async gen to tear down - await chan.send({'stop': True, 'cid': cid}) - 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: - await chan.send({ - 'functype': 'asyncfunc', - 'cid': cid - }) - except ( - trio.ClosedResourceError, - trio.BrokenResourceError, - BrokenPipeError, - ) as ipc_err: - failed_resp = True - if is_rpc: - raise - else: - # TODO: should this be an `.exception()` call? - log.warning( - f'Failed to respond to non-rpc request: {func}\n' - f'{ipc_err}' - ) - - with cancel_scope as cs: - ctx._scope: CancelScope = cs - task_status.started(ctx) - result = await coro - fname: str = func.__name__ - log.runtime( - 'RPC complete:\n' - f'task: {ctx._task}\n' - f'|_cid={ctx.cid}\n' - f'|_{fname}() -> {pformat(result)}\n' - ) - - # NOTE: only send result if we know IPC isn't down - if ( - not failed_resp - and chan.connected() - ): - try: - await chan.send( - {'return': result, - 'cid': cid} - ) - except ( - BrokenPipeError, - trio.BrokenResourceError, - ): - log.warning( - 'Failed to return result:\n' - f'{func}@{actor.uid}\n' - f'remote chan: {chan.uid}' - ) - -@acm -async def _errors_relayed_via_ipc( - actor: Actor, - chan: Channel, - ctx: Context, - is_rpc: bool, - - hide_tb: bool = False, - debug_kbis: bool = False, - task_status: TaskStatus[ - Context | BaseException - ] = trio.TASK_STATUS_IGNORED, - -) -> None: - __tracebackhide__: bool = hide_tb # TODO: use hide_tb here? - try: - yield # run RPC invoke body - - # box and ship RPC errors for wire-transit via - # the task's requesting parent IPC-channel. - except ( - Exception, - BaseExceptionGroup, - KeyboardInterrupt, - ) as err: - - # always hide this frame from debug REPL if the crash - # originated from an rpc task and we DID NOT fail due to - # an IPC transport error! - if ( - is_rpc - and chan.connected() - ): - __tracebackhide__: bool = hide_tb - - if not is_multi_cancelled(err): - - # TODO: maybe we'll want different "levels" of debugging - # eventualy such as ('app', 'supervisory', 'runtime') ? - - # if not isinstance(err, trio.ClosedResourceError) and ( - # if not is_multi_cancelled(err) and ( - - entered_debug: bool = False - if ( - ( - not isinstance(err, ContextCancelled) - or ( - isinstance(err, ContextCancelled) - and ctx._cancel_called - - # if the root blocks the debugger lock request from a child - # we will get a remote-cancelled condition. - and ctx._enter_debugger_on_cancel - ) - ) - and - ( - not isinstance(err, KeyboardInterrupt) - or ( - isinstance(err, KeyboardInterrupt) - and debug_kbis - ) - ) - ): - # await _debug.pause() - # XXX QUESTION XXX: is there any case where we'll - # want to debug IPC disconnects as a default? - # => I can't think of a reason that inspecting this - # type of failure will be useful for respawns or - # recovery logic - the only case is some kind of - # strange bug in our transport layer itself? Going - # to keep this open ended for now. - entered_debug = await _debug._maybe_enter_pm(err) - - if not entered_debug: - log.exception('Actor crashed:\n') - - # always ship errors back to caller - err_msg: dict[str, dict] = pack_error( - err, - # tb=tb, # TODO: special tb fmting? - cid=ctx.cid, - ) - - # NOTE: the src actor should always be packed into the - # error.. but how should we verify this? - # assert err_msg['src_actor_uid'] - # if not err_msg['error'].get('src_actor_uid'): - # import pdbp; pdbp.set_trace() - - if is_rpc: - try: - await chan.send(err_msg) - - # TODO: tests for this scenario: - # - RPC caller closes connection before getting a response - # should **not** crash this actor.. - except ( - trio.ClosedResourceError, - trio.BrokenResourceError, - BrokenPipeError, - ) as ipc_err: - - # if we can't propagate the error that's a big boo boo - log.exception( - f"Failed to ship error to caller @ {chan.uid} !?\n" - f'{ipc_err}' - - ) - - # error is probably from above coro running code *not from - # the target rpc invocation since a scope was never - # allocated around the coroutine await. - if ctx._scope is None: - # we don't ever raise directly here to allow the - # msg-loop-scheduler to continue running for this - # channel. - task_status.started(err) - - # always reraise KBIs so they propagate at the sys-process - # level. - if isinstance(err, KeyboardInterrupt): - raise - - - # RPC task bookeeping - finally: - try: - ctx, func, is_complete = actor._rpc_tasks.pop( - (chan, ctx.cid) - ) - is_complete.set() - - except KeyError: - if is_rpc: - # If we're cancelled before the task returns then the - # cancel scope will not have been inserted yet - log.warning( - 'RPC task likely errored or cancelled before start?' - f'|_{ctx._task}\n' - f' >> {ctx.repr_rpc}\n' - ) - else: - log.cancel( - 'Failed to de-alloc internal runtime cancel task?\n' - f'|_{ctx._task}\n' - f' >> {ctx.repr_rpc}\n' - ) - - finally: - if not actor._rpc_tasks: - log.runtime("All RPC tasks have completed") - actor._ongoing_rpc_tasks.set() - - -async def _invoke( - - actor: 'Actor', - cid: str, - chan: Channel, - func: Callable, - kwargs: dict[str, Any], - - is_rpc: bool = True, - hide_tb: bool = True, - - task_status: TaskStatus[ - Union[Context, BaseException] - ] = trio.TASK_STATUS_IGNORED, -): - ''' - Schedule a `trio` task-as-func and deliver result(s) over - connected IPC channel. - - This is the core "RPC" `trio.Task` scheduling machinery used to start every - remotely invoked function, normally in `Actor._service_n: Nursery`. - - ''' - __tracebackhide__: bool = hide_tb - treat_as_gen: bool = False - - # possibly a traceback (not sure what typing is for this..) - tb = None +tb = None cancel_scope = CancelScope() # activated cancel scope ref @@ -712,9 +293,13 @@ def _get_mod_abspath(module): return os.path.abspath(module.__file__) -async def try_ship_error_to_parent( +async def try_ship_error_to_remote( channel: Channel, - err: Union[Exception, BaseExceptionGroup], + err: Exception|BaseExceptionGroup, + + cid: str|None = None, + remote_descr: str = 'parent', + hide_tb: bool = True, ) -> None: ''' @@ -723,22 +308,39 @@ async def try_ship_error_to_parent( local cancellation ignored but logged as critical(ly bad). ''' + __tracebackhide__: bool = hide_tb with CancelScope(shield=True): try: - await channel.send( - # NOTE: normally only used for internal runtime errors - # so ship to peer actor without a cid. - pack_error(err) + # NOTE: normally only used for internal runtime errors + # so ship to peer actor without a cid. + msg: dict = pack_error( + err, + cid=cid, + + # TODO: special tb fmting for ctxc cases? + # tb=tb, ) + # NOTE: the src actor should always be packed into the + # error.. but how should we verify this? + # actor: Actor = _state.current_actor() + # assert err_msg['src_actor_uid'] + # if not err_msg['error'].get('src_actor_uid'): + # import pdbp; pdbp.set_trace() + await channel.send(msg) + + # XXX NOTE XXX in SC terms this is one of the worst things + # that can happen and provides for a 2-general's dilemma.. except ( trio.ClosedResourceError, trio.BrokenResourceError, + BrokenPipeError, ): - # in SC terms this is one of the worst things that can - # happen and provides for a 2-general's dilemma.. + err_msg: dict = msg['error']['tb_str'] log.critical( - f'Failed to ship error to parent ' - f'{channel.uid}, IPC transport failure!' + 'IPC transport failure -> ' + f'failed to ship error to {remote_descr}!\n\n' + f'X=> {channel.uid}\n\n' + f'{err_msg}\n' ) @@ -896,7 +498,10 @@ class Actor: log.runtime(f"{uid} successfully connected back to us") return event, self._peers[uid][-1] - def load_modules(self) -> None: + def load_modules( + self, + debug_mode: bool = False, + ) -> None: ''' Load allowed RPC modules locally (after fork). @@ -928,7 +533,9 @@ class Actor: except ModuleNotFoundError: # it is expected the corresponding `ModuleNotExposed` error # will be raised later - log.error(f"Failed to import {modpath} in {self.name}") + log.error( + f"Failed to import {modpath} in {self.name}" + ) raise def _get_rpc_func(self, ns, funcname): @@ -1759,7 +1366,7 @@ class Actor: log.cancel( 'Cancel request for RPC task\n\n' - f'<= Actor.cancel_task(): {requesting_uid}\n\n' + f'<= Actor._cancel_task(): {requesting_uid}\n\n' f'=> {ctx._task}\n' f' |_ >> {ctx.repr_rpc}\n' # f' >> Actor._cancel_task() => {ctx._task}\n' @@ -2021,11 +1628,6 @@ async def async_main( if accept_addr_rent is not None: accept_addr = accept_addr_rent - # load exposed/allowed RPC modules - # XXX: do this **after** establishing a channel to the parent - # but **before** starting the message loop for that channel - # such that import errors are properly propagated upwards - actor.load_modules() # The "root" nursery ensures the channel with the immediate # parent is kept alive as a resilient service until @@ -2043,7 +1645,25 @@ async def async_main( actor._service_n = service_nursery assert actor._service_n - # Startup up the channel server with, + # load exposed/allowed RPC modules + # XXX: do this **after** establishing a channel to the parent + # but **before** starting the message loop for that channel + # such that import errors are properly propagated upwards + actor.load_modules() + + # XXX TODO XXX: figuring out debugging of this + # would somemwhat guarantee "self-hosted" runtime + # debugging (since it hits all the ede cases?) + # + # `tractor.pause()` right? + # try: + # actor.load_modules() + # except ModuleNotFoundError as err: + # _debug.pause_from_sync() + # import pdbp; pdbp.set_trace() + # raise + + # Startup up the transport(-channel) server with, # - subactor: the bind address is sent by our parent # over our established channel # - root actor: the ``accept_addr`` passed to this method @@ -2122,7 +1742,7 @@ async def async_main( ) if actor._parent_chan: - await try_ship_error_to_parent( + await try_ship_error_to_remote( actor._parent_chan, err, ) @@ -2532,7 +2152,7 @@ async def process_messages( log.exception("Actor errored:") if actor._parent_chan: - await try_ship_error_to_parent( + await try_ship_error_to_remote( actor._parent_chan, err, ) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index 58c187f0..a2643876 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -215,7 +215,7 @@ async def cancel_on_completion( async def hard_kill( proc: trio.Process, - terminate_after: int = 3, + terminate_after: int = 1.6, # NOTE: for mucking with `.pause()`-ing inside the runtime # whilst also hacking on it XD @@ -281,8 +281,11 @@ async def hard_kill( # zombies (as a feature) we ask the OS to do send in the # removal swad as the last resort. if cs.cancelled_caught: + # TODO: toss in the skynet-logo face as ascii art? log.critical( - 'Well, the #ZOMBIE_LORD_IS_HERE# to collect\n' + # 'Well, the #ZOMBIE_LORD_IS_HERE# to collect\n' + '#T-800 deployed to collect zombie B0\n' + f'|\n' f'|_{proc}\n' ) proc.kill() diff --git a/tractor/_streaming.py b/tractor/_streaming.py index 50a32ae9..149bb350 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -114,13 +114,19 @@ class MsgStream(trio.abc.Channel): stream=self, ) - async def receive(self): + async def receive( + self, + + hide_tb: bool = True, + ): ''' Receive a single msg from the IPC transport, the next in sequence sent by the far end task (possibly in order as determined by the underlying protocol). ''' + __tracebackhide__: bool = hide_tb + # NOTE: `trio.ReceiveChannel` implements # EOC handling as follows (aka uses it # to gracefully exit async for loops): @@ -139,7 +145,7 @@ class MsgStream(trio.abc.Channel): if self._closed: raise self._closed - src_err: Exception|None = None + src_err: Exception|None = None # orig tb try: try: msg = await self._rx_chan.receive() @@ -186,7 +192,7 @@ class MsgStream(trio.abc.Channel): # TODO: Locally, we want to close this stream gracefully, by # terminating any local consumers tasks deterministically. - # One we have broadcast support, we **don't** want to be + # 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. @@ -237,7 +243,12 @@ class MsgStream(trio.abc.Channel): raise_ctxc_from_self_call=True, ) - raise src_err # propagate + # propagate any error but hide low-level frames from + # caller by default. + if hide_tb: + raise type(src_err)(*src_err.args) from src_err + else: + raise src_err async def aclose(self) -> list[Exception|dict]: ''' @@ -475,23 +486,39 @@ class MsgStream(trio.abc.Channel): async def send( self, - data: Any + data: Any, + + hide_tb: bool = True, ) -> None: ''' Send a message over this stream to the far end. ''' - if self._ctx._remote_error: - raise self._ctx._remote_error # from None + __tracebackhide__: bool = hide_tb + self._ctx.maybe_raise() if self._closed: raise self._closed - # raise trio.ClosedResourceError('This stream was already closed') - await self._ctx.chan.send({ - 'yield': data, - 'cid': self._ctx.cid, - }) + try: + await self._ctx.chan.send( + payload={ + 'yield': data, + 'cid': self._ctx.cid, + }, + # hide_tb=hide_tb, + ) + except ( + trio.ClosedResourceError, + trio.BrokenResourceError, + BrokenPipeError, + ) as trans_err: + if hide_tb: + raise type(trans_err)( + *trans_err.args + ) from trans_err + else: + raise def stream(func: Callable) -> Callable: From 4f7823cf557a75b0e43ab3ea06ad3636845ed9bf Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 13 Mar 2024 11:59:39 -0400 Subject: [PATCH 132/146] Move `Portal.open_context()` impl to `._context` Finally, since normally you need the content from `._context.Context` and surroundings in order to effectively grok `Portal.open_context()` anyways, might as well move the impl to the ctx module as `open_context_from_portal()` and just bind it on the `Portal` class def. Associated/required tweaks: - avoid circ import on `.devx` by only import `.maybe_wait_for_debugger()` when debug mode is set. - drop `async_generator` usage, not sure why this hadn't already been changed to `contextlib`? - use `@acm` alias throughout `._portal` --- tractor/_context.py | 545 +++++++++++++++++++++++++++++++++++++++++- tractor/_portal.py | 566 +------------------------------------------- 2 files changed, 553 insertions(+), 558 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index 7a562155..55902281 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -43,7 +43,6 @@ import warnings import trio -from .msg import NamespacePath from ._exceptions import ( ContextCancelled, InternalError, @@ -51,11 +50,16 @@ from ._exceptions import ( StreamOverrun, pack_error, unpack_error, + _raise_from_no_key_in_msg, ) from .log import get_logger +from .msg import NamespacePath from ._ipc import Channel from ._streaming import MsgStream -from ._state import current_actor +from ._state import ( + current_actor, + debug_mode, +) if TYPE_CHECKING: from ._portal import Portal @@ -1021,6 +1025,8 @@ class Context: assert self._scope self._scope.cancel() + # TODO? should we move this to `._streaming` much like we + # moved `Portal.open_context()`'s def to this mod? @acm async def open_stream( self, @@ -1848,6 +1854,541 @@ class Context: return False +@acm +async def open_context_from_portal( + portal: Portal, + func: Callable, + + allow_overruns: bool = False, + + # TODO: if we set this the wrapping `@acm` body will + # still be shown (awkwardly) on pdb REPL entry. Ideally + # we can similarly annotate that frame to NOT show? + hide_tb: bool = True, + + # proxied to RPC + **kwargs, + +) -> AsyncGenerator[tuple[Context, Any], None]: + ''' + Open an inter-actor "task context"; a remote task is + scheduled and cancel-scope-state-linked to a `trio.run()` across + memory boundaries in another actor's runtime. + + This is an `@acm` API bound as `Portal.open_context()` which + allows for deterministic setup and teardown of a remotely + scheduled task in another remote actor. Once opened, the 2 now + "linked" tasks run completely in parallel in each actor's + runtime with their enclosing `trio.CancelScope`s kept in + a synced state wherein if either side errors or cancels an + equivalent error is relayed to the other side via an SC-compat + IPC protocol. + + The yielded `tuple` is a pair delivering a `tractor.Context` + and any first value "sent" by the "callee" task via a call + to `Context.started()`; this side of the + context does not unblock until the "callee" task calls + `.started()` in similar style to `trio.Nursery.start()`. + 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. + + The yielded ``Context`` instance further allows for opening + bidirectional streams, explicit cancellation and + structurred-concurrency-synchronized final result-msg + collection. See ``tractor.Context`` for more details. + + ''' + __tracebackhide__: bool = hide_tb + + # conduct target func method structural checks + if not inspect.iscoroutinefunction(func) and ( + getattr(func, '_tractor_contex_function', False) + ): + raise TypeError( + f'{func} must be an async generator function!') + + # TODO: i think from here onward should probably + # just be factored into an `@acm` inside a new + # a new `_context.py` mod. + nsf = NamespacePath.from_ref(func) + + # XXX NOTE XXX: currenly we do NOT allow opening a contex + # with "self" since the local feeder mem-chan processing + # is not built for it. + if portal.channel.uid == portal.actor.uid: + raise RuntimeError( + '** !! Invalid Operation !! **\n' + 'Can not open an IPC ctx with the local actor!\n' + f'|_{portal.actor}\n' + ) + + ctx: Context = await portal.actor.start_remote_task( + portal.channel, + nsf=nsf, + kwargs=kwargs, + + # NOTE: it's imporant to expose this since you might + # get the case where the parent who opened the context does + # not open a stream until after some slow startup/init + # period, in which case when the first msg is read from + # the feeder mem chan, say when first calling + # `Context.open_stream(allow_overruns=True)`, the overrun condition will be + # raised before any ignoring of overflow msgs can take + # place.. + allow_overruns=allow_overruns, + ) + + assert ctx._remote_func_type == 'context' + msg: dict = await ctx._recv_chan.receive() + + try: + # the "first" value here is delivered by the callee's + # ``Context.started()`` call. + first: Any = msg['started'] + ctx._started_called: bool = True + + except KeyError as src_error: + _raise_from_no_key_in_msg( + ctx=ctx, + msg=msg, + src_err=src_error, + log=log, + expect_key='started', + ) + + ctx._portal: Portal = portal + uid: tuple = portal.channel.uid + cid: str = ctx.cid + + # placeholder for any exception raised in the runtime + # or by user tasks which cause this context's closure. + scope_err: BaseException|None = None + ctxc_from_callee: ContextCancelled|None = None + try: + async with trio.open_nursery() as nurse: + + # NOTE: used to start overrun queuing tasks + ctx._scope_nursery: trio.Nursery = nurse + ctx._scope: trio.CancelScope = nurse.cancel_scope + + # deliver context instance and .started() msg value + # in enter tuple. + yield ctx, first + + # ??TODO??: do we still want to consider this or is + # the `else:` block handling via a `.result()` + # call below enough?? + # -[ ] pretty sure `.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 + # that we can re-use it around the `yield` ^ here + # or vice versa? + # + # NOTE: between the caller exiting and arriving + # here the far end may have sent a ctxc-msg or + # other error, so check for it here immediately + # and maybe raise so as to engage the ctxc + # handling block below! + # + # if re := ctx._remote_error: + # maybe_ctxc: ContextCancelled|None = ctx._maybe_raise_remote_err( + # re, + # # TODO: do we want this to always raise? + # # - means that on self-ctxc, if/when the + # # block is exited before the msg arrives + # # but then the msg during __exit__ + # # calling we may not activate the + # # ctxc-handler block below? should we + # # be? + # # - if there's a remote error that arrives + # # after the child has exited, we won't + # # handle until the `finally:` block + # # where `.result()` is always called, + # # again in which case we handle it + # # differently then in the handler block + # # that would normally engage from THIS + # # block? + # raise_ctxc_from_self_call=True, + # ) + # ctxc_from_callee = maybe_ctxc + + # when in allow_overruns mode there may be + # lingering overflow sender tasks remaining? + if nurse.child_tasks: + # XXX: ensure we are in overrun state + # with ``._allow_overruns=True`` bc otherwise + # there should be no tasks in this nursery! + if ( + not ctx._allow_overruns + or len(nurse.child_tasks) > 1 + ): + raise InternalError( + 'Context has sub-tasks but is ' + 'not in `allow_overruns=True` mode!?' + ) + + # ensure we cancel all overflow sender + # tasks started in the nursery when + # `._allow_overruns == True`. + # + # NOTE: this means `._scope.cancelled_caught` + # will prolly be set! not sure if that's + # non-ideal or not ??? + ctx._scope.cancel() + + # XXX NOTE XXX: maybe shield against + # self-context-cancellation (which raises a local + # `ContextCancelled`) when requested (via + # `Context.cancel()`) by the same task (tree) which entered + # THIS `.open_context()`. + # + # NOTE: There are 2 operating cases for a "graceful cancel" + # of a `Context`. In both cases any `ContextCancelled` + # raised in this scope-block came from a transport msg + # relayed from some remote-actor-task which our runtime set + # as to `Context._remote_error` + # + # the CASES: + # + # - if that context IS THE SAME ONE that called + # `Context.cancel()`, we want to absorb the error + # silently and let this `.open_context()` block to exit + # without raising, ideally eventually receiving the ctxc + # ack msg thus resulting in `ctx.cancel_acked == True`. + # + # - if it is from some OTHER context (we did NOT call + # `.cancel()`), we want to re-RAISE IT whilst also + # setting our own ctx's "reason for cancel" to be that + # other context's cancellation condition; we set our + # `.canceller: tuple[str, str]` to be same value as + # caught here in a `ContextCancelled.canceller`. + # + # AGAIN to restate the above, there are 2 cases: + # + # 1-some other context opened in this `.open_context()` + # block cancelled due to a self or peer cancellation + # request in which case we DO let the error bubble to the + # opener. + # + # 2-THIS "caller" task somewhere invoked `Context.cancel()` + # and received a `ContextCanclled` from the "callee" + # task, in which case we mask the `ContextCancelled` from + # bubbling to this "caller" (much like how `trio.Nursery` + # swallows any `trio.Cancelled` bubbled by a call to + # `Nursery.cancel_scope.cancel()`) + except ContextCancelled as ctxc: + scope_err = ctxc + ctx._local_error: BaseException = scope_err + ctxc_from_callee = ctxc + + # XXX TODO XXX: FIX THIS debug_mode BUGGGG!!! + # using this code and then resuming the REPL will + # cause a SIGINT-ignoring HANG! + # -> prolly due to a stale debug lock entry.. + # -[ ] USE `.stackscope` to demonstrate that (possibly + # documenting it as a definittive example of + # debugging the tractor-runtime itself using it's + # own `.devx.` tooling! + # + # await _debug.pause() + + # CASE 2: context was cancelled by local task calling + # `.cancel()`, we don't raise and the exit block should + # exit silently. + if ( + ctx._cancel_called + and + ctxc is ctx._remote_error + and + ctxc.canceller == portal.actor.uid + ): + log.cancel( + f'Context (cid=[{ctx.cid[-6:]}..] cancelled gracefully with:\n' + f'{ctxc}' + ) + # CASE 1: this context was never cancelled via a local + # task (tree) having called `Context.cancel()`, raise + # the error since it was caused by someone else + # -> probably a remote peer! + else: + raise + + # the above `._scope` can be cancelled due to: + # 1. an explicit self cancel via `Context.cancel()` or + # `Actor.cancel()`, + # 2. any "callee"-side remote error, possibly also a cancellation + # request by some peer, + # 3. any "caller" (aka THIS scope's) local error raised in the above `yield` + except ( + # CASE 3: standard local error in this caller/yieldee + Exception, + + # CASES 1 & 2: can manifest as a `ctx._scope_nursery` + # exception-group of, + # + # 1.-`trio.Cancelled`s, since + # `._scope.cancel()` will have been called + # (transitively by the runtime calling + # `._deliver_msg()`) and any `ContextCancelled` + # eventually absorbed and thus absorbed/supressed in + # any `Context._maybe_raise_remote_err()` call. + # + # 2.-`BaseExceptionGroup[ContextCancelled | RemoteActorError]` + # from any error delivered from the "callee" side + # 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()` + # `._maybe_raise_remote_err()` will be transitively + # called and the remote error raised causing all + # tasks to be cancelled. + # NOTE: ^ this case always can happen if any + # overrun handler tasks were spawned! + BaseExceptionGroup, + + trio.Cancelled, # NOTE: NOT from inside the ctx._scope + KeyboardInterrupt, + + ) as caller_err: + scope_err = caller_err + ctx._local_error: BaseException = scope_err + + # XXX: ALWAYS request the context to CANCEL ON any ERROR. + # NOTE: `Context.cancel()` is conversely NEVER CALLED in + # the `ContextCancelled` "self cancellation absorbed" case + # handled in the block above ^^^ !! + # await _debug.pause() + log.cancel( + 'Context terminated due to\n\n' + f'.outcome => {ctx.repr_outcome()}\n' + ) + + if debug_mode(): + # async with _debug.acquire_debug_lock(portal.actor.uid): + # pass + # TODO: factor ^ into below for non-root cases? + # + from .devx import maybe_wait_for_debugger + was_acquired: bool = await maybe_wait_for_debugger( + header_msg=( + 'Delaying `ctx.cancel()` until debug lock ' + 'acquired..\n' + ), + ) + if was_acquired: + log.pdb( + 'Acquired debug lock! ' + 'Calling `ctx.cancel()`!\n' + ) + + # we don't need to cancel the callee if it already + # told us it's cancelled ;p + if ctxc_from_callee is None: + try: + await ctx.cancel() + except ( + trio.BrokenResourceError, + trio.ClosedResourceError, + ): + log.warning( + 'IPC connection for context is broken?\n' + f'task:{cid}\n' + f'actor:{uid}' + ) + + raise # duh + + # no local scope error, the "clean exit with a result" case. + else: + if ctx.chan.connected(): + log.runtime( + 'Waiting on final context result for\n' + f'peer: {uid}\n' + f'|_{ctx._task}\n' + ) + # XXX NOTE XXX: the below call to + # `Context.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 + # via a call to + # `Context._maybe_cancel_and_set_remote_error()`. + # As per `Context._deliver_msg()`, that error IS + # 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() + except BaseException as berr: + # on normal teardown, if we get some error + # raised in `Context.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 + # this task didn't know until final teardown + # / value collection. + scope_err = berr + ctx._local_error: BaseException = scope_err + raise + + # yes! this worx Bp + # from .devx import _debug + # await _debug.pause() + + # an exception type boxed in a `RemoteActorError` + # is returned (meaning it was obvi not raised) + # that we want to log-report on. + msgdata: str|None = getattr( + result_or_err, + 'msgdata', + None + ) + match (msgdata, result_or_err): + case ( + {'tb_str': tbstr}, + ContextCancelled(), + ): + log.cancel(tbstr) + + case ( + {'tb_str': tbstr}, + RemoteActorError(), + ): + log.exception( + 'Context remotely errored!\n' + f'<= peer: {uid}\n' + f' |_ {nsf}()\n\n' + + f'{tbstr}' + ) + case (None, _): + log.runtime( + 'Context returned final result from callee task:\n' + f'<= peer: {uid}\n' + f' |_ {nsf}()\n\n' + + f'`{result_or_err}`\n' + ) + + finally: + # XXX: (MEGA IMPORTANT) if this is a root opened process we + # wait for any immediate child in debug before popping the + # context from the runtime msg loop otherwise inside + # ``Actor._push_result()`` the msg will be discarded and in + # the case where that msg is global debugger unlock (via + # a "stop" msg for a stream), this can result in a deadlock + # where the root is waiting on the lock to clear but the + # child has already cleared it and clobbered IPC. + if debug_mode(): + from .devx import maybe_wait_for_debugger + await maybe_wait_for_debugger() + + # though it should be impossible for any tasks + # operating *in* this scope to have survived + # we tear down the runtime feeder chan last + # to avoid premature stream clobbers. + if ( + (rxchan := ctx._recv_chan) + + # maybe TODO: yes i know the below check is + # touching `trio` memchan internals..BUT, there are + # only a couple ways to avoid a `trio.Cancelled` + # bubbling from the `.aclose()` call below: + # + # - catch and mask it via the cancel-scope-shielded call + # as we are rn (manual and frowned upon) OR, + # - specially handle the case where `scope_err` is + # one of {`BaseExceptionGroup`, `trio.Cancelled`} + # and then presume that the `.aclose()` call will + # raise a `trio.Cancelled` and just don't call it + # in those cases.. + # + # that latter approach is more logic, LOC, and more + # convoluted so for now stick with the first + # psuedo-hack-workaround where we just try to avoid + # the shielded call as much as we can detect from + # the memchan's `._closed` state.. + # + # XXX MOTIVATION XXX-> we generally want to raise + # any underlying actor-runtime/internals error that + # surfaces from a bug in tractor itself so it can + # be easily detected/fixed AND, we also want to + # minimize noisy runtime tracebacks (normally due + # to the cross-actor linked task scope machinery + # teardown) displayed to user-code and instead only + # displaying `ContextCancelled` traces where the + # cause of crash/exit IS due to something in + # user/app code on either end of the context. + and not rxchan._closed + ): + # XXX NOTE XXX: and again as per above, we mask any + # `trio.Cancelled` raised here so as to NOT mask + # out any exception group or legit (remote) ctx + # error that sourced from the remote task or its + # runtime. + # + # NOTE: further, this should be the only place the + # underlying feeder channel is + # once-and-only-CLOSED! + with trio.CancelScope(shield=True): + await ctx._recv_chan.aclose() + + # XXX: we always raise remote errors locally and + # generally speaking mask runtime-machinery related + # multi-`trio.Cancelled`s. As such, any `scope_error` + # which was the underlying cause of this context's exit + # should be stored as the `Context._local_error` and + # used in determining `Context.cancelled_caught: bool`. + if scope_err is not None: + # sanity, tho can remove? + assert ctx._local_error is scope_err + # ctx._local_error: BaseException = scope_err + # etype: Type[BaseException] = type(scope_err) + + # CASE 2 + if ( + ctx._cancel_called + and ctx.cancel_acked + ): + log.cancel( + 'Context cancelled by caller task\n' + f'|_{ctx._task}\n\n' + + f'{repr(scope_err)}\n' + ) + + # TODO: should we add a `._cancel_req_received` + # flag to determine if the callee manually called + # `ctx.cancel()`? + # -[ ] going to need a cid check no? + + # CASE 1 + else: + outcome_str: str = ctx.repr_outcome( + show_error_fields=True, + # type_only=True, + ) + log.cancel( + f'Context terminated due to local scope error:\n\n' + f'{ctx.chan.uid} => {outcome_str}\n' + ) + + # FINALLY, remove the context from runtime tracking and + # exit! + log.runtime( + 'Removing IPC ctx opened with peer\n' + f'{uid}\n' + f'|_{ctx}\n' + ) + portal.actor._contexts.pop( + (uid, cid), + None, + ) + + def mk_context( chan: Channel, cid: str, diff --git a/tractor/_portal.py b/tractor/_portal.py index bd6cf860..0ca44483 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -24,6 +24,7 @@ OS processes, possibly on different (hardware) hosts. ''' from __future__ import annotations +from contextlib import asynccontextmanager as acm import importlib import inspect from typing import ( @@ -37,30 +38,21 @@ from dataclasses import dataclass import warnings import trio -from async_generator import asynccontextmanager from .trionics import maybe_open_nursery -from .devx import ( - # _debug, - maybe_wait_for_debugger, -) from ._state import ( current_actor, - debug_mode, ) from ._ipc import Channel from .log import get_logger from .msg import NamespacePath from ._exceptions import ( - InternalError, - _raise_from_no_key_in_msg, unpack_error, NoResult, - ContextCancelled, - RemoteActorError, ) from ._context import ( Context, + open_context_from_portal, ) from ._streaming import ( MsgStream, @@ -392,7 +384,7 @@ class Portal: self.channel, ) - @asynccontextmanager + @acm async def open_stream_from( self, async_gen_func: Callable, # typing: ignore @@ -449,550 +441,12 @@ class Portal: # await recv_chan.aclose() self._streams.remove(rchan) - # TODO: move this impl to `._context` mod and - # instead just bind it here as a method so that the logic - # for ctx stuff stays all in one place (instead of frickin - # having to open this file in tandem every gd time!!! XD) - # - @asynccontextmanager - async def open_context( - - self, - func: Callable, - - allow_overruns: bool = False, - - # TODO: if we set this the wrapping `@acm` body will - # still be shown (awkwardly) on pdb REPL entry. Ideally - # we can similarly annotate that frame to NOT show? - hide_tb: bool = True, - - # proxied to RPC - **kwargs, - - ) -> AsyncGenerator[tuple[Context, Any], None]: - ''' - Open an inter-actor "task context"; a remote task is - scheduled and cancel-scope-state-linked to a `trio.run()` across - memory boundaries in another actor's runtime. - - This is an `@acm` API which allows for deterministic setup - and teardown of a remotely scheduled task in another remote - actor. Once opened, the 2 now "linked" tasks run completely - in parallel in each actor's runtime with their enclosing - `trio.CancelScope`s kept in a synced state wherein if - either side errors or cancels an equivalent error is - relayed to the other side via an SC-compat IPC protocol. - - The yielded `tuple` is a pair delivering a `tractor.Context` - and any first value "sent" by the "callee" task via a call - to `Context.started()`; this side of the - context does not unblock until the "callee" task calls - `.started()` in similar style to `trio.Nursery.start()`. - 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. - - The yielded ``Context`` instance further allows for opening - bidirectional streams, explicit cancellation and - structurred-concurrency-synchronized final result-msg - collection. See ``tractor.Context`` for more details. - - ''' - __tracebackhide__: bool = hide_tb - - # conduct target func method structural checks - if not inspect.iscoroutinefunction(func) and ( - getattr(func, '_tractor_contex_function', False) - ): - raise TypeError( - f'{func} must be an async generator function!') - - # TODO: i think from here onward should probably - # just be factored into an `@acm` inside a new - # a new `_context.py` mod. - nsf = NamespacePath.from_ref(func) - - # XXX NOTE XXX: currenly we do NOT allow opening a contex - # with "self" since the local feeder mem-chan processing - # is not built for it. - if self.channel.uid == self.actor.uid: - raise RuntimeError( - '** !! Invalid Operation !! **\n' - 'Can not open an IPC ctx with the local actor!\n' - f'|_{self.actor}\n' - ) - - ctx: Context = await self.actor.start_remote_task( - self.channel, - nsf=nsf, - kwargs=kwargs, - - # NOTE: it's imporant to expose this since you might - # get the case where the parent who opened the context does - # not open a stream until after some slow startup/init - # period, in which case when the first msg is read from - # the feeder mem chan, say when first calling - # `Context.open_stream(allow_overruns=True)`, the overrun condition will be - # raised before any ignoring of overflow msgs can take - # place.. - allow_overruns=allow_overruns, - ) - - assert ctx._remote_func_type == 'context' - msg: dict = await ctx._recv_chan.receive() - - try: - # the "first" value here is delivered by the callee's - # ``Context.started()`` call. - first: Any = msg['started'] - ctx._started_called: bool = True - - except KeyError as src_error: - _raise_from_no_key_in_msg( - ctx=ctx, - msg=msg, - src_err=src_error, - log=log, - expect_key='started', - ) - - ctx._portal: Portal = self - uid: tuple = self.channel.uid - cid: str = ctx.cid - - # placeholder for any exception raised in the runtime - # or by user tasks which cause this context's closure. - scope_err: BaseException|None = None - ctxc_from_callee: ContextCancelled|None = None - try: - async with trio.open_nursery() as nurse: - - # NOTE: used to start overrun queuing tasks - ctx._scope_nursery: trio.Nursery = nurse - ctx._scope: trio.CancelScope = nurse.cancel_scope - - # deliver context instance and .started() msg value - # in enter tuple. - yield ctx, first - - # ??TODO??: do we still want to consider this or is - # the `else:` block handling via a `.result()` - # call below enough?? - # -[ ] pretty sure `.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 - # that we can re-use it around the `yield` ^ here - # or vice versa? - # - # NOTE: between the caller exiting and arriving - # here the far end may have sent a ctxc-msg or - # other error, so check for it here immediately - # and maybe raise so as to engage the ctxc - # handling block below! - # - # if re := ctx._remote_error: - # maybe_ctxc: ContextCancelled|None = ctx._maybe_raise_remote_err( - # re, - # # TODO: do we want this to always raise? - # # - means that on self-ctxc, if/when the - # # block is exited before the msg arrives - # # but then the msg during __exit__ - # # calling we may not activate the - # # ctxc-handler block below? should we - # # be? - # # - if there's a remote error that arrives - # # after the child has exited, we won't - # # handle until the `finally:` block - # # where `.result()` is always called, - # # again in which case we handle it - # # differently then in the handler block - # # that would normally engage from THIS - # # block? - # raise_ctxc_from_self_call=True, - # ) - # ctxc_from_callee = maybe_ctxc - - # when in allow_overruns mode there may be - # lingering overflow sender tasks remaining? - if nurse.child_tasks: - # XXX: ensure we are in overrun state - # with ``._allow_overruns=True`` bc otherwise - # there should be no tasks in this nursery! - if ( - not ctx._allow_overruns - or len(nurse.child_tasks) > 1 - ): - raise InternalError( - 'Context has sub-tasks but is ' - 'not in `allow_overruns=True` mode!?' - ) - - # ensure we cancel all overflow sender - # tasks started in the nursery when - # `._allow_overruns == True`. - # - # NOTE: this means `._scope.cancelled_caught` - # will prolly be set! not sure if that's - # non-ideal or not ??? - ctx._scope.cancel() - - # XXX NOTE XXX: maybe shield against - # self-context-cancellation (which raises a local - # `ContextCancelled`) when requested (via - # `Context.cancel()`) by the same task (tree) which entered - # THIS `.open_context()`. - # - # NOTE: There are 2 operating cases for a "graceful cancel" - # of a `Context`. In both cases any `ContextCancelled` - # raised in this scope-block came from a transport msg - # relayed from some remote-actor-task which our runtime set - # as to `Context._remote_error` - # - # the CASES: - # - # - if that context IS THE SAME ONE that called - # `Context.cancel()`, we want to absorb the error - # silently and let this `.open_context()` block to exit - # without raising, ideally eventually receiving the ctxc - # ack msg thus resulting in `ctx.cancel_acked == True`. - # - # - if it is from some OTHER context (we did NOT call - # `.cancel()`), we want to re-RAISE IT whilst also - # setting our own ctx's "reason for cancel" to be that - # other context's cancellation condition; we set our - # `.canceller: tuple[str, str]` to be same value as - # caught here in a `ContextCancelled.canceller`. - # - # AGAIN to restate the above, there are 2 cases: - # - # 1-some other context opened in this `.open_context()` - # block cancelled due to a self or peer cancellation - # request in which case we DO let the error bubble to the - # opener. - # - # 2-THIS "caller" task somewhere invoked `Context.cancel()` - # and received a `ContextCanclled` from the "callee" - # task, in which case we mask the `ContextCancelled` from - # bubbling to this "caller" (much like how `trio.Nursery` - # swallows any `trio.Cancelled` bubbled by a call to - # `Nursery.cancel_scope.cancel()`) - except ContextCancelled as ctxc: - scope_err = ctxc - ctx._local_error: BaseException = scope_err - ctxc_from_callee = ctxc - - # XXX TODO XXX: FIX THIS debug_mode BUGGGG!!! - # using this code and then resuming the REPL will - # cause a SIGINT-ignoring HANG! - # -> prolly due to a stale debug lock entry.. - # -[ ] USE `.stackscope` to demonstrate that (possibly - # documenting it as a definittive example of - # debugging the tractor-runtime itself using it's - # own `.devx.` tooling! - # - # await _debug.pause() - - # CASE 2: context was cancelled by local task calling - # `.cancel()`, we don't raise and the exit block should - # exit silently. - if ( - ctx._cancel_called - and - ctxc is ctx._remote_error - and - ctxc.canceller == self.actor.uid - ): - log.cancel( - f'Context (cid=[{ctx.cid[-6:]}..] cancelled gracefully with:\n' - f'{ctxc}' - ) - # CASE 1: this context was never cancelled via a local - # task (tree) having called `Context.cancel()`, raise - # the error since it was caused by someone else - # -> probably a remote peer! - else: - raise - - # the above `._scope` can be cancelled due to: - # 1. an explicit self cancel via `Context.cancel()` or - # `Actor.cancel()`, - # 2. any "callee"-side remote error, possibly also a cancellation - # request by some peer, - # 3. any "caller" (aka THIS scope's) local error raised in the above `yield` - except ( - # CASE 3: standard local error in this caller/yieldee - Exception, - - # CASES 1 & 2: can manifest as a `ctx._scope_nursery` - # exception-group of, - # - # 1.-`trio.Cancelled`s, since - # `._scope.cancel()` will have been called - # (transitively by the runtime calling - # `._deliver_msg()`) and any `ContextCancelled` - # eventually absorbed and thus absorbed/supressed in - # any `Context._maybe_raise_remote_err()` call. - # - # 2.-`BaseExceptionGroup[ContextCancelled | RemoteActorError]` - # from any error delivered from the "callee" side - # 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()` - # `._maybe_raise_remote_err()` will be transitively - # called and the remote error raised causing all - # tasks to be cancelled. - # NOTE: ^ this case always can happen if any - # overrun handler tasks were spawned! - BaseExceptionGroup, - - trio.Cancelled, # NOTE: NOT from inside the ctx._scope - KeyboardInterrupt, - - ) as caller_err: - scope_err = caller_err - ctx._local_error: BaseException = scope_err - - # XXX: ALWAYS request the context to CANCEL ON any ERROR. - # NOTE: `Context.cancel()` is conversely NEVER CALLED in - # the `ContextCancelled` "self cancellation absorbed" case - # handled in the block above ^^^ !! - # await _debug.pause() - log.cancel( - 'Context terminated due to\n\n' - f'.outcome => {ctx.repr_outcome()}\n' - ) - - if debug_mode(): - # async with _debug.acquire_debug_lock(self.actor.uid): - # pass - # TODO: factor ^ into below for non-root cases? - was_acquired: bool = await maybe_wait_for_debugger( - header_msg=( - 'Delaying `ctx.cancel()` until debug lock ' - 'acquired..\n' - ), - ) - if was_acquired: - log.pdb( - 'Acquired debug lock! ' - 'Calling `ctx.cancel()`!\n' - ) - - - # we don't need to cancel the callee if it already - # told us it's cancelled ;p - if ctxc_from_callee is None: - try: - await ctx.cancel() - except ( - trio.BrokenResourceError, - trio.ClosedResourceError, - ): - log.warning( - 'IPC connection for context is broken?\n' - f'task:{cid}\n' - f'actor:{uid}' - ) - - raise # duh - - # no local scope error, the "clean exit with a result" case. - else: - if ctx.chan.connected(): - log.runtime( - 'Waiting on final context result for\n' - f'peer: {uid}\n' - f'|_{ctx._task}\n' - ) - # XXX NOTE XXX: the below call to - # `Context.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 - # via a call to - # `Context._maybe_cancel_and_set_remote_error()`. - # As per `Context._deliver_msg()`, that error IS - # 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() - except BaseException as berr: - # on normal teardown, if we get some error - # raised in `Context.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 - # this task didn't know until final teardown - # / value collection. - scope_err = berr - ctx._local_error: BaseException = scope_err - raise - - # yes! this worx Bp - # from .devx import _debug - # await _debug.pause() - - # an exception type boxed in a `RemoteActorError` - # is returned (meaning it was obvi not raised) - # that we want to log-report on. - msgdata: str|None = getattr( - result_or_err, - 'msgdata', - None - ) - match (msgdata, result_or_err): - case ( - {'tb_str': tbstr}, - ContextCancelled(), - ): - log.cancel(tbstr) - - case ( - {'tb_str': tbstr}, - RemoteActorError(), - ): - log.exception( - 'Context remotely errored!\n' - f'<= peer: {uid}\n' - f' |_ {nsf}()\n\n' - - f'{tbstr}' - ) - case (None, _): - log.runtime( - 'Context returned final result from callee task:\n' - f'<= peer: {uid}\n' - f' |_ {nsf}()\n\n' - - f'`{result_or_err}`\n' - ) - - finally: - # XXX: (MEGA IMPORTANT) if this is a root opened process we - # wait for any immediate child in debug before popping the - # context from the runtime msg loop otherwise inside - # ``Actor._push_result()`` the msg will be discarded and in - # the case where that msg is global debugger unlock (via - # a "stop" msg for a stream), this can result in a deadlock - # where the root is waiting on the lock to clear but the - # child has already cleared it and clobbered IPC. - await maybe_wait_for_debugger() - - # though it should be impossible for any tasks - # operating *in* this scope to have survived - # we tear down the runtime feeder chan last - # to avoid premature stream clobbers. - if ( - (rxchan := ctx._recv_chan) - - # maybe TODO: yes i know the below check is - # touching `trio` memchan internals..BUT, there are - # only a couple ways to avoid a `trio.Cancelled` - # bubbling from the `.aclose()` call below: - # - # - catch and mask it via the cancel-scope-shielded call - # as we are rn (manual and frowned upon) OR, - # - specially handle the case where `scope_err` is - # one of {`BaseExceptionGroup`, `trio.Cancelled`} - # and then presume that the `.aclose()` call will - # raise a `trio.Cancelled` and just don't call it - # in those cases.. - # - # that latter approach is more logic, LOC, and more - # convoluted so for now stick with the first - # psuedo-hack-workaround where we just try to avoid - # the shielded call as much as we can detect from - # the memchan's `._closed` state.. - # - # XXX MOTIVATION XXX-> we generally want to raise - # any underlying actor-runtime/internals error that - # surfaces from a bug in tractor itself so it can - # be easily detected/fixed AND, we also want to - # minimize noisy runtime tracebacks (normally due - # to the cross-actor linked task scope machinery - # teardown) displayed to user-code and instead only - # displaying `ContextCancelled` traces where the - # cause of crash/exit IS due to something in - # user/app code on either end of the context. - and not rxchan._closed - ): - # XXX NOTE XXX: and again as per above, we mask any - # `trio.Cancelled` raised here so as to NOT mask - # out any exception group or legit (remote) ctx - # error that sourced from the remote task or its - # runtime. - # - # NOTE: further, this should be the only place the - # underlying feeder channel is - # once-and-only-CLOSED! - with trio.CancelScope(shield=True): - await ctx._recv_chan.aclose() - - # XXX: we always raise remote errors locally and - # generally speaking mask runtime-machinery related - # multi-`trio.Cancelled`s. As such, any `scope_error` - # which was the underlying cause of this context's exit - # should be stored as the `Context._local_error` and - # used in determining `Context.cancelled_caught: bool`. - if scope_err is not None: - # sanity, tho can remove? - assert ctx._local_error is scope_err - # ctx._local_error: BaseException = scope_err - # etype: Type[BaseException] = type(scope_err) - - # CASE 2 - if ( - ctx._cancel_called - and ctx.cancel_acked - ): - log.cancel( - 'Context cancelled by caller task\n' - f'|_{ctx._task}\n\n' - - f'{repr(scope_err)}\n' - ) - - # TODO: should we add a `._cancel_req_received` - # flag to determine if the callee manually called - # `ctx.cancel()`? - # -[ ] going to need a cid check no? - - # CASE 1 - else: - outcome_str: str = ctx.repr_outcome( - show_error_fields=True, - # type_only=True, - ) - log.cancel( - f'Context terminated due to local scope error:\n\n' - f'{ctx.chan.uid} => {outcome_str}\n' - ) - - # XXX: (MEGA IMPORTANT) if this is a root opened process we - # wait for any immediate child in debug before popping the - # context from the runtime msg loop otherwise inside - # ``Actor._push_result()`` the msg will be discarded and in - # the case where that msg is global debugger unlock (via - # a "stop" msg for a stream), this can result in a deadlock - # where the root is waiting on the lock to clear but the - # child has already cleared it and clobbered IPC. - - # FINALLY, remove the context from runtime tracking and - # exit! - log.runtime( - 'Removing IPC ctx opened with peer\n' - f'{uid}\n' - f'|_{ctx}\n' - ) - self.actor._contexts.pop( - (uid, cid), - None, - ) + # NOTE: impl is found in `._context`` mod to make + # reading/groking the details simpler code-org-wise. This + # method does not have to be used over that `@acm` module func + # directly, it is for conventience and from the original API + # design. + open_context = open_context_from_portal @dataclass @@ -1018,7 +472,7 @@ class LocalPortal: return await func(**kwargs) -@asynccontextmanager +@acm async def open_portal( channel: Channel, From a5bc113fde327547af18bd0bc2573df59309eb5d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 13 Mar 2024 15:57:15 -0400 Subject: [PATCH 133/146] Start a `._rpc` module Since `._runtime` was getting pretty long (> 2k LOC) and much of the RPC low-level machinery is fairly isolated to a handful of task-funcs, it makes sense to re-org the RPC task scheduling and driving msg loop to its own code space. The move includes: - `process_messages()` which is the main IPC business logic. - `try_ship_error_to_remote()` helper, to box local errors for the wire. - `_invoke()`, the core task scheduler entrypoing used in the msg loop. - `_invoke_non_context()`, holds impls for non-`@context` task starts. - `_errors_relayed_via_ipc()` which does all error catch-n-boxing for wire-msg shipment using `try_ship_error_to_remote()` internally. Also inside `._runtime` improve some `Actor` methods docs. --- tractor/_rpc.py | 1118 +++++++++++++++++++++++++++++++++++++++++++ tractor/_runtime.py | 907 +++++++---------------------------- 2 files changed, 1289 insertions(+), 736 deletions(-) create mode 100644 tractor/_rpc.py diff --git a/tractor/_rpc.py b/tractor/_rpc.py new file mode 100644 index 00000000..54a60be6 --- /dev/null +++ b/tractor/_rpc.py @@ -0,0 +1,1118 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +''' +Remote (task) Procedure Call (scheduling) with SC transitive semantics. + +''' +from __future__ import annotations +from contextlib import ( + asynccontextmanager as acm, +) +from functools import partial +import inspect +from pprint import pformat +from types import ModuleType +from typing import ( + Any, + Callable, + Coroutine, + TYPE_CHECKING, +) +import warnings + +from async_generator import aclosing +from exceptiongroup import BaseExceptionGroup +import trio +from trio import ( + CancelScope, + Nursery, + TaskStatus, +) +# from trio_typing import ( +# TaskStatus, +# ) + +from .msg import NamespacePath +from ._ipc import Channel +from ._context import ( + Context, +) +from ._exceptions import ( + ModuleNotExposed, + is_multi_cancelled, + ContextCancelled, + pack_error, + unpack_error, + TransportClosed, +) +from .devx import ( + # pause, + maybe_wait_for_debugger, + _debug, +) +from . import _state +from .log import get_logger + + +if TYPE_CHECKING: + from ._runtime import Actor + +log = get_logger('tractor') + + +async def _invoke_non_context( + actor: Actor, + cancel_scope: CancelScope, + ctx: Context, + cid: str, + chan: Channel, + func: Callable, + coro: Coroutine, + kwargs: dict[str, Any], + + treat_as_gen: bool, + is_rpc: bool, + + task_status: TaskStatus[ + Context | BaseException + ] = trio.TASK_STATUS_IGNORED, +): + + # TODO: can we unify this with the `context=True` impl below? + if inspect.isasyncgen(coro): + await chan.send({'functype': 'asyncgen', 'cid': cid}) + # XXX: massive gotcha! If the containing scope + # is cancelled and we execute the below line, + # any ``ActorNursery.__aexit__()`` WON'T be + # triggered in the underlying async gen! So we + # have to properly handle the closing (aclosing) + # of the async gen in order to be sure the cancel + # is propagated! + with cancel_scope as cs: + ctx._scope = cs + task_status.started(ctx) + async with aclosing(coro) as agen: + async for item in agen: + # TODO: can we send values back in here? + # it's gonna require a `while True:` and + # some non-blocking way to retrieve new `asend()` + # values from the channel: + # to_send = await chan.recv_nowait() + # if to_send is not None: + # to_yield = await coro.asend(to_send) + await chan.send({'yield': item, 'cid': cid}) + + log.runtime(f"Finished iterating {coro}") + # TODO: we should really support a proper + # `StopAsyncIteration` system here for returning a final + # value if desired + await chan.send({'stop': True, 'cid': cid}) + + # one way @stream func that gets treated like an async gen + # TODO: can we unify this with the `context=True` impl below? + elif treat_as_gen: + await chan.send({'functype': 'asyncgen', 'cid': cid}) + # 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) + await coro + + if not cs.cancelled_caught: + # task was not cancelled so we can instruct the + # far end async gen to tear down + await chan.send({'stop': True, 'cid': cid}) + 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: + await chan.send({ + 'functype': 'asyncfunc', + 'cid': cid + }) + except ( + trio.ClosedResourceError, + trio.BrokenResourceError, + BrokenPipeError, + ) as ipc_err: + failed_resp = True + if is_rpc: + raise + else: + # TODO: should this be an `.exception()` call? + log.warning( + f'Failed to respond to non-rpc request: {func}\n' + f'{ipc_err}' + ) + + with cancel_scope as cs: + ctx._scope: CancelScope = cs + task_status.started(ctx) + result = await coro + fname: str = func.__name__ + log.runtime( + 'RPC complete:\n' + f'task: {ctx._task}\n' + f'|_cid={ctx.cid}\n' + f'|_{fname}() -> {pformat(result)}\n' + ) + + # NOTE: only send result if we know IPC isn't down + if ( + not failed_resp + and chan.connected() + ): + try: + await chan.send( + {'return': result, + 'cid': cid} + ) + except ( + BrokenPipeError, + trio.BrokenResourceError, + ): + log.warning( + 'Failed to return result:\n' + f'{func}@{actor.uid}\n' + f'remote chan: {chan.uid}' + ) + +@acm +async def _errors_relayed_via_ipc( + actor: Actor, + chan: Channel, + ctx: Context, + is_rpc: bool, + + hide_tb: bool = False, + debug_kbis: bool = False, + task_status: TaskStatus[ + Context | BaseException + ] = trio.TASK_STATUS_IGNORED, + +) -> None: + __tracebackhide__: bool = hide_tb # TODO: use hide_tb here? + try: + yield # run RPC invoke body + + # box and ship RPC errors for wire-transit via + # the task's requesting parent IPC-channel. + except ( + Exception, + BaseExceptionGroup, + KeyboardInterrupt, + ) as err: + + # always hide this frame from debug REPL if the crash + # originated from an rpc task and we DID NOT fail due to + # an IPC transport error! + if ( + is_rpc + and chan.connected() + ): + __tracebackhide__: bool = hide_tb + + if not is_multi_cancelled(err): + + # TODO: maybe we'll want different "levels" of debugging + # eventualy such as ('app', 'supervisory', 'runtime') ? + + # if not isinstance(err, trio.ClosedResourceError) and ( + # if not is_multi_cancelled(err) and ( + + entered_debug: bool = False + if ( + ( + not isinstance(err, ContextCancelled) + or ( + isinstance(err, ContextCancelled) + and ctx._cancel_called + + # if the root blocks the debugger lock request from a child + # we will get a remote-cancelled condition. + and ctx._enter_debugger_on_cancel + ) + ) + and + ( + not isinstance(err, KeyboardInterrupt) + or ( + isinstance(err, KeyboardInterrupt) + and debug_kbis + ) + ) + ): + # await _debug.pause() + # XXX QUESTION XXX: is there any case where we'll + # want to debug IPC disconnects as a default? + # => I can't think of a reason that inspecting this + # type of failure will be useful for respawns or + # recovery logic - the only case is some kind of + # strange bug in our transport layer itself? Going + # to keep this open ended for now. + entered_debug = await _debug._maybe_enter_pm(err) + + if not entered_debug: + log.exception('Actor crashed:\n') + + # always (try to) ship RPC errors back to caller + if is_rpc: + # + # TODO: tests for this scenario: + # - RPC caller closes connection before getting a response + # should **not** crash this actor.. + await try_ship_error_to_remote( + chan, + err, + cid=ctx.cid, + remote_descr='caller', + hide_tb=hide_tb, + ) + + # error is probably from above coro running code *not from + # the target rpc invocation since a scope was never + # allocated around the coroutine await. + if ctx._scope is None: + # we don't ever raise directly here to allow the + # msg-loop-scheduler to continue running for this + # channel. + task_status.started(err) + + # always reraise KBIs so they propagate at the sys-process + # level. + if isinstance(err, KeyboardInterrupt): + raise + + + # RPC task bookeeping + finally: + try: + ctx, func, is_complete = actor._rpc_tasks.pop( + (chan, ctx.cid) + ) + is_complete.set() + + except KeyError: + if is_rpc: + # If we're cancelled before the task returns then the + # cancel scope will not have been inserted yet + log.warning( + 'RPC task likely errored or cancelled before start?' + f'|_{ctx._task}\n' + f' >> {ctx.repr_rpc}\n' + ) + else: + log.cancel( + 'Failed to de-alloc internal runtime cancel task?\n' + f'|_{ctx._task}\n' + f' >> {ctx.repr_rpc}\n' + ) + + finally: + if not actor._rpc_tasks: + log.runtime("All RPC tasks have completed") + actor._ongoing_rpc_tasks.set() + + +_gb_mod: ModuleType|None|False = None + + +async def maybe_import_gb(): + global _gb_mod + if _gb_mod is False: + return + + try: + import greenback + _gb_mod = greenback + await greenback.ensure_portal() + + except ModuleNotFoundError: + log.debug( + '`greenback` is not installed.\n' + 'No sync debug support!\n' + ) + _gb_mod = False + + +async def _invoke( + + actor: Actor, + cid: str, + chan: Channel, + func: Callable, + kwargs: dict[str, Any], + + is_rpc: bool = True, + hide_tb: bool = True, + + task_status: TaskStatus[ + Context | BaseException + ] = trio.TASK_STATUS_IGNORED, +): + ''' + Schedule a `trio` task-as-func and deliver result(s) over + connected IPC channel. + + This is the core "RPC" `trio.Task` scheduling machinery used to start every + remotely invoked function, normally in `Actor._service_n: Nursery`. + + ''' + __tracebackhide__: bool = hide_tb + treat_as_gen: bool = False + + if _state.debug_mode(): + await maybe_import_gb() + + # TODO: possibly a specially formatted traceback + # (not sure what typing is for this..)? + # tb = None + + cancel_scope = CancelScope() + # activated cancel scope ref + cs: CancelScope|None = None + + ctx = actor.get_context( + chan=chan, + cid=cid, + nsf=NamespacePath.from_ref(func), + + # TODO: if we wanted to get cray and support it? + # side='callee', + + # We shouldn't ever need to pass this through right? + # it's up to the soon-to-be called rpc task to + # open the stream with this option. + # allow_overruns=True, + ) + context: bool = False + + # TODO: deprecate this style.. + if getattr(func, '_tractor_stream_function', False): + # handle decorated ``@tractor.stream`` async functions + sig = inspect.signature(func) + params = sig.parameters + + # compat with old api + kwargs['ctx'] = ctx + treat_as_gen = True + + if 'ctx' in params: + warnings.warn( + "`@tractor.stream decorated funcs should now declare " + "a `stream` arg, `ctx` is now designated for use with " + "@tractor.context", + DeprecationWarning, + stacklevel=2, + ) + + elif 'stream' in params: + assert 'stream' in params + kwargs['stream'] = ctx + + + elif getattr(func, '_tractor_context_function', False): + # handle decorated ``@tractor.context`` async function + kwargs['ctx'] = ctx + context = True + + # errors raised inside this block are propgated back to caller + async with _errors_relayed_via_ipc( + actor, + chan, + ctx, + is_rpc, + hide_tb=hide_tb, + task_status=task_status, + ): + if not ( + inspect.isasyncgenfunction(func) or + inspect.iscoroutinefunction(func) + ): + raise TypeError(f'{func} must be an async function!') + + # init coroutine with `kwargs` to immediately catch any + # type-sig errors. + try: + coro = func(**kwargs) + except TypeError: + raise + + # TODO: implement all these cases in terms of the + # `Context` one! + if not context: + await _invoke_non_context( + actor, + cancel_scope, + ctx, + cid, + chan, + func, + coro, + kwargs, + treat_as_gen, + is_rpc, + task_status, + ) + # below is only for `@context` funcs + return + + # our most general case: a remote SC-transitive, + # IPC-linked, cross-actor-task "context" + # ------ - ------ + # TODO: every other "func type" should be implemented from + # a special case of this impl eventually! + # -[ ] streaming funcs should instead of being async-for + # handled directly here wrapped in + # a async-with-open_stream() closure that does the + # normal thing you'd expect a far end streaming context + # to (if written by the app-dev). + # -[ ] one off async funcs can literally just be called + # here and awaited directly, possibly just with a small + # wrapper that calls `Context.started()` and then does + # the `await coro()`? + + # a "context" endpoint type is the most general and + # "least sugary" type of RPC ep with support for + # bi-dir streaming B) + await chan.send({ + 'functype': 'context', + 'cid': cid + }) + + # TODO: should we also use an `.open_context()` equiv + # for this callee side by factoring the impl from + # `Portal.open_context()` into a common helper? + # + # NOTE: there are many different ctx state details + # in a callee side instance according to current impl: + # - `.cancelled_caught` can never be `True`. + # -> the below scope is never exposed to the + # `@context` marked RPC function. + # - `._portal` is never set. + try: + async with trio.open_nursery() as tn: + ctx._scope_nursery = tn + ctx._scope = tn.cancel_scope + task_status.started(ctx) + + # TODO: should would be nice to have our + # `TaskMngr` nursery here! + res: Any = await coro + ctx._result = res + + # deliver final result to caller side. + await chan.send({ + 'return': res, + 'cid': cid + }) + + # NOTE: this happens IFF `ctx._scope.cancel()` is + # called by any of, + # - *this* callee task manually calling `ctx.cancel()`. + # - the runtime calling `ctx._deliver_msg()` which + # itself calls `ctx._maybe_cancel_and_set_remote_error()` + # which cancels the scope presuming the input error + # is not a `.cancel_acked` pleaser. + # - currently a never-should-happen-fallthrough case + # inside ._context._drain_to_final_msg()`.. + # # TODO: remove this ^ right? + if ctx._scope.cancelled_caught: + our_uid: tuple = actor.uid + + # first check for and raise any remote error + # before raising any context cancelled case + # so that real remote errors don't get masked as + # ``ContextCancelled``s. + if re := ctx._remote_error: + ctx._maybe_raise_remote_err(re) + + cs: CancelScope = ctx._scope + + if cs.cancel_called: + + canceller: tuple = ctx.canceller + msg: str = ( + 'actor was cancelled by ' + ) + + # NOTE / TODO: if we end up having + # ``Actor._cancel_task()`` call + # ``Context.cancel()`` directly, we're going to + # need to change this logic branch since it + # will always enter.. + if ctx._cancel_called: + # TODO: test for this!!!!! + canceller: tuple = our_uid + msg += 'itself ' + + # if the channel which spawned the ctx is the + # one that cancelled it then we report that, vs. + # it being some other random actor that for ex. + # some actor who calls `Portal.cancel_actor()` + # and by side-effect cancels this ctx. + elif canceller == ctx.chan.uid: + msg += 'its caller' + + else: + msg += 'a remote peer' + + div_chars: str = '------ - ------' + div_offset: int = ( + round(len(msg)/2)+1 + + + round(len(div_chars)/2)+1 + ) + div_str: str = ( + '\n' + + + ' '*div_offset + + + f'{div_chars}\n' + ) + msg += ( + div_str + + f'<= canceller: {canceller}\n' + f'=> uid: {our_uid}\n' + f' |_{ctx._task}()' + + # TODO: instead just show the + # ctx.__str__() here? + # -[ ] textwrap.indent() it correctly! + # -[ ] BUT we need to wait until + # the state is filled out before emitting + # this msg right ow its kinda empty? bleh.. + # + # f' |_{ctx}' + ) + + # task-contex was either cancelled by request using + # ``Portal.cancel_actor()`` or ``Context.cancel()`` + # on the far end, or it was cancelled by the local + # (callee) task, so relay this cancel signal to the + # other side. + ctxc = ContextCancelled( + msg, + suberror_type=trio.Cancelled, + canceller=canceller, + ) + # assign local error so that the `.outcome` + # resolves to an error for both reporting and + # state checks. + ctx._local_error = ctxc + raise ctxc + + # XXX: do we ever trigger this block any more? + except ( + BaseExceptionGroup, + trio.Cancelled, + BaseException, + + ) as scope_error: + + # always set this (callee) side's exception as the + # local error on the context + ctx._local_error: BaseException = scope_error + + # if a remote error was set then likely the + # exception group was raised due to that, so + # and we instead raise that error immediately! + ctx.maybe_raise() + + # maybe TODO: pack in come kinda + # `trio.Cancelled.__traceback__` here so they can be + # unwrapped and displayed on the caller side? no se.. + raise + + # `@context` entrypoint task bookeeping. + # i.e. only pop the context tracking if used ;) + finally: + assert chan.uid + + # don't pop the local context until we know the + # associated child isn't in debug any more + await maybe_wait_for_debugger() + ctx: Context = actor._contexts.pop(( + chan.uid, + cid, + # ctx.side, + )) + + merr: Exception|None = ctx.maybe_error + + ( + res_type_str, + res_str, + ) = ( + ('error', f'{type(merr)}',) + if merr + else ( + 'result', + f'`{repr(ctx.outcome)}`', + ) + ) + log.cancel( + f'IPC context terminated with a final {res_type_str}\n\n' + f'{ctx}\n' + ) + + +async def try_ship_error_to_remote( + channel: Channel, + err: Exception|BaseExceptionGroup, + + cid: str|None = None, + remote_descr: str = 'parent', + hide_tb: bool = True, + +) -> None: + ''' + Box, pack and encode a local runtime(-internal) exception for + an IPC channel `.send()` with transport/network failures and + local cancellation ignored but logged as critical(ly bad). + + ''' + __tracebackhide__: bool = hide_tb + with CancelScope(shield=True): + try: + # NOTE: normally only used for internal runtime errors + # so ship to peer actor without a cid. + msg: dict = pack_error( + err, + cid=cid, + + # TODO: special tb fmting for ctxc cases? + # tb=tb, + ) + # NOTE: the src actor should always be packed into the + # error.. but how should we verify this? + # actor: Actor = _state.current_actor() + # assert err_msg['src_actor_uid'] + # if not err_msg['error'].get('src_actor_uid'): + # import pdbp; pdbp.set_trace() + await channel.send(msg) + + # XXX NOTE XXX in SC terms this is one of the worst things + # that can happen and provides for a 2-general's dilemma.. + except ( + trio.ClosedResourceError, + trio.BrokenResourceError, + BrokenPipeError, + ): + err_msg: dict = msg['error']['tb_str'] + log.critical( + 'IPC transport failure -> ' + f'failed to ship error to {remote_descr}!\n\n' + f'X=> {channel.uid}\n\n' + f'{err_msg}\n' + ) + + +async def process_messages( + actor: Actor, + chan: Channel, + shield: bool = False, + task_status: TaskStatus[CancelScope] = trio.TASK_STATUS_IGNORED, + +) -> bool: + ''' + This is the low-level, per-IPC-channel, RPC task scheduler loop. + + Receive (multiplexed) per-`Channel` RPC requests as msgs from + remote processes; schedule target async funcs as local + `trio.Task`s inside the `Actor._service_n: Nursery`. + + Depending on msg type, non-`cmd` (task spawning/starting) + request payloads (eg. `started`, `yield`, `return`, `error`) + are delivered to locally running, linked-via-`Context`, tasks + with any (boxed) errors and/or final results shipped back to + the remote side. + + All higher level inter-actor comms ops are delivered in some + form by the msg processing here, including: + + - lookup and invocation of any (async) funcs-as-tasks requested + by remote actors presuming the local actor has enabled their + containing module. + + - IPC-session oriented `Context` and `MsgStream` msg payload + delivery such as `started`, `yield` and `return` msgs. + + - cancellation handling for both `Context.cancel()` (which + translate to `Actor._cancel_task()` RPCs server side) + and `Actor.cancel()` process-wide-runtime-shutdown requests + (as utilized inside `Portal.cancel_actor()` ). + + + ''' + # TODO: once `trio` get's an "obvious way" for req/resp we + # should use it? + # https://github.com/python-trio/trio/issues/467 + log.runtime( + 'Entering IPC msg loop:\n' + f'peer: {chan.uid}\n' + f'|_{chan}\n' + ) + nursery_cancelled_before_task: bool = False + msg: dict | None = None + try: + # NOTE: this internal scope allows for keeping this + # message loop running despite the current task having + # been cancelled (eg. `open_portal()` may call this method + # from a locally spawned task) and recieve this scope + # using ``scope = Nursery.start()`` + with CancelScope(shield=shield) as loop_cs: + task_status.started(loop_cs) + async for msg in chan: + + # dedicated loop terminate sentinel + if msg is None: + + tasks: dict[ + tuple[Channel, str], + tuple[Context, Callable, trio.Event] + ] = actor._rpc_tasks.copy() + log.cancel( + f'Peer IPC channel terminated via `None` setinel msg?\n' + f'=> Cancelling all {len(tasks)} local RPC tasks..\n' + f'peer: {chan.uid}\n' + f'|_{chan}\n' + ) + for (channel, cid) in tasks: + if channel is chan: + await actor._cancel_task( + cid, + channel, + requesting_uid=channel.uid, + + ipc_msg=msg, + ) + break + + log.transport( # type: ignore + f'<= IPC msg from peer: {chan.uid}\n\n' + + # TODO: conditionally avoid fmting depending + # on log level (for perf)? + # => specifically `pformat()` sub-call..? + f'{pformat(msg)}\n' + ) + + cid = msg.get('cid') + if cid: + # deliver response to local caller/waiter + # via its per-remote-context memory channel. + await actor._push_result( + chan, + cid, + msg, + ) + + log.runtime( + 'Waiting on next IPC msg from\n' + f'peer: {chan.uid}:\n' + f'|_{chan}\n' + + # f'last msg: {msg}\n' + ) + continue + + # process a 'cmd' request-msg upack + # TODO: impl with native `msgspec.Struct` support !! + # -[ ] implement with ``match:`` syntax? + # -[ ] discard un-authed msgs as per, + # + try: + ( + ns, + funcname, + kwargs, + actorid, + cid, + ) = msg['cmd'] + + except KeyError: + # This is the non-rpc error case, that is, an + # error **not** raised inside a call to ``_invoke()`` + # (i.e. no cid was provided in the msg - see above). + # Push this error to all local channel consumers + # (normally portals) by marking the channel as errored + assert chan.uid + exc = unpack_error(msg, chan=chan) + chan._exc = exc + raise exc + + log.runtime( + 'Handling RPC cmd from\n' + f'peer: {actorid}\n' + '\n' + f'=> {ns}.{funcname}({kwargs})\n' + ) + if ns == 'self': + if funcname == 'cancel': + func: Callable = actor.cancel + kwargs |= { + 'req_chan': chan, + } + + # don't start entire actor runtime cancellation + # if this actor is currently in debug mode! + pdb_complete: trio.Event|None = _debug.Lock.local_pdb_complete + if pdb_complete: + await pdb_complete.wait() + + # Either of `Actor.cancel()`/`.cancel_soon()` + # was called, so terminate this IPC msg + # loop, exit back out into `async_main()`, + # and immediately start the core runtime + # machinery shutdown! + with CancelScope(shield=True): + await _invoke( + actor, + cid, + chan, + func, + kwargs, + is_rpc=False, + ) + + log.runtime( + 'Cancelling IPC transport msg-loop with peer:\n' + f'|_{chan}\n' + ) + loop_cs.cancel() + break + + if funcname == '_cancel_task': + func: Callable = actor._cancel_task + + # we immediately start the runtime machinery + # shutdown + # with CancelScope(shield=True): + target_cid: str = kwargs['cid'] + kwargs |= { + # NOTE: ONLY the rpc-task-owning + # parent IPC channel should be able to + # cancel it! + 'parent_chan': chan, + 'requesting_uid': chan.uid, + 'ipc_msg': msg, + } + # TODO: remove? already have emit in meth. + # log.runtime( + # f'Rx RPC task cancel request\n' + # f'<= canceller: {chan.uid}\n' + # f' |_{chan}\n\n' + # f'=> {actor}\n' + # f' |_cid: {target_cid}\n' + # ) + try: + await _invoke( + actor, + cid, + chan, + func, + kwargs, + is_rpc=False, + ) + except BaseException: + log.exception( + 'Failed to cancel task?\n' + f'<= canceller: {chan.uid}\n' + f' |_{chan}\n\n' + f'=> {actor}\n' + f' |_cid: {target_cid}\n' + ) + continue + else: + # normally registry methods, eg. + # ``.register_actor()`` etc. + func: Callable = getattr(actor, funcname) + + else: + # complain to client about restricted modules + try: + func = actor._get_rpc_func(ns, funcname) + except ( + ModuleNotExposed, + AttributeError, + ) as err: + err_msg: dict[str, dict] = pack_error( + err, + cid=cid, + ) + await chan.send(err_msg) + continue + + # schedule a task for the requested RPC function + # in the actor's main "service nursery". + # TODO: possibly a service-tn per IPC channel for + # supervision isolation? would avoid having to + # manage RPC tasks individually in `._rpc_tasks` + # table? + log.runtime( + f'Spawning task for RPC request\n' + f'<= caller: {chan.uid}\n' + f' |_{chan}\n\n' + # TODO: maddr style repr? + # f' |_@ /ipv4/{chan.raddr}/tcp/{chan.rport}/' + # f'cid="{cid[-16:]} .."\n\n' + + f'=> {actor}\n' + f' |_cid: {cid}\n' + f' |>> {func}()\n' + ) + assert actor._service_n # wait why? do it at top? + try: + ctx: Context = await actor._service_n.start( + partial( + _invoke, + actor, + cid, + chan, + func, + kwargs, + ), + name=funcname, + ) + + except ( + RuntimeError, + BaseExceptionGroup, + ): + # avoid reporting a benign race condition + # during actor runtime teardown. + nursery_cancelled_before_task: bool = True + break + + # in the lone case where a ``Context`` is not + # delivered, it's likely going to be a locally + # scoped exception from ``_invoke()`` itself. + if isinstance(err := ctx, Exception): + log.warning( + 'Task for RPC failed?' + f'|_ {func}()\n\n' + + f'{err}' + ) + continue + + else: + # mark that we have ongoing rpc tasks + actor._ongoing_rpc_tasks = trio.Event() + + # store cancel scope such that the rpc task can be + # cancelled gracefully if requested + actor._rpc_tasks[(chan, cid)] = ( + ctx, + func, + trio.Event(), + ) + + log.runtime( + 'Waiting on next IPC msg from\n' + f'peer: {chan.uid}\n' + f'|_{chan}\n' + ) + + # end of async for, channel disconnect vis + # ``trio.EndOfChannel`` + log.runtime( + f"{chan} for {chan.uid} disconnected, cancelling tasks" + ) + await actor.cancel_rpc_tasks( + req_uid=actor.uid, + # a "self cancel" in terms of the lifetime of the + # IPC connection which is presumed to be the + # source of any requests for spawned tasks. + parent_chan=chan, + ) + + except ( + TransportClosed, + ): + # 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 + # the message loop and expect the teardown sequence to clean + # up. + # TODO: don't show this msg if it's an emphemeral + # discovery ep call? + log.runtime( + f'channel closed abruptly with\n' + f'peer: {chan.uid}\n' + f'|_{chan.raddr}\n' + ) + + # transport **was** disconnected + return True + + except ( + Exception, + BaseExceptionGroup, + ) as err: + + if nursery_cancelled_before_task: + sn: Nursery = actor._service_n + assert sn and sn.cancel_scope.cancel_called # sanity + log.cancel( + f'Service nursery cancelled before it handled {funcname}' + ) + else: + # ship any "internal" exception (i.e. one from internal + # machinery not from an rpc task) to parent + match err: + case ContextCancelled(): + log.cancel( + f'Actor: {actor.uid} was context-cancelled with,\n' + f'str(err)' + ) + case _: + log.exception("Actor errored:") + + if actor._parent_chan: + await try_ship_error_to_remote( + actor._parent_chan, + err, + ) + + # if this is the `MainProcess` we expect the error broadcasting + # above to trigger an error at consuming portal "checkpoints" + raise + + finally: + # msg debugging for when he machinery is brokey + log.runtime( + 'Exiting IPC msg loop with\n' + f'peer: {chan.uid}\n' + f'|_{chan}\n\n' + 'final msg:\n' + f'{pformat(msg)}\n' + ) + + # transport **was not** disconnected + return False diff --git a/tractor/_runtime.py b/tractor/_runtime.py index c034bd86..84ad8cea 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -1,349 +1,106 @@ -tb = None +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. - cancel_scope = CancelScope() - # activated cancel scope ref - cs: CancelScope|None = None +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. - ctx = actor.get_context( - chan=chan, - cid=cid, - nsf=NamespacePath.from_ref(func), +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. - # TODO: if we wanted to get cray and support it? - # side='callee', +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . - # We shouldn't ever need to pass this through right? - # it's up to the soon-to-be called rpc task to - # open the stream with this option. - # allow_overruns=True, - ) - context: bool = False +''' +The fundamental core machinery implementing every "actor" +including the process-local, or "python-interpreter (aka global) +singleton) `Actor` primitive(s) and its internal `trio` machinery +implementing the low level runtime system supporting the +discovery, communication, spawning, supervision and cancellation +of other actors in a hierarchincal process tree. - # TODO: deprecate this style.. - if getattr(func, '_tractor_stream_function', False): - # handle decorated ``@tractor.stream`` async functions - sig = inspect.signature(func) - params = sig.parameters +The runtime's main entry point: `async_main()` opens the top level +supervision and service `trio.Nursery`s which manage the tasks responsible +for running all lower level spawning, supervision and msging layers: - # compat with old api - kwargs['ctx'] = ctx - treat_as_gen = True +- lowlevel transport-protocol init and persistent connectivity on + top of `._ipc` primitives; the transport layer. +- bootstrapping of connection/runtime config from the spawning + parent (actor). +- starting and supervising IPC-channel msg processing loops around + tranport connections from parent/peer actors in order to deliver + SC-transitive RPC via scheduling of `trio` tasks. +- registration of newly spawned actors with the discovery sys. - if 'ctx' in params: - warnings.warn( - "`@tractor.stream decorated funcs should now declare " - "a `stream` arg, `ctx` is now designated for use with " - "@tractor.context", - DeprecationWarning, - stacklevel=2, - ) +''' +from __future__ import annotations +from contextlib import ( + ExitStack, +) +from collections import defaultdict +from functools import partial +from itertools import chain +import importlib +import importlib.util +from pprint import pformat +import signal +import sys +from typing import ( + Any, + Callable, + TYPE_CHECKING, +) +import uuid +from types import ModuleType +import os - elif 'stream' in params: - assert 'stream' in params - kwargs['stream'] = ctx +import trio +from trio import ( + CancelScope, +) +from trio_typing import ( + Nursery, + TaskStatus, +) + +from .msg import NamespacePath +from ._ipc import Channel +from ._context import ( + mk_context, + Context, +) +from .log import get_logger +from ._exceptions import ( + unpack_error, + ModuleNotExposed, + ContextCancelled, + TransportClosed, +) +from ._discovery import get_arbiter +from . import _debug +from ._portal import Portal +from . import _state +from . import _mp_fixup_main +from ._rpc import ( + process_messages, + try_ship_error_to_remote, +) - elif getattr(func, '_tractor_context_function', False): - # handle decorated ``@tractor.context`` async function - kwargs['ctx'] = ctx - context = True +if TYPE_CHECKING: + from ._supervise import ActorNursery - # errors raised inside this block are propgated back to caller - async with _errors_relayed_via_ipc( - actor, - chan, - ctx, - is_rpc, - hide_tb=hide_tb, - task_status=task_status, - ): - if not ( - inspect.isasyncgenfunction(func) or - inspect.iscoroutinefunction(func) - ): - raise TypeError(f'{func} must be an async function!') - # init coroutine with `kwargs` to immediately catch any - # type-sig errors. - try: - coro = func(**kwargs) - except TypeError: - raise - - # TODO: implement all these cases in terms of the - # `Context` one! - if not context: - await _invoke_non_context( - actor, - cancel_scope, - ctx, - cid, - chan, - func, - coro, - kwargs, - treat_as_gen, - is_rpc, - task_status, - ) - # below is only for `@context` funcs - return - - # our most general case: a remote SC-transitive, - # IPC-linked, cross-actor-task "context" - # ------ - ------ - # TODO: every other "func type" should be implemented from - # a special case of this impl eventually! - # -[ ] streaming funcs should instead of being async-for - # handled directly here wrapped in - # a async-with-open_stream() closure that does the - # normal thing you'd expect a far end streaming context - # to (if written by the app-dev). - # -[ ] one off async funcs can literally just be called - # here and awaited directly, possibly just with a small - # wrapper that calls `Context.started()` and then does - # the `await coro()`? - - # a "context" endpoint type is the most general and - # "least sugary" type of RPC ep with support for - # bi-dir streaming B) - await chan.send({ - 'functype': 'context', - 'cid': cid - }) - - # TODO: should we also use an `.open_context()` equiv - # for this callee side by factoring the impl from - # `Portal.open_context()` into a common helper? - # - # NOTE: there are many different ctx state details - # in a callee side instance according to current impl: - # - `.cancelled_caught` can never be `True`. - # -> the below scope is never exposed to the - # `@context` marked RPC function. - # - `._portal` is never set. - try: - async with trio.open_nursery() as tn: - ctx._scope_nursery = tn - ctx._scope = tn.cancel_scope - task_status.started(ctx) - - # TODO: should would be nice to have our - # `TaskMngr` nursery here! - res: Any = await coro - ctx._result = res - - # deliver final result to caller side. - await chan.send({ - 'return': res, - 'cid': cid - }) - - # NOTE: this happens IFF `ctx._scope.cancel()` is - # called by any of, - # - *this* callee task manually calling `ctx.cancel()`. - # - the runtime calling `ctx._deliver_msg()` which - # itself calls `ctx._maybe_cancel_and_set_remote_error()` - # which cancels the scope presuming the input error - # is not a `.cancel_acked` pleaser. - # - currently a never-should-happen-fallthrough case - # inside ._context._drain_to_final_msg()`.. - # # TODO: remove this ^ right? - if ctx._scope.cancelled_caught: - our_uid: tuple = actor.uid - - # first check for and raise any remote error - # before raising any context cancelled case - # so that real remote errors don't get masked as - # ``ContextCancelled``s. - if re := ctx._remote_error: - ctx._maybe_raise_remote_err(re) - - cs: CancelScope = ctx._scope - - if cs.cancel_called: - - canceller: tuple = ctx.canceller - msg: str = ( - 'actor was cancelled by ' - ) - - # NOTE / TODO: if we end up having - # ``Actor._cancel_task()`` call - # ``Context.cancel()`` directly, we're going to - # need to change this logic branch since it - # will always enter.. - if ctx._cancel_called: - # TODO: test for this!!!!! - canceller: tuple = our_uid - msg += 'itself ' - - # if the channel which spawned the ctx is the - # one that cancelled it then we report that, vs. - # it being some other random actor that for ex. - # some actor who calls `Portal.cancel_actor()` - # and by side-effect cancels this ctx. - elif canceller == ctx.chan.uid: - msg += 'its caller' - - else: - msg += 'a remote peer' - - div_chars: str = '------ - ------' - div_offset: int = ( - round(len(msg)/2)+1 - + - round(len(div_chars)/2)+1 - ) - div_str: str = ( - '\n' - + - ' '*div_offset - + - f'{div_chars}\n' - ) - msg += ( - div_str + - f'<= canceller: {canceller}\n' - f'=> uid: {our_uid}\n' - f' |_{ctx._task}()' - - # TODO: instead just show the - # ctx.__str__() here? - # -[ ] textwrap.indent() it correctly! - # -[ ] BUT we need to wait until - # the state is filled out before emitting - # this msg right ow its kinda empty? bleh.. - # - # f' |_{ctx}' - ) - - # task-contex was either cancelled by request using - # ``Portal.cancel_actor()`` or ``Context.cancel()`` - # on the far end, or it was cancelled by the local - # (callee) task, so relay this cancel signal to the - # other side. - ctxc = ContextCancelled( - msg, - suberror_type=trio.Cancelled, - canceller=canceller, - ) - # assign local error so that the `.outcome` - # resolves to an error for both reporting and - # state checks. - ctx._local_error = ctxc - raise ctxc - - # XXX: do we ever trigger this block any more? - except ( - BaseExceptionGroup, - trio.Cancelled, - BaseException, - - ) as scope_error: - - # always set this (callee) side's exception as the - # local error on the context - ctx._local_error: BaseException = scope_error - - # if a remote error was set then likely the - # exception group was raised due to that, so - # and we instead raise that error immediately! - ctx.maybe_raise() - - # maybe TODO: pack in come kinda - # `trio.Cancelled.__traceback__` here so they can be - # unwrapped and displayed on the caller side? no se.. - raise - - # `@context` entrypoint task bookeeping. - # i.e. only pop the context tracking if used ;) - finally: - assert chan.uid - - # don't pop the local context until we know the - # associated child isn't in debug any more - await _debug.maybe_wait_for_debugger() - ctx: Context = actor._contexts.pop(( - chan.uid, - cid, - # ctx.side, - )) - - merr: Exception|None = ctx.maybe_error - - ( - res_type_str, - res_str, - ) = ( - ('error', f'{type(merr)}',) - if merr - else ( - 'result', - f'`{repr(ctx.outcome)}`', - ) - ) - log.cancel( - f'IPC context terminated with a final {res_type_str}\n\n' - f'{ctx}\n' - ) +log = get_logger('tractor') def _get_mod_abspath(module): return os.path.abspath(module.__file__) -async def try_ship_error_to_remote( - channel: Channel, - err: Exception|BaseExceptionGroup, - - cid: str|None = None, - remote_descr: str = 'parent', - hide_tb: bool = True, - -) -> None: - ''' - Box, pack and encode a local runtime(-internal) exception for - an IPC channel `.send()` with transport/network failures and - local cancellation ignored but logged as critical(ly bad). - - ''' - __tracebackhide__: bool = hide_tb - with CancelScope(shield=True): - try: - # NOTE: normally only used for internal runtime errors - # so ship to peer actor without a cid. - msg: dict = pack_error( - err, - cid=cid, - - # TODO: special tb fmting for ctxc cases? - # tb=tb, - ) - # NOTE: the src actor should always be packed into the - # error.. but how should we verify this? - # actor: Actor = _state.current_actor() - # assert err_msg['src_actor_uid'] - # if not err_msg['error'].get('src_actor_uid'): - # import pdbp; pdbp.set_trace() - await channel.send(msg) - - # XXX NOTE XXX in SC terms this is one of the worst things - # that can happen and provides for a 2-general's dilemma.. - except ( - trio.ClosedResourceError, - trio.BrokenResourceError, - BrokenPipeError, - ): - err_msg: dict = msg['error']['tb_str'] - log.critical( - 'IPC transport failure -> ' - f'failed to ship error to {remote_descr}!\n\n' - f'X=> {channel.uid}\n\n' - f'{err_msg}\n' - ) - - class Actor: ''' The fundamental "runtime" concurrency primitive. @@ -488,8 +245,8 @@ class Actor: self, uid: tuple[str, str] ) -> tuple[trio.Event, Channel]: ''' - Wait for a connection back from a spawned actor with a given - ``uid``. + Wait for a connection back from a spawned actor with a `uid` + using a `trio.Event` for sync. ''' log.runtime(f"Waiting for peer {uid} to connect") @@ -503,11 +260,11 @@ class Actor: debug_mode: bool = False, ) -> None: ''' - Load allowed RPC modules locally (after fork). + Load enabled RPC py-modules locally (after process fork/spawn). Since this actor may be spawned on a different machine from the original nursery we need to try and load the local module - code (if it exists). + code (presuming it exists). ''' try: @@ -539,6 +296,11 @@ class Actor: raise def _get_rpc_func(self, ns, funcname): + ''' + Try to lookup and return a target RPC func from the + post-fork enabled module set. + + ''' try: return getattr(self._mods[ns], funcname) except KeyError as err: @@ -569,7 +331,8 @@ class Actor: ) -> None: ''' - Entry point for new inbound connections to the channel server. + Entry point for new inbound IPC connections on a specific + transport server. ''' self._no_more_peers = trio.Event() # unset by making new @@ -905,6 +668,8 @@ class Actor: except trio.BrokenResourceError: log.runtime(f"Channel {chan.uid} was already closed") + # TODO: rename to `._deliver_payload()` since this handles + # more then just `result` msgs now obvi XD async def _push_result( self, chan: Channel, @@ -913,7 +678,8 @@ class Actor: ) -> None|bool: ''' - Push an RPC result to the local consumer's queue. + Push an RPC msg-payload to the local consumer peer-task's + queue. ''' uid: tuple[str, str] = chan.uid @@ -959,11 +725,16 @@ class Actor: ) -> Context: ''' - Look up or create a new inter-actor-task-IPC-linked task - "context" which encapsulates the local task's scheduling - enviroment including a ``trio`` cancel scope, a pair of IPC - messaging "feeder" channels, and an RPC id unique to the - task-as-function invocation. + Look-up (existing) or create a new + inter-actor-SC-linked task "context" (a `Context`) which + encapsulates the local RPC task's execution enviroment + around `Channel` relayed msg handling including, + + - a dedicated `trio` cancel scope (`Context._scope`), + - a pair of IPC-msg-relay "feeder" mem-channels + (`Context._recv/send_chan`), + - and a "context id" (cid) unique to the task-pair + msging session's lifetime. ''' actor_uid = chan.uid @@ -1020,15 +791,17 @@ class Actor: ) -> Context: ''' - Send a ``'cmd'`` message to a remote actor, which starts - a remote task-as-function entrypoint. + Send a `'cmd'` msg to a remote actor, which requests the + start and schedule of a remote task-as-function's + entrypoint. - Synchronously validates the endpoint type and return a caller - side task ``Context`` that can be used to wait for responses - delivered by the local runtime's message processing loop. + Synchronously validates the endpoint type and returns + a (caller side) `Context` that can be used to accept + delivery of msg payloads from the local runtime's + processing loop: `._rpc.process_messages()`. ''' - cid = str(uuid.uuid4()) + cid: str = str(uuid.uuid4()) assert chan.uid ctx = self.get_context( chan=chan, @@ -1087,7 +860,17 @@ class Actor: async def _from_parent( self, parent_addr: tuple[str, int] | None, - ) -> tuple[Channel, tuple[str, int] | None]: + + ) -> tuple[ + Channel, + list[tuple[str, int]] | None, + ]: + ''' + Bootstrap this local actor's runtime config from its parent by + connecting back via the IPC transport, handshaking and then + `Channel.recv()`-ing seeded data. + + ''' try: # Connect back to the parent actor and conduct initial # handshake. From this point on if we error, we @@ -1168,10 +951,11 @@ class Actor: task_status: TaskStatus[trio.Nursery] = trio.TASK_STATUS_IGNORED, ) -> None: ''' - Start the channel server, begin listening for new connections. + Start the IPC transport server, begin listening for new connections. - This will cause an actor to continue living (blocking) until - ``cancel_server()`` is called. + This will cause an actor to continue living (and thus + blocking at the process/OS-thread level) until + `.cancel_server()` is called. ''' self._server_down = trio.Event() @@ -1206,8 +990,8 @@ class Actor: ''' Cancel this actor asap; can be called from a sync context. - Schedules `.cancel()` to be run immediately just like when - cancelled by the parent. + Schedules runtime cancellation via `Actor.cancel()` inside + the RPC service nursery. ''' assert self._service_n @@ -1229,15 +1013,15 @@ class Actor: ) -> bool: ''' Cancel this actor's runtime, eventually resulting in - the exit its containing process. + termination of its containing OS process. The ideal "deterministic" teardown sequence in order is: - - cancel all ongoing rpc tasks by cancel scope + - cancel all ongoing rpc tasks by cancel scope. - cancel the channel server to prevent new inbound - connections + connections. - cancel the "service" nursery reponsible for - spawning new rpc tasks - - return control the parent channel message loop + spawning new rpc tasks. + - return control the parent channel message loop. ''' ( @@ -1325,11 +1109,9 @@ class Actor: ) -> bool: ''' - Cancel a local task by call-id / channel. - - Note this method will be treated as a streaming function - by remote actor-callers due to the declaration of ``ctx`` - in the signature (for now). + Cancel a local (RPC) task by context-id/channel by calling + `trio.CancelScope.cancel()` on it's surrounding cancel + scope. ''' @@ -1441,8 +1223,9 @@ class Actor: ) -> None: ''' - Cancel all existing RPC responder tasks using the cancel scope - registered for each. + Cancel all ongoing RPC tasks owned/spawned for a given + `parent_chan: Channel` or simply all tasks (inside + `._service_n`) when `parent_chan=None`. ''' tasks: dict = self._rpc_tasks @@ -1527,8 +1310,8 @@ class Actor: def cancel_server(self) -> None: ''' - Cancel the internal channel server nursery thereby - preventing any new inbound connections from being established. + Cancel the internal IPC transport server nursery thereby + preventing any new inbound IPC connections establishing. ''' if self._server_n: @@ -1536,9 +1319,11 @@ class Actor: self._server_n.cancel_scope.cancel() @property - def accept_addr(self) -> tuple[str, int] | None: + @property + def accept_addr(self) -> tuple[str, int]: ''' - Primary address to which the channel server is bound. + Primary address to which the IPC transport server is + bound. ''' # throws OSError on failure @@ -1546,7 +1331,7 @@ class Actor: def get_parent(self) -> Portal: ''' - Return a portal to our parent actor. + Return a `Portal` to our parent. ''' assert self._parent_chan, "No parent channel for this actor?" @@ -1554,7 +1339,7 @@ class Actor: def get_chans(self, uid: tuple[str, str]) -> list[Channel]: ''' - Return all channels to the actor with provided uid. + Return all IPC channels to the actor with provided `uid`. ''' return self._peers[uid] @@ -1566,10 +1351,10 @@ class Actor: ) -> tuple[str, str]: ''' Exchange `(name, UUIDs)` identifiers as the first - communication step. + communication step with any (peer) remote `Actor`. These are essentially the "mailbox addresses" found in - actor model parlance. + "actor model" parlance. ''' await chan.send(self.uid) @@ -1583,6 +1368,13 @@ class Actor: return uid def is_infected_aio(self) -> bool: + ''' + If `True`, this actor is running `trio` in guest mode on + the `asyncio` event loop and thus can use the APIs in + `.to_asyncio` to coordinate tasks running in each + framework but within the same actor runtime. + + ''' return self._infected_aio @@ -1602,11 +1394,14 @@ async def async_main( ) -> None: ''' - Actor runtime entrypoint; start the IPC channel server, maybe connect - back to the parent, and startup all core machinery tasks. + Main `Actor` runtime entrypoint; start the transport-specific + IPC channel server, (maybe) connect back to parent (to receive + additional config), startup all core `trio` machinery for + delivering RPCs, register with the discovery system. - A "root" (or "top-level") nursery for this actor is opened here and - when cancelled/terminated effectively closes the actor's "runtime". + The "root" (or "top-level") and "service" `trio.Nursery`s are + opened here and when cancelled/terminated effectively shutdown + the actor's "runtime" and all thus all ongoing RPC tasks. ''' # attempt to retreive ``trio``'s sigint handler and stash it @@ -1814,367 +1609,7 @@ async def async_main( log.runtime("Runtime completed") -async def process_messages( - actor: Actor, - chan: Channel, - shield: bool = False, - task_status: TaskStatus[CancelScope] = trio.TASK_STATUS_IGNORED, - -) -> bool: - ''' - This is the per-channel, low level RPC task scheduler loop. - - Receive multiplexed RPC request messages from some remote process, - spawn handler tasks depending on request type and deliver responses - or boxed errors back to the remote caller (task). - - ''' - # TODO: once `trio` get's an "obvious way" for req/resp we - # should use it? - # https://github.com/python-trio/trio/issues/467 - log.runtime( - 'Entering IPC msg loop:\n' - f'peer: {chan.uid}\n' - f'|_{chan}\n' - ) - nursery_cancelled_before_task: bool = False - msg: dict | None = None - try: - # NOTE: this internal scope allows for keeping this - # message loop running despite the current task having - # been cancelled (eg. `open_portal()` may call this method - # from a locally spawned task) and recieve this scope - # using ``scope = Nursery.start()`` - with CancelScope(shield=shield) as loop_cs: - task_status.started(loop_cs) - async for msg in chan: - - # dedicated loop terminate sentinel - if msg is None: - - tasks: dict[ - tuple[Channel, str], - tuple[Context, Callable, trio.Event] - ] = actor._rpc_tasks.copy() - log.cancel( - f'Peer IPC channel terminated via `None` setinel msg?\n' - f'=> Cancelling all {len(tasks)} local RPC tasks..\n' - f'peer: {chan.uid}\n' - f'|_{chan}\n' - ) - for (channel, cid) in tasks: - if channel is chan: - await actor._cancel_task( - cid, - channel, - requesting_uid=channel.uid, - - ipc_msg=msg, - ) - break - - log.transport( # type: ignore - f'<= IPC msg from peer: {chan.uid}\n\n' - - # TODO: conditionally avoid fmting depending - # on log level (for perf)? - # => specifically `pformat()` sub-call..? - f'{pformat(msg)}\n' - ) - - cid = msg.get('cid') - if cid: - # deliver response to local caller/waiter - # via its per-remote-context memory channel. - await actor._push_result( - chan, - cid, - msg, - ) - - log.runtime( - 'Waiting on next IPC msg from\n' - f'peer: {chan.uid}:\n' - f'|_{chan}\n' - - # f'last msg: {msg}\n' - ) - continue - - # process a 'cmd' request-msg upack - # TODO: impl with native `msgspec.Struct` support !! - # -[ ] implement with ``match:`` syntax? - # -[ ] discard un-authed msgs as per, - # - try: - ( - ns, - funcname, - kwargs, - actorid, - cid, - ) = msg['cmd'] - - except KeyError: - # This is the non-rpc error case, that is, an - # error **not** raised inside a call to ``_invoke()`` - # (i.e. no cid was provided in the msg - see above). - # Push this error to all local channel consumers - # (normally portals) by marking the channel as errored - assert chan.uid - exc = unpack_error(msg, chan=chan) - chan._exc = exc - raise exc - - log.runtime( - 'Handling RPC cmd from\n' - f'peer: {actorid}\n' - '\n' - f'=> {ns}.{funcname}({kwargs})\n' - ) - if ns == 'self': - if funcname == 'cancel': - func: Callable = actor.cancel - kwargs |= { - 'req_chan': chan, - } - - # don't start entire actor runtime cancellation - # if this actor is currently in debug mode! - pdb_complete: trio.Event|None = _debug.Lock.local_pdb_complete - if pdb_complete: - await pdb_complete.wait() - - # Either of `Actor.cancel()`/`.cancel_soon()` - # was called, so terminate this IPC msg - # loop, exit back out into `async_main()`, - # and immediately start the core runtime - # machinery shutdown! - with CancelScope(shield=True): - await _invoke( - actor, - cid, - chan, - func, - kwargs, - is_rpc=False, - ) - - log.runtime( - 'Cancelling IPC transport msg-loop with peer:\n' - f'|_{chan}\n' - ) - loop_cs.cancel() - break - - if funcname == '_cancel_task': - func: Callable = actor._cancel_task - - # we immediately start the runtime machinery - # shutdown - # with CancelScope(shield=True): - target_cid: str = kwargs['cid'] - kwargs |= { - # NOTE: ONLY the rpc-task-owning - # parent IPC channel should be able to - # cancel it! - 'parent_chan': chan, - 'requesting_uid': chan.uid, - 'ipc_msg': msg, - } - # TODO: remove? already have emit in meth. - # log.runtime( - # f'Rx RPC task cancel request\n' - # f'<= canceller: {chan.uid}\n' - # f' |_{chan}\n\n' - # f'=> {actor}\n' - # f' |_cid: {target_cid}\n' - # ) - try: - await _invoke( - actor, - cid, - chan, - func, - kwargs, - is_rpc=False, - ) - except BaseException: - log.exception( - 'Failed to cancel task?\n' - f'<= canceller: {chan.uid}\n' - f' |_{chan}\n\n' - f'=> {actor}\n' - f' |_cid: {target_cid}\n' - ) - continue - else: - # normally registry methods, eg. - # ``.register_actor()`` etc. - func: Callable = getattr(actor, funcname) - - else: - # complain to client about restricted modules - try: - func = actor._get_rpc_func(ns, funcname) - except (ModuleNotExposed, AttributeError) as err: - err_msg: dict[str, dict] = pack_error( - err, - cid=cid, - ) - await chan.send(err_msg) - continue - - # schedule a task for the requested RPC function - # in the actor's main "service nursery". - # TODO: possibly a service-tn per IPC channel for - # supervision isolation? would avoid having to - # manage RPC tasks individually in `._rpc_tasks` - # table? - log.runtime( - f'Spawning task for RPC request\n' - f'<= caller: {chan.uid}\n' - f' |_{chan}\n\n' - # TODO: maddr style repr? - # f' |_@ /ipv4/{chan.raddr}/tcp/{chan.rport}/' - # f'cid="{cid[-16:]} .."\n\n' - - f'=> {actor}\n' - f' |_cid: {cid}\n' - f' |>> {func}()\n' - ) - assert actor._service_n # wait why? do it at top? - try: - ctx: Context = await actor._service_n.start( - partial( - _invoke, - actor, - cid, - chan, - func, - kwargs, - ), - name=funcname, - ) - - except ( - RuntimeError, - BaseExceptionGroup, - ): - # avoid reporting a benign race condition - # during actor runtime teardown. - nursery_cancelled_before_task: bool = True - break - - # in the lone case where a ``Context`` is not - # delivered, it's likely going to be a locally - # scoped exception from ``_invoke()`` itself. - if isinstance(err := ctx, Exception): - log.warning( - 'Task for RPC failed?' - f'|_ {func}()\n\n' - - f'{err}' - ) - continue - - else: - # mark that we have ongoing rpc tasks - actor._ongoing_rpc_tasks = trio.Event() - - # store cancel scope such that the rpc task can be - # cancelled gracefully if requested - actor._rpc_tasks[(chan, cid)] = ( - ctx, - func, - trio.Event(), - ) - - log.runtime( - 'Waiting on next IPC msg from\n' - f'peer: {chan.uid}\n' - f'|_{chan}\n' - ) - - # end of async for, channel disconnect vis - # ``trio.EndOfChannel`` - log.runtime( - f"{chan} for {chan.uid} disconnected, cancelling tasks" - ) - await actor.cancel_rpc_tasks( - req_uid=actor.uid, - # a "self cancel" in terms of the lifetime of the - # IPC connection which is presumed to be the - # source of any requests for spawned tasks. - parent_chan=chan, - ) - - except ( - TransportClosed, - ): - # 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 - # the message loop and expect the teardown sequence to clean - # up. - # TODO: don't show this msg if it's an emphemeral - # discovery ep call? - log.runtime( - f'channel closed abruptly with\n' - f'peer: {chan.uid}\n' - f'|_{chan.raddr}\n' - ) - - # transport **was** disconnected - return True - - except ( - Exception, - BaseExceptionGroup, - ) as err: - - if nursery_cancelled_before_task: - sn: Nursery = actor._service_n - assert sn and sn.cancel_scope.cancel_called # sanity - log.cancel( - f'Service nursery cancelled before it handled {funcname}' - ) - else: - # ship any "internal" exception (i.e. one from internal - # machinery not from an rpc task) to parent - match err: - case ContextCancelled(): - log.cancel( - f'Actor: {actor.uid} was context-cancelled with,\n' - f'str(err)' - ) - case _: - log.exception("Actor errored:") - - if actor._parent_chan: - await try_ship_error_to_remote( - actor._parent_chan, - err, - ) - - # if this is the `MainProcess` we expect the error broadcasting - # above to trigger an error at consuming portal "checkpoints" - raise - - finally: - # msg debugging for when he machinery is brokey - log.runtime( - 'Exiting IPC msg loop with\n' - f'peer: {chan.uid}\n' - f'|_{chan}\n\n' - 'final msg:\n' - f'{pformat(msg)}\n' - ) - - # transport **was not** disconnected - return False - - +# TODO: rename to `Registry` and move to `._discovery`! class Arbiter(Actor): ''' A special actor who knows all the other actors and always has From 85825cdd76df2cc087034a10695fd5bd88be708e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 13 Mar 2024 16:09:31 -0400 Subject: [PATCH 134/146] Add `.trionics._broadcast` todos for py 3.12 --- tractor/trionics/_broadcast.py | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/tractor/trionics/_broadcast.py b/tractor/trionics/_broadcast.py index 244a42d4..a5d31871 100644 --- a/tractor/trionics/_broadcast.py +++ b/tractor/trionics/_broadcast.py @@ -26,7 +26,6 @@ from contextlib import asynccontextmanager from functools import partial from operator import ne from typing import ( - Optional, Callable, Awaitable, Any, @@ -45,6 +44,11 @@ from tractor.log import get_logger log = get_logger(__name__) +# TODO: use new type-vars syntax from 3.12 +# https://realpython.com/python312-new-features/#dedicated-type-variable-syntax +# https://docs.python.org/3/whatsnew/3.12.html#whatsnew312-pep695 +# https://docs.python.org/3/reference/simple_stmts.html#type +# # A regular invariant generic type T = TypeVar("T") @@ -110,7 +114,7 @@ class BroadcastState(Struct): # broadcast event to wake up all sleeping consumer tasks # on a newly produced value from the sender. - recv_ready: Optional[tuple[int, trio.Event]] = None + recv_ready: tuple[int, trio.Event]|None = None # if a ``trio.EndOfChannel`` is received on any # consumer all consumers should be placed in this state @@ -164,7 +168,7 @@ class BroadcastReceiver(ReceiveChannel): rx_chan: AsyncReceiver, state: BroadcastState, - receive_afunc: Optional[Callable[[], Awaitable[Any]]] = None, + receive_afunc: Callable[[], Awaitable[Any]]|None = None, raise_on_lag: bool = True, ) -> None: @@ -452,7 +456,7 @@ def broadcast_receiver( recv_chan: AsyncReceiver, max_buffer_size: int, - receive_afunc: Optional[Callable[[], Awaitable[Any]]] = None, + receive_afunc: Callable[[], Awaitable[Any]]|None = None, raise_on_lag: bool = True, ) -> BroadcastReceiver: From 05f28c87282868a1a7f6a23bf0941d2ae42ac3fc Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 13 Mar 2024 16:21:30 -0400 Subject: [PATCH 135/146] Pin to `trio>=0.24` to avoid `trio_typing` --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index b8f915ff..b35b6b3c 100755 --- a/setup.py +++ b/setup.py @@ -45,7 +45,7 @@ setup( # trio related # proper range spec: # https://packaging.python.org/en/latest/discussions/install-requires-vs-requirements/#id5 - 'trio >= 0.22', + 'trio >= 0.24', 'async_generator', 'trio_typing', 'exceptiongroup', From a87df3009f0deabc6accba6ef4b45b2067b0b670 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 13 Mar 2024 18:41:24 -0400 Subject: [PATCH 136/146] Drop now-deprecated deps on modern `trio`/Python - `trio_typing` is nearly obsolete since `trio >= 0.23` - `exceptiongroup` is built-in to python 3.11 - `async_generator` primitives have lived in `contextlib` for quite a while! --- examples/debugging/debug_mode_hang.py | 9 +++++++++ examples/parallelism/concurrent_actors_primes.py | 8 +++++--- examples/rpc_bidir_streaming.py | 2 +- setup.py | 7 ++++--- tests/test_cancellation.py | 4 ---- tests/test_child_manages_service_nursery.py | 6 ++++-- tests/test_infected_asyncio.py | 1 - tests/test_runtime.py | 3 ++- tests/test_trioisms.py | 2 +- tractor/_exceptions.py | 11 +++++------ tractor/_root.py | 3 ++- tractor/_rpc.py | 6 +----- tractor/_runtime.py | 2 -- tractor/_spawn.py | 15 +++++++-------- tractor/_supervise.py | 1 - tractor/experimental/_pubsub.py | 2 +- tractor/trionics/_mngrs.py | 7 +++---- 17 files changed, 45 insertions(+), 44 deletions(-) create mode 100644 examples/debugging/debug_mode_hang.py diff --git a/examples/debugging/debug_mode_hang.py b/examples/debugging/debug_mode_hang.py new file mode 100644 index 00000000..a81890ed --- /dev/null +++ b/examples/debugging/debug_mode_hang.py @@ -0,0 +1,9 @@ +''' +Reproduce a bug where enabling debug mode for a sub-actor actually causes +a hang on teardown... + +''' +import asyncio + +import trio +import tractor diff --git a/examples/parallelism/concurrent_actors_primes.py b/examples/parallelism/concurrent_actors_primes.py index feaaca79..748861e6 100644 --- a/examples/parallelism/concurrent_actors_primes.py +++ b/examples/parallelism/concurrent_actors_primes.py @@ -8,7 +8,10 @@ This uses no extra threads, fancy semaphores or futures; all we need is ``tractor``'s channels. """ -from contextlib import asynccontextmanager +from contextlib import ( + asynccontextmanager as acm, + aclosing, +) from typing import Callable import itertools import math @@ -16,7 +19,6 @@ import time import tractor import trio -from async_generator import aclosing PRIMES = [ @@ -44,7 +46,7 @@ async def is_prime(n): return True -@asynccontextmanager +@acm async def worker_pool(workers=4): """Though it's a trivial special case for ``tractor``, the well known "worker pool" seems to be the defacto "but, I want this diff --git a/examples/rpc_bidir_streaming.py b/examples/rpc_bidir_streaming.py index 73200814..c961bf20 100644 --- a/examples/rpc_bidir_streaming.py +++ b/examples/rpc_bidir_streaming.py @@ -13,7 +13,7 @@ async def simple_rpc( ''' # signal to parent that we're up much like - # ``trio_typing.TaskStatus.started()`` + # ``trio.TaskStatus.started()`` await ctx.started(data + 1) async with ctx.open_stream() as stream: diff --git a/setup.py b/setup.py index b35b6b3c..612f4d57 100755 --- a/setup.py +++ b/setup.py @@ -46,9 +46,10 @@ setup( # proper range spec: # https://packaging.python.org/en/latest/discussions/install-requires-vs-requirements/#id5 'trio >= 0.24', - 'async_generator', - 'trio_typing', - 'exceptiongroup', + + # 'async_generator', # in stdlib mostly! + # 'trio_typing', # trio==0.23.0 has type hints! + # 'exceptiongroup', # in stdlib as of 3.11! # tooling 'tricycle', diff --git a/tests/test_cancellation.py b/tests/test_cancellation.py index 5b589f6a..b8c14af3 100644 --- a/tests/test_cancellation.py +++ b/tests/test_cancellation.py @@ -8,10 +8,6 @@ import platform import time from itertools import repeat -from exceptiongroup import ( - BaseExceptionGroup, - ExceptionGroup, -) import pytest import trio import tractor diff --git a/tests/test_child_manages_service_nursery.py b/tests/test_child_manages_service_nursery.py index 228d6ade..350f939b 100644 --- a/tests/test_child_manages_service_nursery.py +++ b/tests/test_child_manages_service_nursery.py @@ -6,13 +6,15 @@ sub-sub-actor daemons. ''' from typing import Optional import asyncio -from contextlib import asynccontextmanager as acm +from contextlib import ( + asynccontextmanager as acm, + aclosing, +) import pytest import trio import tractor from tractor import RemoteActorError -from async_generator import aclosing async def aio_streamer( diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index a3f96ee8..568708a2 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -8,7 +8,6 @@ import builtins import itertools import importlib -from exceptiongroup import BaseExceptionGroup import pytest import trio import tractor diff --git a/tests/test_runtime.py b/tests/test_runtime.py index 3755af1b..55553dd9 100644 --- a/tests/test_runtime.py +++ b/tests/test_runtime.py @@ -64,7 +64,8 @@ async def test_lifetime_stack_wipes_tmpfile( except ( tractor.RemoteActorError, - tractor.BaseExceptionGroup, + # tractor.BaseExceptionGroup, + BaseExceptionGroup, ): pass diff --git a/tests/test_trioisms.py b/tests/test_trioisms.py index 5b19f50d..27dc6c34 100644 --- a/tests/test_trioisms.py +++ b/tests/test_trioisms.py @@ -5,7 +5,7 @@ want to see changed. ''' import pytest import trio -from trio_typing import TaskStatus +from trio import TaskStatus @pytest.mark.parametrize( diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 259a28a7..344f0c33 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -30,11 +30,10 @@ from typing import ( import textwrap import traceback -import exceptiongroup as eg import trio -from ._state import current_actor -from .log import get_logger +from tractor._state import current_actor +from tractor.log import get_logger if TYPE_CHECKING: from ._context import Context @@ -373,7 +372,6 @@ def unpack_error( for ns in [ builtins, _this_mod, - eg, trio, ]: if suberror_type := getattr( @@ -396,12 +394,13 @@ def unpack_error( def is_multi_cancelled(exc: BaseException) -> bool: ''' - Predicate to determine if a possible ``eg.BaseExceptionGroup`` contains + Predicate to determine if a possible ``BaseExceptionGroup`` contains only ``trio.Cancelled`` sub-exceptions (and is likely the result of cancelling a collection of subtasks. ''' - if isinstance(exc, eg.BaseExceptionGroup): + # if isinstance(exc, eg.BaseExceptionGroup): + if isinstance(exc, BaseExceptionGroup): return exc.subgroup( lambda exc: isinstance(exc, trio.Cancelled) ) is not None diff --git a/tractor/_root.py b/tractor/_root.py index 1ec745c9..881dc90f 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -29,12 +29,13 @@ import typing import warnings -from exceptiongroup import BaseExceptionGroup import trio from ._runtime import ( Actor, Arbiter, + # TODO: rename and make a non-actor subtype? + # Arbiter as Registry, async_main, ) from . import _debug diff --git a/tractor/_rpc.py b/tractor/_rpc.py index 54a60be6..6bdc0c6b 100644 --- a/tractor/_rpc.py +++ b/tractor/_rpc.py @@ -21,6 +21,7 @@ Remote (task) Procedure Call (scheduling) with SC transitive semantics. from __future__ import annotations from contextlib import ( asynccontextmanager as acm, + aclosing, ) from functools import partial import inspect @@ -34,17 +35,12 @@ from typing import ( ) import warnings -from async_generator import aclosing -from exceptiongroup import BaseExceptionGroup import trio from trio import ( CancelScope, Nursery, TaskStatus, ) -# from trio_typing import ( -# TaskStatus, -# ) from .msg import NamespacePath from ._ipc import Channel diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 84ad8cea..d28ea263 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -60,8 +60,6 @@ import os import trio from trio import ( CancelScope, -) -from trio_typing import ( Nursery, TaskStatus, ) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index a2643876..e91638bc 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -31,25 +31,24 @@ from typing import ( TYPE_CHECKING, ) -from exceptiongroup import BaseExceptionGroup import trio -from trio_typing import TaskStatus +from trio import TaskStatus from ._debug import ( maybe_wait_for_debugger, acquire_debug_lock, ) -from ._state import ( +from tractor._state import ( current_actor, is_main_process, is_root_process, debug_mode, ) -from .log import get_logger -from ._portal import Portal -from ._runtime import Actor -from ._entry import _mp_main -from ._exceptions import ActorFailure +from tractor.log import get_logger +from tractor._portal import Portal +from tractor._runtime import Actor +from tractor._entry import _mp_main +from tractor._exceptions import ActorFailure if TYPE_CHECKING: diff --git a/tractor/_supervise.py b/tractor/_supervise.py index 507e2544..1e5ea387 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -29,7 +29,6 @@ from typing import ( import typing import warnings -from exceptiongroup import BaseExceptionGroup import trio from ._debug import maybe_wait_for_debugger diff --git a/tractor/experimental/_pubsub.py b/tractor/experimental/_pubsub.py index 89f286d2..b894ed49 100644 --- a/tractor/experimental/_pubsub.py +++ b/tractor/experimental/_pubsub.py @@ -31,7 +31,7 @@ from typing import ( Callable, ) from functools import partial -from async_generator import aclosing +from contextlib import aclosing import trio import wrapt diff --git a/tractor/trionics/_mngrs.py b/tractor/trionics/_mngrs.py index 1c079cdb..f57be0a7 100644 --- a/tractor/trionics/_mngrs.py +++ b/tractor/trionics/_mngrs.py @@ -33,10 +33,9 @@ from typing import ( ) import trio -from trio_typing import TaskStatus -from .._state import current_actor -from ..log import get_logger +from tractor._state import current_actor +from tractor.log import get_logger log = get_logger(__name__) @@ -184,7 +183,7 @@ class _Cache: cls, mng, ctx_key: tuple, - task_status: TaskStatus[T] = trio.TASK_STATUS_IGNORED, + task_status: trio.TaskStatus[T] = trio.TASK_STATUS_IGNORED, ) -> None: async with mng as value: From 771fc33801f439dff99bc31f126165bf6147c477 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 19 Mar 2024 18:40:50 -0400 Subject: [PATCH 137/146] Absorb EoCs via `Context.open_stream()` silently I swear long ago it used to operate this way but, I guess this finalizes the design decision. It makes a lot more sense to *not* propagate any `trio.EndOfChannel` raised from a `Context.open_stream() as stream:` block when that EoC is due to graceful-explicit stream termination. We use the EoC much like a `StopAsyncIteration` where the error indicates termination of the stream due to either: - reception of a stop IPC msg indicating the far end ended the stream (gracecfully), - closure of the underlying `Context._recv_chan` either by the runtime or due to user code having called `MsgStream.aclose()`. User code shouldn't expect to handle EoC outside the block since the `@acm` having closed should indicate the exactly same lifetime state (of said stream) ;) Deats: - add special EoC handler in `.open_stream()` which silently "absorbs" the error only when the stream is already marked as closed (meaning the EoC indeed corresponds to IPC closure) with an assert for now ensuring the error is the same as set to `MsgStream._eoc`. - in `MsgStream.receive()` break up the handlers for EoC and `trio.ClosedResourceError` since the error instances are saved to different variables and we **don't** want to rewrite the exception in the eoc case (normally to mask `trio` internals in tbs) bc we need the instance to be the exact one for doing checks inside `.open_stream().__aexit__()` to absorb it. Other surrounding "improvements": - start using the new `Context.maybe_raise()` helper where it can easily replace existing equivalent block-sections. - use new `RemoteActorError.src_uid` as required. --- tractor/_context.py | 117 ++++++++++++++++++++++++++---------------- tractor/_streaming.py | 89 ++++++++++++++++++-------------- 2 files changed, 123 insertions(+), 83 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index 55902281..11975bae 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -169,8 +169,7 @@ async def _drain_to_final_msg( # only when we are sure the remote error is # the source cause of this local task's # cancellation. - if re := ctx._remote_error: - ctx._maybe_raise_remote_err(re) + ctx.maybe_raise() # CASE 1: we DID request the cancel we simply # continue to bubble up as normal. @@ -257,6 +256,13 @@ async def _drain_to_final_msg( ) # XXX fallthrough to handle expected error XXX + # TODO: replace this with `ctx.maybe_raise()` + # + # TODO: would this be handier for this case maybe? + # async with maybe_raise_on_exit() as raises: + # if raises: + # log.error('some msg about raising..') + re: Exception|None = ctx._remote_error if re: log.critical( @@ -595,7 +601,7 @@ class Context: if not re: return False - if from_uid := re.src_actor_uid: + if from_uid := re.src_uid: from_uid: tuple = tuple(from_uid) our_uid: tuple = self._actor.uid @@ -825,7 +831,7 @@ class Context: # cancellation. maybe_error_src: tuple = getattr( error, - 'src_actor_uid', + 'src_uid', None, ) self._canceller = ( @@ -1030,8 +1036,8 @@ class Context: @acm async def open_stream( self, - allow_overruns: bool | None = False, - msg_buffer_size: int | None = None, + allow_overruns: bool|None = False, + msg_buffer_size: int|None = None, ) -> AsyncGenerator[MsgStream, None]: ''' @@ -1071,13 +1077,16 @@ class Context: # absorbed there (silently) and we DO NOT want to # actually try to stream - a cancel msg was already # sent to the other side! - if self._remote_error: - # NOTE: this is diff then calling - # `._maybe_raise_remote_err()` specifically - # because any task entering this `.open_stream()` - # AFTER cancellation has already been requested, - # we DO NOT want to absorb any ctxc ACK silently! - raise self._remote_error + self.maybe_raise( + raise_ctxc_from_self_call=True, + ) + # NOTE: this is diff then calling + # `._maybe_raise_remote_err()` specifically + # because we want to raise a ctxc on any task entering this `.open_stream()` + # AFTER cancellation was already been requested, + # we DO NOT want to absorb any ctxc ACK silently! + # if self._remote_error: + # raise self._remote_error # XXX NOTE: if no `ContextCancelled` has been responded # back from the other side (yet), we raise a different @@ -1158,7 +1167,6 @@ class Context: # await trio.lowlevel.checkpoint() yield stream - # XXX: (MEGA IMPORTANT) if this is a root opened process we # wait for any immediate child in debug before popping the # context from the runtime msg loop otherwise inside @@ -1183,12 +1191,23 @@ class Context: # # await stream.aclose() - # if re := ctx._remote_error: - # ctx._maybe_raise_remote_err( - # re, - # raise_ctxc_from_self_call=True, - # ) - # await trio.lowlevel.checkpoint() + # NOTE: absorb and do not raise any + # EoC received from the other side such that + # it is not raised inside the surrounding + # context block's scope! + except trio.EndOfChannel as eoc: + if ( + eoc + and stream.closed + ): + # sanity, can remove? + assert eoc is stream._eoc + # from .devx import pause + # await pause() + log.warning( + 'Stream was terminated by EoC\n\n' + f'{repr(eoc)}\n' + ) finally: if self._portal: @@ -1204,7 +1223,6 @@ class Context: # TODO: replace all the instances of this!! XD def maybe_raise( self, - hide_tb: bool = True, **kwargs, @@ -1388,33 +1406,41 @@ class Context: f'{drained_msgs}' ) - if ( - (re := self._remote_error) - # and self._result == res_placeholder - ): - self._maybe_raise_remote_err( - re, - # NOTE: obvi we don't care if we - # overran the far end if we're already - # waiting on a final result (msg). - # raise_overrun_from_self=False, - raise_overrun_from_self=( - raise_overrun - and - # only when we ARE NOT the canceller - # should we raise overruns, bc ow we're - # raising something we know might happen - # during cancellation ;) - (not self._cancel_called) - ), + self.maybe_raise( + raise_overrun_from_self=( + raise_overrun + and + # only when we ARE NOT the canceller + # should we raise overruns, bc ow we're + # raising something we know might happen + # during cancellation ;) + (not self._cancel_called) ) + ) + # if ( + # (re := self._remote_error) + # # and self._result == res_placeholder + # ): + # self._maybe_raise_remote_err( + # re, + # # NOTE: obvi we don't care if we + # # overran the far end if we're already + # # waiting on a final result (msg). + # # raise_overrun_from_self=False, + # raise_overrun_from_self=( + # raise_overrun + # and + # # only when we ARE NOT the canceller + # # should we raise overruns, bc ow we're + # # raising something we know might happen + # # during cancellation ;) + # (not self._cancel_called) + # ), + # ) # if maybe_err: # self._result = maybe_err return self.outcome - # None if self._result == res_placeholder - # else self._result - # ) # TODO: switch this with above which should be named # `.wait_for_outcome()` and instead do @@ -1863,8 +1889,9 @@ async def open_context_from_portal( # TODO: if we set this the wrapping `@acm` body will # still be shown (awkwardly) on pdb REPL entry. Ideally - # we can similarly annotate that frame to NOT show? - hide_tb: bool = True, + # we can similarly annotate that frame to NOT show? for now + # we DO SHOW this frame since it's awkward ow.. + hide_tb: bool = False, # proxied to RPC **kwargs, diff --git a/tractor/_streaming.py b/tractor/_streaming.py index 149bb350..e0015fe4 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -136,7 +136,7 @@ class MsgStream(trio.abc.Channel): # return await self.receive() # except trio.EndOfChannel: # raise StopAsyncIteration - + # # see ``.aclose()`` for notes on the old behaviour prior to # introducing this if self._eoc: @@ -152,7 +152,6 @@ class MsgStream(trio.abc.Channel): return msg['yield'] except KeyError as kerr: - # log.exception('GOT KEYERROR') src_err = kerr # NOTE: may raise any of the below error types @@ -166,30 +165,20 @@ class MsgStream(trio.abc.Channel): stream=self, ) - # XXX: we close the stream on any of these error conditions: + # 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._push_result()`, but now the 'stop' message handling + # has been put just above inside `_raise_from_no_key_in_msg()`. except ( - # trio.ClosedResourceError, # by self._rx_chan - trio.EndOfChannel, # by self._rx_chan or `stop` msg from far end + trio.EndOfChannel, ) as eoc: - # log.exception('GOT EOC') src_err = eoc self._eoc = 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. - - # an ``EndOfChannel`` indicates either the internal recv - # memchan exhausted **or** we raisesd it just above after - # receiving a `stop` message from the far end of the stream. - - # Previously this was triggered by calling ``.aclose()`` on - # the send side of the channel inside - # ``Actor._push_result()`` (should still be commented code - # there - which should eventually get removed), but now the - # 'stop' message handling has been put just above. - # 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 @@ -210,8 +199,11 @@ class MsgStream(trio.abc.Channel): # raise eoc - except trio.ClosedResourceError as cre: # by self._rx_chan - # log.exception('GOT CRE') + # 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( '`Context._rx_chan` was already closed?' @@ -237,15 +229,30 @@ class MsgStream(trio.abc.Channel): # over the end-of-stream connection error since likely # the remote error was the source cause? ctx: Context = self._ctx - if re := ctx._remote_error: - ctx._maybe_raise_remote_err( - re, - raise_ctxc_from_self_call=True, - ) + ctx.maybe_raise( + raise_ctxc_from_self_call=True, + ) - # propagate any error but hide low-level frames from - # caller by default. - if hide_tb: + # propagate any error but hide low-level frame details + # from the caller by default for debug noise reduction. + if ( + hide_tb + + # 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 + + # - `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) + ): raise type(src_err)(*src_err.args) from src_err else: raise src_err @@ -370,6 +377,10 @@ class MsgStream(trio.abc.Channel): # await rx_chan.aclose() if not self._eoc: + log.cancel( + 'Stream closed before it received an EoC?\n' + 'Setting eoc manually..\n..' + ) self._eoc: bool = trio.EndOfChannel( f'Context stream closed by {self._ctx.side}\n' f'|_{self}\n' @@ -414,13 +425,11 @@ class MsgStream(trio.abc.Channel): @property def closed(self) -> bool: - if ( - (rxc := self._rx_chan._closed) - or - (_closed := self._closed) - or - (_eoc := self._eoc) - ): + + rxc: bool = self._rx_chan._closed + _closed: bool|Exception = self._closed + _eoc: bool|trio.EndOfChannel = self._eoc + if rxc or _closed or _eoc: log.runtime( f'`MsgStream` is already closed\n' f'{self}\n' @@ -496,7 +505,11 @@ class MsgStream(trio.abc.Channel): ''' __tracebackhide__: bool = hide_tb + # raise any alreay known error immediately self._ctx.maybe_raise() + if self._eoc: + raise self._eoc + if self._closed: raise self._closed From 3706abca71e75384fc812977b37b2b263cc5c778 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 19 Mar 2024 19:33:06 -0400 Subject: [PATCH 138/146] Adjust advanced faults test(s) for absorbed EoCs More or less just simplifies to not seeing the stream closure errors and instead expecting KBIs from the simulated user who 'ctl-cs after hang'. Toss in a little `stuff_hangin_ctlc()` to the script to wrap all that and always check stream closure before sending the final KBI. --- .../ipc_failure_during_stream.py | 120 ++++++++++-------- tests/test_advanced_faults.py | 74 +++++++---- 2 files changed, 116 insertions(+), 78 deletions(-) diff --git a/examples/advanced_faults/ipc_failure_during_stream.py b/examples/advanced_faults/ipc_failure_during_stream.py index c7322a7c..9dca92b1 100644 --- a/examples/advanced_faults/ipc_failure_during_stream.py +++ b/examples/advanced_faults/ipc_failure_during_stream.py @@ -6,6 +6,7 @@ been an outage) and we want to ensure that despite being in debug mode actor tree will eventually be cancelled without leaving any zombies. ''' +from contextlib import asynccontextmanager as acm from functools import partial from tractor import ( @@ -17,6 +18,7 @@ from tractor import ( _testing, ) import trio +import pytest async def break_ipc( @@ -41,6 +43,13 @@ async def break_ipc( await stream.aclose() method: str = method or def_method + print( + '#################################\n' + 'Simulating CHILD-side IPC BREAK!\n' + f'method: {method}\n' + f'pre `.aclose()`: {pre_close}\n' + '#################################\n' + ) match method: case 'trans_aclose': @@ -80,17 +89,17 @@ async def break_ipc_then_error( break_ipc_with: str|None = None, pre_close: bool = False, ): + await break_ipc( + stream=stream, + method=break_ipc_with, + pre_close=pre_close, + ) async for msg in stream: await stream.send(msg) - await break_ipc( - stream=stream, - method=break_ipc_with, - pre_close=pre_close, - ) - assert 0 + + assert 0 -# async def close_stream_and_error( async def iter_ipc_stream( stream: MsgStream, break_ipc_with: str|None = None, @@ -99,20 +108,6 @@ async def iter_ipc_stream( async for msg in stream: await stream.send(msg) - # wipe out channel right before raising - # await break_ipc( - # stream=stream, - # method=break_ipc_with, - # pre_close=pre_close, - # ) - - # send channel close msg at SC-prot level - # - # TODO: what should get raised here if anything? - # await stream.aclose() - - # assert 0 - @context async def recv_and_spawn_net_killers( @@ -134,14 +129,16 @@ async def recv_and_spawn_net_killers( async for i in stream: print(f'child echoing {i}') await stream.send(i) + if ( break_ipc_after and - i > break_ipc_after + i >= break_ipc_after ): - '#################################\n' - 'Simulating CHILD-side IPC BREAK!\n' - '#################################\n' + n.start_soon( + iter_ipc_stream, + stream, + ) n.start_soon( partial( break_ipc_then_error, @@ -149,10 +146,23 @@ async def recv_and_spawn_net_killers( pre_close=pre_close, ) ) - n.start_soon( - iter_ipc_stream, - stream, - ) + + +@acm +async def stuff_hangin_ctlc(timeout: float = 1) -> None: + + with trio.move_on_after(timeout) as cs: + yield timeout + + if cs.cancelled_caught: + # pretend to be a user seeing no streaming action + # thinking it's a hang, and then hitting ctl-c.. + print( + f"i'm a user on the PARENT side and thingz hangin " + f'after timeout={timeout} ???\n\n' + 'MASHING CTlR-C..!?\n' + ) + raise KeyboardInterrupt async def main( @@ -169,9 +179,6 @@ async def main( ) -> None: - # from tractor._state import _runtime_vars as rtv - # rtv['_debug_mode'] = debug_mode - async with ( open_nursery( start_method=start_method, @@ -190,10 +197,11 @@ async def main( ) async with ( + stuff_hangin_ctlc(timeout=2) as timeout, _testing.expect_ctxc( yay=( break_parent_ipc_after - or break_child_ipc_after, + or break_child_ipc_after ), # TODO: we CAN'T remove this right? # since we need the ctxc to bubble up from either @@ -205,12 +213,14 @@ async def main( # and KBI in an eg? reraise=True, ), + portal.open_context( recv_and_spawn_net_killers, break_ipc_after=break_child_ipc_after, pre_close=pre_close, ) as (ctx, sent), ): + rx_eoc: bool = False ipc_break_sent: bool = False async with ctx.open_stream() as stream: for i in range(1000): @@ -228,6 +238,7 @@ async def main( '#################################\n' ) + # TODO: other methods? see break func above. # await stream._ctx.chan.send(None) # await stream._ctx.chan.transport.stream.send_eof() await stream._ctx.chan.transport.stream.aclose() @@ -251,10 +262,12 @@ async def main( # TODO: is this needed or no? raise - timeout: int = 1 - print(f'Entering `stream.receive()` with timeout={timeout}\n') - with trio.move_on_after(timeout) as cs: - + # timeout: int = 1 + # with trio.move_on_after(timeout) as cs: + async with stuff_hangin_ctlc() as timeout: + print( + f'PARENT `stream.receive()` with timeout={timeout}\n' + ) # NOTE: in the parent side IPC failure case this # will raise an ``EndOfChannel`` after the child # is killed and sends a stop msg back to it's @@ -266,23 +279,30 @@ async def main( f'{rx}\n' ) except trio.EndOfChannel: + rx_eoc: bool = True print('MsgStream got EoC for PARENT') raise - if cs.cancelled_caught: - # pretend to be a user seeing no streaming action - # thinking it's a hang, and then hitting ctl-c.. - print( - f"YOO i'm a PARENT user anddd thingz hangin..\n" - f'after timeout={timeout}\n' - ) + print( + 'Streaming finished and we got Eoc.\n' + 'Canceling `.open_context()` in root with\n' + 'CTlR-C..' + ) + if rx_eoc: + assert stream.closed + try: + await stream.send(i) + pytest.fail('stream not closed?') + except ( + trio.ClosedResourceError, + trio.EndOfChannel, + ) as send_err: + if rx_eoc: + assert send_err is stream._eoc + else: + assert send_err is stream._closed - print( - "YOO i'm mad!\n" - 'The send side is dun but thingz hangin..\n' - 'MASHING CTlR-C Ctl-c..' - ) - raise KeyboardInterrupt + raise KeyboardInterrupt if __name__ == '__main__': diff --git a/tests/test_advanced_faults.py b/tests/test_advanced_faults.py index 8b73b4c2..5f73ac6c 100644 --- a/tests/test_advanced_faults.py +++ b/tests/test_advanced_faults.py @@ -85,8 +85,8 @@ def test_ipc_channel_break_during_stream( ''' if spawn_backend != 'trio': - # if debug_mode: - # pytest.skip('`debug_mode` only supported on `trio` spawner') + if debug_mode: + pytest.skip('`debug_mode` only supported on `trio` spawner') # non-`trio` spawners should never hit the hang condition that # requires the user to do ctl-c to cancel the actor tree. @@ -107,7 +107,10 @@ def test_ipc_channel_break_during_stream( # AND we tell the child to call `MsgStream.aclose()`. and pre_aclose_msgstream ): - expect_final_exc = trio.EndOfChannel + # expect_final_exc = trio.EndOfChannel + # ^XXX NOPE! XXX^ since now `.open_stream()` absorbs this + # gracefully! + expect_final_exc = KeyboardInterrupt # NOTE when ONLY the child breaks or it breaks BEFORE the # parent we expect the parent to get a closed resource error @@ -120,11 +123,25 @@ def test_ipc_channel_break_during_stream( and ipc_break['break_parent_ipc_after'] is False ): - expect_final_exc = trio.ClosedResourceError + # NOTE: we DO NOT expect this any more since + # the child side's channel will be broken silently + # and nothing on the parent side will indicate this! + # expect_final_exc = trio.ClosedResourceError - # if child calls `MsgStream.aclose()` then expect EoC. + # NOTE: child will send a 'stop' msg before it breaks + # the transport channel BUT, that will be absorbed by the + # `ctx.open_stream()` block and thus the `.open_context()` + # should hang, after which the test script simulates + # a user sending ctl-c by raising a KBI. if pre_aclose_msgstream: - expect_final_exc = trio.EndOfChannel + expect_final_exc = KeyboardInterrupt + + # XXX OLD XXX + # if child calls `MsgStream.aclose()` then expect EoC. + # ^ XXX not any more ^ since eoc is always absorbed + # gracefully and NOT bubbled to the `.open_context()` + # block! + # expect_final_exc = trio.EndOfChannel # BOTH but, CHILD breaks FIRST elif ( @@ -134,12 +151,8 @@ def test_ipc_channel_break_during_stream( > ipc_break['break_child_ipc_after'] ) ): - expect_final_exc = trio.ClosedResourceError - - # child will send a 'stop' msg before it breaks - # the transport channel. if pre_aclose_msgstream: - expect_final_exc = trio.EndOfChannel + expect_final_exc = KeyboardInterrupt # NOTE when the parent IPC side dies (even if the child's does as well # but the child fails BEFORE the parent) we always expect the @@ -160,7 +173,8 @@ def test_ipc_channel_break_during_stream( ipc_break['break_parent_ipc_after'] is not False and ( ipc_break['break_child_ipc_after'] - > ipc_break['break_parent_ipc_after'] + > + ipc_break['break_parent_ipc_after'] ) ): expect_final_exc = trio.ClosedResourceError @@ -224,25 +238,29 @@ def test_stream_closed_right_after_ipc_break_and_zombie_lord_engages(): ''' async def main(): - async with tractor.open_nursery() as n: - portal = await n.start_actor( - 'ipc_breaker', - enable_modules=[__name__], - ) + with trio.fail_after(3): + async with tractor.open_nursery() as n: + portal = await n.start_actor( + 'ipc_breaker', + enable_modules=[__name__], + ) - with trio.move_on_after(1): - async with ( - portal.open_context( - break_ipc_after_started - ) as (ctx, sent), - ): - async with ctx.open_stream(): - await trio.sleep(0.5) + with trio.move_on_after(1): + async with ( + portal.open_context( + break_ipc_after_started + ) as (ctx, sent), + ): + async with ctx.open_stream(): + await trio.sleep(0.5) - print('parent waiting on context') + print('parent waiting on context') - print('parent exited context') - raise KeyboardInterrupt + print( + 'parent exited context\n' + 'parent raising KBI..\n' + ) + raise KeyboardInterrupt with pytest.raises(KeyboardInterrupt): trio.run(main) From 9a8cd1389404d876ed2335fb7e57fa964686ee07 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 20 Mar 2024 10:42:17 -0400 Subject: [PATCH 139/146] Another cancel-req-invalid log msg fmt tweak --- tractor/_runtime.py | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index d28ea263..d1f8fc48 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -656,10 +656,12 @@ class Actor: f'|_{chan}\n' ) try: - # send a msg loop terminate sentinel + # send msg loop terminate sentinel which + # triggers cancellation of all remotely + # started tasks. await chan.send(None) - # XXX: do we want this? + # XXX: do we want this? no right? # causes "[104] connection reset by peer" on other end # await chan.aclose() @@ -1137,10 +1139,10 @@ class Actor: # - callee self raises ctxc before caller send request, # - callee errors prior to cancel req. log.cancel( - 'Cancel request invalid, RPC task already completed?\n' + 'Cancel request invalid, RPC task already completed?\n\n' f'<= canceller: {requesting_uid}\n\n' - f'=>{parent_chan}\n' - f' |_ctx-id: {cid}\n' + f'=> {cid}@{parent_chan.uid}\n' + f' |_{parent_chan}\n' ) return True @@ -1421,7 +1423,6 @@ async def async_main( if accept_addr_rent is not None: accept_addr = accept_addr_rent - # The "root" nursery ensures the channel with the immediate # parent is kept alive as a resilient service until # cancellation steps have (mostly) occurred in From 11bab13a0607e534f02881ceea19103901cf2627 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 18 Mar 2025 20:55:21 -0400 Subject: [PATCH 140/146] Various adjustments to fix breakage after rebase - Remove `exceptiongroup` import, - pin to py 3.11 in `setup.py` - revert any lingering `tractor.devx` imports; sub-pkg is coming in a downstream PR! - remove weird double `@property` lingering from conflict reso.. - modern `pytest` requires conftest mod mods to be relative imported. --- setup.py | 2 +- tests/test_debugger.py | 2 +- tests/test_multi_program.py | 2 +- tractor/__init__.py | 2 -- tractor/_context.py | 14 +++++++------- tractor/_rpc.py | 9 ++------- tractor/_runtime.py | 1 - 7 files changed, 12 insertions(+), 20 deletions(-) diff --git a/setup.py b/setup.py index 612f4d57..ac3e94b6 100755 --- a/setup.py +++ b/setup.py @@ -75,7 +75,7 @@ setup( ], tests_require=['pytest'], - python_requires=">=3.10", + python_requires=">=3.11", keywords=[ 'trio', 'async', diff --git a/tests/test_debugger.py b/tests/test_debugger.py index f665d850..25f6dad2 100644 --- a/tests/test_debugger.py +++ b/tests/test_debugger.py @@ -26,7 +26,7 @@ from pexpect.exceptions import ( from tractor._testing import ( examples_dir, ) -from conftest import ( +from .conftest import ( _ci_env, ) diff --git a/tests/test_multi_program.py b/tests/test_multi_program.py index 0b6b5baf..92f4c52d 100644 --- a/tests/test_multi_program.py +++ b/tests/test_multi_program.py @@ -10,7 +10,7 @@ import tractor from tractor._testing import ( tractor_test, ) -from conftest import ( +from .conftest import ( sig_prog, _INT_SIGNAL, _INT_RETURN_CODE, diff --git a/tractor/__init__.py b/tractor/__init__.py index 64c81e99..7af40c6e 100644 --- a/tractor/__init__.py +++ b/tractor/__init__.py @@ -18,8 +18,6 @@ tractor: structured concurrent ``trio``-"actors". """ -from exceptiongroup import BaseExceptionGroup - from ._clustering import open_actor_cluster from ._ipc import Channel from ._context import ( diff --git a/tractor/_context.py b/tractor/_context.py index 11975bae..50f7bfa5 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -313,7 +313,7 @@ async def _drain_to_final_msg( log.critical('SHOULD NEVER GET HERE!?') assert msg is ctx._cancel_msg assert error.msgdata == ctx._remote_error.msgdata - from .devx._debug import pause + from ._debug import pause await pause() ctx._maybe_cancel_and_set_remote_error(error) ctx._maybe_raise_remote_err(error) @@ -2199,12 +2199,12 @@ async def open_context_from_portal( # pass # TODO: factor ^ into below for non-root cases? # - from .devx import maybe_wait_for_debugger + from ._debug import maybe_wait_for_debugger was_acquired: bool = await maybe_wait_for_debugger( - header_msg=( - 'Delaying `ctx.cancel()` until debug lock ' - 'acquired..\n' - ), + # header_msg=( + # 'Delaying `ctx.cancel()` until debug lock ' + # 'acquired..\n' + # ), ) if was_acquired: log.pdb( @@ -2310,7 +2310,7 @@ async def open_context_from_portal( # where the root is waiting on the lock to clear but the # child has already cleared it and clobbered IPC. if debug_mode(): - from .devx import maybe_wait_for_debugger + from ._debug import maybe_wait_for_debugger await maybe_wait_for_debugger() # though it should be impossible for any tasks diff --git a/tractor/_rpc.py b/tractor/_rpc.py index 6bdc0c6b..47548106 100644 --- a/tractor/_rpc.py +++ b/tractor/_rpc.py @@ -55,15 +55,10 @@ from ._exceptions import ( unpack_error, TransportClosed, ) -from .devx import ( - # pause, - maybe_wait_for_debugger, - _debug, -) +from . import _debug from . import _state from .log import get_logger - if TYPE_CHECKING: from ._runtime import Actor @@ -651,7 +646,7 @@ async def _invoke( # don't pop the local context until we know the # associated child isn't in debug any more - await maybe_wait_for_debugger() + await _debug.maybe_wait_for_debugger() ctx: Context = actor._contexts.pop(( chan.uid, cid, diff --git a/tractor/_runtime.py b/tractor/_runtime.py index d1f8fc48..cc8eaf5f 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -1318,7 +1318,6 @@ class Actor: log.runtime("Shutting down channel server") self._server_n.cancel_scope.cancel() - @property @property def accept_addr(self) -> tuple[str, int]: ''' From 6b282bfa0691341feda9389f57ae557629387f1a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 18 Mar 2025 21:00:14 -0400 Subject: [PATCH 141/146] Add `._testing` as subpkg.. --- setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/setup.py b/setup.py index ac3e94b6..1dbdcc54 100755 --- a/setup.py +++ b/setup.py @@ -39,6 +39,7 @@ setup( 'tractor.experimental', # wacky ideas 'tractor.trionics', # trio extensions 'tractor.msg', # lowlevel data types + 'tractor._testing', # internal cross-subsys suite utils ], install_requires=[ From f2d3f0cc219e64c616d0bdd1ef5f5491c002d53b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 19 Mar 2025 09:58:04 -0400 Subject: [PATCH 142/146] Backport skipping `examples/multihost/` in tests This was actually fixed on a downstream dev branch (adding py3.13 support i think?); so backport it here to get us running again on 3.11. --- tests/test_docs_examples.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/test_docs_examples.py b/tests/test_docs_examples.py index 5099aeba..7a923343 100644 --- a/tests/test_docs_examples.py +++ b/tests/test_docs_examples.py @@ -88,6 +88,7 @@ def run_example_in_subproc( and 'debugging' not in p[0] and 'integration' not in p[0] and 'advanced_faults' not in p[0] + and 'multihost' not in p[0] ], ids=lambda t: t[1], From 2a5ff820617a5010dfe164a42c67d4021121577a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 18 Mar 2025 20:58:37 -0400 Subject: [PATCH 143/146] Only run CI on py3.11 --- .github/workflows/ci.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 9d91f17d..571bd1db 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -20,7 +20,7 @@ jobs: - name: Setup python uses: actions/setup-python@v2 with: - python-version: '3.10' + python-version: '3.11' - name: Install dependencies run: pip install -U . --upgrade-strategy eager -r requirements-test.txt @@ -41,7 +41,7 @@ jobs: - name: Setup python uses: actions/setup-python@v2 with: - python-version: '3.10' + python-version: '3.11' - name: Build sdist run: python setup.py sdist --formats=zip @@ -59,7 +59,7 @@ jobs: fail-fast: false matrix: os: [ubuntu-latest] - python: ['3.10'] + python: ['3.11'] spawn_backend: [ 'trio', 'mp_spawn', From 54576851e958e4ba9d010ba5dc2257a711bd7754 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 20 Feb 2024 08:53:37 -0500 Subject: [PATCH 144/146] Add a `debug_mode: bool` fixture via `--tpdb` flag Allows tests (including any `@tractor_test`s) to subscribe to a CLI flag `--tpdb` (for "tractor python debugger") which the session can provide to tests which can then proxy the value to `open_root_actor()` (via `open_nursery()`) when booting the runtime - thus enabling our debug mode globally to any subscribers B) This is real handy if you have some failures but can't determine the root issue without jumping into a `pdbp` REPL inside a (sub-)actor's spawned-task. --- tests/conftest.py | 25 +++++++++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index 4bedc4f1..01811b56 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -41,22 +41,43 @@ no_windows = pytest.mark.skipif( def pytest_addoption(parser): parser.addoption( - "--ll", action="store", dest='loglevel', + "--ll", + action="store", + dest='loglevel', default='ERROR', help="logging level to set when testing" ) parser.addoption( - "--spawn-backend", action="store", dest='spawn_backend', + "--spawn-backend", + action="store", + dest='spawn_backend', default='trio', help="Processing spawning backend to use for test run", ) + parser.addoption( + "--tpdb", "--debug-mode", + action="store_true", + dest='tractor_debug_mode', + # default=False, + help=( + 'Enable a flag that can be used by tests to to set the ' + '`debug_mode: bool` for engaging the internal ' + 'multi-proc debugger sys.' + ), + ) + def pytest_configure(config): backend = config.option.spawn_backend tractor._spawn.try_set_start_method(backend) +@pytest.fixture(scope='session') +def debug_mode(request): + return request.config.option.tractor_debug_mode + + @pytest.fixture(scope='session', autouse=True) def loglevel(request): orig = tractor.log._default_loglevel From bc13599e1f38cefb5f936bc22311aa62925747f6 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 18 Mar 2025 21:44:50 -0400 Subject: [PATCH 145/146] Revert "Port all tests to new `reg_addr` fixture name" This reverts commit 715348c5c2d2d0ec793fb2dc47479c38b8a54c49. --- tests/test_cancellation.py | 31 ++++----- tests/test_child_manages_service_nursery.py | 2 +- tests/test_debugger.py | 2 +- tests/test_discovery.py | 76 ++++++++------------- tests/test_infected_asyncio.py | 26 +++---- tests/test_legacy_one_way_streaming.py | 22 +++--- tests/test_local.py | 12 ++-- tests/test_multi_program.py | 10 +-- tests/test_pubsub.py | 8 +-- tests/test_rpc.py | 4 +- tests/test_spawning.py | 14 ++-- tests/test_task_broadcasting.py | 18 ++--- 12 files changed, 99 insertions(+), 126 deletions(-) diff --git a/tests/test_cancellation.py b/tests/test_cancellation.py index b8c14af3..14e4d0ae 100644 --- a/tests/test_cancellation.py +++ b/tests/test_cancellation.py @@ -14,7 +14,7 @@ import tractor from tractor._testing import ( tractor_test, ) -from conftest import no_windows +from .conftest import no_windows def is_win(): @@ -45,7 +45,7 @@ async def do_nuthin(): ], ids=['no_args', 'unexpected_args'], ) -def test_remote_error(reg_addr, args_err): +def test_remote_error(arb_addr, args_err): ''' Verify an error raised in a subactor that is propagated to the parent nursery, contains the underlying boxed builtin @@ -57,7 +57,7 @@ def test_remote_error(reg_addr, args_err): async def main(): async with tractor.open_nursery( - registry_addrs=[reg_addr], + arbiter_addr=arb_addr, ) as nursery: # on a remote type error caused by bad input args @@ -99,7 +99,7 @@ def test_remote_error(reg_addr, args_err): assert exc.type == errtype -def test_multierror(reg_addr): +def test_multierror(arb_addr): ''' Verify we raise a ``BaseExceptionGroup`` out of a nursery where more then one actor errors. @@ -107,7 +107,7 @@ def test_multierror(reg_addr): ''' async def main(): async with tractor.open_nursery( - registry_addrs=[reg_addr], + arbiter_addr=arb_addr, ) as nursery: await nursery.run_in_actor(assert_err, name='errorer1') @@ -132,14 +132,14 @@ def test_multierror(reg_addr): @pytest.mark.parametrize( 'num_subactors', range(25, 26), ) -def test_multierror_fast_nursery(reg_addr, start_method, num_subactors, delay): +def test_multierror_fast_nursery(arb_addr, start_method, num_subactors, delay): """Verify we raise a ``BaseExceptionGroup`` out of a nursery where more then one actor errors and also with a delay before failure to test failure during an ongoing spawning. """ async def main(): async with tractor.open_nursery( - registry_addrs=[reg_addr], + arbiter_addr=arb_addr, ) as nursery: for i in range(num_subactors): @@ -177,20 +177,15 @@ async def do_nothing(): @pytest.mark.parametrize('mechanism', ['nursery_cancel', KeyboardInterrupt]) -def test_cancel_single_subactor(reg_addr, mechanism): - ''' - Ensure a ``ActorNursery.start_actor()`` spawned subactor +def test_cancel_single_subactor(arb_addr, mechanism): + """Ensure a ``ActorNursery.start_actor()`` spawned subactor cancels when the nursery is cancelled. - - ''' + """ async def spawn_actor(): - ''' - Spawn an actor that blocks indefinitely then cancel via - either `ActorNursery.cancel()` or an exception raise. - - ''' + """Spawn an actor that blocks indefinitely. + """ async with tractor.open_nursery( - registry_addrs=[reg_addr], + arbiter_addr=arb_addr, ) as nursery: portal = await nursery.start_actor( diff --git a/tests/test_child_manages_service_nursery.py b/tests/test_child_manages_service_nursery.py index 350f939b..1dcbe031 100644 --- a/tests/test_child_manages_service_nursery.py +++ b/tests/test_child_manages_service_nursery.py @@ -142,7 +142,7 @@ async def open_actor_local_nursery( ) def test_actor_managed_trio_nursery_task_error_cancels_aio( asyncio_mode: bool, - reg_addr: tuple, + arb_addr ): ''' Verify that a ``trio`` nursery created managed in a child actor diff --git a/tests/test_debugger.py b/tests/test_debugger.py index 25f6dad2..889e7c74 100644 --- a/tests/test_debugger.py +++ b/tests/test_debugger.py @@ -78,7 +78,7 @@ has_nested_actors = pytest.mark.has_nested_actors def spawn( start_method, testdir, - reg_addr, + arb_addr, ) -> 'pexpect.spawn': if start_method != 'trio': diff --git a/tests/test_discovery.py b/tests/test_discovery.py index cd9dc022..b56c3a2e 100644 --- a/tests/test_discovery.py +++ b/tests/test_discovery.py @@ -14,19 +14,19 @@ import trio @tractor_test -async def test_reg_then_unreg(reg_addr): +async def test_reg_then_unreg(arb_addr): actor = tractor.current_actor() assert actor.is_arbiter assert len(actor._registry) == 1 # only self is registered async with tractor.open_nursery( - registry_addrs=[reg_addr], + arbiter_addr=arb_addr, ) as n: 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_arbiter(*arb_addr) as aportal: # this local actor should be the arbiter assert actor is aportal.actor @@ -52,27 +52,15 @@ async def hi(): return the_line.format(tractor.current_actor().name) -async def say_hello( - other_actor: str, - reg_addr: tuple[str, int], -): +async def say_hello(other_actor): await trio.sleep(1) # wait for other actor to spawn - async with tractor.find_actor( - other_actor, - registry_addrs=[reg_addr], - ) as portal: + async with tractor.find_actor(other_actor) as portal: assert portal is not None return await portal.run(__name__, 'hi') -async def say_hello_use_wait( - other_actor: str, - reg_addr: tuple[str, int], -): - async with tractor.wait_for_actor( - other_actor, - registry_addr=reg_addr, - ) as portal: +async def say_hello_use_wait(other_actor): + async with tractor.wait_for_actor(other_actor) as portal: assert portal is not None result = await portal.run(__name__, 'hi') return result @@ -80,29 +68,21 @@ async def say_hello_use_wait( @tractor_test @pytest.mark.parametrize('func', [say_hello, say_hello_use_wait]) -async def test_trynamic_trio( - func, - start_method, - reg_addr, -): - ''' - Root actor acting as the "director" and running one-shot-task-actors - for the directed subs. - - ''' +async def test_trynamic_trio(func, start_method, arb_addr): + """Main tractor entry point, the "master" process (for now + acts as the "director"). + """ async with tractor.open_nursery() as n: print("Alright... Action!") donny = await n.run_in_actor( func, other_actor='gretchen', - reg_addr=reg_addr, name='donny', ) gretchen = await n.run_in_actor( func, other_actor='donny', - reg_addr=reg_addr, name='gretchen', ) print(await gretchen.result()) @@ -150,7 +130,7 @@ async def unpack_reg(actor_or_portal): async def spawn_and_check_registry( - reg_addr: tuple, + arb_addr: tuple, use_signal: bool, remote_arbiter: bool = False, with_streaming: bool = False, @@ -158,9 +138,9 @@ async def spawn_and_check_registry( ) -> None: async with tractor.open_root_actor( - registry_addrs=[reg_addr], + arbiter_addr=arb_addr, ): - async with tractor.get_arbiter(*reg_addr) as portal: + async with tractor.get_arbiter(*arb_addr) as portal: # runtime needs to be up to call this actor = tractor.current_actor() @@ -232,19 +212,17 @@ async def spawn_and_check_registry( def test_subactors_unregister_on_cancel( start_method, use_signal, - reg_addr, + arb_addr, with_streaming, ): - ''' - Verify that cancelling a nursery results in all subactors + """Verify that cancelling a nursery results in all subactors deregistering themselves with the arbiter. - - ''' + """ with pytest.raises(KeyboardInterrupt): trio.run( partial( spawn_and_check_registry, - reg_addr, + arb_addr, use_signal, remote_arbiter=False, with_streaming=with_streaming, @@ -258,7 +236,7 @@ def test_subactors_unregister_on_cancel_remote_daemon( daemon, start_method, use_signal, - reg_addr, + arb_addr, with_streaming, ): """Verify that cancelling a nursery results in all subactors @@ -269,7 +247,7 @@ def test_subactors_unregister_on_cancel_remote_daemon( trio.run( partial( spawn_and_check_registry, - reg_addr, + arb_addr, use_signal, remote_arbiter=True, with_streaming=with_streaming, @@ -283,7 +261,7 @@ async def streamer(agen): async def close_chans_before_nursery( - reg_addr: tuple, + arb_addr: tuple, use_signal: bool, remote_arbiter: bool = False, ) -> None: @@ -296,9 +274,9 @@ async def close_chans_before_nursery( entries_at_end = 1 async with tractor.open_root_actor( - registry_addrs=[reg_addr], + arbiter_addr=arb_addr, ): - async with tractor.get_arbiter(*reg_addr) as aportal: + async with tractor.get_arbiter(*arb_addr) as aportal: try: get_reg = partial(unpack_reg, aportal) @@ -350,7 +328,7 @@ async def close_chans_before_nursery( def test_close_channel_explicit( start_method, use_signal, - reg_addr, + arb_addr, ): """Verify that closing a stream explicitly and killing the actor's "root nursery" **before** the containing nursery tears down also @@ -360,7 +338,7 @@ def test_close_channel_explicit( trio.run( partial( close_chans_before_nursery, - reg_addr, + arb_addr, use_signal, remote_arbiter=False, ), @@ -372,7 +350,7 @@ def test_close_channel_explicit_remote_arbiter( daemon, start_method, use_signal, - reg_addr, + arb_addr, ): """Verify that closing a stream explicitly and killing the actor's "root nursery" **before** the containing nursery tears down also @@ -382,7 +360,7 @@ def test_close_channel_explicit_remote_arbiter( trio.run( partial( close_chans_before_nursery, - reg_addr, + arb_addr, use_signal, remote_arbiter=True, ), diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 568708a2..f9670225 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -47,7 +47,7 @@ async def trio_cancels_single_aio_task(): await tractor.to_asyncio.run_task(sleep_forever) -def test_trio_cancels_aio_on_actor_side(reg_addr): +def test_trio_cancels_aio_on_actor_side(arb_addr): ''' Spawn an infected actor that is cancelled by the ``trio`` side task using std cancel scope apis. @@ -55,7 +55,7 @@ def test_trio_cancels_aio_on_actor_side(reg_addr): ''' async def main(): async with tractor.open_nursery( - registry_addrs=[reg_addr] + arbiter_addr=arb_addr ) as n: await n.run_in_actor( trio_cancels_single_aio_task, @@ -94,7 +94,7 @@ async def asyncio_actor( raise -def test_aio_simple_error(reg_addr): +def test_aio_simple_error(arb_addr): ''' Verify a simple remote asyncio error propagates back through trio to the parent actor. @@ -103,7 +103,7 @@ def test_aio_simple_error(reg_addr): ''' async def main(): async with tractor.open_nursery( - registry_addrs=[reg_addr] + arbiter_addr=arb_addr ) as n: await n.run_in_actor( asyncio_actor, @@ -131,7 +131,7 @@ def test_aio_simple_error(reg_addr): assert err.type == AssertionError -def test_tractor_cancels_aio(reg_addr): +def test_tractor_cancels_aio(arb_addr): ''' Verify we can cancel a spawned asyncio task gracefully. @@ -150,7 +150,7 @@ def test_tractor_cancels_aio(reg_addr): trio.run(main) -def test_trio_cancels_aio(reg_addr): +def test_trio_cancels_aio(arb_addr): ''' Much like the above test with ``tractor.Portal.cancel_actor()`` except we just use a standard ``trio`` cancellation api. @@ -206,7 +206,7 @@ async def trio_ctx( ids='parent_actor_cancels_child={}'.format ) def test_context_spawns_aio_task_that_errors( - reg_addr, + arb_addr, parent_cancels: bool, ): ''' @@ -288,7 +288,7 @@ async def aio_cancel(): await sleep_forever() -def test_aio_cancelled_from_aio_causes_trio_cancelled(reg_addr): +def test_aio_cancelled_from_aio_causes_trio_cancelled(arb_addr): async def main(): async with tractor.open_nursery() as n: @@ -436,7 +436,7 @@ async def stream_from_aio( 'fan_out', [False, True], ids='fan_out_w_chan_subscribe={}'.format ) -def test_basic_interloop_channel_stream(reg_addr, fan_out): +def test_basic_interloop_channel_stream(arb_addr, fan_out): async def main(): async with tractor.open_nursery() as n: portal = await n.run_in_actor( @@ -450,7 +450,7 @@ def test_basic_interloop_channel_stream(reg_addr, fan_out): # TODO: parametrize the above test and avoid the duplication here? -def test_trio_error_cancels_intertask_chan(reg_addr): +def test_trio_error_cancels_intertask_chan(arb_addr): async def main(): async with tractor.open_nursery() as n: portal = await n.run_in_actor( @@ -469,7 +469,7 @@ def test_trio_error_cancels_intertask_chan(reg_addr): assert exc.type == Exception -def test_trio_closes_early_and_channel_exits(reg_addr): +def test_trio_closes_early_and_channel_exits(arb_addr): async def main(): async with tractor.open_nursery() as n: portal = await n.run_in_actor( @@ -484,7 +484,7 @@ def test_trio_closes_early_and_channel_exits(reg_addr): trio.run(main) -def test_aio_errors_and_channel_propagates_and_closes(reg_addr): +def test_aio_errors_and_channel_propagates_and_closes(arb_addr): async def main(): async with tractor.open_nursery() as n: portal = await n.run_in_actor( @@ -561,7 +561,7 @@ async def trio_to_aio_echo_server( ids='raise_error={}'.format, ) def test_echoserver_detailed_mechanics( - reg_addr, + arb_addr, raise_error_mid_stream, ): diff --git a/tests/test_legacy_one_way_streaming.py b/tests/test_legacy_one_way_streaming.py index 1e7ec987..5d7787fa 100644 --- a/tests/test_legacy_one_way_streaming.py +++ b/tests/test_legacy_one_way_streaming.py @@ -55,7 +55,7 @@ async def context_stream( async def stream_from_single_subactor( - reg_addr, + arb_addr, start_method, stream_func, ): @@ -64,7 +64,7 @@ async def stream_from_single_subactor( # only one per host address, spawns an actor if None async with tractor.open_nursery( - registry_addrs=[reg_addr], + arbiter_addr=arb_addr, start_method=start_method, ) as nursery: @@ -115,13 +115,13 @@ async def stream_from_single_subactor( @pytest.mark.parametrize( 'stream_func', [async_gen_stream, context_stream] ) -def test_stream_from_single_subactor(reg_addr, start_method, stream_func): +def test_stream_from_single_subactor(arb_addr, start_method, stream_func): """Verify streaming from a spawned async generator. """ trio.run( partial( stream_from_single_subactor, - reg_addr, + arb_addr, start_method, stream_func=stream_func, ), @@ -225,14 +225,14 @@ async def a_quadruple_example(): return result_stream -async def cancel_after(wait, reg_addr): - async with tractor.open_root_actor(registry_addrs=[reg_addr]): +async def cancel_after(wait, arb_addr): + async with tractor.open_root_actor(arbiter_addr=arb_addr): with trio.move_on_after(wait): return await a_quadruple_example() @pytest.fixture(scope='module') -def time_quad_ex(reg_addr, ci_env, spawn_backend): +def time_quad_ex(arb_addr, ci_env, spawn_backend): if spawn_backend == 'mp': """no idea but the mp *nix runs are flaking out here often... """ @@ -240,7 +240,7 @@ def time_quad_ex(reg_addr, ci_env, spawn_backend): timeout = 7 if platform.system() in ('Windows', 'Darwin') else 4 start = time.time() - results = trio.run(cancel_after, timeout, reg_addr) + results = trio.run(cancel_after, timeout, arb_addr) diff = time.time() - start assert results return results, diff @@ -260,14 +260,14 @@ def test_a_quadruple_example(time_quad_ex, ci_env, spawn_backend): list(map(lambda i: i/10, range(3, 9))) ) def test_not_fast_enough_quad( - reg_addr, time_quad_ex, cancel_delay, ci_env, spawn_backend + arb_addr, time_quad_ex, cancel_delay, ci_env, spawn_backend ): """Verify we can cancel midway through the quad example and all actors cancel gracefully. """ results, diff = time_quad_ex delay = max(diff - cancel_delay, 0) - results = trio.run(cancel_after, delay, reg_addr) + results = trio.run(cancel_after, delay, arb_addr) system = platform.system() if system in ('Windows', 'Darwin') and results is not None: # In CI envoirments it seems later runs are quicker then the first @@ -280,7 +280,7 @@ def test_not_fast_enough_quad( @tractor_test async def test_respawn_consumer_task( - reg_addr, + arb_addr, spawn_backend, loglevel, ): diff --git a/tests/test_local.py b/tests/test_local.py index a019d771..bb013043 100644 --- a/tests/test_local.py +++ b/tests/test_local.py @@ -24,7 +24,7 @@ async def test_no_runtime(): @tractor_test -async def test_self_is_registered(reg_addr): +async def test_self_is_registered(arb_addr): "Verify waiting on the arbiter to register itself using the standard api." actor = tractor.current_actor() assert actor.is_arbiter @@ -34,20 +34,20 @@ async def test_self_is_registered(reg_addr): @tractor_test -async def test_self_is_registered_localportal(reg_addr): +async def test_self_is_registered_localportal(arb_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_arbiter(*arb_addr) as portal: assert isinstance(portal, tractor._portal.LocalPortal) with trio.fail_after(0.2): sockaddr = await portal.run_from_ns( 'self', 'wait_for_actor', name='root') - assert sockaddr[0] == reg_addr + assert sockaddr[0] == arb_addr -def test_local_actor_async_func(reg_addr): +def test_local_actor_async_func(arb_addr): """Verify a simple async function in-process. """ nums = [] @@ -55,7 +55,7 @@ def test_local_actor_async_func(reg_addr): async def print_loop(): async with tractor.open_root_actor( - registry_addrs=[reg_addr], + arbiter_addr=arb_addr, ): # arbiter is started in-proc if dne assert tractor.current_actor().is_arbiter diff --git a/tests/test_multi_program.py b/tests/test_multi_program.py index 92f4c52d..d3eadabf 100644 --- a/tests/test_multi_program.py +++ b/tests/test_multi_program.py @@ -30,9 +30,9 @@ def test_abort_on_sigint(daemon): @tractor_test -async def test_cancel_remote_arbiter(daemon, reg_addr): +async def test_cancel_remote_arbiter(daemon, arb_addr): assert not tractor.current_actor().is_arbiter - async with tractor.get_arbiter(*reg_addr) as portal: + async with tractor.get_arbiter(*arb_addr) as portal: await portal.cancel_actor() time.sleep(0.1) @@ -41,16 +41,16 @@ 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_arbiter(*arb_addr) as portal: pass -def test_register_duplicate_name(daemon, reg_addr): +def test_register_duplicate_name(daemon, arb_addr): async def main(): async with tractor.open_nursery( - registry_addrs=[reg_addr], + arbiter_addr=arb_addr, ) as n: assert not tractor.current_actor().is_arbiter diff --git a/tests/test_pubsub.py b/tests/test_pubsub.py index 6d416f89..69f4c513 100644 --- a/tests/test_pubsub.py +++ b/tests/test_pubsub.py @@ -159,7 +159,7 @@ async def test_required_args(callwith_expecterror): ) def test_multi_actor_subs_arbiter_pub( loglevel, - reg_addr, + arb_addr, pub_actor, ): """Try out the neato @pub decorator system. @@ -169,7 +169,7 @@ def test_multi_actor_subs_arbiter_pub( async def main(): async with tractor.open_nursery( - registry_addrs=[reg_addr], + arbiter_addr=arb_addr, enable_modules=[__name__], ) as n: @@ -254,12 +254,12 @@ def test_multi_actor_subs_arbiter_pub( def test_single_subactor_pub_multitask_subs( loglevel, - reg_addr, + arb_addr, ): async def main(): async with tractor.open_nursery( - registry_addrs=[reg_addr], + arbiter_addr=arb_addr, enable_modules=[__name__], ) as n: diff --git a/tests/test_rpc.py b/tests/test_rpc.py index 71f3258b..b16f2f1d 100644 --- a/tests/test_rpc.py +++ b/tests/test_rpc.py @@ -52,7 +52,7 @@ async def short_sleep(): 'fail_on_syntax', ], ) -def test_rpc_errors(reg_addr, to_call, testdir): +def test_rpc_errors(arb_addr, to_call, testdir): """Test errors when making various RPC requests to an actor that either doesn't have the requested module exposed or doesn't define the named function. @@ -84,7 +84,7 @@ def test_rpc_errors(reg_addr, to_call, testdir): # spawn a subactor which calls us back async with tractor.open_nursery( - arbiter_addr=reg_addr, + arbiter_addr=arb_addr, enable_modules=exposed_mods.copy(), ) as n: diff --git a/tests/test_spawning.py b/tests/test_spawning.py index 6a4b2988..3f4772e9 100644 --- a/tests/test_spawning.py +++ b/tests/test_spawning.py @@ -16,14 +16,14 @@ data_to_pass_down = {'doggy': 10, 'kitty': 4} async def spawn( is_arbiter: bool, data: dict, - reg_addr: tuple[str, int], + arb_addr: tuple[str, int], ): namespaces = [__name__] await trio.sleep(0.1) async with tractor.open_root_actor( - arbiter_addr=reg_addr, + arbiter_addr=arb_addr, ): actor = tractor.current_actor() @@ -41,7 +41,7 @@ async def spawn( is_arbiter=False, name='sub-actor', data=data, - reg_addr=reg_addr, + arb_addr=arb_addr, enable_modules=namespaces, ) @@ -55,12 +55,12 @@ async def spawn( return 10 -def test_local_arbiter_subactor_global_state(reg_addr): +def test_local_arbiter_subactor_global_state(arb_addr): result = trio.run( spawn, True, data_to_pass_down, - reg_addr, + arb_addr, ) assert result == 10 @@ -140,7 +140,7 @@ async def check_loglevel(level): def test_loglevel_propagated_to_subactor( start_method, capfd, - reg_addr, + arb_addr, ): if start_method == 'mp_forkserver': pytest.skip( @@ -152,7 +152,7 @@ def test_loglevel_propagated_to_subactor( async with tractor.open_nursery( name='arbiter', start_method=start_method, - arbiter_addr=reg_addr, + arbiter_addr=arb_addr, ) as tn: await tn.run_in_actor( diff --git a/tests/test_task_broadcasting.py b/tests/test_task_broadcasting.py index d7a29134..5e18e10a 100644 --- a/tests/test_task_broadcasting.py +++ b/tests/test_task_broadcasting.py @@ -66,13 +66,13 @@ async def ensure_sequence( async def open_sequence_streamer( sequence: list[int], - reg_addr: tuple[str, int], + arb_addr: tuple[str, int], start_method: str, ) -> tractor.MsgStream: async with tractor.open_nursery( - arbiter_addr=reg_addr, + arbiter_addr=arb_addr, start_method=start_method, ) as tn: @@ -93,7 +93,7 @@ async def open_sequence_streamer( def test_stream_fan_out_to_local_subscriptions( - reg_addr, + arb_addr, start_method, ): @@ -103,7 +103,7 @@ def test_stream_fan_out_to_local_subscriptions( async with open_sequence_streamer( sequence, - reg_addr, + arb_addr, start_method, ) as stream: @@ -138,7 +138,7 @@ def test_stream_fan_out_to_local_subscriptions( ] ) def test_consumer_and_parent_maybe_lag( - reg_addr, + arb_addr, start_method, task_delays, ): @@ -150,7 +150,7 @@ def test_consumer_and_parent_maybe_lag( async with open_sequence_streamer( sequence, - reg_addr, + arb_addr, start_method, ) as stream: @@ -211,7 +211,7 @@ def test_consumer_and_parent_maybe_lag( def test_faster_task_to_recv_is_cancelled_by_slower( - reg_addr, + arb_addr, start_method, ): ''' @@ -225,7 +225,7 @@ def test_faster_task_to_recv_is_cancelled_by_slower( async with open_sequence_streamer( sequence, - reg_addr, + arb_addr, start_method, ) as stream: @@ -302,7 +302,7 @@ def test_subscribe_errors_after_close(): def test_ensure_slow_consumers_lag_out( - reg_addr, + arb_addr, start_method, ): '''This is a pure local task test; no tractor From 96cdcd8f3902d2abb6fcd736afa4d584bb92df26 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 20 Mar 2025 14:24:21 -0400 Subject: [PATCH 146/146] Pin to exact `trio` version that still has loose egs --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 1dbdcc54..a4e5e1ed 100755 --- a/setup.py +++ b/setup.py @@ -46,7 +46,7 @@ setup( # trio related # proper range spec: # https://packaging.python.org/en/latest/discussions/install-requires-vs-requirements/#id5 - 'trio >= 0.24', + 'trio == 0.24', # 'async_generator', # in stdlib mostly! # 'trio_typing', # trio==0.23.0 has type hints!