diff --git a/docs/README.rst b/docs/README.rst index 9dfe2f60..9dd7faf4 100644 --- a/docs/README.rst +++ b/docs/README.rst @@ -3,8 +3,8 @@ |gh_actions| |docs| -``tractor`` is a `structured concurrent`_, multi-processing_ runtime -built on trio_. +``tractor`` is a `structured concurrent`_, (optionally +distributed_) multi-processing_ runtime built on trio_. Fundamentally, ``tractor`` gives you parallelism via ``trio``-"*actors*": independent Python processes (aka @@ -17,11 +17,20 @@ protocol" constructed on top of multiple Pythons each running a ``trio`` scheduled runtime - a call to ``trio.run()``. We believe the system adheres to the `3 axioms`_ of an "`actor model`_" -but likely *does not* look like what *you* probably think an "actor -model" looks like, and that's *intentional*. +but likely **does not** look like what **you** probably *think* an "actor +model" looks like, and that's **intentional**. -The first step to grok ``tractor`` is to get the basics of ``trio`` down. -A great place to start is the `trio docs`_ and this `blog post`_. + +Where do i start!? +------------------ +The first step to grok ``tractor`` is to get an intermediate +knowledge of ``trio`` and **structured concurrency** B) + +Some great places to start are, +- the seminal `blog post`_ +- obviously the `trio docs`_ +- wikipedia's nascent SC_ page +- the fancy diagrams @ libdill-docs_ Features @@ -593,6 +602,7 @@ matrix seems too hip, we're also mostly all in the the `trio gitter channel`_! .. _structured concurrent: https://trio.discourse.group/t/concise-definition-of-structured-concurrency/228 +.. _distributed: https://en.wikipedia.org/wiki/Distributed_computing .. _multi-processing: https://en.wikipedia.org/wiki/Multiprocessing .. _trio: https://github.com/python-trio/trio .. _nurseries: https://vorpus.org/blog/notes-on-structured-concurrency-or-go-statement-considered-harmful/#nurseries-a-structured-replacement-for-go-statements @@ -611,8 +621,9 @@ channel`_! .. _trio docs: https://trio.readthedocs.io/en/latest/ .. _blog post: https://vorpus.org/blog/notes-on-structured-concurrency-or-go-statement-considered-harmful/ .. _structured concurrency: https://en.wikipedia.org/wiki/Structured_concurrency +.. _SC: https://en.wikipedia.org/wiki/Structured_concurrency +.. _libdill-docs: https://sustrik.github.io/libdill/structured-concurrency.html .. _structured chadcurrency: https://en.wikipedia.org/wiki/Structured_concurrency -.. _structured concurrency: https://en.wikipedia.org/wiki/Structured_concurrency .. _unrequirements: https://en.wikipedia.org/wiki/Actor_model#Direct_communication_and_asynchrony .. _async generators: https://www.python.org/dev/peps/pep-0525/ .. _trio-parallel: https://github.com/richardsheridan/trio-parallel diff --git a/examples/advanced_faults/ipc_failure_during_stream.py b/examples/advanced_faults/ipc_failure_during_stream.py index 6728b8d2..60b28c3e 100644 --- a/examples/advanced_faults/ipc_failure_during_stream.py +++ b/examples/advanced_faults/ipc_failure_during_stream.py @@ -6,53 +6,59 @@ 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 contextlib import asynccontextmanager as acm +from functools import partial + from tractor import ( open_nursery, context, Context, + ContextCancelled, MsgStream, + _testing, ) +import trio +import pytest -async def break_channel_silently_then_error( +async def break_ipc_then_error( stream: MsgStream, + break_ipc_with: str|None = None, + pre_close: bool = False, ): + await _testing.break_ipc( + stream=stream, + method=break_ipc_with, + pre_close=pre_close, + ) 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) - assert 0 + assert 0 -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 - @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: ''' Receive stream msgs and spawn some IPC killers mid-stream. ''' + broke_ipc: bool = False await ctx.started() async with ( ctx.open_stream() as stream, @@ -60,27 +66,58 @@ async def recv_and_spawn_net_killers( ): async for i in stream: print(f'child echoing {i}') - await stream.send(i) + if not broke_ipc: + await stream.send(i) + else: + await trio.sleep(0.01) + 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) + broke_ipc = True + n.start_soon( + iter_ipc_stream, + stream, + ) + n.start_soon( + partial( + break_ipc_then_error, + stream=stream, + pre_close=pre_close, + ) + ) + + +@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( 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: @@ -91,60 +128,129 @@ 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, + async with ( + stuff_hangin_ctlc(timeout=2) as timeout, + _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, + ), - ) as (ctx, sent): + 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): 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) + + # 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() + 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) + # mp_spawn/forkserver backends and thus the + # zombie reaper never even kicks in? + try: + print(f'parent sending {i}') + 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 + except trio.ClosedResourceError: + # NOTE: don't send if we already broke the + # connection to avoid raising a closed-error + # such that we drop through to the ctl-c + # mashing by user. + await trio.sleep(0.01) + + # 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 # caller/this-parent. - rx = await stream.receive() + try: + rx = await stream.receive() + print( + "I'm a happy PARENT user and echoed to me is\n" + f'{rx}\n' + ) + except trio.EndOfChannel: + rx_eoc: bool = True + print('MsgStream got EoC for PARENT') + raise - print(f"I'm a happy user and echoed to me is {rx}") + 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 - 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( - "YOO i'm mad send side dun but thingz hangin..\n" - 'MASHING CTlR-C Ctl-c..' - ) - raise KeyboardInterrupt + raise KeyboardInterrupt if __name__ == '__main__': diff --git a/examples/debugging/asyncio_bp.py b/examples/debugging/asyncio_bp.py new file mode 100644 index 00000000..baddfe03 --- /dev/null +++ b/examples/debugging/asyncio_bp.py @@ -0,0 +1,119 @@ +import asyncio + +import trio +import tractor +from tractor import to_asyncio + + +async def aio_sleep_forever(): + await asyncio.sleep(float('inf')) + + +async def bp_then_error( + to_trio: trio.MemorySendChannel, + from_trio: asyncio.Queue, + + raise_after_bp: bool = True, + +) -> None: + + # sync with ``trio``-side (caller) task + to_trio.send_nowait('start') + + # NOTE: what happens here inside the hook needs some refinement.. + # => seems like it's still `._debug._set_trace()` but + # we set `Lock.local_task_in_debug = 'sync'`, we probably want + # some further, at least, meta-data about the task/actoq in debug + # in terms of making it clear it's asyncio mucking about. + breakpoint() + + # short checkpoint / delay + await asyncio.sleep(0.5) + + if raise_after_bp: + raise ValueError('blah') + + # TODO: test case with this so that it gets cancelled? + else: + # XXX NOTE: this is required in order to get the SIGINT-ignored + # hang case documented in the module script section! + await aio_sleep_forever() + + +@tractor.context +async def trio_ctx( + ctx: tractor.Context, + bp_before_started: bool = False, +): + + # this will block until the ``asyncio`` task sends a "first" + # message, see first line in above func. + async with ( + + to_asyncio.open_channel_from( + bp_then_error, + raise_after_bp=not bp_before_started, + ) as (first, chan), + + trio.open_nursery() as n, + ): + + assert first == 'start' + + if bp_before_started: + await tractor.breakpoint() + + await ctx.started(first) + + n.start_soon( + to_asyncio.run_task, + aio_sleep_forever, + ) + await trio.sleep_forever() + + +async def main( + bps_all_over: bool = False, + +) -> None: + + async with tractor.open_nursery( + # debug_mode=True, + ) as n: + + p = await n.start_actor( + 'aio_daemon', + enable_modules=[__name__], + infect_asyncio=True, + debug_mode=True, + loglevel='cancel', + ) + + async with p.open_context( + trio_ctx, + bp_before_started=bps_all_over, + ) as (ctx, first): + + assert first == 'start' + + if bps_all_over: + await tractor.breakpoint() + + # await trio.sleep_forever() + await ctx.cancel() + assert 0 + + # TODO: case where we cancel from trio-side while asyncio task + # has debugger lock? + # await p.cancel_actor() + + +if __name__ == '__main__': + + # works fine B) + trio.run(main) + + # will hang and ignores SIGINT !! + # NOTE: you'll need to send a SIGQUIT (via ctl-\) to kill it + # manually.. + # trio.run(main, True) 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/debugging/multi_daemon_subactors.py b/examples/debugging/multi_daemon_subactors.py index 6c2d5750..80ef933c 100644 --- a/examples/debugging/multi_daemon_subactors.py +++ b/examples/debugging/multi_daemon_subactors.py @@ -4,9 +4,15 @@ import trio async def breakpoint_forever(): "Indefinitely re-enter debugger in child actor." - while True: - yield 'yo' - await tractor.breakpoint() + try: + while True: + yield 'yo' + await tractor.breakpoint() + except BaseException: + tractor.log.get_console_log().exception( + 'Cancelled while trying to enter pause point!' + ) + raise async def name_error(): @@ -19,7 +25,7 @@ async def main(): """ async with tractor.open_nursery( debug_mode=True, - loglevel='error', + loglevel='cancel', ) as n: p0 = await n.start_actor('bp_forever', enable_modules=[__name__]) @@ -32,7 +38,7 @@ async def main(): try: await p1.run(name_error) except tractor.RemoteActorError as rae: - assert rae.type is NameError + assert rae.boxed_type is NameError async for i in stream: diff --git a/examples/debugging/multi_nested_subactors_error_up_through_nurseries.py b/examples/debugging/multi_nested_subactors_error_up_through_nurseries.py index 348a5ee9..8df52e3b 100644 --- a/examples/debugging/multi_nested_subactors_error_up_through_nurseries.py +++ b/examples/debugging/multi_nested_subactors_error_up_through_nurseries.py @@ -45,6 +45,7 @@ async def spawn_until(depth=0): ) +# TODO: notes on the new boxed-relayed errors through proxy actors async def main(): """The main ``tractor`` routine. diff --git a/examples/debugging/multi_subactors.py b/examples/debugging/multi_subactors.py index 259d5268..22b13ac8 100644 --- a/examples/debugging/multi_subactors.py +++ b/examples/debugging/multi_subactors.py @@ -38,6 +38,7 @@ async def main(): """ async with tractor.open_nursery( debug_mode=True, + # loglevel='runtime', ) as n: # Spawn both actors, don't bother with collecting results diff --git a/examples/debugging/per_actor_debug.py b/examples/debugging/per_actor_debug.py index 1db56981..c1bf5cab 100644 --- a/examples/debugging/per_actor_debug.py +++ b/examples/debugging/per_actor_debug.py @@ -23,5 +23,6 @@ async def main(): n.start_soon(debug_actor.run, die) n.start_soon(crash_boi.run, die) + if __name__ == '__main__': trio.run(main) diff --git a/examples/debugging/pm_in_subactor.py b/examples/debugging/pm_in_subactor.py new file mode 100644 index 00000000..a8f5048e --- /dev/null +++ b/examples/debugging/pm_in_subactor.py @@ -0,0 +1,56 @@ +import trio +import tractor + + +@tractor.context +async def name_error( + ctx: tractor.Context, +): + ''' + Raise a `NameError`, catch it and enter `.post_mortem()`, then + expect the `._rpc._invoke()` crash handler to also engage. + + ''' + try: + getattr(doggypants) # noqa (on purpose) + except NameError: + await tractor.post_mortem() + raise + + +async def main(): + ''' + Test 3 `PdbREPL` entries: + - one in the child due to manual `.post_mortem()`, + - another in the child due to runtime RPC crash handling. + - final one here in parent from the RAE. + + ''' + # XXX NOTE: ideally the REPL arrives at this frame in the parent + # ONE UP FROM the inner ctx block below! + async with tractor.open_nursery( + debug_mode=True, + # loglevel='cancel', + ) as an: + p: tractor.Portal = await an.start_actor( + 'child', + enable_modules=[__name__], + ) + + # XXX should raise `RemoteActorError[NameError]` + # AND be the active frame when REPL enters! + try: + async with p.open_context(name_error) as (ctx, first): + assert first + except tractor.RemoteActorError as rae: + assert rae.boxed_type is NameError + + # manually handle in root's parent task + await tractor.post_mortem() + raise + else: + raise RuntimeError('IPC ctx should have remote errored!?') + + +if __name__ == '__main__': + trio.run(main) diff --git a/examples/debugging/root_actor_breakpoint_forever.py b/examples/debugging/root_actor_breakpoint_forever.py index 3536a751..88a6e0e9 100644 --- a/examples/debugging/root_actor_breakpoint_forever.py +++ b/examples/debugging/root_actor_breakpoint_forever.py @@ -2,10 +2,13 @@ import trio import tractor -async def main(): +async def main( + registry_addrs: tuple[str, int]|None = None +): async with tractor.open_root_actor( debug_mode=True, + # loglevel='runtime', ): while True: await tractor.breakpoint() diff --git a/examples/debugging/shielded_pause.py b/examples/debugging/shielded_pause.py new file mode 100644 index 00000000..3e34d8fc --- /dev/null +++ b/examples/debugging/shielded_pause.py @@ -0,0 +1,88 @@ +import trio +import tractor + + +async def cancellable_pause_loop( + task_status: trio.TaskStatus[trio.CancelScope] = trio.TASK_STATUS_IGNORED +): + with trio.CancelScope() as cs: + task_status.started(cs) + for _ in range(3): + try: + # ON first entry, there is no level triggered + # cancellation yet, so this cp does a parent task + # ctx-switch so that this scope raises for the NEXT + # checkpoint we hit. + await trio.lowlevel.checkpoint() + await tractor.pause() + + cs.cancel() + + # parent should have called `cs.cancel()` by now + await trio.lowlevel.checkpoint() + + except trio.Cancelled: + print('INSIDE SHIELDED PAUSE') + await tractor.pause(shield=True) + else: + # should raise it again, bubbling up to parent + print('BUBBLING trio.Cancelled to parent task-nursery') + await trio.lowlevel.checkpoint() + + +async def pm_on_cancelled(): + async with trio.open_nursery() as tn: + tn.cancel_scope.cancel() + try: + await trio.sleep_forever() + except trio.Cancelled: + # should also raise `Cancelled` since + # we didn't pass `shield=True`. + try: + await tractor.post_mortem(hide_tb=False) + except trio.Cancelled as taskc: + + # should enter just fine, in fact it should + # be debugging the internals of the previous + # sin-shield call above Bo + await tractor.post_mortem( + hide_tb=False, + shield=True, + ) + raise taskc + + else: + raise RuntimeError('Dint cancel as expected!?') + + +async def cancelled_before_pause( +): + ''' + Verify that using a shielded pause works despite surrounding + cancellation called state in the calling task. + + ''' + async with trio.open_nursery() as tn: + cs: trio.CancelScope = await tn.start(cancellable_pause_loop) + await trio.sleep(0.1) + + assert cs.cancelled_caught + + await pm_on_cancelled() + + +async def main(): + async with tractor.open_nursery( + debug_mode=True, + ) as n: + portal: tractor.Portal = await n.run_in_actor( + cancelled_before_pause, + ) + await portal.result() + + # ensure the same works in the root actor! + await pm_on_cancelled() + + +if __name__ == '__main__': + trio.run(main) diff --git a/examples/debugging/subactor_breakpoint.py b/examples/debugging/subactor_breakpoint.py index bcc304d1..4fdff484 100644 --- a/examples/debugging/subactor_breakpoint.py +++ b/examples/debugging/subactor_breakpoint.py @@ -3,17 +3,20 @@ import tractor async def breakpoint_forever(): - """Indefinitely re-enter debugger in child actor. - """ + ''' + Indefinitely re-enter debugger in child actor. + + ''' while True: await trio.sleep(0.1) - await tractor.breakpoint() + await tractor.pause() async def main(): async with tractor.open_nursery( debug_mode=True, + loglevel='cancel', ) as n: portal = await n.run_in_actor( diff --git a/examples/debugging/subactor_error.py b/examples/debugging/subactor_error.py index e38c1614..d7aee447 100644 --- a/examples/debugging/subactor_error.py +++ b/examples/debugging/subactor_error.py @@ -3,16 +3,26 @@ import tractor async def name_error(): - getattr(doggypants) + getattr(doggypants) # noqa (on purpose) async def main(): async with tractor.open_nursery( debug_mode=True, - ) as n: + # loglevel='transport', + ) as an: - portal = await n.run_in_actor(name_error) - await portal.result() + # TODO: ideally the REPL arrives at this frame in the parent, + # ABOVE the @api_frame of `Portal.run_in_actor()` (which + # should eventually not even be a portal method ... XD) + # await tractor.pause() + p: tractor.Portal = await an.run_in_actor(name_error) + + # with this style, should raise on this line + await p.result() + + # with this alt style should raise at `open_nusery()` + # return await p.result() if __name__ == '__main__': diff --git a/examples/debugging/sync_bp.py b/examples/debugging/sync_bp.py new file mode 100644 index 00000000..137710fc --- /dev/null +++ b/examples/debugging/sync_bp.py @@ -0,0 +1,153 @@ +from functools import partial +import time + +import trio +import tractor + + +def sync_pause( + use_builtin: bool = False, + error: bool = False, + hide_tb: bool = True, + pre_sleep: float|None = None, +): + if pre_sleep: + time.sleep(pre_sleep) + + if use_builtin: + breakpoint(hide_tb=hide_tb) + + else: + tractor.pause_from_sync() + + if error: + raise RuntimeError('yoyo sync code error') + + +@tractor.context +async def start_n_sync_pause( + ctx: tractor.Context, +): + actor: tractor.Actor = tractor.current_actor() + + # sync to parent-side task + await ctx.started() + + print(f'Entering `sync_pause()` in subactor: {actor.uid}\n') + sync_pause() + print(f'Exited `sync_pause()` in subactor: {actor.uid}\n') + + +async def main() -> None: + async with ( + tractor.open_nursery( + # NOTE: required for pausing from sync funcs + maybe_enable_greenback=True, + debug_mode=True, + # loglevel='cancel', + ) as an, + trio.open_nursery() as tn, + ): + # just from root task + sync_pause() + + p: tractor.Portal = await an.start_actor( + 'subactor', + enable_modules=[__name__], + # infect_asyncio=True, + debug_mode=True, + ) + + # TODO: 3 sub-actor usage cases: + # -[x] via a `.open_context()` + # -[ ] via a `.run_in_actor()` call + # -[ ] via a `.run()` + # -[ ] via a `.to_thread.run_sync()` in subactor + async with p.open_context( + start_n_sync_pause, + ) as (ctx, first): + assert first is None + + # TODO: handle bg-thread-in-root-actor special cases! + # + # there are a couple very subtle situations possible here + # and they are likely to become more important as cpython + # moves to support no-GIL. + # + # Cases: + # 1. root-actor bg-threads that call `.pause_from_sync()` + # whilst an in-tree subactor also is using ` .pause()`. + # |_ since the root-actor bg thread can not + # `Lock._debug_lock.acquire_nowait()` without running + # a `trio.Task`, AND because the + # `PdbREPL.set_continue()` is called from that + # bg-thread, we can not `._debug_lock.release()` + # either! + # |_ this results in no actor-tree `Lock` being used + # on behalf of the bg-thread and thus the subactor's + # task and the thread trying to to use stdio + # simultaneously which results in the classic TTY + # clobbering! + # + # 2. mutiple sync-bg-threads that call + # `.pause_from_sync()` where one is scheduled via + # `Nursery.start_soon(to_thread.run_sync)` in a bg + # task. + # + # Due to the GIL, the threads never truly try to step + # through the REPL simultaneously, BUT their `logging` + # and traceback outputs are interleaved since the GIL + # (seemingly) on every REPL-input from the user + # switches threads.. + # + # Soo, the context switching semantics of the GIL + # result in a very confusing and messy interaction UX + # since eval and (tb) print output is NOT synced to + # each REPL-cycle (like we normally make it via + # a `.set_continue()` callback triggering the + # `Lock.release()`). Ideally we can solve this + # usability issue NOW because this will of course be + # that much more important when eventually there is no + # GIL! + + # XXX should cause double REPL entry and thus TTY + # clobbering due to case 1. above! + tn.start_soon( + partial( + trio.to_thread.run_sync, + partial( + sync_pause, + use_builtin=False, + # pre_sleep=0.5, + ), + abandon_on_cancel=True, + thread_name='start_soon_root_bg_thread', + ) + ) + + await tractor.pause() + + # XXX should cause double REPL entry and thus TTY + # clobbering due to case 2. above! + await trio.to_thread.run_sync( + partial( + sync_pause, + # NOTE this already works fine since in the new + # thread the `breakpoint()` built-in is never + # overloaded, thus NO locking is used, HOWEVER + # the case 2. from above still exists! + use_builtin=True, + ), + abandon_on_cancel=False, + thread_name='inline_root_bg_thread', + ) + + await ctx.cancel() + + # TODO: case where we cancel from trio-side while asyncio task + # has debugger lock? + await p.cancel_actor() + + +if __name__ == '__main__': + trio.run(main) diff --git a/examples/full_fledged_streaming_service.py b/examples/full_fledged_streaming_service.py index 1650b583..be4c372e 100644 --- a/examples/full_fledged_streaming_service.py +++ b/examples/full_fledged_streaming_service.py @@ -1,6 +1,11 @@ import time import trio import tractor +from tractor import ( + ActorNursery, + MsgStream, + Portal, +) # this is the first 2 actors, streamer_1 and streamer_2 @@ -12,14 +17,18 @@ async def stream_data(seed): # this is the third actor; the aggregator async def aggregate(seed): - """Ensure that the two streams we receive match but only stream + ''' + Ensure that the two streams we receive match but only stream a single set of values to the parent. - """ - async with tractor.open_nursery() as nursery: - portals = [] + + ''' + an: ActorNursery + async with tractor.open_nursery() as an: + portals: list[Portal] = [] for i in range(1, 3): - # fork point - portal = await nursery.start_actor( + + # fork/spawn call + portal = await an.start_actor( name=f'streamer_{i}', enable_modules=[__name__], ) @@ -43,7 +52,11 @@ async def aggregate(seed): async with trio.open_nursery() as n: for portal in portals: - n.start_soon(push_to_chan, portal, send_chan.clone()) + n.start_soon( + push_to_chan, + portal, + send_chan.clone(), + ) # close this local task's reference to send side await send_chan.aclose() @@ -60,26 +73,36 @@ async def aggregate(seed): print("FINISHED ITERATING in aggregator") - await nursery.cancel() + await an.cancel() print("WAITING on `ActorNursery` to finish") print("AGGREGATOR COMPLETE!") -# this is the main actor and *arbiter* -async def main(): - # a nursery which spawns "actors" +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) + an: ActorNursery async with tractor.open_nursery( - arbiter_addr=('127.0.0.1', 1616) - ) as nursery: + loglevel='cancel', + debug_mode=True, + ) as an: seed = int(1e3) pre_start = time.time() - portal = await nursery.start_actor( + portal: Portal = await an.start_actor( name='aggregator', enable_modules=[__name__], ) + stream: MsgStream async with portal.open_stream_from( aggregate, seed=seed, @@ -88,11 +111,12 @@ async def main(): start = time.time() # the portal call returns exactly what you'd expect # as if the remote "aggregate" function was called locally - result_stream = [] + result_stream: list[int] = [] async for value in stream: result_stream.append(value) - await portal.cancel_actor() + cancelled: bool = await portal.cancel_actor() + assert cancelled print(f"STREAM TIME = {time.time() - start}") print(f"STREAM + SPAWN TIME = {time.time() - pre_start}") 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/pyproject.toml b/pyproject.toml index e52aa476..45847b14 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -1,3 +1,72 @@ +[build-system] +requires = ["poetry-core"] +build-backend = "poetry.core.masonry.api" + +# ------ - ------ + +[tool.poetry] +name = "tractor" +version = "0.1.0a6dev0" +description='structured concurrent `trio`-"actors"' +authors = ["Tyler Goodlet "] +license = "AGPlv3" +readme = "docs/README.rst" + +# TODO: do we need this xontrib loader at all given pep420 +# and xonsh's xontrib global-autoload-via-setuptools? +# https://xon.sh/tutorial_xontrib.html#authoring-xontribs +packages = [ + {include = 'tractor' }, + # {include = 'tractor.experimental' }, + # {include = 'tractor.trionics' }, + # {include = 'tractor.msg' }, + # {include = 'tractor.devx' }, +] + +# ------ - ------ + +[tool.poetry.dependencies] +python = "^3.11" + +# trio runtime related +# proper range spec: +# https://packaging.python.org/en/latest/discussions/install-requires-vs-requirements/#id5 +trio='^0.24' +tricycle = "^0.4.1" +trio-typing = "^0.10.0" + +msgspec='^0.18.5' # interchange +wrapt = "^1.16.0" # decorators +colorlog = "^6.8.2" # logging + +# built-in multi-actor `pdb` REPL +pdbp = "^1.5.0" + + +# TODO: distributed transport using +# linux kernel networking +# 'pyroute2 + +# ------ - ------ + +[tool.poetry.group.dev] +optional = false +[tool.poetry.group.dev.dependencies] +# testing +pytest = "^8.2.0" +pexpect = "^4.9.0" + +# .devx tooling +greenback = "^1.2.1" +stackscope = "^0.2.2" + +# (light) xonsh usage/integration +xontrib-vox = "^0.0.1" +prompt-toolkit = "^3.0.43" +xonsh-vox-tabcomplete = "^0.5" + +# ------ - ------ + [tool.towncrier] package = "tractor" filename = "NEWS.rst" @@ -26,3 +95,48 @@ 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" + +# ------ - ------ + +[project] +name = "tractor" +keywords = [ + 'trio', + 'async', + 'concurrency', + 'structured concurrency', + 'actor model', + 'distributed', + 'multiprocessing' +] +classifiers = [ + "Development Status :: 3 - Alpha", + "Operating System :: POSIX :: Linux", + "Framework :: Trio", + "License :: OSI Approved :: GNU Affero General Public License v3 or later (AGPLv3+)", + "Programming Language :: Python :: Implementation :: CPython", + "Programming Language :: Python :: 3 :: Only", + "Programming Language :: Python :: 3.11", + "Topic :: System :: Distributed Computing", +] diff --git a/requirements-test.txt b/requirements-test.txt index 8070f2c7..b589bd12 100644 --- a/requirements-test.txt +++ b/requirements-test.txt @@ -6,3 +6,4 @@ mypy trio_typing pexpect towncrier +numpy diff --git a/setup.py b/setup.py index d26deb9b..a2219372 100755 --- a/setup.py +++ b/setup.py @@ -26,7 +26,7 @@ with open('docs/README.rst', encoding='utf-8') as f: setup( name="tractor", version='0.1.0a6dev0', # alpha zone - description='structured concurrrent `trio`-"actors"', + description='structured concurrent `trio`-"actors"', long_description=readme, license='AGPLv3', author='Tyler Goodlet', @@ -36,41 +36,44 @@ setup( platforms=['linux', 'windows'], packages=[ 'tractor', - 'tractor.experimental', - 'tractor.trionics', + 'tractor.experimental', # wacky ideas + 'tractor.trionics', # trio extensions + 'tractor.msg', # lowlevel data types + 'tractor.devx', # "dev-experience" ], install_requires=[ # trio related # proper range spec: # https://packaging.python.org/en/latest/discussions/install-requires-vs-requirements/#id5 - 'trio >= 0.22', - 'async_generator', - 'trio_typing', - 'exceptiongroup', + 'trio >= 0.24', + + # 'async_generator', # in stdlib mostly! + # 'trio_typing', # trio==0.23.0 has type hints! + # 'exceptiongroup', # in stdlib as of 3.11! # tooling + 'stackscope', 'tricycle', 'trio_typing', 'colorlog', 'wrapt', # IPC serialization - 'msgspec', + 'msgspec>=0.18.5', # debug mode REPL 'pdbp', + # TODO: distributed transport using + # linux kernel networking + # 'pyroute2', + # pip ref docs on these specs: # https://pip.pypa.io/en/stable/reference/requirement-specifiers/#examples # and pep: # https://peps.python.org/pep-0440/#version-specifiers - # windows deps workaround for ``pdbpp`` - # https://github.com/pdbpp/pdbpp/issues/498 - # https://github.com/pdbpp/fancycompleter/issues/37 - 'pyreadline3 ; platform_system == "Windows"', - ], tests_require=['pytest'], python_requires=">=3.10", diff --git a/tests/conftest.py b/tests/conftest.py index 3363cf56..5ce84425 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -7,94 +7,19 @@ 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 @@ -114,41 +39,45 @@ 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', + "--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 @@ -168,14 +97,35 @@ _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), +) + + @pytest.fixture(scope='session') -def arb_addr(): - return _arb_addr +def reg_addr() -> tuple[str, int]: + + # globally override the runtime to the per-test-session-dynamic + # addr so that all tests never conflict with any other actor + # tree using the default. + from tractor import _root + _root._default_lo_addrs = [_reg_addr] + + return _reg_addr def pytest_generate_tests(metafunc): @@ -212,34 +162,40 @@ def sig_prog(proc, sig): assert ret +# TODO: factor into @cm and move to `._testing`? @pytest.fixture def daemon( loglevel: str, testdir, - arb_addr: tuple[str, int], + reg_addr: tuple[str, int], ): ''' - Run a daemon actor as a "remote arbiter". + Run a daemon root actor as a separate actor-process tree and + "remote registrar" for discovery-protocol related tests. ''' if loglevel in ('trace', 'debug'): - # too much logging will lock up the subproc (smh) - loglevel = 'info' + # XXX: too much logging will lock up the subproc (smh) + loglevel: str = 'info' - cmdargs = [ - sys.executable, '-c', - "import tractor; tractor.run_daemon([], registry_addr={}, loglevel={})" - .format( - arb_addr, - "'{}'".format(loglevel) if loglevel else None) + code: str = ( + "import tractor; " + "tractor.run_daemon([], registry_addrs={reg_addrs}, loglevel={ll})" + ).format( + reg_addrs=str([reg_addr]), + ll="'{}'".format(loglevel) if loglevel else None, + ) + cmd: list[str] = [ + sys.executable, + '-c', code, ] - kwargs = dict() + kwargs = {} if platform.system() == 'Windows': # without this, tests hang on windows forever kwargs['creationflags'] = subprocess.CREATE_NEW_PROCESS_GROUP proc = testdir.popen( - cmdargs, + cmd, stdout=subprocess.PIPE, stderr=subprocess.PIPE, **kwargs, diff --git a/tests/test_advanced_faults.py b/tests/test_advanced_faults.py index a48866ea..dfaeb68e 100644 --- a/tests/test_advanced_faults.py +++ b/tests/test_advanced_faults.py @@ -3,22 +3,30 @@ 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, + break_ipc, ) @pytest.mark.parametrize( - 'debug_mode', - [False, True], - ids=['no_debug_mode', 'debug_mode'], + 'pre_aclose_msgstream', + [ + False, + True, + ], + ids=[ + 'no_msgstream_aclose', + 'pre_aclose_msgstream', + ], ) @pytest.mark.parametrize( 'ipc_break', @@ -63,8 +71,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 @@ -83,70 +93,132 @@ def test_ipc_channel_break_during_stream( # requires the user to do ctl-c to cancel the actor tree. expect_final_exc = trio.ClosedResourceError - mod = import_path( - examples_dir() / 'advanced_faults' / 'ipc_failure_during_stream.py', + mod: ModuleType = import_path( + examples_dir() / 'advanced_faults' + / 'ipc_failure_during_stream.py', root=examples_dir(), + consider_namespace_packages=False, ) - 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 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 + # ^XXX NOPE! XXX^ since now `.open_stream()` absorbs this + # gracefully! + expect_final_exc = KeyboardInterrupt - # 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'] - ) + # 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 + ): + # 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 + + # 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 = 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 ( + ipc_break['break_child_ipc_after'] is not False + and ( + ipc_break['break_parent_ipc_after'] + > ipc_break['break_child_ipc_after'] ) + ): + if pre_aclose_msgstream: + 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 + # 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 - # 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. + # BOTH but, PARENT breaks FIRST elif ( - # only parent breaks - ( + ipc_break['break_parent_ipc_after'] is not False + and ( + ipc_break['break_child_ipc_after'] + > 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'] - ) ) ): - expect_final_exc = trio.EndOfChannel + expect_final_exc = trio.ClosedResourceError - with pytest.raises(expect_final_exc): - trio.run( - partial( - mod.main, - debug_mode=debug_mode, - start_method=spawn_backend, - **ipc_break, + 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 @@ -155,9 +227,15 @@ async def break_ipc_after_started( ) -> None: await ctx.started() async with ctx.open_stream() as stream: - await stream.aclose() - await trio.sleep(0.2) - await ctx.chan.send(None) + + # TODO: make a test which verifies the error + # for this, i.e. raises a `MsgTypeError` + # await ctx.chan.send(None) + + await break_ipc( + stream=stream, + pre_close=True, + ) print('child broke IPC and terminating') @@ -169,25 +247,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) diff --git a/tests/test_advanced_streaming.py b/tests/test_advanced_streaming.py index 799a0897..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 @@ -298,44 +307,69 @@ 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('transport 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__], ) @@ -343,7 +377,6 @@ def test_local_task_fanout_from_stream(): 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 +385,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 +395,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') diff --git a/tests/test_cancellation.py b/tests/test_cancellation.py index 657ab8e4..bf41ddd1 100644 --- a/tests/test_cancellation.py +++ b/tests/test_cancellation.py @@ -8,15 +8,13 @@ import platform import time from itertools import repeat -from exceptiongroup import ( - BaseExceptionGroup, - ExceptionGroup, -) 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(): @@ -47,17 +45,19 @@ async def do_nuthin(): ], ids=['no_args', 'unexpected_args'], ) -def test_remote_error(arb_addr, args_err): - """Verify an error raised in a subactor that is propagated +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 way up the stack. - """ + + ''' args, errtype = 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 @@ -65,7 +65,9 @@ def test_remote_error(arb_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 @@ -75,7 +77,7 @@ def test_remote_error(arb_addr, args_err): # of this actor nursery. await portal.result() except tractor.RemoteActorError as err: - assert err.type == errtype + assert err.boxed_type == errtype print("Look Maa that actor failed hard, hehh") raise @@ -84,20 +86,33 @@ def test_remote_error(arb_addr, args_err): with pytest.raises(tractor.RemoteActorError) as excinfo: trio.run(main) - assert excinfo.value.type == errtype + assert excinfo.value.boxed_type == errtype else: - # the root task will also error on the `.result()` call - # so we expect an error from there AND the child. - with pytest.raises(BaseExceptionGroup) as excinfo: + # the root task will also error on the `Portal.result()` + # call so we expect an error from there AND the child. + # |_ tho seems like on new `trio` this doesn't always + # happen? + with pytest.raises(( + BaseExceptionGroup, + tractor.RemoteActorError, + )) as excinfo: trio.run(main) - # ensure boxed errors - for exc in excinfo.value.exceptions: - assert exc.type == errtype + # ensure boxed errors are `errtype` + err: BaseException = excinfo.value + if isinstance(err, BaseExceptionGroup): + suberrs: list[BaseException] = err.exceptions + else: + suberrs: list[BaseException] = [err] + + for exc in suberrs: + assert exc.boxed_type == errtype -def test_multierror(arb_addr): +def test_multierror( + reg_addr: tuple[str, int], +): ''' Verify we raise a ``BaseExceptionGroup`` out of a nursery where more then one actor errors. @@ -105,7 +120,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') @@ -115,7 +130,7 @@ def test_multierror(arb_addr): try: await portal2.result() except tractor.RemoteActorError as err: - assert err.type == AssertionError + assert err.boxed_type == AssertionError print("Look Maa that first actor failed hard, hehh") raise @@ -130,14 +145,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): @@ -167,7 +182,7 @@ def test_multierror_fast_nursery(arb_addr, start_method, num_subactors, delay): for exc in exceptions: assert isinstance(exc, tractor.RemoteActorError) - assert exc.type == AssertionError + assert exc.boxed_type == AssertionError async def do_nothing(): @@ -175,15 +190,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( @@ -303,7 +323,7 @@ async def test_some_cancels_all(num_actors_and_errs, start_method, loglevel): await portal.run(func, **kwargs) except tractor.RemoteActorError as err: - assert err.type == err_type + assert err.boxed_type == err_type # we only expect this first error to propogate # (all other daemons are cancelled before they # can be scheduled) @@ -322,11 +342,11 @@ async def test_some_cancels_all(num_actors_and_errs, start_method, loglevel): assert len(err.exceptions) == num_actors for exc in err.exceptions: if isinstance(exc, tractor.RemoteActorError): - assert exc.type == err_type + assert exc.boxed_type == err_type else: assert isinstance(exc, trio.Cancelled) elif isinstance(err, tractor.RemoteActorError): - assert err.type == err_type + assert err.boxed_type == err_type assert n.cancelled is True assert not n._children @@ -405,7 +425,7 @@ async def test_nested_multierrors(loglevel, start_method): elif isinstance(subexc, tractor.RemoteActorError): # on windows it seems we can't exactly be sure wtf # will happen.. - assert subexc.type in ( + assert subexc.boxed_type in ( tractor.RemoteActorError, trio.Cancelled, BaseExceptionGroup, @@ -415,7 +435,7 @@ async def test_nested_multierrors(loglevel, start_method): for subsub in subexc.exceptions: if subsub in (tractor.RemoteActorError,): - subsub = subsub.type + subsub = subsub.boxed_type assert type(subsub) in ( trio.Cancelled, @@ -430,16 +450,16 @@ async def test_nested_multierrors(loglevel, start_method): # we get back the (sent) cancel signal instead if is_win(): if isinstance(subexc, tractor.RemoteActorError): - assert subexc.type in ( + assert subexc.boxed_type in ( BaseExceptionGroup, tractor.RemoteActorError ) else: assert isinstance(subexc, BaseExceptionGroup) else: - assert subexc.type is ExceptionGroup + assert subexc.boxed_type is ExceptionGroup else: - assert subexc.type in ( + assert subexc.boxed_type in ( tractor.RemoteActorError, trio.Cancelled ) diff --git a/tests/test_caps_based_msging.py b/tests/test_caps_based_msging.py new file mode 100644 index 00000000..9a73ba8d --- /dev/null +++ b/tests/test_caps_based_msging.py @@ -0,0 +1,930 @@ +''' +Low-level functional audits for our +"capability based messaging"-spec feats. + +B~) + +''' +import typing +from typing import ( + Any, + Type, + Union, +) +from contextvars import ( + Context, +) + +from msgspec import ( + structs, + msgpack, + Struct, + ValidationError, +) +import pytest + +import tractor +from tractor import ( + _state, + MsgTypeError, +) +from tractor.msg import ( + _codec, + _ctxvar_MsgCodec, + + NamespacePath, + MsgCodec, + mk_codec, + apply_codec, + current_codec, +) +from tractor.msg.types import ( + _payload_msgs, + log, + Msg, + Started, + mk_msg_spec, +) +import trio + + +def mk_custom_codec( + pld_spec: Union[Type]|Any, + add_hooks: bool, + +) -> MsgCodec: + ''' + Create custom `msgpack` enc/dec-hooks and set a `Decoder` + which only loads `pld_spec` (like `NamespacePath`) types. + + ''' + uid: tuple[str, str] = tractor.current_actor().uid + + # XXX NOTE XXX: despite defining `NamespacePath` as a type + # field on our `Msg.pld`, we still need a enc/dec_hook() pair + # to cast to/from that type on the wire. See the docs: + # https://jcristharif.com/msgspec/extending.html#mapping-to-from-native-types + + def enc_nsp(obj: Any) -> Any: + print(f'{uid} ENC HOOK') + match obj: + case NamespacePath(): + print( + f'{uid}: `NamespacePath`-Only ENCODE?\n' + f'obj-> `{obj}`: {type(obj)}\n' + ) + # if type(obj) != NamespacePath: + # breakpoint() + return str(obj) + + print( + f'{uid}\n' + 'CUSTOM ENCODE\n' + f'obj-arg-> `{obj}`: {type(obj)}\n' + ) + logmsg: str = ( + f'{uid}\n' + 'FAILED ENCODE\n' + f'obj-> `{obj}: {type(obj)}`\n' + ) + raise NotImplementedError(logmsg) + + def dec_nsp( + obj_type: Type, + obj: Any, + + ) -> Any: + print( + f'{uid}\n' + 'CUSTOM DECODE\n' + f'type-arg-> {obj_type}\n' + f'obj-arg-> `{obj}`: {type(obj)}\n' + ) + nsp = None + + if ( + obj_type is NamespacePath + and isinstance(obj, str) + and ':' in obj + ): + nsp = NamespacePath(obj) + # TODO: we could built a generic handler using + # JUST matching the obj_type part? + # nsp = obj_type(obj) + + if nsp: + print(f'Returning NSP instance: {nsp}') + return nsp + + logmsg: str = ( + f'{uid}\n' + 'FAILED DECODE\n' + f'type-> {obj_type}\n' + f'obj-arg-> `{obj}`: {type(obj)}\n\n' + f'current codec:\n' + f'{current_codec()}\n' + ) + # TODO: figure out the ignore subsys for this! + # -[ ] option whether to defense-relay backc the msg + # inside an `Invalid`/`Ignore` + # -[ ] how to make this handling pluggable such that a + # `Channel`/`MsgTransport` can intercept and process + # back msgs either via exception handling or some other + # signal? + log.warning(logmsg) + # NOTE: this delivers the invalid + # value up to `msgspec`'s decoding + # machinery for error raising. + return obj + # raise NotImplementedError(logmsg) + + nsp_codec: MsgCodec = mk_codec( + ipc_pld_spec=pld_spec, + + # NOTE XXX: the encode hook MUST be used no matter what since + # our `NamespacePath` is not any of a `Any` native type nor + # a `msgspec.Struct` subtype - so `msgspec` has no way to know + # how to encode it unless we provide the custom hook. + # + # AGAIN that is, regardless of whether we spec an + # `Any`-decoded-pld the enc has no knowledge (by default) + # how to enc `NamespacePath` (nsp), so we add a custom + # hook to do that ALWAYS. + enc_hook=enc_nsp if add_hooks else None, + + # XXX NOTE: pretty sure this is mutex with the `type=` to + # `Decoder`? so it won't work in tandem with the + # `ipc_pld_spec` passed above? + dec_hook=dec_nsp if add_hooks else None, + ) + return nsp_codec + + +def chk_codec_applied( + expect_codec: MsgCodec, + enter_value: MsgCodec|None = None, + +) -> MsgCodec: + ''' + buncha sanity checks ensuring that the IPC channel's + context-vars are set to the expected codec and that are + ctx-var wrapper APIs match the same. + + ''' + # TODO: play with tricyle again, bc this is supposed to work + # the way we want? + # + # TreeVar + # task: trio.Task = trio.lowlevel.current_task() + # curr_codec = _ctxvar_MsgCodec.get_in(task) + + # ContextVar + # task_ctx: Context = task.context + # assert _ctxvar_MsgCodec in task_ctx + # curr_codec: MsgCodec = task.context[_ctxvar_MsgCodec] + + # NOTE: currently we use this! + # RunVar + curr_codec: MsgCodec = current_codec() + last_read_codec = _ctxvar_MsgCodec.get() + # assert curr_codec is last_read_codec + + assert ( + (same_codec := expect_codec) is + # returned from `mk_codec()` + + # yielded value from `apply_codec()` + + # read from current task's `contextvars.Context` + curr_codec is + last_read_codec + + # the default `msgspec` settings + is not _codec._def_msgspec_codec + is not _codec._def_tractor_codec + ) + + if enter_value: + enter_value is same_codec + + +def iter_maybe_sends( + send_items: dict[Union[Type], Any] | list[tuple], + ipc_pld_spec: Union[Type] | Any, + add_codec_hooks: bool, + + codec: MsgCodec|None = None, + +) -> tuple[Any, bool]: + + if isinstance(send_items, dict): + send_items = send_items.items() + + for ( + send_type_spec, + send_value, + ) in send_items: + + expect_roundtrip: bool = False + + # values-to-typespec santiy + send_type = type(send_value) + assert send_type == send_type_spec or ( + (subtypes := getattr(send_type_spec, '__args__', None)) + and send_type in subtypes + ) + + spec_subtypes: set[Union[Type]] = ( + getattr( + ipc_pld_spec, + '__args__', + {ipc_pld_spec,}, + ) + ) + send_in_spec: bool = ( + send_type == ipc_pld_spec + or ( + ipc_pld_spec != Any + and # presume `Union` of types + send_type in spec_subtypes + ) + or ( + ipc_pld_spec == Any + and + send_type != NamespacePath + ) + ) + expect_roundtrip = ( + send_in_spec + # any spec should support all other + # builtin py values that we send + # except our custom nsp type which + # we should be able to send as long + # as we provide the custom codec hooks. + or ( + ipc_pld_spec == Any + and + send_type == NamespacePath + and + add_codec_hooks + ) + ) + + if codec is not None: + # XXX FIRST XXX ensure roundtripping works + # before touching any IPC primitives/APIs. + wire_bytes: bytes = codec.encode( + Started( + cid='blahblah', + pld=send_value, + ) + ) + # NOTE: demonstrates the decoder loading + # to via our native SCIPP msg-spec + # (structurred-conc-inter-proc-protocol) + # implemented as per, + try: + msg: Started = codec.decode(wire_bytes) + if not expect_roundtrip: + pytest.fail( + f'NOT-EXPECTED able to roundtrip value given spec:\n' + f'ipc_pld_spec -> {ipc_pld_spec}\n' + f'value -> {send_value}: {send_type}\n' + ) + + pld = msg.pld + assert pld == send_value + + except ValidationError: + if expect_roundtrip: + pytest.fail( + f'EXPECTED to roundtrip value given spec:\n' + f'ipc_pld_spec -> {ipc_pld_spec}\n' + f'value -> {send_value}: {send_type}\n' + ) + + yield ( + str(send_type), + send_value, + expect_roundtrip, + ) + + +def dec_type_union( + type_names: list[str], +) -> Type: + ''' + Look up types by name, compile into a list and then create and + return a `typing.Union` from the full set. + + ''' + import importlib + types: list[Type] = [] + for type_name in type_names: + for ns in [ + typing, + importlib.import_module(__name__), + ]: + if type_ref := getattr( + ns, + type_name, + False, + ): + types.append(type_ref) + + # special case handling only.. + # ipc_pld_spec: Union[Type] = eval( + # pld_spec_str, + # {}, # globals + # {'typing': typing}, # locals + # ) + + return Union[*types] + + +def enc_type_union( + union_or_type: Union[Type]|Type, +) -> list[str]: + ''' + Encode a type-union or single type to a list of type-name-strings + ready for IPC interchange. + + ''' + type_strs: list[str] = [] + for typ in getattr( + union_or_type, + '__args__', + {union_or_type,}, + ): + type_strs.append(typ.__qualname__) + + return type_strs + + +@tractor.context +async def send_back_values( + ctx: Context, + expect_debug: bool, + pld_spec_type_strs: list[str], + add_hooks: bool, + started_msg_bytes: bytes, + expect_ipc_send: dict[str, tuple[Any, bool]], + +) -> None: + ''' + Setup up a custom codec to load instances of `NamespacePath` + and ensure we can round trip a func ref with our parent. + + ''' + uid: tuple = tractor.current_actor().uid + + # debug mode sanity check (prolly superfluous but, meh) + assert expect_debug == _state.debug_mode() + + # init state in sub-actor should be default + chk_codec_applied( + expect_codec=_codec._def_tractor_codec, + ) + + # load pld spec from input str + ipc_pld_spec = dec_type_union( + pld_spec_type_strs, + ) + pld_spec_str = str(ipc_pld_spec) + + # same as on parent side config. + nsp_codec: MsgCodec = mk_custom_codec( + pld_spec=ipc_pld_spec, + add_hooks=add_hooks, + ) + with ( + apply_codec(nsp_codec) as codec, + ): + chk_codec_applied( + expect_codec=nsp_codec, + enter_value=codec, + ) + + print( + f'{uid}: attempting `Started`-bytes DECODE..\n' + ) + try: + msg: Started = nsp_codec.decode(started_msg_bytes) + expected_pld_spec_str: str = msg.pld + assert pld_spec_str == expected_pld_spec_str + + # TODO: maybe we should add our own wrapper error so as to + # be interchange-lib agnostic? + # -[ ] the error type is wtv is raised from the hook so we + # could also require a type-class of errors for + # indicating whether the hook-failure can be handled by + # a nasty-dialog-unprot sub-sys? + except ValidationError: + + # NOTE: only in the `Any` spec case do we expect this to + # work since otherwise no spec covers a plain-ol' + # `.pld: str` + if pld_spec_str == 'Any': + raise + else: + print( + f'{uid}: (correctly) unable to DECODE `Started`-bytes\n' + f'{started_msg_bytes}\n' + ) + + iter_send_val_items = iter(expect_ipc_send.values()) + sent: list[Any] = [] + for send_value, expect_send in iter_send_val_items: + try: + print( + f'{uid}: attempting to `.started({send_value})`\n' + f'=> expect_send: {expect_send}\n' + f'SINCE, ipc_pld_spec: {ipc_pld_spec}\n' + f'AND, codec: {codec}\n' + ) + await ctx.started(send_value) + sent.append(send_value) + if not expect_send: + + # XXX NOTE XXX THIS WON'T WORK WITHOUT SPECIAL + # `str` handling! or special debug mode IPC + # msgs! + await tractor.pause() + + raise RuntimeError( + f'NOT-EXPECTED able to roundtrip value given spec:\n' + f'ipc_pld_spec -> {ipc_pld_spec}\n' + f'value -> {send_value}: {type(send_value)}\n' + ) + + break # move on to streaming block.. + + except tractor.MsgTypeError: + await tractor.pause() + + if expect_send: + raise RuntimeError( + f'EXPECTED to `.started()` value given spec:\n' + f'ipc_pld_spec -> {ipc_pld_spec}\n' + f'value -> {send_value}: {type(send_value)}\n' + ) + + async with ctx.open_stream() as ipc: + print( + f'{uid}: Entering streaming block to send remaining values..' + ) + + for send_value, expect_send in iter_send_val_items: + send_type: Type = type(send_value) + print( + '------ - ------\n' + f'{uid}: SENDING NEXT VALUE\n' + f'ipc_pld_spec: {ipc_pld_spec}\n' + f'expect_send: {expect_send}\n' + f'val: {send_value}\n' + '------ - ------\n' + ) + try: + await ipc.send(send_value) + print(f'***\n{uid}-CHILD sent {send_value!r}\n***\n') + sent.append(send_value) + + # NOTE: should only raise above on + # `.started()` or a `Return` + # if not expect_send: + # raise RuntimeError( + # f'NOT-EXPECTED able to roundtrip value given spec:\n' + # f'ipc_pld_spec -> {ipc_pld_spec}\n' + # f'value -> {send_value}: {send_type}\n' + # ) + + except ValidationError: + print(f'{uid} FAILED TO SEND {send_value}!') + + # await tractor.pause() + if expect_send: + raise RuntimeError( + f'EXPECTED to roundtrip value given spec:\n' + f'ipc_pld_spec -> {ipc_pld_spec}\n' + f'value -> {send_value}: {send_type}\n' + ) + # continue + + else: + print( + f'{uid}: finished sending all values\n' + 'Should be exiting stream block!\n' + ) + + print(f'{uid}: exited streaming block!') + + # TODO: this won't be true bc in streaming phase we DO NOT + # msgspec check outbound msgs! + # -[ ] once we implement the receiver side `InvalidMsg` + # then we can expect it here? + # assert ( + # len(sent) + # == + # len([val + # for val, expect in + # expect_ipc_send.values() + # if expect is True]) + # ) + + +def ex_func(*args): + print(f'ex_func({args})') + + +@pytest.mark.parametrize( + 'ipc_pld_spec', + [ + Any, + NamespacePath, + NamespacePath|None, # the "maybe" spec Bo + ], + ids=[ + 'any_type', + 'nsp_type', + 'maybe_nsp_type', + ] +) +@pytest.mark.parametrize( + 'add_codec_hooks', + [ + True, + False, + ], + ids=['use_codec_hooks', 'no_codec_hooks'], +) +def test_codec_hooks_mod( + debug_mode: bool, + ipc_pld_spec: Union[Type]|Any, + # send_value: None|str|NamespacePath, + add_codec_hooks: bool, +): + ''' + Audit the `.msg.MsgCodec` override apis details given our impl + uses `contextvars` to accomplish per `trio` task codec + application around an inter-proc-task-comms context. + + ''' + async def main(): + nsp = NamespacePath.from_ref(ex_func) + send_items: dict[Union, Any] = { + Union[None]: None, + Union[NamespacePath]: nsp, + Union[str]: str(nsp), + } + + # init default state for actor + chk_codec_applied( + expect_codec=_codec._def_tractor_codec, + ) + + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + p: tractor.Portal = await an.start_actor( + 'sub', + enable_modules=[__name__], + ) + + # TODO: 2 cases: + # - codec not modified -> decode nsp as `str` + # - codec modified with hooks -> decode nsp as + # `NamespacePath` + nsp_codec: MsgCodec = mk_custom_codec( + pld_spec=ipc_pld_spec, + add_hooks=add_codec_hooks, + ) + with apply_codec(nsp_codec) as codec: + chk_codec_applied( + expect_codec=nsp_codec, + enter_value=codec, + ) + + expect_ipc_send: dict[str, tuple[Any, bool]] = {} + + report: str = ( + 'Parent report on send values with\n' + f'ipc_pld_spec: {ipc_pld_spec}\n' + ' ------ - ------\n' + ) + for val_type_str, val, expect_send in iter_maybe_sends( + send_items, + ipc_pld_spec, + add_codec_hooks=add_codec_hooks, + ): + report += ( + f'send_value: {val}: {type(val)} ' + f'=> expect_send: {expect_send}\n' + ) + expect_ipc_send[val_type_str] = (val, expect_send) + + print( + report + + ' ------ - ------\n' + ) + assert len(expect_ipc_send) == len(send_items) + # now try over real IPC with a the subactor + # expect_ipc_rountrip: bool = True + expected_started = Started( + cid='cid', + pld=str(ipc_pld_spec), + ) + # build list of values we expect to receive from + # the subactor. + expect_to_send: list[Any] = [ + val + for val, expect_send in expect_ipc_send.values() + if expect_send + ] + + pld_spec_type_strs: list[str] = enc_type_union(ipc_pld_spec) + + # XXX should raise an mte (`MsgTypeError`) + # when `add_codec_hooks == False` bc the input + # `expect_ipc_send` kwarg has a nsp which can't be + # serialized! + # + # TODO:can we ensure this happens from the + # `Return`-side (aka the sub) as well? + if not add_codec_hooks: + try: + async with p.open_context( + send_back_values, + expect_debug=debug_mode, + pld_spec_type_strs=pld_spec_type_strs, + add_hooks=add_codec_hooks, + started_msg_bytes=nsp_codec.encode(expected_started), + + # XXX NOTE bc we send a `NamespacePath` in this kwarg + expect_ipc_send=expect_ipc_send, + + ) as (ctx, first): + pytest.fail('ctx should fail to open without custom enc_hook!?') + + # this test passes bc we can go no further! + except MsgTypeError: + # teardown nursery + await p.cancel_actor() + return + + # TODO: send the original nsp here and + # test with `limit_msg_spec()` above? + # await tractor.pause() + print('PARENT opening IPC ctx!\n') + async with ( + + # XXX should raise an mte (`MsgTypeError`) + # when `add_codec_hooks == False`.. + p.open_context( + send_back_values, + expect_debug=debug_mode, + pld_spec_type_strs=pld_spec_type_strs, + add_hooks=add_codec_hooks, + started_msg_bytes=nsp_codec.encode(expected_started), + expect_ipc_send=expect_ipc_send, + ) as (ctx, first), + + ctx.open_stream() as ipc, + ): + # ensure codec is still applied across + # `tractor.Context` + its embedded nursery. + chk_codec_applied( + expect_codec=nsp_codec, + enter_value=codec, + ) + print( + 'root: ENTERING CONTEXT BLOCK\n' + f'type(first): {type(first)}\n' + f'first: {first}\n' + ) + expect_to_send.remove(first) + + # TODO: explicit values we expect depending on + # codec config! + # assert first == first_val + # assert first == f'{__name__}:ex_func' + + async for next_sent in ipc: + print( + 'Parent: child sent next value\n' + f'{next_sent}: {type(next_sent)}\n' + ) + if expect_to_send: + expect_to_send.remove(next_sent) + else: + print('PARENT should terminate stream loop + block!') + + # all sent values should have arrived! + assert not expect_to_send + + await p.cancel_actor() + + trio.run(main) + + +def chk_pld_type( + payload_spec: Type[Struct]|Any, + pld: Any, + + expect_roundtrip: bool|None = None, + +) -> bool: + + pld_val_type: Type = type(pld) + + # TODO: verify that the overridden subtypes + # DO NOT have modified type-annots from original! + # 'Start', .pld: FuncSpec + # 'StartAck', .pld: IpcCtxSpec + # 'Stop', .pld: UNSEt + # 'Error', .pld: ErrorData + + codec: MsgCodec = mk_codec( + # NOTE: this ONLY accepts `Msg.pld` fields of a specified + # type union. + ipc_pld_spec=payload_spec, + ) + + # make a one-off dec to compare with our `MsgCodec` instance + # which does the below `mk_msg_spec()` call internally + ipc_msg_spec: Union[Type[Struct]] + msg_types: list[Msg[payload_spec]] + ( + ipc_msg_spec, + msg_types, + ) = mk_msg_spec( + payload_type_union=payload_spec, + ) + _enc = msgpack.Encoder() + _dec = msgpack.Decoder( + type=ipc_msg_spec or Any, # like `Msg[Any]` + ) + + assert ( + payload_spec + == + codec.pld_spec + ) + + # assert codec.dec == dec + # + # ^-XXX-^ not sure why these aren't "equal" but when cast + # to `str` they seem to match ?? .. kk + + assert ( + str(ipc_msg_spec) + == + str(codec.msg_spec) + == + str(_dec.type) + == + str(codec.dec.type) + ) + + # verify the boxed-type for all variable payload-type msgs. + if not msg_types: + breakpoint() + + roundtrip: bool|None = None + pld_spec_msg_names: list[str] = [ + td.__name__ for td in _payload_msgs + ] + for typedef in msg_types: + + skip_runtime_msg: bool = typedef.__name__ not in pld_spec_msg_names + if skip_runtime_msg: + continue + + pld_field = structs.fields(typedef)[1] + assert pld_field.type is payload_spec # TODO-^ does this need to work to get all subtypes to adhere? + + kwargs: dict[str, Any] = { + 'cid': '666', + 'pld': pld, + } + enc_msg: Msg = typedef(**kwargs) + + _wire_bytes: bytes = _enc.encode(enc_msg) + wire_bytes: bytes = codec.enc.encode(enc_msg) + assert _wire_bytes == wire_bytes + + ve: ValidationError|None = None + try: + dec_msg = codec.dec.decode(wire_bytes) + _dec_msg = _dec.decode(wire_bytes) + + # decoded msg and thus payload should be exactly same! + assert (roundtrip := ( + _dec_msg + == + dec_msg + == + enc_msg + )) + + if ( + expect_roundtrip is not None + and expect_roundtrip != roundtrip + ): + breakpoint() + + assert ( + pld + == + dec_msg.pld + == + enc_msg.pld + ) + # assert (roundtrip := (_dec_msg == enc_msg)) + + except ValidationError as _ve: + ve = _ve + roundtrip: bool = False + if pld_val_type is payload_spec: + raise ValueError( + 'Got `ValidationError` despite type-var match!?\n' + f'pld_val_type: {pld_val_type}\n' + f'payload_type: {payload_spec}\n' + ) from ve + + else: + # ow we good cuz the pld spec mismatched. + print( + 'Got expected `ValidationError` since,\n' + f'{pld_val_type} is not {payload_spec}\n' + ) + else: + if ( + payload_spec is not Any + and + pld_val_type is not payload_spec + ): + raise ValueError( + 'DID NOT `ValidationError` despite expected type match!?\n' + f'pld_val_type: {pld_val_type}\n' + f'payload_type: {payload_spec}\n' + ) + + # full code decode should always be attempted! + if roundtrip is None: + breakpoint() + + return roundtrip + + +def test_limit_msgspec(): + + async def main(): + async with tractor.open_root_actor( + debug_mode=True + ): + + # ensure we can round-trip a boxing `Msg` + assert chk_pld_type( + # Msg, + Any, + None, + expect_roundtrip=True, + ) + + # TODO: don't need this any more right since + # `msgspec>=0.15` has the nice generics stuff yah?? + # + # manually override the type annot of the payload + # field and ensure it propagates to all msg-subtypes. + # Msg.__annotations__['pld'] = Any + + # verify that a mis-typed payload value won't decode + assert not chk_pld_type( + # Msg, + int, + pld='doggy', + ) + + # parametrize the boxed `.pld` type as a custom-struct + # and ensure that parametrization propagates + # to all payload-msg-spec-able subtypes! + class CustomPayload(Struct): + name: str + value: Any + + assert not chk_pld_type( + # Msg, + CustomPayload, + pld='doggy', + ) + + assert chk_pld_type( + # Msg, + CustomPayload, + pld=CustomPayload(name='doggy', value='urmom') + ) + + # uhh bc we can `.pause_from_sync()` now! :surfer: + # breakpoint() + + trio.run(main) diff --git a/tests/test_child_manages_service_nursery.py b/tests/test_child_manages_service_nursery.py index 806e6d7e..21fb3920 100644 --- a/tests/test_child_manages_service_nursery.py +++ b/tests/test_child_manages_service_nursery.py @@ -6,14 +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 -from trio_typing import TaskStatus import tractor from tractor import RemoteActorError -from async_generator import aclosing async def aio_streamer( @@ -141,7 +142,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 @@ -170,4 +171,4 @@ def test_actor_managed_trio_nursery_task_error_cancels_aio( # verify boxed error err = excinfo.value - assert isinstance(err.type(), NameError) + assert err.boxed_type is NameError 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 4efc6319..8edea510 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -1,24 +1,36 @@ ''' ``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 from itertools import count +import math import platform -from typing import Optional +from pprint import pformat +from typing import ( + Callable, +) import pytest import trio import tractor +from tractor import ( + Actor, + Context, + current_actor, +) from tractor._exceptions import ( StreamOverrun, ContextCancelled, ) +from tractor._state import current_ipc_ctx -from conftest import tractor_test +from tractor._testing import ( + tractor_test, + expect_ctxc, +) # ``Context`` semantics are as follows, # ------------------------------------ @@ -64,7 +76,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). @@ -79,7 +91,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()``. @@ -100,11 +112,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__], ) @@ -119,7 +135,7 @@ def test_started_misuse(target): @tractor.context async def simple_setup_teardown( - ctx: tractor.Context, + ctx: Context, data: int, block_forever: bool = False, @@ -129,6 +145,8 @@ async def simple_setup_teardown( global _state _state = True + assert current_ipc_ctx() is ctx + # signal to parent that we're up await ctx.started(data + 1) @@ -165,6 +183,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 @@ -172,20 +191,23 @@ 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__], ) 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 current_ipc_ctx() is ctx assert sent == 11 if callee_blocks_forever: @@ -193,9 +215,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 +227,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 @@ -226,10 +250,10 @@ def test_simple_context( trio.run(main) except error_parent: pass - except trio.MultiError as me: + except BaseExceptionGroup as beg: # XXX: on windows it seems we may have to expect the group error from tractor._exceptions import is_multi_cancelled - assert is_multi_cancelled(me) + assert is_multi_cancelled(beg) else: trio.run(main) @@ -253,6 +277,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) @@ -261,37 +286,100 @@ def test_caller_cancels( ''' async def check_canceller( - ctx: tractor.Context, + 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 + _ctxc: ContextCancelled|None = None + if ( + cancel_method == 'portal' + and not callee_returns_early + ): + try: + res = await ctx.result() + 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 + + # NOTE: should not ever raise even in the `ctx` + # case since self-cancellation should swallow the ctxc + # silently! + else: + 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: 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) == - tractor.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 + + # TODO: don't need this right? + # if _ctxc: + # raise _ctxc + 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__], ) - 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 portal.open_context( - simple_setup_teardown, - data=10, - block_forever=not callee_returns_early, - ) as (ctx, sent): + async with ( + expect_ctxc( + yay=( + not callee_returns_early + and 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 @@ -300,10 +388,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) @@ -313,6 +409,23 @@ def test_caller_cancels( if cancel_method != 'portal': await portal.cancel_actor() + # 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' + ): + assert not ctx._scope.cancelled_caught + trio.run(main) @@ -331,7 +444,7 @@ def test_caller_cancels( @tractor.context async def close_ctx_immediately( - ctx: tractor.Context, + ctx: Context, ) -> None: @@ -343,17 +456,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, @@ -361,10 +490,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`` @@ -372,12 +500,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 @@ -397,8 +528,7 @@ async def test_callee_closes_ctx_after_stream_open(): @tractor.context async def expect_cancelled( - - ctx: tractor.Context, + ctx: Context, ) -> None: global _state @@ -412,12 +542,29 @@ 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 else: - assert 0, "Wasn't cancelled!?" + assert 0, "callee wasn't cancelled !?" @pytest.mark.parametrize( @@ -427,12 +574,18 @@ 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' + ''' + caller context closes without using/opening stream - async with tractor.open_nursery() as n: + ''' + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: - portal = await n.start_actor( + root: Actor = current_actor() + portal = await an.start_actor( 'ctx_cancelled', enable_modules=[__name__], ) @@ -440,22 +593,37 @@ 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 - # call cancel explicitly - if use_ctx_cancel_method: + await portal.run(assert_state, value=True) + # 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: 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?" @@ -464,7 +632,10 @@ 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: @@ -472,7 +643,25 @@ 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() + + # NOTE: local scope should have absorbed the cancellation since + # in this case we call `ctx.cancel()` and the local + # `._scope` does not get `.cancel_called` and thus + # `.cancelled_caught` neither will ever bet set. + if use_ctx_cancel_method: + assert not 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: async for msg in stream: @@ -494,11 +683,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__], ) @@ -545,25 +736,31 @@ async def test_multitask_caller_cancels_from_nonroot_task(): @tractor.context async def cancel_self( - ctx: tractor.Context, + ctx: Context, ) -> None: 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,17 +771,22 @@ 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(): +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__], ) @@ -598,10 +800,12 @@ async def test_callee_cancels_before_started(): # raises a special cancel signal except tractor.ContextCancelled as ce: - ce.type == trio.Cancelled + _ce = ce # for debug on crash + ce.boxed_type == trio.Cancelled # the traceback should be informative - assert 'cancelled itself' in ce.msgdata['tb_str'] + assert 'itself' in ce.tb_str + assert ce.tb_str == ce.msgdata['tb_str'] # teardown the actor await portal.cancel_actor() @@ -610,7 +814,7 @@ async def test_callee_cancels_before_started(): @tractor.context async def never_open_stream( - ctx: tractor.Context, + ctx: Context, ) -> None: ''' @@ -624,8 +828,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: ''' @@ -648,9 +852,15 @@ async def keep_sending_from_callee( ('caller', 1, never_open_stream), ('callee', 0, keep_sending_from_callee), ], - ids='overrun_condition={}'.format, + ids=[ + ('caller_1buf_never_open_stream'), + ('callee_0buf_keep_sending_from_callee'), + ] ) -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 @@ -661,35 +871,38 @@ 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__], ) - async with portal.open_context( - entrypoint, - ) as (ctx, sent): - assert sent is None + with trio.fail_after(1): + async with portal.open_context( + entrypoint, + ) as (ctx, sent): + assert sent is None - if 'caller' in overrunner: + if 'caller' in overrunner: - async with ctx.open_stream() as stream: + 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) + # 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) - else: - # expect overrun error to be relayed back - # and this sleep interrupted - await trio.sleep_forever() + else: + # expect overrun error to be relayed back + # and this sleep interrupted + await trio.sleep_forever() - else: - # callee overruns caller case so we do nothing here - await trio.sleep_forever() + else: + # callee overruns caller case so we do nothing here + await trio.sleep_forever() await portal.cancel_actor() @@ -701,7 +914,7 @@ def test_one_end_stream_not_opened(overrun_by): with pytest.raises(tractor.RemoteActorError) as excinfo: trio.run(main) - assert excinfo.value.type == StreamOverrun + assert excinfo.value.boxed_type == StreamOverrun elif overrunner == 'callee': with pytest.raises(tractor.RemoteActorError) as excinfo: @@ -710,7 +923,7 @@ def test_one_end_stream_not_opened(overrun_by): # TODO: embedded remote errors so that we can verify the source # error? the callee delivers an error which is an overrun # wrapped in a remote actor error. - assert excinfo.value.type == tractor.RemoteActorError + assert excinfo.value.boxed_type == tractor.RemoteActorError else: trio.run(main) @@ -719,7 +932,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, @@ -736,7 +949,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() @@ -755,8 +971,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: @@ -767,15 +998,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.cancel_called_remote}' - ) - return 'yo' + try: + return 'yo' + finally: + print( + 'exiting callee with context:\n' + f'{pformat(ctx)}\n' + ) @pytest.mark.parametrize( @@ -801,7 +1035,10 @@ def test_maybe_allow_overruns_stream( cancel_ctx: bool, slow_side: str, allow_overruns_side: str, + + # conftest wide loglevel: str, + debug_mode: bool, ): ''' Demonstrate small overruns of each task back and forth @@ -820,62 +1057,72 @@ 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( - 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 + # stream-sequence batch info with send delay to determine + # approx timeout determining whether test has hung. + total_batches: int = 2 + num_items: int = 10 + seq = list(range(num_items)) + parent_send_delay: float = 0.16 + timeout: float = math.ceil( + total_batches * num_items * parent_send_delay + ) + with trio.fail_after(timeout): + 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, - async with ctx.open_stream( - msg_buffer_size=1 if slow_side == 'parent' else None, - allow_overruns=(allow_overruns_side in {'parent', 'both'}), - ) as stream: + ) as (ctx, sent): + assert sent is None - total_batches: int = 2 - for _ in range(total_batches): - for msg in seq: - # print(f'root tx {msg}') - await stream.send(msg) - 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) + async with ctx.open_stream( + msg_buffer_size=1 if slow_side == 'parent' else None, + allow_overruns=(allow_overruns_side in {'parent', 'both'}), + ) as stream: - batch = [] - async for msg in stream: - print(f'root rx {msg}') - batch.append(msg) - if batch == seq: - break + for _ in range(total_batches): + for msg in seq: + # print(f'root tx {msg}') + await stream.send(msg) + 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(parent_send_delay) + + batch = [] + async for msg in stream: + print(f'root rx {msg}') + batch.append(msg) + if batch == seq: + break + + if cancel_ctx: + # cancel the remote task + print('Requesting `ctx.cancel()` in parent!') + await ctx.cancel() + + res: str|ContextCancelled = await ctx.result() if cancel_ctx: - # cancel the remote task - print('sending root side cancel') - await ctx.cancel() + assert isinstance(res, ContextCancelled) + assert tuple(res.canceller) == current_actor().uid - res = await ctx.result() - - 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' + else: + print(f'RX ROOT SIDE RESULT {res}') + assert res == 'yo' # cancel the daemon await portal.cancel_actor() @@ -904,7 +1151,7 @@ def test_maybe_allow_overruns_stream( # 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 ( + assert err.boxed_type in ( tractor.RemoteActorError, StreamOverrun, ) @@ -912,11 +1159,12 @@ def test_maybe_allow_overruns_stream( elif ( slow_side == 'child' ): - assert err.type == StreamOverrun + assert err.boxed_type == StreamOverrun elif slow_side == 'parent': - assert err.type == tractor.RemoteActorError - assert 'StreamOverrun' in err.msgdata['tb_str'] + assert err.boxed_type == tractor.RemoteActorError + assert 'StreamOverrun' in err.tb_str + assert err.tb_str == err.msgdata['tb_str'] else: # if this hits the logic blocks from above are not @@ -924,91 +1172,50 @@ def test_maybe_allow_overruns_stream( 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(): +def test_ctx_with_self_actor( + loglevel: str, + debug_mode: bool, +): ''' - 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. + NOTE: for now this is an INVALID OP! - ''' - 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() + BUT, eventually presuming we add a "side" key to `Actor.get_context()`, + we might be able to get this working symmetrically, but should we?? - -@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. + 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() as n: - portal = await n.start_actor( - 'errorer', - enable_modules=[__name__], - ) - await n.start_actor( - 'sleeper', - enable_modules=[__name__], - ) - + 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( - error_before_started - ) as (ctx, sent), - ): - await trio.sleep_forever() + expect_cancelled, + # echo_back_sequence, + # seq=seq, + # wait_for_cancel=cancel_ctx, + # be_slow=(slow_side == 'child'), + # allow_overruns_side=allow_overruns_side, - with pytest.raises(tractor.RemoteActorError) as excinfo: + ) 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 excinfo.value.type == TypeError + assert 'Invalid Operation' in repr(excinfo.value) diff --git a/tests/test_debugger.py b/tests/test_debugger.py index a44a3138..43dadbb0 100644 --- a/tests/test_debugger.py +++ b/tests/test_debugger.py @@ -10,12 +10,10 @@ TODO: - wonder if any of it'll work on OS X? """ +from functools import partial import itertools -from os import path -from typing import Optional import platform import pathlib -import sys import time import pytest @@ -25,8 +23,15 @@ from pexpect.exceptions import ( EOF, ) -from conftest import ( +from tractor.devx._debug import ( + _pause_msg, + _crash_msg, + _repl_fail_msg, +) +from tractor._testing import ( examples_dir, +) +from conftest import ( _ci_env, ) @@ -78,7 +83,7 @@ has_nested_actors = pytest.mark.has_nested_actors def spawn( start_method, testdir, - arb_addr, + reg_addr, ) -> 'pexpect.spawn': if start_method != 'trio': @@ -123,20 +128,58 @@ def expect( raise +def in_prompt_msg( + prompt: str, + parts: list[str], + + pause_on_false: bool = False, + print_prompt_on_false: bool = True, + +) -> bool: + ''' + Predicate check if (the prompt's) std-streams output has all + `str`-parts in it. + + Can be used in test asserts for bulk matching expected + log/REPL output for a given `pdb` interact point. + + ''' + __tracebackhide__: bool = False + + for part in parts: + if part not in prompt: + if pause_on_false: + import pdbp + pdbp.set_trace() + + if print_prompt_on_false: + print(prompt) + + return False + + return True + + +# TODO: todo support terminal color-chars stripping so we can match +# against call stack frame output from the the 'll' command the like! +# -[ ] SO answer for stipping ANSI codes: https://stackoverflow.com/a/14693789 def assert_before( child, patts: list[str], + **kwargs, + ) -> None: + __tracebackhide__: bool = False - before = str(child.before.decode()) + # as in before the prompt end + before: str = str(child.before.decode()) + assert in_prompt_msg( + prompt=before, + parts=patts, - for patt in patts: - try: - assert patt in before - except AssertionError: - print(before) - raise + **kwargs + ) @pytest.fixture( @@ -166,7 +209,7 @@ def ctlc( # XXX: disable pygments highlighting for auto-tests # since some envs (like actions CI) will struggle # the the added color-char encoding.. - from tractor._debug import TractorConfig + from tractor.devx._debug import TractorConfig TractorConfig.use_pygements = False yield use_ctlc @@ -180,7 +223,10 @@ def ctlc( ], ids=lambda item: f'{item[0]} -> {item[1]}', ) -def test_root_actor_error(spawn, user_in_out): +def test_root_actor_error( + spawn, + user_in_out, +): ''' Demonstrate crash handler entering pdb from basic error in root actor. @@ -195,7 +241,10 @@ def test_root_actor_error(spawn, user_in_out): before = str(child.before.decode()) # make sure expected logging and error arrives - assert "Attaching to pdb in crashed actor: ('root'" in before + assert in_prompt_msg( + before, + [_crash_msg, "('root'"] + ) assert 'AssertionError' in before # send user command @@ -242,7 +291,7 @@ def do_ctlc( child, count: int = 3, delay: float = 0.1, - patt: Optional[str] = None, + patt: str|None = None, # expect repl UX to reprint the prompt after every # ctrl-c send. @@ -332,7 +381,10 @@ def test_subactor_error( child.expect(PROMPT) before = str(child.before.decode()) - assert "Attaching to pdb in crashed actor: ('name_error'" in before + assert in_prompt_msg( + before, + [_crash_msg, "('name_error'"] + ) if do_next: child.sendline('n') @@ -353,9 +405,15 @@ def test_subactor_error( before = str(child.before.decode()) # root actor gets debugger engaged - assert "Attaching to pdb in crashed actor: ('root'" in before + assert in_prompt_msg( + before, + [_crash_msg, "('root'"] + ) # error is a remote error propagated from the subactor - assert "RemoteActorError: ('name_error'" in before + assert in_prompt_msg( + before, + [_crash_msg, "('name_error'"] + ) # another round if ctlc: @@ -380,7 +438,10 @@ def test_subactor_breakpoint( child.expect(PROMPT) before = str(child.before.decode()) - assert "Attaching pdb to actor: ('breakpoint_forever'" in before + assert in_prompt_msg( + before, + [_pause_msg, "('breakpoint_forever'"] + ) # do some "next" commands to demonstrate recurrent breakpoint # entries @@ -396,7 +457,10 @@ def test_subactor_breakpoint( child.sendline('continue') child.expect(PROMPT) before = str(child.before.decode()) - assert "Attaching pdb to actor: ('breakpoint_forever'" in before + assert in_prompt_msg( + before, + [_pause_msg, "('breakpoint_forever'"] + ) if ctlc: do_ctlc(child) @@ -408,8 +472,12 @@ def test_subactor_breakpoint( child.expect(PROMPT) before = str(child.before.decode()) - assert "RemoteActorError: ('breakpoint_forever'" in before - assert 'bdb.BdbQuit' in before + assert in_prompt_msg( + before, + ['RemoteActorError:', + "('breakpoint_forever'", + 'bdb.BdbQuit',] + ) if ctlc: do_ctlc(child) @@ -421,8 +489,12 @@ def test_subactor_breakpoint( child.expect(pexpect.EOF) before = str(child.before.decode()) - assert "RemoteActorError: ('breakpoint_forever'" in before - assert 'bdb.BdbQuit' in before + assert in_prompt_msg( + before, + ['RemoteActorError:', + "('breakpoint_forever'", + 'bdb.BdbQuit',] + ) @has_nested_actors @@ -441,7 +513,10 @@ def test_multi_subactors( child.expect(PROMPT) before = str(child.before.decode()) - assert "Attaching pdb to actor: ('breakpoint_forever'" in before + assert in_prompt_msg( + before, + [_pause_msg, "('breakpoint_forever'"] + ) if ctlc: do_ctlc(child) @@ -461,7 +536,10 @@ def test_multi_subactors( # first name_error failure child.expect(PROMPT) before = str(child.before.decode()) - assert "Attaching to pdb in crashed actor: ('name_error'" in before + assert in_prompt_msg( + before, + [_crash_msg, "('name_error'"] + ) assert "NameError" in before if ctlc: @@ -487,7 +565,10 @@ def test_multi_subactors( child.sendline('c') child.expect(PROMPT) before = str(child.before.decode()) - assert "Attaching pdb to actor: ('breakpoint_forever'" in before + assert in_prompt_msg( + before, + [_pause_msg, "('breakpoint_forever'"] + ) if ctlc: do_ctlc(child) @@ -527,17 +608,21 @@ def test_multi_subactors( child.expect(PROMPT) before = str(child.before.decode()) - assert_before(child, [ - # debugger attaches to root - "Attaching to pdb in crashed actor: ('root'", + assert_before( + child, [ + # debugger attaches to root + # "Attaching to pdb in crashed actor: ('root'", + _crash_msg, + "('root'", - # expect a multierror with exceptions for each sub-actor - "RemoteActorError: ('breakpoint_forever'", - "RemoteActorError: ('name_error'", - "RemoteActorError: ('spawn_error'", - "RemoteActorError: ('name_error_1'", - 'bdb.BdbQuit', - ]) + # expect a multierror with exceptions for each sub-actor + "RemoteActorError: ('breakpoint_forever'", + "RemoteActorError: ('name_error'", + "RemoteActorError: ('spawn_error'", + "RemoteActorError: ('name_error_1'", + 'bdb.BdbQuit', + ] + ) if ctlc: do_ctlc(child) @@ -574,15 +659,22 @@ def test_multi_daemon_subactors( # the root's tty lock first so anticipate either crash # message on the first entry. - bp_forever_msg = "Attaching pdb to actor: ('bp_forever'" + bp_forev_parts = [_pause_msg, "('bp_forever'"] + bp_forev_in_msg = partial( + in_prompt_msg, + parts=bp_forev_parts, + ) + name_error_msg = "NameError: name 'doggypants' is not defined" + name_error_parts = [name_error_msg] before = str(child.before.decode()) - if bp_forever_msg in before: - next_msg = name_error_msg + + if bp_forev_in_msg(prompt=before): + next_parts = name_error_parts elif name_error_msg in before: - next_msg = bp_forever_msg + next_parts = bp_forev_parts else: raise ValueError("Neither log msg was found !?") @@ -599,7 +691,10 @@ def test_multi_daemon_subactors( child.sendline('c') child.expect(PROMPT) - assert_before(child, [next_msg]) + assert_before( + child, + next_parts, + ) # XXX: hooray the root clobbering the child here was fixed! # IMO, this demonstrates the true power of SC system design. @@ -607,7 +702,7 @@ def test_multi_daemon_subactors( # now the root actor won't clobber the bp_forever child # during it's first access to the debug lock, but will instead # wait for the lock to release, by the edge triggered - # ``_debug.Lock.no_remote_has_tty`` event before sending cancel messages + # ``devx._debug.Lock.no_remote_has_tty`` event before sending cancel messages # (via portals) to its underlings B) # at some point here there should have been some warning msg from @@ -623,9 +718,15 @@ def test_multi_daemon_subactors( child.expect(PROMPT) try: - assert_before(child, [bp_forever_msg]) + assert_before( + child, + bp_forev_parts, + ) except AssertionError: - assert_before(child, [name_error_msg]) + assert_before( + child, + name_error_parts, + ) else: if ctlc: @@ -637,7 +738,10 @@ def test_multi_daemon_subactors( child.sendline('c') child.expect(PROMPT) - assert_before(child, [name_error_msg]) + assert_before( + child, + name_error_parts, + ) # wait for final error in root # where it crashs with boxed error @@ -647,7 +751,7 @@ def test_multi_daemon_subactors( child.expect(PROMPT) assert_before( child, - [bp_forever_msg] + bp_forev_parts ) except AssertionError: break @@ -656,8 +760,11 @@ def test_multi_daemon_subactors( child, [ # boxed error raised in root task - "Attaching to pdb in crashed actor: ('root'", - "_exceptions.RemoteActorError: ('name_error'", + # "Attaching to pdb in crashed actor: ('root'", + _crash_msg, + "('root'", # should attach in root + "_exceptions.RemoteActorError:", # with an embedded RAE for.. + "('name_error'", # the src subactor which raised ] ) @@ -758,10 +865,11 @@ def test_multi_nested_subactors_error_through_nurseries( # https://github.com/goodboy/tractor/issues/320 # ctlc: bool, ): - """Verify deeply nested actors that error trigger debugger entries + ''' + Verify deeply nested actors that error trigger debugger entries at each actor nurserly (level) all the way up the tree. - """ + ''' # NOTE: previously, inside this script was a bug where if the # parent errors before a 2-levels-lower actor has released the lock, # the parent tries to cancel it but it's stuck in the debugger? @@ -770,7 +878,7 @@ def test_multi_nested_subactors_error_through_nurseries( child = spawn('multi_nested_subactors_error_up_through_nurseries') - timed_out_early: bool = False + # timed_out_early: bool = False for send_char in itertools.cycle(['c', 'q']): try: @@ -781,22 +889,31 @@ def test_multi_nested_subactors_error_through_nurseries( except EOF: break - assert_before(child, [ + assert_before( + child, + [ # boxed source errors + "NameError: name 'doggypants' is not defined", + "tractor._exceptions.RemoteActorError:", + "('name_error'", + "bdb.BdbQuit", - # boxed source errors - "NameError: name 'doggypants' is not defined", - "tractor._exceptions.RemoteActorError: ('name_error'", - "bdb.BdbQuit", + # first level subtrees + # "tractor._exceptions.RemoteActorError: ('spawner0'", + "src_uid=('spawner0'", - # first level subtrees - "tractor._exceptions.RemoteActorError: ('spawner0'", - # "tractor._exceptions.RemoteActorError: ('spawner1'", + # "tractor._exceptions.RemoteActorError: ('spawner1'", - # propagation of errors up through nested subtrees - "tractor._exceptions.RemoteActorError: ('spawn_until_0'", - "tractor._exceptions.RemoteActorError: ('spawn_until_1'", - "tractor._exceptions.RemoteActorError: ('spawn_until_2'", - ]) + # propagation of errors up through nested subtrees + # "tractor._exceptions.RemoteActorError: ('spawn_until_0'", + # "tractor._exceptions.RemoteActorError: ('spawn_until_1'", + # "tractor._exceptions.RemoteActorError: ('spawn_until_2'", + # ^-NOTE-^ old RAE repr, new one is below with a field + # showing the src actor's uid. + "src_uid=('spawn_until_0'", + "relay_uid=('spawn_until_1'", + "src_uid=('spawn_until_2'", + ] + ) @pytest.mark.timeout(15) @@ -871,11 +988,14 @@ def test_root_nursery_cancels_before_child_releases_tty_lock( if not timed_out_early: before = str(child.before.decode()) - assert_before(child, [ - "tractor._exceptions.RemoteActorError: ('spawner0'", - "tractor._exceptions.RemoteActorError: ('name_error'", - "NameError: name 'doggypants' is not defined", - ]) + assert_before( + child, + [ + "tractor._exceptions.RemoteActorError: ('spawner0'", + "tractor._exceptions.RemoteActorError: ('name_error'", + "NameError: name 'doggypants' is not defined", + ], + ) def test_root_cancels_child_context_during_startup( @@ -909,8 +1029,10 @@ def test_different_debug_mode_per_actor( # only one actor should enter the debugger before = str(child.before.decode()) - assert "Attaching to pdb in crashed actor: ('debugged_boi'" in before - assert "RuntimeError" in before + assert in_prompt_msg( + before, + [_crash_msg, "('debugged_boi'", "RuntimeError"], + ) if ctlc: do_ctlc(child) @@ -925,9 +1047,313 @@ def test_different_debug_mode_per_actor( # msg reported back from the debug mode actor is processed. # assert "tractor._exceptions.RemoteActorError: ('debugged_boi'" in before - assert "tractor._exceptions.RemoteActorError: ('crash_boi'" in before - # the crash boi should not have made a debugger request but # instead crashed completely - assert "tractor._exceptions.RemoteActorError: ('crash_boi'" in before - assert "RuntimeError" in before + assert_before( + child, + [ + "tractor._exceptions.RemoteActorError:", + "src_uid=('crash_boi'", + "RuntimeError", + ] + ) + + +def test_pause_from_sync( + spawn, + ctlc: bool +): + ''' + Verify we can use the `pdbp` REPL from sync functions AND from + any thread spawned with `trio.to_thread.run_sync()`. + + `examples/debugging/sync_bp.py` + + ''' + child = spawn('sync_bp') + + # first `sync_pause()` after nurseries open + child.expect(PROMPT) + assert_before( + child, + [ + # pre-prompt line + _pause_msg, + " async with p.open_context(name_error) as (ctx, first):', + # ] + # ) + + # # step up a frame to ensure the it's the root's nursery + # child.sendline('u') + # child.expect(PROMPT) + # assert_before( + # child, + # [ + # # handler block annotation + # '-> async with tractor.open_nursery(', + # ] + # ) + + child.sendline('c') + child.expect(pexpect.EOF) + + +def test_shield_pause( + spawn, +): + ''' + Verify the `tractor.pause()/.post_mortem()` API works inside an + already cancelled `trio.CancelScope` and that you can step to the + next checkpoint wherein the cancelled will get raised. + + ''' + child = spawn('shielded_pause') + + # First entry is via manual `.post_mortem()` + child.expect(PROMPT) + assert_before( + child, + [ + _pause_msg, + "cancellable_pause_loop'", + "('cancelled_before_pause'", # actor name + ] + ) + + # since 3 tries in ex. shield pause loop + for i in range(3): + child.sendline('c') + child.expect(PROMPT) + assert_before( + child, + [ + _pause_msg, + "INSIDE SHIELDED PAUSE", + "('cancelled_before_pause'", # actor name + ] + ) + + # back inside parent task that opened nursery + child.sendline('c') + child.expect(PROMPT) + assert_before( + child, + [ + _crash_msg, + "('cancelled_before_pause'", # actor name + _repl_fail_msg, + "trio.Cancelled", + "raise Cancelled._create()", + + # we should be handling a taskc inside + # the first `.port_mortem()` sin-shield! + 'await DebugStatus.req_finished.wait()', + ] + ) + + # same as above but in the root actor's task + child.sendline('c') + child.expect(PROMPT) + assert_before( + child, + [ + _crash_msg, + "('root'", # actor name + _repl_fail_msg, + "trio.Cancelled", + "raise Cancelled._create()", + + # handling a taskc inside the first unshielded + # `.port_mortem()`. + # BUT in this case in the root-proc path ;) + 'wait Lock._debug_lock.acquire()', + ] + ) + child.sendline('c') + child.expect(pexpect.EOF) + + +# TODO: needs ANSI code stripping tho, see `assert_before()` # above! +def test_correct_frames_below_hidden(): + ''' + Ensure that once a `tractor.pause()` enages, when the user + inputs a "next"/"n" command the actual next line steps + and that using a "step"/"s" into the next LOC, particuarly + `tractor` APIs, you can step down into that code. + + ''' + ... + + +def test_cant_pause_from_paused_task(): + ''' + Pausing from with an already paused task should raise an error. + + Normally this should only happen in practise while debugging the call stack of `tractor.pause()` itself, likely + by a `.pause()` line somewhere inside our runtime. + + ''' + ... diff --git a/tests/test_discovery.py b/tests/test_discovery.py index 8ba4ebee..cd9dc022 100644 --- a/tests/test_discovery.py +++ b/tests/test_discovery.py @@ -9,25 +9,24 @@ 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(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 +52,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 +80,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 +150,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 +158,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 +232,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 +258,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 +269,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 +283,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 +296,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 +350,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 +360,7 @@ def test_close_channel_explicit( trio.run( partial( close_chans_before_nursery, - arb_addr, + reg_addr, use_signal, remote_arbiter=False, ), @@ -351,7 +372,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 +382,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_docs_examples.py b/tests/test_docs_examples.py index f134c71b..79a22009 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, ) @@ -21,7 +20,7 @@ from conftest import ( def run_example_in_subproc( loglevel: str, testdir, - arb_addr: tuple[str, int], + reg_addr: tuple[str, int], ): @contextmanager diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index dd9d681a..45722a63 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 @@ -18,6 +17,7 @@ from tractor import ( ContextCancelled, ) from tractor.trionics import BroadcastReceiver +from tractor._testing import expect_ctxc async def sleep_and_err( @@ -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, @@ -68,7 +68,7 @@ def test_trio_cancels_aio_on_actor_side(arb_addr): async def asyncio_actor( target: str, - expect_err: Optional[Exception] = None + expect_err: Exception|None = None ) -> None: @@ -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, @@ -112,15 +112,26 @@ def test_aio_simple_error(arb_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 + assert err.boxed_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 +150,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. @@ -190,11 +201,12 @@ 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( - arb_addr, + reg_addr, parent_cancels: bool, ): ''' @@ -214,18 +226,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, @@ -242,7 +272,7 @@ def test_context_spawns_aio_task_that_errors( err = excinfo.value assert isinstance(err, expect) - assert err.type == AssertionError + assert err.boxed_type == AssertionError async def aio_cancel(): @@ -258,7 +288,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: @@ -269,11 +299,22 @@ def test_aio_cancelled_from_aio_causes_trio_cancelled(arb_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.boxed_type == to_asyncio.AsyncioCancelled # TODO: verify open_channel_from will fail on this.. @@ -395,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(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( @@ -403,13 +444,14 @@ def test_basic_interloop_channel_stream(arb_addr, fan_out): infect_asyncio=True, fan_out=fan_out, ) + # should raise RAE diectly await portal.result() trio.run(main) # 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( @@ -420,15 +462,14 @@ def test_trio_error_cancels_intertask_chan(arb_addr): # should trigger remote actor error await portal.result() - with pytest.raises(BaseExceptionGroup) as excinfo: + with pytest.raises(RemoteActorError) as excinfo: trio.run(main) - # ensure boxed errors - for exc in excinfo.value.exceptions: - assert exc.type == Exception + # ensure boxed error type + excinfo.value.boxed_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( @@ -436,14 +477,14 @@ def test_trio_closes_early_and_channel_exits(arb_addr): exit_early=True, infect_asyncio=True, ) - # should trigger remote actor error + # should raise RAE diectly await portal.result() # should be a quiet exit on a simple channel exit 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( @@ -451,15 +492,17 @@ def test_aio_errors_and_channel_propagates_and_closes(arb_addr): aio_raise_err=True, infect_asyncio=True, ) - # should trigger remote actor error + # should trigger RAE directly, not an eg. await portal.result() - with pytest.raises(BaseExceptionGroup) as excinfo: + with pytest.raises( + # NOTE: bc we directly wait on `Portal.result()` instead + # of capturing it inside the `ActorNursery` machinery. + expected_exception=RemoteActorError, + ) as excinfo: trio.run(main) - # ensure boxed errors - for exc in excinfo.value.exceptions: - assert exc.type == Exception + excinfo.value.boxed_type == Exception @tractor.context @@ -520,7 +563,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, ): @@ -560,7 +603,8 @@ def test_echoserver_detailed_mechanics( pass else: pytest.fail( - "stream wasn't stopped after sentinel?!") + 'stream not stopped after sentinel ?!' + ) # TODO: the case where this blocks and # is cancelled by kbi or out of task cancellation @@ -572,3 +616,37 @@ def test_echoserver_detailed_mechanics( else: trio.run(main) + + +# TODO: debug_mode tests once we get support for `asyncio`! +# +# -[ ] need tests to wrap both scripts: +# - [ ] infected_asyncio_echo_server.py +# - [ ] debugging/asyncio_bp.py +# -[ ] consider moving ^ (some of) these ^ to `test_debugger`? +# +# -[ ] missing impl outstanding includes: +# - [x] for sync pauses we need to ensure we open yet another +# `greenback` portal in the asyncio task +# => completed using `.bestow_portal(task)` inside +# `.to_asyncio._run_asyncio_task()` right? +# -[ ] translation func to get from `asyncio` task calling to +# `._debug.wait_for_parent_stdin_hijack()` which does root +# call to do TTY locking. +# +def test_sync_breakpoint(): + ''' + Verify we can do sync-func/code breakpointing using the + `breakpoint()` builtin inside infected mode actors. + + ''' + pytest.xfail('This support is not implemented yet!') + + +def test_debug_mode_crash_handling(): + ''' + Verify mult-actor crash handling works with a combo of infected-`asyncio`-mode + and normal `trio` actors despite nested process trees. + + ''' + pytest.xfail('This support is not implemented yet!') diff --git a/tests/test_inter_peer_cancellation.py b/tests/test_inter_peer_cancellation.py new file mode 100644 index 00000000..7bf9a2bd --- /dev/null +++ b/tests/test_inter_peer_cancellation.py @@ -0,0 +1,1251 @@ +''' +Codify the cancellation request semantics in terms +of one remote actor cancelling another. + +''' +# from contextlib import asynccontextmanager as acm +import itertools + +import pytest +import trio +import tractor +from tractor import ( # typing + Actor, + current_actor, + open_nursery, + Portal, + Context, + ContextCancelled, + RemoteActorError, +) +from tractor._testing import ( + # tractor_test, + expect_ctxc, +) + +# 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(): +# ''' +# 2 cases: +# - calls `Actor.cancel()` locally in some task +# - calls LocalPortal.cancel_actor()` ? + +# ''' +# ... + + +@tractor.context +async def open_stream_then_sleep_forever( + ctx: Context, + expect_ctxc: bool = False, + +) -> None: + ''' + Sync the context, open a stream then just sleep. + + Allow checking for (context) cancellation locally. + + ''' + try: + await ctx.started() + + # NOTE: the below means this child will send a `Stop` + # to it's parent-side task despite that side never + # opening a stream itself. + 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 +async def error_before_started( + ctx: Context, +) -> None: + ''' + 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: + async with ( + p2.open_context(open_stream_then_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: + # 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 + # 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() + + +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 + override the original error that caused the cancellation of the + secondary context. + + ''' + async def main(): + async with tractor.open_nursery( + debug_mode=debug_mode, + ) 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(RemoteActorError) as excinfo: + trio.run(main) + + rae = excinfo.value + assert rae.boxed_type == TypeError + + +@tractor.context +async def sleep_a_bit_then_cancel_peer( + ctx: Context, + peer_name: str = 'sleeper', + cancel_after: float = .5, + +) -> None: + ''' + Connect to peer, sleep as per input delay, cancel the peer. + + ''' + peer: Portal + async with tractor.wait_for_actor(peer_name) as peer: + await ctx.started() + await trio.sleep(cancel_after) + await peer.cancel_actor() + + # such that we're cancelled by our rent ctx-task + await trio.sleep(3) + print('CANCELLER RETURNING!') + + +@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) + await trio.sleep(0.01) + + +@tractor.context +async def stream_from_peer( + ctx: Context, + debug_mode: bool, + peer_name: str = 'sleeper', +) -> None: + + # sanity + assert tractor._state.debug_mode() == debug_mode + + 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 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 ctxc 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 actor. + except ContextCancelled as _ctxc: + ctxc = _ctxc + + # print("TRYING TO ENTER PAUSSE!!!") + # await tractor.pause(shield=True) + re: ContextCancelled = peer_ctx._remote_error + + # XXX YES XXX, remote error should be unpacked only once! + assert ( + re + is + peer_ctx.maybe_error + is + ctxc + is + peer_ctx._local_error + ) + # NOTE: these errors should all match! + # ------ - ------ + # XXX [2024-05-03] XXX + # ------ - ------ + # broke this due to a re-raise inside `.msg._ops.drain_to_final_msg()` + # where the `Error()` msg was directly raising the ctxc + # instead of just returning up to the caller inside + # `Context.return()` which would results in a diff instance of + # the same remote error bubbling out above vs what was + # already unpacked and set inside `Context. + assert ( + peer_ctx._remote_error.msgdata + == + ctxc.msgdata + ) + # ^-XXX-^ notice the data is of course the exact same.. so + # the above larger assert makes sense to also always be true! + + # XXX YES XXX, bc should be exact same msg instances + assert peer_ctx._remote_error._ipc_msg is ctxc._ipc_msg + + # XXX NO XXX, bc new one always created for property accesss + assert peer_ctx._remote_error.ipc_msg != ctxc.ipc_msg + + # 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 ctxc.canceller + + # 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 + # sets this AFTER the nursery block this task + # was started in, exits. + assert not ctx._scope.cancelled_caught + + # 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 not peer_ctx._scope.cancelled_caught + + # TODO / NOTE `.canceller` won't have been set yet + # here because that machinery is inside + # `Portal.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 + # + # await tractor.pause() + # assert 'canceller' in ctx.canceller + + # root/parent actor task should NEVER HAVE cancelled us! + assert not ctx.canceller + + raise + # 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 RuntimeError('Never triggered local `ContextCancelled` ?!?') + + +@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, + debug_mode: 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()` + + ''' + async def main(): + async with tractor.open_nursery( + # NOTE: to halt the peer tasks on ctxc, uncomment this. + debug_mode=debug_mode, + ) as an: + canceller: Portal = await an.start_actor( + 'canceller', + enable_modules=[__name__], + ) + 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__], + ) + root: Actor = current_actor() + + try: + async with ( + sleeper.open_context( + open_stream_then_sleep_forever, + expect_ctxc=True, + ) as (sleeper_ctx, sent), + + just_caller.open_context( + stream_from_peer, + debug_mode=debug_mode, + ) as (caller_ctx, sent), + + canceller.open_context( + sleep_a_bit_then_cancel_peer, + ) as (canceller_ctx, sent), + + ): + ctxs: dict[str, Context] = { + 'sleeper': sleeper_ctx, + 'caller': caller_ctx, + 'canceller': canceller_ctx, + } + + try: + print('PRE CONTEXT RESULT') + res = await sleeper_ctx.result() + assert res + + # should never get here + pytest.fail( + 'Context.result() did not raise ctx-cancelled?' + ) + + # should always raise since this root task does + # not request the sleeper cancellation ;) + except ContextCancelled as _ctxc: + ctxc = _ctxc + print( + 'CAUGHT REMOTE CONTEXT CANCEL\n\n' + f'{ctxc}\n' + ) + + # canceller and caller peers should not + # have been remotely cancelled. + 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 ctxc.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._scope.cancelled_caught + + # CASE_1: error-during-ctxc-handling, + if error_during_ctxerr_handling: + raise RuntimeError('Simulated RTE re-raise during ctxc handling') + + # CASE_2: standard teardown inside in `.open_context()` block + raise + + # XXX SHOULD NEVER EVER GET HERE XXX + except BaseException as berr: + 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 ctxc ?!?\n\n' + f'{ctxs}\n' + ) + + except ( + ContextCancelled, + RuntimeError, + )as loc_err: + _loc_err = loc_err + + # NOTE: the main state to check on `Context` is: + # - `.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_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: + print(f'loc_err: {_loc_err}\n') + assert isinstance(loc_err, RuntimeError) + + # 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 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 + # `.__aexit__()` to every opened ctx. + 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: + assert 'canceller' in re.canceller + assert 'sleeper' in ctx.canceller + + if ctx is canceller_ctx: + assert ( + re.canceller + == + root.uid + ) + + else: # the other 2 ctxs + assert ( + isinstance(re, ContextCancelled) + and ( + 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 + # `.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: + # 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) + + # 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! + final_err = sleeper_ctx.outcome + assert ( + final_err is loc_err + is sleeper_ctx.maybe_error + is sleeper_ctx._remote_error + ) + + for name, ctx in ctxs.items(): + + re: BaseException|None = ctx._remote_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 + 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 + + # 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 not ctx._scope.cancelled_caught + + elif ctx in ( + caller_ctx, + canceller_ctx, + ): + + assert not ctx._remote_error + + # 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? + # + 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._scope.cancelled_caught + + 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 + # 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 + + else: + pytest.fail( + 'Uhh wut ctx is this?\n' + f'{ctx}\n' + ) + + # TODO: do we even need this flag? + # -> each context should have received + # a silently absorbed context cancellation + # in its remote nursery scope. + # assert ctx.chan.uid == ctx.canceller + + # 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 + or + sleeper_ctx.cancel_acked + ) + + # XXX NOTE XXX: and see matching comment above but, + # 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 not 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 + + 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.boxed_type == ContextCancelled + assert excinfo.value.canceller[0] == 'canceller' + + +@tractor.context +async def basic_echo_server( + ctx: Context, + peer_name: str = 'wittle_bruv', + + err_after: int|None = None, + +) -> 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) + + if ( + err_after + and i > err_after + ): + raise RuntimeError( + f'Simulated error in `{peer_name}`' + ) + + +@tractor.context +async def serve_subactors( + ctx: Context, + peer_name: str, + debug_mode: bool, + +) -> None: + async with open_nursery() as an: + + # sanity + assert tractor._state.debug_mode() == debug_mode + + await ctx.started(peer_name) + async with ctx.open_stream() as ipc: + async for msg in ipc: + 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 ipc.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, + debug_mode: bool, + + # 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, + sub_err_after: int|None = None, + +) -> None: + # sanity + if debug_mode: + assert tractor._state.debug_mode() + + # 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, + debug_mode=debug_mode, + ) 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', + err_after=sub_err_after, + ) + + # 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 = '', + err_after: int|None = None, +): + # contact target actor, do a stream dialog. + async with ( + tractor.wait_for_actor( + name=actor_name + ) as lb, + lb.open_context( + basic_echo_server, + + # XXX proxy any delayed err condition + err_after=err_after, + ) as (sub_ctx, first), + + sub_ctx.open_stream() 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'], +) +@pytest.mark.parametrize( + 'raise_sub_spawn_error_after', + [None, 50], +) +def test_peer_spawns_and_cancels_service_subactor( + debug_mode: bool, + raise_client_error: str, + reg_addr: tuple[str, int], + raise_sub_spawn_error_after: int|None, +): + # 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' + + def check_inner_rte(rae: RemoteActorError): + ''' + Validate the little_bro's relayed inception! + + ''' + assert rae.boxed_type is RemoteActorError + assert rae.src_type is RuntimeError + assert 'client' in rae.relay_uid + assert peer_name in rae.src_uid + + async def main(): + async with tractor.open_nursery( + # NOTE: to halt the peer tasks on ctxc, uncomment this. + debug_mode=debug_mode, + registry_addrs=[reg_addr], + ) 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, + debug_mode=debug_mode, + + ) as (spawn_ctx, first), + + client.open_context( + client_req_subactor, + peer_name=peer_name, + debug_mode=debug_mode, + reraise_on_cancel=raise_client_error, + + # trigger for error condition in sub + # during streaming. + sub_err_after=raise_sub_spawn_error_after, + + ) as (client_ctx, client_says), + ): + root: Actor = current_actor() + spawner_uid: tuple = spawn_ctx.chan.uid + 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. + sub: Portal + 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' + ) + + async with expect_ctxc( + yay=raise_sub_spawn_error_after, + reraise=False, + ): + await tell_little_bro( + actor_name=peer_name, + caller='root', + ) + + if not raise_sub_spawn_error_after: + + # signal client to cancel and maybe raise a KBI + await client_ctx.cancel() + print( + '-> root cancelling client,\n' + '-> root checking `client_ctx.result()`,\n' + f'-> checking that sub-spawn {peer_name} is down\n' + ) + + try: + res = await client_ctx.result(hide_tb=False) + + # in remote (relayed inception) error + # case, we should error on the line above! + if raise_sub_spawn_error_after: + pytest.fail( + 'Never rxed proxied `RemoteActorError[RuntimeError]` !?' + ) + + assert isinstance(res, ContextCancelled) + assert client_ctx.cancel_acked + assert res.canceller == root.uid + + except RemoteActorError as rae: + _err = rae + assert raise_sub_spawn_error_after + + # since this is a "relayed error" via the client + # sub-actor, it is expected to be + # a `RemoteActorError` boxing another + # `RemoteActorError` otherwise known as + # an "inception" (from `trio`'s parlance) + # ((or maybe a "Matryoshka" and/or "matron" + # in our own working parlance)) which + # contains the source error from the + # little_bro: a `RuntimeError`. + # + check_inner_rte(rae) + assert rae.relay_uid == client.chan.uid + assert rae.src_uid == sub.chan.uid + + assert not client_ctx.cancel_acked + assert ( + client_ctx.maybe_error + is client_ctx.outcome + is rae + ) + raise + # await tractor.pause() + + else: + assert not raise_sub_spawn_error_after + + # cancelling the spawner sub should + # transitively cancel it's sub, the little + # bruv. + print('root cancelling server/client sub-actors') + await spawn_ctx.cancel() + async with tractor.find_actor( + name=peer_name, + ) as sub: + assert not sub + + # await server.cancel_actor() + + except RemoteActorError as rae: + # XXX more-or-less same as above handler + # this is just making sure the error bubbles out + # of the + _err = rae + assert raise_sub_spawn_error_after + raise + + # 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: + _ctxc = ctxc + print( + f'{root.uid} caught ctxc from ctx with {client_ctx.chan.uid}\n' + f'{repr(ctxc)}\n' + ) + + if not raise_sub_spawn_error_after: + assert ctxc.canceller == root.uid + else: + assert ctxc.canceller == spawner_uid + + assert ctxc is spawn_ctx.outcome + assert ctxc is spawn_ctx.maybe_error + raise + + if raise_sub_spawn_error_after: + pytest.fail( + 'context block(s) in PARENT never raised?!?' + ) + + if not raise_sub_spawn_error_after: + # 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 + + if raise_sub_spawn_error_after: + with pytest.raises(RemoteActorError) as excinfo: + trio.run(main) + + rae: RemoteActorError = excinfo.value + check_inner_rte(rae) + + else: + trio.run(main) diff --git a/tests/test_legacy_one_way_streaming.py b/tests/test_legacy_one_way_streaming.py index 17e94ba3..6092bca7 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(): @@ -38,10 +38,13 @@ async def async_gen_stream(sequence): assert cs.cancelled_caught +# TODO: deprecated either remove entirely +# or re-impl in terms of `MsgStream` one-sides +# wrapper, but at least remove `Portal.open_stream_from()` @tractor.stream async def context_stream( ctx: tractor.Context, - sequence + sequence: list[int], ): for i in sequence: await ctx.send_yield(i) @@ -55,7 +58,7 @@ async def context_stream( async def stream_from_single_subactor( - arb_addr, + reg_addr, start_method, stream_func, ): @@ -64,7 +67,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 +118,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 +228,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 +243,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 +263,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 +283,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..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 @@ -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..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, @@ -28,9 +30,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 +41,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_pldrx_limiting.py b/tests/test_pldrx_limiting.py new file mode 100644 index 00000000..e5ce691a --- /dev/null +++ b/tests/test_pldrx_limiting.py @@ -0,0 +1,362 @@ +''' +Audit sub-sys APIs from `.msg._ops` +mostly for ensuring correct `contextvars` +related settings around IPC contexts. + +''' +from contextlib import ( + asynccontextmanager as acm, +) + +from msgspec import ( + Struct, +) +import pytest +import trio + +import tractor +from tractor import ( + Context, + MsgTypeError, + current_ipc_ctx, + Portal, +) +from tractor.msg import ( + _ops as msgops, + Return, +) +from tractor.msg import ( + _codec, +) +from tractor.msg.types import ( + log, +) + + +class PldMsg( + Struct, + + # TODO: with multiple structs in-spec we need to tag them! + # -[ ] offer a built-in `PldMsg` type to inherit from which takes + # case of these details? + # + # https://jcristharif.com/msgspec/structs.html#tagged-unions + # tag=True, + # tag_field='msg_type', +): + field: str + + +maybe_msg_spec = PldMsg|None + + +@acm +async def maybe_expect_raises( + raises: BaseException|None = None, + ensure_in_message: list[str]|None = None, + post_mortem: bool = False, + timeout: int = 3, +) -> None: + ''' + Async wrapper for ensuring errors propagate from the inner scope. + + ''' + if tractor._state.debug_mode(): + timeout += 999 + + with trio.fail_after(timeout): + try: + yield + except BaseException as _inner_err: + inner_err = _inner_err + # wasn't-expected to error.. + if raises is None: + raise + + else: + assert type(inner_err) is raises + + # maybe check for error txt content + if ensure_in_message: + part: str + err_repr: str = repr(inner_err) + for part in ensure_in_message: + for i, arg in enumerate(inner_err.args): + if part in err_repr: + break + # if part never matches an arg, then we're + # missing a match. + else: + raise ValueError( + 'Failed to find error message content?\n\n' + f'expected: {ensure_in_message!r}\n' + f'part: {part!r}\n\n' + f'{inner_err.args}' + ) + + if post_mortem: + await tractor.post_mortem() + + else: + if raises: + raise RuntimeError( + f'Expected a {raises.__name__!r} to be raised?' + ) + + +@tractor.context( + pld_spec=maybe_msg_spec, +) +async def child( + ctx: Context, + started_value: int|PldMsg|None, + return_value: str|None, + validate_pld_spec: bool, + raise_on_started_mte: bool = True, + +) -> None: + ''' + Call ``Context.started()`` more then once (an error). + + ''' + expect_started_mte: bool = started_value == 10 + + # sanaity check that child RPC context is the current one + curr_ctx: Context = current_ipc_ctx() + assert ctx is curr_ctx + + rx: msgops.PldRx = ctx._pld_rx + curr_pldec: _codec.MsgDec = rx.pld_dec + + ctx_meta: dict = getattr( + child, + '_tractor_context_meta', + None, + ) + if ctx_meta: + assert ( + ctx_meta['pld_spec'] + is curr_pldec.spec + is curr_pldec.pld_spec + ) + + # 2 cases: hdndle send-side and recv-only validation + # - when `raise_on_started_mte == True`, send validate + # - else, parent-recv-side only validation + mte: MsgTypeError|None = None + try: + await ctx.started( + value=started_value, + validate_pld_spec=validate_pld_spec, + ) + + except MsgTypeError as _mte: + mte = _mte + log.exception('started()` raised an MTE!\n') + if not expect_started_mte: + raise RuntimeError( + 'Child-ctx-task SHOULD NOT HAVE raised an MTE for\n\n' + f'{started_value!r}\n' + ) + + boxed_div: str = '------ - ------' + assert boxed_div not in mte._message + assert boxed_div not in mte.tb_str + assert boxed_div not in repr(mte) + assert boxed_div not in str(mte) + mte_repr: str = repr(mte) + for line in mte.message.splitlines(): + assert line in mte_repr + + # since this is a *local error* there should be no + # boxed traceback content! + assert not mte.tb_str + + # propagate to parent? + if raise_on_started_mte: + raise + + # no-send-side-error fallthrough + if ( + validate_pld_spec + and + expect_started_mte + ): + raise RuntimeError( + 'Child-ctx-task SHOULD HAVE raised an MTE for\n\n' + f'{started_value!r}\n' + ) + + assert ( + not expect_started_mte + or + not validate_pld_spec + ) + + # if wait_for_parent_to_cancel: + # ... + # + # ^-TODO-^ logic for diff validation policies on each side: + # + # -[ ] ensure that if we don't validate on the send + # side, that we are eventually error-cancelled by our + # parent due to the bad `Started` payload! + # -[ ] the boxed error should be srced from the parent's + # runtime NOT ours! + # -[ ] we should still error on bad `return_value`s + # despite the parent not yet error-cancelling us? + # |_ how do we want the parent side to look in that + # case? + # -[ ] maybe the equiv of "during handling of the + # above error another occurred" for the case where + # the parent sends a MTE to this child and while + # waiting for the child to terminate it gets back + # the MTE for this case? + # + + # XXX should always fail on recv side since we can't + # really do much else beside terminate and relay the + # msg-type-error from this RPC task ;) + return return_value + + +@pytest.mark.parametrize( + 'return_value', + [ + 'yo', + None, + ], + ids=[ + 'return[invalid-"yo"]', + 'return[valid-None]', + ], +) +@pytest.mark.parametrize( + 'started_value', + [ + 10, + PldMsg(field='yo'), + ], + ids=[ + 'Started[invalid-10]', + 'Started[valid-PldMsg]', + ], +) +@pytest.mark.parametrize( + 'pld_check_started_value', + [ + True, + False, + ], + ids=[ + 'check-started-pld', + 'no-started-pld-validate', + ], +) +def test_basic_payload_spec( + debug_mode: bool, + loglevel: str, + return_value: str|None, + started_value: int|PldMsg, + pld_check_started_value: bool, +): + ''' + Validate the most basic `PldRx` msg-type-spec semantics around + a IPC `Context` endpoint start, started-sync, and final return + value depending on set payload types and the currently applied + pld-spec. + + ''' + invalid_return: bool = return_value == 'yo' + invalid_started: bool = started_value == 10 + + async def main(): + async with tractor.open_nursery( + debug_mode=debug_mode, + loglevel=loglevel, + ) as an: + p: Portal = await an.start_actor( + 'child', + enable_modules=[__name__], + ) + + # since not opened yet. + assert current_ipc_ctx() is None + + if invalid_started: + msg_type_str: str = 'Started' + bad_value_str: str = '10' + elif invalid_return: + msg_type_str: str = 'Return' + bad_value_str: str = "'yo'" + else: + # XXX but should never be used below then.. + msg_type_str: str = '' + bad_value_str: str = '' + + maybe_mte: MsgTypeError|None = None + should_raise: Exception|None = ( + MsgTypeError if ( + invalid_return + or + invalid_started + ) else None + ) + async with ( + maybe_expect_raises( + raises=should_raise, + ensure_in_message=[ + f"invalid `{msg_type_str}` msg payload", + f"value: `{bad_value_str}` does not " + f"match type-spec: `{msg_type_str}.pld: PldMsg|NoneType`", + ], + # only for debug + # post_mortem=True, + ), + p.open_context( + child, + return_value=return_value, + started_value=started_value, + validate_pld_spec=pld_check_started_value, + ) as (ctx, first), + ): + # now opened with 'child' sub + assert current_ipc_ctx() is ctx + + assert type(first) is PldMsg + assert first.field == 'yo' + + try: + res: None|PldMsg = await ctx.result(hide_tb=False) + assert res is None + except MsgTypeError as mte: + maybe_mte = mte + if not invalid_return: + raise + + # expected this invalid `Return.pld` so audit + # the error state + meta-data + assert mte.expected_msg_type is Return + assert mte.cid == ctx.cid + mte_repr: str = repr(mte) + for line in mte.message.splitlines(): + assert line in mte_repr + + assert mte.tb_str + # await tractor.pause(shield=True) + + # verify expected remote mte deats + assert ctx._local_error is None + assert ( + mte is + ctx._remote_error is + ctx.maybe_error is + ctx.outcome + ) + + if should_raise is None: + assert maybe_mte is None + + await p.cancel_actor() + + trio.run(main) diff --git a/tests/test_pubsub.py b/tests/test_pubsub.py index ababcb51..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(): @@ -160,7 +159,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 +169,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 +254,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_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(): diff --git a/tests/test_rpc.py b/tests/test_rpc.py index 6d158961..9581708f 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 @@ -13,9 +15,19 @@ async def sleep_back_actor( func_name, func_defined, exposed_mods, + *, + reg_addr: tuple, ): if actor_name: - async with tractor.find_actor(actor_name) as portal: + async with tractor.find_actor( + actor_name, + # NOTE: must be set manually since + # the subactor doesn't have the reg_addr + # fixture code run in it! + # TODO: maybe we should just set this once in the + # _state mod and derive to all children? + registry_addrs=[reg_addr], + ) as portal: try: await portal.run(__name__, func_name) except tractor.RemoteActorError as err: @@ -24,7 +36,7 @@ async def sleep_back_actor( if not exposed_mods: expect = tractor.ModuleNotExposed - assert err.type is expect + assert err.boxed_type is expect raise else: await trio.sleep(float('inf')) @@ -42,14 +54,25 @@ 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(arb_addr, to_call, testdir): - """Test errors when making various RPC requests to an actor +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. - """ + + ''' exposed_mods, funcname, inside_err = to_call subactor_exposed_mods = [] func_defined = globals().get(funcname, False) @@ -77,8 +100,13 @@ 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, + registry_addrs=[reg_addr], enable_modules=exposed_mods.copy(), + + # NOTE: will halt test in REPL if uncommented, so only + # do that if actually debugging subactor but keep it + # disabled for the test. + # debug_mode=True, ) as n: actor = tractor.current_actor() @@ -95,6 +123,7 @@ def test_rpc_errors(arb_addr, to_call, testdir): exposed_mods=exposed_mods, func_defined=True if func_defined else False, enable_modules=subactor_exposed_mods, + reg_addr=reg_addr, ) def run(): @@ -105,18 +134,20 @@ def test_rpc_errors(arb_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 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 )) if getattr(value, 'type', None): - assert value.type is inside_err + assert value.boxed_type is inside_err diff --git a/tests/test_runtime.py b/tests/test_runtime.py index 127138c2..55553dd9 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 = '' @@ -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_shm.py b/tests/test_shm.py new file mode 100644 index 00000000..2b7a382f --- /dev/null +++ b/tests/test_shm.py @@ -0,0 +1,167 @@ +""" +Shared mem primitives and APIs. + +""" +import uuid + +# import numpy +import pytest +import trio +import tractor +from tractor._shm import ( + open_shm_list, + attach_shm_list, +) + + +@tractor.context +async def child_attach_shml_alot( + ctx: tractor.Context, + shm_key: str, +) -> None: + + await ctx.started(shm_key) + + # now try to attach a boatload of times in a loop.. + for _ in range(1000): + shml = attach_shm_list( + key=shm_key, + readonly=False, + ) + assert shml.shm.name == shm_key + await trio.sleep(0.001) + + +def test_child_attaches_alot(): + async def main(): + async with tractor.open_nursery() as an: + + # allocate writeable list in parent + key = f'shml_{uuid.uuid4()}' + shml = open_shm_list( + key=key, + ) + + portal = await an.start_actor( + 'shm_attacher', + enable_modules=[__name__], + ) + + async with ( + portal.open_context( + child_attach_shml_alot, + shm_key=shml.key, + ) as (ctx, start_val), + ): + assert start_val == key + await ctx.result() + + await portal.cancel_actor() + + trio.run(main) + + +@tractor.context +async def child_read_shm_list( + ctx: tractor.Context, + shm_key: str, + use_str: bool, + frame_size: int, +) -> None: + + # attach in child + shml = attach_shm_list( + key=shm_key, + # dtype=str if use_str else float, + ) + await ctx.started(shml.key) + + async with ctx.open_stream() as stream: + async for i in stream: + print(f'(child): reading shm list index: {i}') + + if use_str: + expect = str(float(i)) + else: + expect = float(i) + + if frame_size == 1: + val = shml[i] + assert expect == val + print(f'(child): reading value: {val}') + else: + frame = shml[i - frame_size:i] + print(f'(child): reading frame: {frame}') + + +@pytest.mark.parametrize( + 'use_str', + [False, True], + ids=lambda i: f'use_str_values={i}', +) +@pytest.mark.parametrize( + 'frame_size', + [1, 2**6, 2**10], + ids=lambda i: f'frame_size={i}', +) +def test_parent_writer_child_reader( + use_str: bool, + frame_size: int, +): + + async def main(): + async with tractor.open_nursery( + # debug_mode=True, + ) as an: + + portal = await an.start_actor( + 'shm_reader', + enable_modules=[__name__], + debug_mode=True, + ) + + # allocate writeable list in parent + key = 'shm_list' + seq_size = int(2 * 2 ** 10) + shml = open_shm_list( + key=key, + size=seq_size, + dtype=str if use_str else float, + readonly=False, + ) + + async with ( + portal.open_context( + child_read_shm_list, + shm_key=key, + use_str=use_str, + frame_size=frame_size, + ) as (ctx, sent), + + ctx.open_stream() as stream, + ): + + assert sent == key + + for i in range(seq_size): + + val = float(i) + if use_str: + val = str(val) + + # print(f'(parent): writing {val}') + shml[i] = val + + # only on frame fills do we + # signal to the child that a frame's + # worth is ready. + if (i % frame_size) == 0: + print(f'(parent): signalling frame full on {val}') + await stream.send(i) + else: + print(f'(parent): signalling final frame on {val}') + await stream.send(i) + + await portal.cancel_actor() + + trio.run(main) diff --git a/tests/test_spawning.py b/tests/test_spawning.py index 17798c09..99ec9abc 100644 --- a/tests/test_spawning.py +++ b/tests/test_spawning.py @@ -2,13 +2,15 @@ Spawning basics """ -from typing import Optional +from typing import ( + Any, +) 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} @@ -16,24 +18,21 @@ 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() assert actor.is_arbiter == is_arbiter data = data_to_pass_down if actor.is_arbiter: - - async with tractor.open_nursery( - ) as nursery: + async with tractor.open_nursery() as nursery: # forks here portal = await nursery.run_in_actor( @@ -41,7 +40,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 +54,14 @@ 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 @@ -94,7 +95,9 @@ async def test_movie_theatre_convo(start_method): await portal.cancel_actor() -async def cellar_door(return_value: Optional[str]): +async def cellar_door( + return_value: str|None, +): return return_value @@ -104,16 +107,18 @@ async def cellar_door(return_value: Optional[str]): ) @tractor_test async def test_most_beautiful_word( - start_method, - return_value + start_method: str, + return_value: Any, + debug_mode: bool, ): ''' The main ``tractor`` routine. ''' with trio.fail_after(1): - async with tractor.open_nursery() as n: - + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as n: portal = await n.run_in_actor( cellar_door, return_value=return_value, @@ -140,7 +145,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 +157,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 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/__init__.py b/tractor/__init__.py index aa262105..ad3144dc 100644 --- a/tractor/__init__.py +++ b/tractor/__init__.py @@ -18,71 +18,49 @@ tractor: structured concurrent ``trio``-"actors". """ -from exceptiongroup import BaseExceptionGroup - -from ._clustering import open_actor_cluster -from ._ipc import Channel +from ._clustering import ( + open_actor_cluster as open_actor_cluster, +) from ._context import ( - Context, - context, + Context as Context, # the type + context as context, # a func-decorator ) from ._streaming import ( - MsgStream, - stream, + MsgStream as MsgStream, + stream as stream, ) from ._discovery import ( - get_arbiter, - find_actor, - wait_for_actor, - query_actor, + get_arbiter as get_arbiter, + find_actor as find_actor, + wait_for_actor as wait_for_actor, + query_actor as query_actor, +) +from ._supervise import ( + open_nursery as open_nursery, + ActorNursery as ActorNursery, ) -from ._supervise import open_nursery from ._state import ( - current_actor, - is_root_process, + current_actor as current_actor, + is_root_process as is_root_process, + current_ipc_ctx as current_ipc_ctx, ) from ._exceptions import ( - RemoteActorError, - ModuleNotExposed, - ContextCancelled, + ContextCancelled as ContextCancelled, + ModuleNotExposed as ModuleNotExposed, + MsgTypeError as MsgTypeError, + RemoteActorError as RemoteActorError, ) -from ._debug import ( - breakpoint, - post_mortem, +from .devx import ( + breakpoint as breakpoint, + pause as pause, + pause_from_sync as pause_from_sync, + post_mortem as post_mortem, ) -from . import msg +from . import msg as msg from ._root import ( - run_daemon, - open_root_actor, + run_daemon as run_daemon, + open_root_actor as open_root_actor, ) -from ._portal import Portal -from ._runtime import Actor - - -__all__ = [ - 'Actor', - 'Channel', - 'Context', - 'ContextCancelled', - 'ModuleNotExposed', - 'MsgStream', - 'BaseExceptionGroup', - 'Portal', - 'RemoteActorError', - 'breakpoint', - 'context', - 'current_actor', - 'find_actor', - 'get_arbiter', - 'is_root_process', - 'msg', - 'open_actor_cluster', - 'open_nursery', - 'open_root_actor', - 'post_mortem', - 'query_actor', - 'run_daemon', - 'stream', - 'to_asyncio', - 'wait_for_actor', -] +from ._ipc import Channel as Channel +from ._portal import Portal as Portal +from ._runtime import Actor as Actor diff --git a/tractor/_child.py b/tractor/_child.py index 91aaec4f..4226ae90 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,9 +35,8 @@ def parse_ipaddr(arg): return (str(host), int(port)) -from ._entry import _trio_main - if __name__ == "__main__": + __tracebackhide__: bool = True parser = argparse.ArgumentParser() parser.add_argument("--uid", type=parse_uid) diff --git a/tractor/_context.py b/tractor/_context.py index 054f7832..32acf831 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -15,17 +15,29 @@ # along with this program. If not, see . ''' -The fundamental cross process SC abstraction: an inter-actor, -cancel-scope linked task "context". +The fundamental cross-process SC abstraction: an inter-actor, +transitively cancel-scope linked, (dual) task IPC coupled "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. +A `Context` is very similar to the look and feel of the +`.cancel_scope: trio.CancelScope` built into each `trio.Nursery` +except that it links the lifetimes of 2 memory space disjoint, +parallel executing, tasks scheduled in separate "actors". + +So while a `trio.Nursery` has a `.parent_task` which exists both +before (open) and then inside the body of the `async with` of the +nursery's scope (/block), a `Context` contains 2 tasks, a "parent" +and a "child" side, where both execute independently in separate +memory domains of different (host's) processes linked through +a SC-transitive IPC "shuttle dialog protocol". The underlying IPC +dialog-(un)protocol allows for the maintainance of SC properties +end-2-end between the tasks. ''' from __future__ import annotations from collections import deque -from contextlib import asynccontextmanager as acm +from contextlib import ( + asynccontextmanager as acm, +) from dataclasses import ( dataclass, field, @@ -35,49 +47,100 @@ import inspect from pprint import pformat from typing import ( Any, - Callable, AsyncGenerator, + Callable, + Type, + TypeAlias, TYPE_CHECKING, + Union, ) import warnings - +# ------ - ------ import trio - +from trio.lowlevel import Task +# ------ - ------ from ._exceptions import ( - unpack_error, - pack_error, ContextCancelled, + InternalError, + MsgTypeError, + RemoteActorError, StreamOverrun, + pack_from_raise, + unpack_error, ) -from .log import get_logger -from ._ipc import Channel -from ._streaming import MsgStream -from ._state import current_actor - +from .log import ( + get_logger, + at_least_level, +) +from .msg import ( + Error, + MsgType, + NamespacePath, + PayloadT, + Started, + Stop, + Yield, + pretty_struct, + _ops as msgops, +) +from ._ipc import ( + Channel, +) +from ._streaming import ( + MsgStream, + open_stream_from_ctx, +) +from ._state import ( + current_actor, + debug_mode, + _ctxvar_Context, +) +# ------ - ------ if TYPE_CHECKING: from ._portal import Portal - + from ._runtime import Actor + from ._ipc import MsgTransport + from .devx._frame_stack import ( + CallerInfo, + ) log = get_logger(__name__) +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: ''' - An inter-actor, ``trio``-task communication context. + An inter-actor, SC transitive, `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 "parent" task or, + - by the RPC machinery's `._rpc._invoke()` as a `ctx` arg + to a remotely scheduled "child" 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 + `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 `._rpc._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 @@ -86,58 +149,137 @@ 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. - _recv_chan: trio.MemoryReceiveChannel + _actor: Actor + + # the "feeder" channels for delivering message values to the + # local task from the runtime's msg processing loop. + _rx_chan: trio.MemoryReceiveChannel _send_chan: trio.MemorySendChannel + # payload receiver + _pld_rx: msgops.PldRx + + @property + def pld_rx(self) -> msgops.PldRx: + ''' + The current `tractor.Context`'s msg-payload-receiver. + + A payload receiver is the IPC-msg processing sub-sys which + filters inter-actor-task communicated payload data, i.e. the + `PayloadMsg.pld: PayloadT` field value, AFTER its container + shuttlle msg (eg. `Started`/`Yield`/`Return) has been + delivered up from `tractor`'s transport layer but BEFORE the + data is yielded to `tractor` application code. + + The "IPC-primitive API" is normally one of a `Context` (this)` or a `MsgStream` + or some higher level API using one of them. + + For ex. `pld_data: PayloadT = MsgStream.receive()` implicitly + calls into the stream's parent `Context.pld_rx.recv_pld().` to + receive the latest `PayloadMsg.pld` value. + + Modification of the current payload spec via `limit_plds()` + allows a `tractor` application to contextually filter IPC + payload content with a type specification as supported by the + interchange backend. + + - for `msgspec` see . + + Note that the `PldRx` itself is a per-`Context` instance that + normally only changes when some (sub-)task, on a given "side" + of the IPC ctx (either a "child"-side RPC or inside + a "parent"-side `Portal.open_context()` block), modifies it + using the `.msg._ops.limit_plds()` API. + + ''' + return self._pld_rx + + # 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()`. _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 + # 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 - # cancellation state - _cancel_called: bool = False - _cancelled_remote: tuple | None = None - _cancel_msg: str | None = None - _scope: trio.CancelScope | 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 + _task: Task|None = None + + # TODO: cs around result waiting so we can cancel any + # permanently blocking `._rx_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|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 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 + + # 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 + # 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 + # 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 is 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_stdio_for_peer()` 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 - 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._cancelled_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 + _started_msg: MsgType|None = None + _started_pld: Any = None _stream_opened: bool = False + _stream: MsgStream|None = None + + # caller of `Portal.open_context()` for + # logging purposes mostly + _caller_info: CallerInfo|None = None # overrun handling machinery # NOTE: none of this provides "backpressure" to the remote @@ -150,148 +292,643 @@ 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 + # TODO: figure out how we can enforce this without losing our minds.. + _strict_started: bool = False + _cancel_on_msgerr: bool = True + + def pformat( + self, + extra_fields: dict[str, Any]|None = None, + # ^-TODO-^ some built-in extra state fields + # we'll want in some devx specific cases? + + ) -> 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' + ) + + outcome_str: str = self.repr_outcome( + show_error_fields=True + ) + fmtstr: str = ( + 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: + __str__ = pformat + __repr__ = pformat + + @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. + + ''' + 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 + ) + # XXX we only report "this context" as self-cancelled + # once we've received a ctxc from our direct-peer task + # (aka we're `.cancel_acked`). + if not re: + return False + + if from_uid := re.src_uid: + from_uid: tuple = tuple(from_uid) + + our_uid: tuple = self._actor.uid + our_canceller = self.canceller + + return bool( + isinstance((ctxc := re), ContextCancelled) + and from_uid == self.chan.uid + and ctxc.canceller == our_uid + and our_canceller == our_uid + ) + + @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. + + ''' + return self._is_self_cancelled() + + @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 'parent' if self._portal else 'child' + + @staticmethod + def _peer_side(side: str) -> str: + match side: + case 'child': + return 'parent' + case 'parent': + return 'child' + + @property + def peer_side(self) -> str: + return self._peer_side(self.side) + + # TODO: remove stat! + # -[ ] re-implement the `.experiemental._pubsub` stuff + # with `MsgStream` and that should be last usage? + # -[ ] remove from `tests/legacy_one_way_streaming.py`! async def send_yield( self, data: Any, - ) -> None: + ''' + Deprecated method for what now is implemented in `MsgStream`. + We need to rework / remove some stuff tho, see above. + + ''' warnings.warn( "`Context.send_yield()` is now deprecated. " "Use ``MessageStream.send()``. ", DeprecationWarning, stacklevel=2, ) - await self.chan.send({'yield': data, 'cid': self.cid}) + await self.chan.send( + Yield( + cid=self.cid, + pld=data, + ) + ) async def send_stop(self) -> None: - await self.chan.send({'stop': True, 'cid': self.cid}) + ''' + Terminate a `MsgStream` dialog-phase by sending the IPC + equiv of a `StopIteration`. - async def _maybe_cancel_and_set_remote_error( + ''' + await self.chan.send(Stop(cid=self.cid)) + + 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: + - 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*. + + TODOs: + - If this is an error message from a context opened by + `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 + any time soon; in such cases there are 2 possible + approaches depending on the current task's work and + wrapping "thread" type: + + - 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 + `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. + + # TODO: never do this right? + # if self._remote_error: + # return + + # 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'<= {self.peer_side!r}: {self.chan.uid}\n' + f'=> {self.side!r}: {self._actor.uid}\n\n' + f'{error!r}' + ) + self._remote_error: BaseException = error + + msgerr: bool = False + + # self-cancel (ack) or, + # peer propagated remote cancellation. + if isinstance(error, ContextCancelled): + # NOTE in the case error is a ctxc the canceller will + # either be another peer or us. in the case where it's us + # we mark ourself as the canceller of ourselves (a ctx + # "self cancel" from this side's perspective), if instead + # the far end was cancelled by some other (inter-) peer, + # we want to mark our canceller as the actor that was + # cancelled, NOT their reported canceller. IOW in the + # latter case we're cancelled by someone else getting + # cancelled. + if (canc := error.canceller) == self._actor.uid: + whom: str = 'us' + self._canceller = canc + else: + whom = 'a remote peer (not us)' + self._canceller = error.src_uid + + log.cancel( + f'IPC context was cancelled by {whom}!\n\n' + f'{error}' + ) + + elif isinstance(error, MsgTypeError): + msgerr = True + self._canceller = error.src_uid + log.error( + f'IPC dialog error due to msg-type caused by {self.peer_side!r} side\n\n' + f'{error}\n' + f'{pformat(self)}\n' + ) + + else: + # 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_uid: tuple = getattr( + error, + 'src_uid', + None, + ) + # we mark the source actor as our canceller + self._canceller = maybe_error_src_uid + log.error( + f'Remote context error:\n\n' + # f'{pformat(self)}\n' + f'{error!r}' + ) + + if self._canceller is None: + log.error('Ctx has no canceller set!?') + + # 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 + + # 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 + ): + if ( + msgerr + + # NOTE: we allow user to config not cancelling the + # local scope on `MsgTypeError`s + and + not self._cancel_on_msgerr + ): + message: str = ( + 'NOT Cancelling `Context._scope` since,\n' + f'Context._cancel_on_msgerr = {self._cancel_on_msgerr}\n\n' + f'AND we got a msg-type-error!\n' + f'{error}\n' + ) + else: + # TODO: it'd sure be handy to inject our own + # `trio.Cancelled` subtype here ;) + # https://github.com/goodboy/tractor/issues/368 + message: str = 'Cancelling `Context._scope` !\n\n' + self._scope.cancel() + + else: + message: str = 'NOT cancelling `Context._scope` !\n\n' + # from .devx import mk_pdb + # mk_pdb().set_trace() + + fmt_str: str = 'No `self._scope: CancelScope` was set/used ?\n' + if ( + cs + and + at_least_level(log=log, level='cancel') + ): + fmt_str: str = self.pformat( + extra_fields={ + '._is_self_cancelled()': self._is_self_cancelled(), + '._cancel_on_msgerr': self._cancel_on_msgerr, + } + ) + from .devx.pformat import pformat_cs + cs_fmt: str = pformat_cs( + cs, + var_name='Context._scope', + ) + fmt_str += ( + '\n' + + + cs_fmt + ) + log.cancel( + message + + + fmt_str ) - # 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 + # TODO: also 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}' - # 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 + # f'/cid={cid_head}..{cid_tail}' + # TODO: ? not use this ^ right ? + ) - if ( - isinstance(error, ContextCancelled) - ): - log.cancel( - 'Remote task-context sucessfully cancelled for ' - f'{self.chan.uid}:{self.cid}' + dmaddr = dst_maddr + + @property + def repr_rpc(self) -> 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}() -> {outcome_str}' + ) + + @property + def repr_caller(self) -> str: + ci: CallerInfo|None = self._caller_info + if ci: + return ( + f'{ci.caller_nsp}()' + # f'|_api: {ci.api_nsp}' ) - if self._cancel_called: - # from ._debug import breakpoint - # await breakpoint() + return '' - # 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. - 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._cancelled_remote = self.chan.uid - self._scope.cancel() + @property + def repr_api(self) -> str: + # ci: CallerInfo|None = self._caller_info + # if ci: + # return ( + # f'{ci.api_nsp}()\n' + # ) - # 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() + # TODO: use `.dev._frame_stack` scanning to find caller! + return 'Portal.open_context()' async def cancel( self, - msg: str | None = None, timeout: float = 0.616, - # timeout: float = 1000, ) -> None: ''' - Cancel this inter-actor-task context. + Cancel this inter-actor IPC context by requestng the + remote side's cancel-scope-linked `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 = 'caller' if self._portal else 'callee' - if msg: - assert side == 'callee', 'Only callee side can provide cancel msg' + side: str = self.side + self._cancel_called: bool = True - log.cancel(f'Cancelling {side} side of context to {self.chan.uid}') + header: str = ( + f'Cancelling ctx with peer from {side.upper()} side\n\n' + ) + reminfo: str = ( + # ' =>\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 + ) - self._cancel_called = True - # await _debug.breakpoint() - # breakpoint() - - if side == 'caller': + # 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 + # when the runtime finally receives it during teardown + # (normally in `.result()` called from + # `Portal.open_context().__aexit__()`) + if side == 'parent': 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 {self.side!r}-side ctx task missing it?!?' ) - 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}") + header + + + reminfo + ) # NOTE: we're telling the far end actor to cancel a task # corresponding to *this actor*. The far end local channel @@ -301,7 +938,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 @@ -310,17 +946,33 @@ 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 cancelling remote task " - f"{cid} for {self._portal.channel.uid}") + 'Timed out on cancel request of remote task?\n' + f'{reminfo}' + ) - # callee side remote task + # 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: - self._cancel_msg = msg - + 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? @@ -329,137 +981,140 @@ class Context: assert self._scope self._scope.cancel() - @acm - async def open_stream( - + # TODO: replace all the `._maybe_raise_remote_err()` usage + # with instances of this!! + def maybe_raise( self, - allow_overruns: bool | None = False, - msg_buffer_size: int | None = None, + hide_tb: bool = True, + **kwargs, - ) -> AsyncGenerator[MsgStream, None]: + ) -> Exception|None: ''' - Open a ``MsgStream``, a bi-directional stream connected to the - cross-actor (far end) task for this ``Context``. + Check for for a remote error delivered by the runtime from + our peer (task); if set immediately raise. - 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. + This is a convenience wrapper for + `._maybe_raise_remote_err(self._remote_error)`. ''' - 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!' + __tracebackhide__: bool = hide_tb + if re := self._remote_error: + return self._maybe_raise_remote_err( + re, + hide_tb=hide_tb, + **kwargs, ) - 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() + remote_error: Exception, + from_src_exc: BaseException|None|bool = False, + raise_ctxc_from_self_call: bool = False, + raise_overrun_from_self: bool = True, + hide_tb: bool = True, + + ) -> ( + ContextCancelled # `.cancel()` request to far side + |RemoteActorError # stream overrun caused and ignored by us + ): + ''' + 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). + + ''' + __tracebackhide__: bool = hide_tb + our_uid: tuple = self.chan.uid + + # XXX NOTE XXX: `ContextCancelled`/`StreamOverrun` absorption + # for "graceful cancellation" case: + # + # Whenever a "side" of a context (a `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 ( - isinstance(err, ContextCancelled) - and ( - self._cancel_called - or self.chan._cancel_called - or tuple(err.canceller) == current_actor().uid - ) + not raise_ctxc_from_self_call + and self._is_self_cancelled(remote_error) + + # TODO: ?potentially it is useful to emit certain + # warning/cancel logs for the cases where the + # 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? + + ) 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 + # `msg._ops.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.boxed_type is StreamOverrun + + # and tuple(remote_error.msgdata['sender']) == our_uid + and tuple(remote_error.sender) == our_uid ): - return err + # 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 = remote_error - raise err # from None + else: + log.warning( + 'Local error already set for ctx?\n' + f'{self._local_error}\n\n' + f'{self}' + ) - async def result(self) -> Any | Exception: + 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: + # - [ ] 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 + if from_src_exc is not False: + raise remote_error from from_src_exc + + raise remote_error + + async def wait_for_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. @@ -485,70 +1140,312 @@ class Context: 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 + __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._rx_chan + raise_overrun: bool = not self._allow_overruns if ( - self._result == id(self) - and not self._remote_error - and not self._recv_chan._closed # type: ignore + self.maybe_error is None + and + not self._rx_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'] + # 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. + # + # XXX NOTE XXX: this call shouldn't really ever raise + # (other then internal error), instead delivering an + # `Error`-msg and that being `.maybe_raise()`-ed below + # since every message should be delivered via the normal + # `._deliver_msg()` route which will appropriately set + # any `.maybe_error`. + ( + outcome_msg, + drained_msgs, + ) = await msgops.drain_to_final_msg( + ctx=self, + hide_tb=hide_tb, + ) - # 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() + drained_status: str = ( + 'Ctx drained to final outcome msg\n\n' + f'{outcome_msg}\n' + ) + if drained_msgs: + drained_status += ( + '\n' + f'The pre-drained msgs are\n' + f'{pformat(drained_msgs)}\n' + ) - break - except KeyError: # as msgerr: + log.cancel(drained_status) - if 'yield' in msg: - # far end task is still streaming to us so discard - log.warning(f'Discarding stream delivered {msg}') - continue + self.maybe_raise( + # 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 + 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) + ) + ) + # TODO: eventually make `.outcome: Outcome` and thus return + # `self.outcome.unwrap()` here! + return self.outcome - elif 'stop' in msg: - log.debug('Remote stream terminated') - continue + # TODO: switch this with above! + # -[ ] should be named `.wait_for_outcome()` and instead do + # a `.outcome.Outcome.unwrap()` ? + # + async def result( + self, + *args, + **kwargs, + ) -> Any|Exception: + log.warning( + '`Context.result()` is DEPRECATED!\n' + 'Use `Context.[no]wait_for_result()` instead!\n' + ) + return await self.wait_for_result( + *args, + **kwargs, + ) - # internal error should never get here - assert msg.get('cid'), ( - "Received internal error at portal?") + @property + def maybe_error(self) -> BaseException|None: + le: BaseException|None = self._local_error + re: RemoteActorError|ContextCancelled|None = self._remote_error - err = unpack_error( - msg, - self._portal.channel - ) # from msgerr + 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 + ) - err = self._maybe_raise_remote_err(err) - self._remote_err = err + # 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=), + # ): - return self._remote_error or self._result + error: BaseException|None = le or re + if error: + return error + + 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: + return self._result is not Unresolved + + # def get_result_nowait(self) -> Any|None: + # def get_outcome_nowait(self) -> Any|None: + # def recv_result_nowait(self) -> Any|None: + # def receive_outcome_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 has_outcome(self) -> bool: + return bool(self.maybe_error) or self._final_result_is_set() + + def repr_outcome( + self, + show_error_fields: bool = False, + type_only: bool = False, + + ) -> str: + ''' + Deliver a (simplified) `str` representation (as in + `.__repr__()`) of the final `.outcome` + + ''' + 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. + 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 ( + str(type(merr).__name__) + or + repr(merr) + ) + + # for all other errors show their regular output + return ( + str(merr) + or + repr(merr) + ) + + return ( + str(self._result) + or + repr(self._result) + ) + + @property + def repr_state(self) -> str: + ''' + A `str`-status describing the current state of this + inter-actor IPC context in terms of the current "phase" state + of the SC shuttling dialog protocol. + + ''' + merr: Exception|None = self.maybe_error + outcome: Unresolved|Exception|Any = self.outcome + status: str|None = None + match ( + outcome, + merr, + ): + # "graceful" ctx cancellation + case ( + Unresolved, + ContextCancelled(), + ): + if self._is_self_cancelled(): + status = 'self-cancelled' + elif ( + self.canceller + and not self._cancel_called + ): + status = 'peer-cancelled' + + # (remote) error condition + case ( + Unresolved, + BaseException(), # any error-type + ): + status = 'errored' + + # result already returned + case ( + _, # any non-unresolved value + None, + ) if self._final_result_is_set(): + status = 'result-returned' + + # normal operation but still in a pre-`Return`-result + # dialog phase + case ( + Unresolved, # noqa (ruff, you so weird..) + None, # no (remote) error set + ): + if stream := self._stream: + if stream.closed: + status = 'streaming-finished' + else: + status = 'streaming' + + elif self._started_called: + status = 'started' + + else: + if self.side == 'child': + status = 'pre-started' + else: + status = 'syncing-to-child' + + if status is None: + status = '??unknown??' + # from tractor.devx import mk_pdb + # mk_pdb().set_trace() + + return status async def started( self, - value: Any | None = None + + value: PayloadT|None = None, + validate_pld_spec: bool = True, + strict_pld_parity: bool = False, + + # TODO: this will always emit for msgpack for any () vs. [] + # inside the value.. do we want to offer warnings on that? + # complain_no_parity: bool = False, + + hide_tb: bool = True, ) -> 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()``. @@ -556,18 +1453,60 @@ 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 + started_msg = Started( + cid=self.cid, + pld=value, + ) + # XXX MEGA NOTE XXX: ONLY on the first msg sent with + # `Context.started()` do we STRINGENTLY roundtrip-check + # the first payload such that the child side can't send an + # incorrect value according to the currently applied + # msg-spec! + # + # HOWEVER, once a stream is opened via + # `Context.open_stream()` then this check is NEVER done on + # `MsgStream.send()` and instead both the parent and child + # sides are expected to relay back msg-type errors when + # decode failures exhibit on `MsgStream.receive()` calls thus + # enabling a so-called (by the holy 0mq lords) + # "cheap-or-nasty pattern" un-protocol design Bo + # + # https://zguide.zeromq.org/docs/chapter7/#The-Cheap-or-Nasty-Pattern + # + __tracebackhide__: bool = hide_tb + if validate_pld_spec: + # TODO: prolly wrap this as a `show_frame_when_not()` + try: + msgops.validate_payload_msg( + pld_msg=started_msg, + pld_value=value, + ipc=self, + strict_pld_parity=strict_pld_parity, + hide_tb=hide_tb, + ) + except BaseException as err: + if not isinstance(err, MsgTypeError): + __tracebackhide__: bool = False - # TODO: do we need a restart api? - # async def restart(self) -> None: - # pass + raise + + + # TODO: maybe a flag to by-pass encode op if already done + # here in caller? + await self.chan.send(started_msg) + + # set msg-related internal runtime-state + self._started_called: bool = True + self._started_msg: Started = started_msg + self._started_pld = value async def _drain_overflows( self, @@ -587,7 +1526,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) @@ -621,37 +1560,115 @@ class Context: async def _deliver_msg( self, - msg: dict, - - draining: bool = False, + msg: MsgType, ) -> bool: + ''' + Deliver an IPC msg received from a transport-channel to + this context's underlying mem chan for handling by local + user application tasks; deliver `bool` indicating whether + the msg was able to be delivered. - cid = self.cid - chan = self.chan - uid = chan.uid + 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 queued up and eventually sent if possible. + + XXX RULES XXX + ------ - ------ + - NEVER raise remote errors from this method; a runtime task caller. + An error "delivered" to a ctx should always be raised by + the corresponding local task operating on the + `Portal`/`Context` APIs. + + - NEVER `return` early before delivering the msg! + bc if the error is a ctxc and there is a task waiting on + `.result()` we need the msg to be + `send_chan.send_nowait()`-ed over the `._rx_chan` so + that the error is relayed to that waiter task and thus + raised in user code! + + ''' + 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 - log.runtime( - f"Delivering {msg} from {uid} to caller {cid}" + side: str = self.side + if side == 'child': + assert not self._portal + + flow_body: str = ( + f'<= peer {self.peer_side!r}: {from_uid}\n' + f' |_<{nsf}()>\n\n' + + f'=> {side!r}: {self._task}\n' + f' |_<{self.repr_api} @ {self.repr_caller}>\n\n' ) - error = msg.get('error') - if error: - await self._maybe_cancel_and_set_remote_error(msg) - - if ( - self._in_overrun + re: Exception|None + if re := unpack_error( + msg, + self.chan, ): + if not isinstance(re, ContextCancelled): + log_meth = log.error + else: + log_meth = log.runtime + + log_meth( + f'Delivering IPC ctx error from {self.peer_side!r} to {side!r} task\n\n' + + f'{flow_body}' + f'{pformat(re)}\n' + ) + self._cancel_msg: dict = msg + + # XXX 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. a `ContextCancelled` + # or `RemoteActorError`). + self._maybe_cancel_and_set_remote_error(re) + + # TODO: expose as mod func instead! + structfmt = pretty_struct.Struct.pformat + if self._in_overrun: + log.warning( + f'Queueing OVERRUN msg on caller task:\n\n' + + f'{flow_body}' + + f'{structfmt(msg)}\n' + ) self._overflow_q.append(msg) + + # XXX NOTE XXX + # overrun is the ONLY case where returning early is fine! return False try: + log.runtime( + f'Delivering msg from IPC ctx:\n\n' + + f'{flow_body}' + + f'{structfmt(msg)}\n' + ) + + # 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 @@ -662,7 +1679,14 @@ 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 @@ -670,43 +1694,55 @@ 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) - 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}' - ] + # 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 = self._actor.uid + txt: str = ( + 'on IPC context:\n' + + 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: - lines.insert( - 1, - f'\n*** No stream open on `{local_uid[0]}` side! ***\n' + txt += ( + f'\n*** No stream open on `{local_uid[0]}` side! ***\n\n' + f'{msg}\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) + 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 @@ -714,25 +1750,608 @@ class Context: # anything different. return False else: + # raise local overrun and immediately pack as IPC + # msg for far end. + err_msg: Error = pack_from_raise( + local_err=StreamOverrun( + txt, + sender=from_uid, + ), + cid=cid, + ) 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") + # relay condition to sender side remote task + await chan.send(err_msg) + return True + # XXX: local consumer has closed their side of + # the IPC so cancel the far end streaming task + except trio.BrokenResourceError: + log.warning( + 'Channel for ctx is already closed?\n' + f'|_{chan}\n' + ) + + # ow, indicate unable to deliver by default return False + # NOTE: similar to `Portal.open_context()`, this impl is found in + # the `._streaming`` mod to make reading/groking the details + # simpler code-org-wise. + open_stream = open_stream_from_ctx + + +# TODO: exception tb masking by using a manual +# `.__aexit__()`/.__aenter__()` pair on a type? +# => currently this is one of the few places we can't easily +# mask errors - on the exit side of a `Portal.open_context()`.. +# there's # => currently this is one of the few places we can't +# there's 2 ways to approach it: +# - manually write an @acm type as per above +# - use `contextlib.AsyncContextDecorator` to override the default +# impl to suppress traceback frames: +# * https://docs.python.org/3/library/contextlib.html#contextlib.AsyncContextDecorator +# * https://docs.python.org/3/library/contextlib.html#contextlib.ContextDecorator +# - also we could just override directly the underlying +# `contextlib._AsyncGeneratorContextManager`? +@acm +async def open_context_from_portal( + portal: Portal, + func: Callable, + + allow_overruns: bool = False, + 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 + + # denote this frame as a "runtime frame" for stack + # introspection where we report the caller code in logging + # and error message content. + # NOTE: 2 bc of the wrapping `@acm` + __runtimeframe__: int = 2 # noqa + + # if NOT an async func but decorated with `@context`, error. + if ( + not inspect.iscoroutinefunction(func) + and getattr(func, '_tractor_context_meta', False) + ): + raise TypeError( + f'{func!r} must be an async function!' + ) + + ctx_meta: dict[str, Any]|None = getattr( + func, + '_tractor_context_meta', + None, + ) + + # 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 (uid := 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, + + portal=portal, + + # 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' + assert ctx._caller_info + _ctxvar_Context.set(ctx) + + # 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 tn, + msgops.maybe_limit_plds( + ctx=ctx, + spec=ctx_meta.get('pld_spec'), + ), + ): + # NOTE: this in an implicit runtime nursery used to, + # - start overrun queuing tasks when as well as + # for cancellation of the scope opened by the user. + ctx._scope_nursery: trio.Nursery = tn + ctx._scope: trio.CancelScope = tn.cancel_scope + + # XXX NOTE since `._scope` is NOT set BEFORE we retreive the + # `Started`-msg any cancellation triggered + # in `._maybe_cancel_and_set_remote_error()` will + # NOT actually cancel the below line! + # -> it's expected that if there is an error in this phase of + # the dialog, the `Error` msg should be raised from the `msg` + # handling block below. + try: + started_msg, first = await ctx._pld_rx.recv_msg_w_pld( + ipc=ctx, + expect_msg=Started, + passthrough_non_pld_msgs=False, + hide_tb=hide_tb, + ) + except trio.Cancelled as taskc: + ctx_cs: trio.CancelScope = ctx._scope + if not ctx_cs.cancel_called: + raise + + # from .devx import pause + # await pause(shield=True) + + log.cancel( + 'IPC ctx was cancelled during "child" task sync due to\n\n' + f'{ctx.maybe_error}\n' + ) + # OW if the ctx's scope was cancelled manually, + # likely the `Context` was cancelled via a call to + # `._maybe_cancel_and_set_remote_error()` so ensure + # we raise the underlying `._remote_error` directly + # instead of bubbling that taskc. + ctx.maybe_raise( + # mask the above taskc from the tb + from_src_exc=None, + hide_tb=hide_tb, + ) + + # OW, some other unexpected cancel condition + # that should prolly never happen right? + raise InternalError( + 'Invalid cancellation during IPC ctx sync phase?\n' + ) from taskc + + ctx._started_called: bool = True + ctx._started_msg: bool = started_msg + ctx._started_pld: bool = first + + # deliver context ref and `.started()` msg payload value + # in `__aenter__` 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? + # + # maybe TODO NOTE: between the caller exiting and + # arriving here the far end may have sent a ctxc-msg or + # other error, so the quetion is whether we should check + # for it here immediately and maybe raise so as to engage + # the ctxc handling block below ???? + # + # self.maybe_raise() + + # when in allow_overruns mode there may be + # lingering overflow sender tasks remaining? + if tn.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(tn.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 + # finish 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( + log.exception( + f'{ctx.side}-side of `Context` terminated with ' + 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: {ctx.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! + # 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. + match result_or_err: + case ContextCancelled() as ctxc: + log.cancel(ctxc.tb_str) + + case RemoteActorError() as rae: + log.exception( + 'Context remotely errored!\n' + f'<= peer: {uid}\n' + f' |_ {nsf}()\n\n' + + f'{rae.tb_str}' + ) + 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._deliver_ctx_payload()`` 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._rx_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-forever-and-only-CLOSED! + with trio.CancelScope(shield=True): + await ctx._rx_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( + f'Context cancelled by {ctx.side!r}-side 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 {ctx.side!r}-side error:\n\n' + f'{ctx.chan.uid} => {outcome_str}\n' + ) + + # FINALLY, remove the context from runtime tracking and + # exit! + log.runtime( + f'De-allocating IPC ctx opened with {ctx.side!r} peer \n' + f'uid: {uid}\n' + f'cid: {ctx.cid}\n' + ) + portal.actor._contexts.pop( + (uid, ctx.cid), + None, + ) + def mk_context( chan: Channel, cid: str, + nsf: NamespacePath, + msg_buffer_size: int = 2**6, + pld_spec: Union[Type] = Any, **kwargs, @@ -748,31 +2367,114 @@ def mk_context( recv_chan: trio.MemoryReceiveChannel send_chan, recv_chan = trio.open_memory_channel(msg_buffer_size) + # TODO: only scan caller-info if log level so high! + from .devx._frame_stack import find_caller_info + caller_info: CallerInfo|None = find_caller_info() + + pld_rx = msgops.PldRx( + _pld_dec=msgops._def_any_pldec, + ) + ctx = Context( - chan, - cid, + chan=chan, + cid=cid, + _actor=current_actor(), _send_chan=send_chan, - _recv_chan=recv_chan, + _rx_chan=recv_chan, + _pld_rx=pld_rx, + _nsf=nsf, + _task=trio.lowlevel.current_task(), + _caller_info=caller_info, **kwargs, ) - ctx._result: int | Any = id(ctx) + pld_rx._ctx = ctx + ctx._result = Unresolved return ctx -def context(func: Callable) -> Callable: +# TODO: use the new type-parameters to annotate this in 3.13? +# -[ ] https://peps.python.org/pep-0718/#unknown-types +# -[ ] allow for `pld_spec` input(s) ideally breaking down, +# |_ `start: ParameterSpec`, +# |_ `started: TypeAlias`, +# |_ `yields: TypeAlias`, +# |_ `return: TypeAlias`, +# |_ `invalid_policy: str|Callable` ? +# -[ ] prolly implement the `@acm` wrapper using +# a `contextlib.ContextDecorator`, i guess not if +# we don't need an `__aexit__` block right? +# |_ de hecho, @acm can already be used as a decorator as of 3.10 +# but i dunno how that's gonna play with `trio.Nursery.start[_soon]()` +# |_ https://docs.python.org/3/library/contextlib.html#using-a-context-manager-as-a-function-decorator +# +def context( + func: Callable|None = None, + + *, + + # must be named! + pld_spec: Union[Type]|TypeAlias = Any, + dec_hook: Callable|None = None, + enc_hook: Callable|None = None, + +) -> Callable: ''' - Mark an async function as a streaming routine with ``@context``. + Mark an async function as an SC-supervised, inter-`Actor`, RPC + scheduled child-side `Task`, IPC endpoint otherwise + known more colloquially as a (RPC) "context". + + Functions annotated the fundamental IPC endpoint type offered by + `tractor`. ''' + # XXX for the `@context(pld_spec=MyMsg|None)` case + if func is None: + return partial( + context, + pld_spec=pld_spec, + dec_hook=dec_hook, + enc_hook=enc_hook, + ) + + # TODO: from this, enforcing a `Start.sig` type + # check when invoking RPC tasks by ensuring the input + # args validate against the endpoint def. + sig: inspect.Signature = inspect.signature(func) + # params: inspect.Parameters = sig.parameters + + # https://docs.python.org/3/library/inspect.html#inspect.get_annotations + annots: dict[str, Type] = inspect.get_annotations( + func, + eval_str=True, + ) + name: str + param: Type + for name, param in annots.items(): + if param is Context: + ctx_var_name: str = name + break + else: + raise TypeError( + 'At least one (normally the first) argument to the `@context` function ' + f'{func.__name__!r} must be typed as `tractor.Context`, for ex,\n\n' + f'`ctx: tractor.Context`\n' + ) + # 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 + # func._tractor_context_function = True # type: ignore + func._tractor_context_meta: dict[str, Any] = { + 'ctx_var_name': ctx_var_name, + # `msgspec` related settings + 'pld_spec': pld_spec, + 'enc_hook': enc_hook, + 'dec_hook': dec_hook, - 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`" - ) + # TODO: eventually we need to "signature-check" with these + # vs. the `Start` msg fields! + # => this would allow for TPC endpoint argument-type-spec + # limiting and we could then error on + # invalid inputs passed to `.open_context(rpc_ep, arg0='blah')` + 'sig': sig, + } return func diff --git a/tractor/_debug.py b/tractor/_debug.py deleted file mode 100644 index b0482f18..00000000 --- a/tractor/_debug.py +++ /dev/null @@ -1,922 +0,0 @@ -# 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 . - -""" -Multi-core debugging for da peeps! - -""" -from __future__ import annotations -import bdb -import os -import sys -import signal -from functools import ( - partial, - cached_property, -) -from contextlib import asynccontextmanager as acm -from typing import ( - Any, - Optional, - Callable, - AsyncIterator, - AsyncGenerator, -) -from types import FrameType - -import pdbp -import tractor -import trio -from trio_typing import TaskStatus - -from .log import get_logger -from ._discovery import get_root -from ._state import ( - is_root_process, - debug_mode, -) -from ._exceptions import ( - is_multi_cancelled, - ContextCancelled, -) -from ._ipc import Channel - -log = get_logger(__name__) - - -__all__ = ['breakpoint', 'post_mortem'] - - -class Lock: - ''' - Actor global debug lock state. - - Mostly to avoid a lot of ``global`` declarations for now XD. - - ''' - repl: MultiActorPdb | None = None - # placeholder for function to set a ``trio.Event`` on debugger exit - # pdb_release_hook: Optional[Callable] = None - - _trio_handler: Callable[ - [int, Optional[FrameType]], Any - ] | int | None = None - - # actor-wide variable pointing to current task name using debugger - local_task_in_debug: str | None = None - - # NOTE: set by the current task waiting on the root tty lock from - # the CALLER side of the `lock_tty_for_child()` context entry-call - # and must be cancelled if this actor is cancelled via IPC - # request-message otherwise deadlocks with the parent actor may - # ensure - _debugger_request_cs: Optional[trio.CancelScope] = None - - # NOTE: set only in the root actor for the **local** root spawned task - # which has acquired the lock (i.e. this is on the callee side of - # the `lock_tty_for_child()` context entry). - _root_local_task_cs_in_debug: Optional[trio.CancelScope] = None - - # actor tree-wide actor uid that supposedly has the tty lock - global_actor_in_debug: Optional[tuple[str, str]] = None - - local_pdb_complete: Optional[trio.Event] = None - no_remote_has_tty: Optional[trio.Event] = None - - # lock in root actor preventing multi-access to local tty - _debug_lock: trio.StrictFIFOLock = trio.StrictFIFOLock() - - _orig_sigint_handler: Optional[Callable] = None - _blocked: set[tuple[str, str]] = set() - - @classmethod - def shield_sigint(cls): - cls._orig_sigint_handler = signal.signal( - signal.SIGINT, - shield_sigint_handler, - ) - - @classmethod - def unshield_sigint(cls): - # always restore ``trio``'s sigint handler. see notes below in - # the pdb factory about the nightmare that is that code swapping - # out the handler when the repl activates... - signal.signal(signal.SIGINT, cls._trio_handler) - cls._orig_sigint_handler = None - - @classmethod - def release(cls): - try: - cls._debug_lock.release() - except RuntimeError: - # uhhh makes no sense but been seeing the non-owner - # release error even though this is definitely the task - # that locked? - owner = cls._debug_lock.statistics().owner - if owner: - raise - - # actor-local state, irrelevant for non-root. - cls.global_actor_in_debug = None - cls.local_task_in_debug = None - - try: - # sometimes the ``trio`` might already be terminated in - # which case this call will raise. - if cls.local_pdb_complete is not None: - cls.local_pdb_complete.set() - finally: - # restore original sigint handler - cls.unshield_sigint() - cls.repl = None - - -class TractorConfig(pdbp.DefaultConfig): - ''' - Custom ``pdbp`` goodness :surfer: - - ''' - use_pygments: bool = True - sticky_by_default: bool = False - enable_hidden_frames: bool = False - - # much thanks @mdmintz for the hot tip! - # fixes line spacing issue when resizing terminal B) - truncate_long_lines: bool = False - - -class MultiActorPdb(pdbp.Pdb): - ''' - Add teardown hooks to the regular ``pdbp.Pdb``. - - ''' - # override the pdbp config with our coolio one - DefaultConfig = TractorConfig - - # def preloop(self): - # print('IN PRELOOP') - # super().preloop() - - # TODO: figure out how to disallow recursive .set_trace() entry - # since that'll cause deadlock for us. - def set_continue(self): - try: - super().set_continue() - finally: - Lock.release() - - def set_quit(self): - try: - super().set_quit() - finally: - Lock.release() - - # XXX NOTE: we only override this because apparently the stdlib pdb - # bois likes to touch the SIGINT handler as much as i like to touch - # my d$%&. - def _cmdloop(self): - self.cmdloop() - - @cached_property - def shname(self) -> str | None: - ''' - Attempt to return the login shell name with a special check for - the infamous `xonsh` since it seems to have some issues much - different from std shells when it comes to flushing the prompt? - - ''' - # SUPER HACKY and only really works if `xonsh` is not used - # before spawning further sub-shells.. - shpath = os.getenv('SHELL', None) - - if shpath: - if ( - os.getenv('XONSH_LOGIN', default=False) - or 'xonsh' in shpath - ): - return 'xonsh' - - return os.path.basename(shpath) - - return None - - -@acm -async def _acquire_debug_lock_from_root_task( - uid: tuple[str, str] - -) -> AsyncIterator[trio.StrictFIFOLock]: - ''' - Acquire a root-actor local FIFO lock which tracks mutex access of - the process tree's global debugger breakpoint. - - This lock avoids tty clobbering (by preventing multiple processes - reading from stdstreams) and ensures multi-actor, sequential access - to the ``pdb`` repl. - - ''' - task_name = trio.lowlevel.current_task().name - - log.runtime( - f"Attempting to acquire TTY lock, remote task: {task_name}:{uid}" - ) - - we_acquired = False - - try: - log.runtime( - f"entering lock checkpoint, remote task: {task_name}:{uid}" - ) - we_acquired = True - - # NOTE: if the surrounding cancel scope from the - # `lock_tty_for_child()` caller is cancelled, this line should - # unblock and NOT leave us in some kind of - # a "child-locked-TTY-but-child-is-uncontactable-over-IPC" - # condition. - await Lock._debug_lock.acquire() - - if Lock.no_remote_has_tty is None: - # mark the tty lock as being in use so that the runtime - # can try to avoid clobbering any connection from a child - # that's currently relying on it. - Lock.no_remote_has_tty = trio.Event() - - Lock.global_actor_in_debug = uid - log.runtime(f"TTY lock acquired, remote task: {task_name}:{uid}") - - # NOTE: critical section: this yield is unshielded! - - # IF we received a cancel during the shielded lock entry of some - # next-in-queue requesting task, then the resumption here will - # result in that ``trio.Cancelled`` being raised to our caller - # (likely from ``lock_tty_for_child()`` below)! In - # this case the ``finally:`` below should trigger and the - # surrounding caller side context should cancel normally - # relaying back to the caller. - - yield Lock._debug_lock - - finally: - if ( - we_acquired - and Lock._debug_lock.locked() - ): - Lock._debug_lock.release() - - # IFF there are no more requesting tasks queued up fire, the - # "tty-unlocked" event thereby alerting any monitors of the lock that - # we are now back in the "tty unlocked" state. This is basically - # and edge triggered signal around an empty queue of sub-actor - # tasks that may have tried to acquire the lock. - stats = Lock._debug_lock.statistics() - if ( - not stats.owner - ): - log.runtime(f"No more tasks waiting on tty lock! says {uid}") - if Lock.no_remote_has_tty is not None: - Lock.no_remote_has_tty.set() - Lock.no_remote_has_tty = None - - Lock.global_actor_in_debug = None - - log.runtime( - f"TTY lock released, remote task: {task_name}:{uid}" - ) - - -@tractor.context -async def lock_tty_for_child( - - ctx: tractor.Context, - subactor_uid: tuple[str, str] - -) -> str: - ''' - Lock the TTY in the root process of an actor tree in a new - inter-actor-context-task such that the ``pdbp`` debugger console - can be mutex-allocated to the calling sub-actor for REPL control - without interference by other processes / threads. - - NOTE: this task must be invoked in the root process of the actor - tree. It is meant to be invoked as an rpc-task and should be - highly reliable at releasing the mutex complete! - - ''' - task_name = trio.lowlevel.current_task().name - - if tuple(subactor_uid) in Lock._blocked: - log.warning( - f'Actor {subactor_uid} is blocked from acquiring debug lock\n' - f"remote task: {task_name}:{subactor_uid}" - ) - ctx._enter_debugger_on_cancel = False - await ctx.cancel(f'Debug lock blocked for {subactor_uid}') - return 'pdb_lock_blocked' - - # TODO: when we get to true remote debugging - # this will deliver stdin data? - - log.debug( - "Attempting to acquire TTY lock\n" - f"remote task: {task_name}:{subactor_uid}" - ) - - log.debug(f"Actor {subactor_uid} is WAITING on stdin hijack lock") - Lock.shield_sigint() - - try: - with ( - trio.CancelScope(shield=True) as debug_lock_cs, - ): - Lock._root_local_task_cs_in_debug = debug_lock_cs - async with _acquire_debug_lock_from_root_task(subactor_uid): - - # indicate to child that we've locked stdio - await ctx.started('Locked') - log.debug( - f"Actor {subactor_uid} acquired stdin hijack lock" - ) - - # wait for unlock pdb by child - async with ctx.open_stream() as stream: - assert await stream.receive() == 'pdb_unlock' - - return "pdb_unlock_complete" - - finally: - Lock._root_local_task_cs_in_debug = None - Lock.unshield_sigint() - - -async def wait_for_parent_stdin_hijack( - actor_uid: tuple[str, str], - task_status: TaskStatus[trio.CancelScope] = trio.TASK_STATUS_IGNORED -): - ''' - Connect to the root actor via a ``Context`` and invoke a task which - locks a root-local TTY lock: ``lock_tty_for_child()``; this func - should be called in a new task from a child actor **and never the - root*. - - This function is used by any sub-actor to acquire mutex access to - the ``pdb`` REPL and thus the root's TTY for interactive debugging - (see below inside ``_breakpoint()``). It can be used to ensure that - an intermediate nursery-owning actor does not clobber its children - if they are in debug (see below inside - ``maybe_wait_for_debugger()``). - - ''' - with trio.CancelScope(shield=True) as cs: - Lock._debugger_request_cs = cs - - try: - async with get_root() as portal: - - # this syncs to child's ``Context.started()`` call. - async with portal.open_context( - - tractor._debug.lock_tty_for_child, - subactor_uid=actor_uid, - - ) as (ctx, val): - - log.debug('locked context') - assert val == 'Locked' - - async with ctx.open_stream() as stream: - # unblock local caller - - try: - assert Lock.local_pdb_complete - task_status.started(cs) - await Lock.local_pdb_complete.wait() - - finally: - # TODO: shielding currently can cause hangs... - # with trio.CancelScope(shield=True): - await stream.send('pdb_unlock') - - # sync with callee termination - assert await ctx.result() == "pdb_unlock_complete" - - log.debug('exitting child side locking task context') - - except ContextCancelled: - log.warning('Root actor cancelled debug lock') - raise - - finally: - Lock.local_task_in_debug = None - log.debug('Exiting debugger from child') - - -def mk_mpdb() -> tuple[MultiActorPdb, Callable]: - - pdb = MultiActorPdb() - # signal.signal = pdbp.hideframe(signal.signal) - - Lock.shield_sigint() - - # XXX: These are the important flags mentioned in - # https://github.com/python-trio/trio/issues/1155 - # which resolve the traceback spews to console. - pdb.allow_kbdint = True - pdb.nosigint = True - - return pdb, Lock.unshield_sigint - - -async def _breakpoint( - - debug_func, - - # TODO: - # shield: bool = False - -) -> None: - ''' - Breakpoint entry for engaging debugger instance sync-interaction, - from async code, executing in actor runtime (task). - - ''' - __tracebackhide__ = True - actor = tractor.current_actor() - pdb, undo_sigint = mk_mpdb() - task_name = trio.lowlevel.current_task().name - - # TODO: is it possible to debug a trio.Cancelled except block? - # right now it seems like we can kinda do with by shielding - # around ``tractor.breakpoint()`` but not if we move the shielded - # scope here??? - # with trio.CancelScope(shield=shield): - # await trio.lowlevel.checkpoint() - - if ( - not Lock.local_pdb_complete - or Lock.local_pdb_complete.is_set() - ): - Lock.local_pdb_complete = trio.Event() - - # TODO: need a more robust check for the "root" actor - if ( - not is_root_process() - and actor._parent_chan # a connected child - ): - - if Lock.local_task_in_debug: - - # Recurrence entry case: this task already has the lock and - # is likely recurrently entering a breakpoint - if Lock.local_task_in_debug == task_name: - # noop on recurrent entry case but we want to trigger - # a checkpoint to allow other actors error-propagate and - # potetially avoid infinite re-entries in some subactor. - await trio.lowlevel.checkpoint() - return - - # if **this** actor is already in debug mode block here - # waiting for the control to be released - this allows - # support for recursive entries to `tractor.breakpoint()` - log.warning(f"{actor.uid} already has a debug lock, waiting...") - - await Lock.local_pdb_complete.wait() - await trio.sleep(0.1) - - # mark local actor as "in debug mode" to avoid recurrent - # entries/requests to the root process - Lock.local_task_in_debug = task_name - - # this **must** be awaited by the caller and is done using the - # root nursery so that the debugger can continue to run without - # being restricted by the scope of a new task nursery. - - # TODO: if we want to debug a trio.Cancelled triggered exception - # we have to figure out how to avoid having the service nursery - # cancel on this task start? I *think* this works below: - # ```python - # actor._service_n.cancel_scope.shield = shield - # ``` - # but not entirely sure if that's a sane way to implement it? - try: - with trio.CancelScope(shield=True): - await actor._service_n.start( - wait_for_parent_stdin_hijack, - actor.uid, - ) - Lock.repl = pdb - except RuntimeError: - Lock.release() - - if actor._cancel_called: - # service nursery won't be usable and we - # don't want to lock up the root either way since - # we're in (the midst of) cancellation. - return - - raise - - elif is_root_process(): - - # we also wait in the root-parent for any child that - # may have the tty locked prior - # TODO: wait, what about multiple root tasks acquiring it though? - if Lock.global_actor_in_debug == actor.uid: - # re-entrant root process already has it: noop. - return - - # XXX: since we need to enter pdb synchronously below, - # we have to release the lock manually from pdb completion - # callbacks. Can't think of a nicer way then this atm. - if Lock._debug_lock.locked(): - log.warning( - 'Root actor attempting to shield-acquire active tty lock' - f' owned by {Lock.global_actor_in_debug}') - - # must shield here to avoid hitting a ``Cancelled`` and - # a child getting stuck bc we clobbered the tty - with trio.CancelScope(shield=True): - await Lock._debug_lock.acquire() - else: - # may be cancelled - await Lock._debug_lock.acquire() - - Lock.global_actor_in_debug = actor.uid - Lock.local_task_in_debug = task_name - Lock.repl = pdb - - try: - # block here one (at the appropriate frame *up*) where - # ``breakpoint()`` was awaited and begin handling stdio. - log.debug("Entering the synchronous world of pdb") - debug_func(actor, pdb) - - except bdb.BdbQuit: - Lock.release() - raise - - # XXX: apparently we can't do this without showing this frame - # in the backtrace on first entry to the REPL? Seems like an odd - # behaviour that should have been fixed by now. This is also why - # we scrapped all the @cm approaches that were tried previously. - # finally: - # __tracebackhide__ = True - # # frame = sys._getframe() - # # last_f = frame.f_back - # # last_f.f_globals['__tracebackhide__'] = True - # # signal.signal = pdbp.hideframe(signal.signal) - - -def shield_sigint_handler( - signum: int, - frame: 'frame', # type: ignore # noqa - # pdb_obj: Optional[MultiActorPdb] = None, - *args, - -) -> None: - ''' - Specialized, debugger-aware SIGINT handler. - - In childred we always ignore to avoid deadlocks since cancellation - should always be managed by the parent supervising actor. The root - is always cancelled on ctrl-c. - - ''' - __tracebackhide__ = True - - uid_in_debug = Lock.global_actor_in_debug - - actor = tractor.current_actor() - # print(f'{actor.uid} in HANDLER with ') - - def do_cancel(): - # If we haven't tried to cancel the runtime then do that instead - # of raising a KBI (which may non-gracefully destroy - # a ``trio.run()``). - if not actor._cancel_called: - actor.cancel_soon() - - # If the runtime is already cancelled it likely means the user - # hit ctrl-c again because teardown didn't full take place in - # which case we do the "hard" raising of a local KBI. - else: - raise KeyboardInterrupt - - any_connected = False - - if uid_in_debug is not None: - # try to see if the supposed (sub)actor in debug still - # has an active connection to *this* actor, and if not - # it's likely they aren't using the TTY lock / debugger - # and we should propagate SIGINT normally. - chans = actor._peers.get(tuple(uid_in_debug)) - if chans: - any_connected = any(chan.connected() for chan in chans) - if not any_connected: - log.warning( - 'A global actor reported to be in debug ' - 'but no connection exists for this child:\n' - f'{uid_in_debug}\n' - 'Allowing SIGINT propagation..' - ) - return do_cancel() - - # only set in the actor actually running the REPL - pdb_obj = Lock.repl - - # root actor branch that reports whether or not a child - # has locked debugger. - if ( - is_root_process() - and uid_in_debug is not None - - # XXX: only if there is an existing connection to the - # (sub-)actor in debug do we ignore SIGINT in this - # parent! Otherwise we may hang waiting for an actor - # which has already terminated to unlock. - and any_connected - ): - # we are root and some actor is in debug mode - # if uid_in_debug is not None: - - if pdb_obj: - name = uid_in_debug[0] - if name != 'root': - log.pdb( - f"Ignoring SIGINT, child in debug mode: `{uid_in_debug}`" - ) - - else: - log.pdb( - "Ignoring SIGINT while in debug mode" - ) - elif ( - is_root_process() - ): - if pdb_obj: - log.pdb( - "Ignoring SIGINT since debug mode is enabled" - ) - - if ( - Lock._root_local_task_cs_in_debug - and not Lock._root_local_task_cs_in_debug.cancel_called - ): - Lock._root_local_task_cs_in_debug.cancel() - - # revert back to ``trio`` handler asap! - Lock.unshield_sigint() - - # child actor that has locked the debugger - elif not is_root_process(): - - chan: Channel = actor._parent_chan - if not chan or not chan.connected(): - log.warning( - 'A global actor reported to be in debug ' - 'but no connection exists for its parent:\n' - f'{uid_in_debug}\n' - 'Allowing SIGINT propagation..' - ) - return do_cancel() - - task = Lock.local_task_in_debug - if ( - task - and pdb_obj - ): - log.pdb( - f"Ignoring SIGINT while task in debug mode: `{task}`" - ) - - # TODO: how to handle the case of an intermediary-child actor - # that **is not** marked in debug mode? See oustanding issue: - # https://github.com/goodboy/tractor/issues/320 - # elif debug_mode(): - - else: # XXX: shouldn't ever get here? - print("WTFWTFWTF") - raise KeyboardInterrupt - - # NOTE: currently (at least on ``fancycompleter`` 0.9.2) - # it looks to be that the last command that was run (eg. ll) - # will be repeated by default. - - # maybe redraw/print last REPL output to console since - # we want to alert the user that more input is expect since - # nothing has been done dur to ignoring sigint. - if ( - pdb_obj # only when this actor has a REPL engaged - ): - # XXX: yah, mega hack, but how else do we catch this madness XD - if pdb_obj.shname == 'xonsh': - pdb_obj.stdout.write(pdb_obj.prompt) - - pdb_obj.stdout.flush() - - # TODO: make this work like sticky mode where if there is output - # detected as written to the tty we redraw this part underneath - # and erase the past draw of this same bit above? - # pdb_obj.sticky = True - # pdb_obj._print_if_sticky() - - # also see these links for an approach from ``ptk``: - # https://github.com/goodboy/tractor/issues/130#issuecomment-663752040 - # https://github.com/prompt-toolkit/python-prompt-toolkit/blob/c2c6af8a0308f9e5d7c0e28cb8a02963fe0ce07a/prompt_toolkit/patch_stdout.py - - # XXX LEGACY: lol, see ``pdbpp`` issue: - # https://github.com/pdbpp/pdbpp/issues/496 - - -def _set_trace( - actor: tractor.Actor | None = None, - pdb: MultiActorPdb | None = None, -): - __tracebackhide__ = True - actor = actor or tractor.current_actor() - - # start 2 levels up in user code - frame: Optional[FrameType] = sys._getframe() - if frame: - frame = frame.f_back # type: ignore - - if ( - frame - and pdb - and actor is not None - ): - log.pdb(f"\nAttaching pdb to actor: {actor.uid}\n") - # no f!#$&* idea, but when we're in async land - # we need 2x frames up? - frame = frame.f_back - - else: - pdb, undo_sigint = mk_mpdb() - - # we entered the global ``breakpoint()`` built-in from sync - # code? - Lock.local_task_in_debug = 'sync' - - pdb.set_trace(frame=frame) - - -breakpoint = partial( - _breakpoint, - _set_trace, -) - - -def _post_mortem( - actor: tractor.Actor, - pdb: MultiActorPdb, - -) -> None: - ''' - Enter the ``pdbpp`` port mortem entrypoint using our custom - debugger instance. - - ''' - log.pdb(f"\nAttaching to pdb in crashed actor: {actor.uid}\n") - - # TODO: you need ``pdbpp`` master (at least this commit - # https://github.com/pdbpp/pdbpp/commit/b757794857f98d53e3ebbe70879663d7d843a6c2) - # to fix this and avoid the hang it causes. See issue: - # https://github.com/pdbpp/pdbpp/issues/480 - # TODO: help with a 3.10+ major release if/when it arrives. - - pdbp.xpm(Pdb=lambda: pdb) - - -post_mortem = partial( - _breakpoint, - _post_mortem, -) - - -async def _maybe_enter_pm(err): - if ( - debug_mode() - - # NOTE: don't enter debug mode recursively after quitting pdb - # Iow, don't re-enter the repl if the `quit` command was issued - # by the user. - and not isinstance(err, bdb.BdbQuit) - - # XXX: if the error is the likely result of runtime-wide - # cancellation, we don't want to enter the debugger since - # there's races between when the parent actor has killed all - # comms and when the child tries to contact said parent to - # acquire the tty lock. - - # Really we just want to mostly avoid catching KBIs here so there - # might be a simpler check we can do? - and not is_multi_cancelled(err) - ): - log.debug("Actor crashed, entering debug mode") - try: - await post_mortem() - finally: - Lock.release() - return True - - else: - return False - - -@acm -async def acquire_debug_lock( - subactor_uid: tuple[str, str], -) -> AsyncGenerator[None, tuple]: - ''' - Grab root's debug lock on entry, release on exit. - - This helper is for actor's who don't actually need - to acquired the debugger but want to wait until the - lock is free in the process-tree root. - - ''' - if not debug_mode(): - yield None - return - - async with trio.open_nursery() as n: - cs = await n.start( - wait_for_parent_stdin_hijack, - subactor_uid, - ) - yield None - cs.cancel() - - -async def maybe_wait_for_debugger( - poll_steps: int = 2, - poll_delay: float = 0.1, - child_in_debug: bool = False, - -) -> None: - - if ( - not debug_mode() - and not child_in_debug - ): - return - - if ( - is_root_process() - ): - # If we error in the root but the debugger is - # engaged we don't want to prematurely kill (and - # thus clobber access to) the local tty since it - # will make the pdb repl unusable. - # Instead try to wait for pdb to be released before - # tearing down. - - sub_in_debug = None - - for _ in range(poll_steps): - - if Lock.global_actor_in_debug: - sub_in_debug = tuple(Lock.global_actor_in_debug) - - log.debug('Root polling for debug') - - with trio.CancelScope(shield=True): - await trio.sleep(poll_delay) - - # TODO: could this make things more deterministic? wait - # to see if a sub-actor task will be scheduled and grab - # the tty lock on the next tick? - # XXX: doesn't seem to work - # await trio.testing.wait_all_tasks_blocked(cushion=0) - - debug_complete = Lock.no_remote_has_tty - if ( - (debug_complete and - not debug_complete.is_set()) - ): - log.debug( - 'Root has errored but pdb is in use by ' - f'child {sub_in_debug}\n' - 'Waiting on tty lock to release..') - - await debug_complete.wait() - - await trio.sleep(poll_delay) - continue - else: - log.debug( - 'Root acquired TTY LOCK' - ) diff --git a/tractor/_discovery.py b/tractor/_discovery.py index 03775ac2..99a4dd68 100644 --- a/tractor/_discovery.py +++ b/tractor/_discovery.py @@ -15,32 +15,45 @@ # along with this program. If not, see . """ -Actor discovery API. +Discovery (protocols) API for automatic addressing and location +management of (service) actors. """ +from __future__ import annotations from typing import ( - Optional, - Union, AsyncGenerator, + AsyncContextManager, + TYPE_CHECKING, ) from contextlib import asynccontextmanager as acm +import warnings +from .trionics import gather_contexts from ._ipc import _connect_chan, Channel from ._portal import ( Portal, open_portal, LocalPortal, ) -from ._state import current_actor, _runtime_vars +from ._state import ( + current_actor, + _runtime_vars, +) + + +if TYPE_CHECKING: + from ._runtime import Actor @acm -async def get_arbiter( - +async def get_registry( host: str, port: int, -) -> AsyncGenerator[Union[Portal, LocalPortal], None]: +) -> AsyncGenerator[ + Portal | LocalPortal | None, + None, +]: ''' Return a portal instance connected to a local or remote arbiter. @@ -51,16 +64,33 @@ async def get_arbiter( if not actor: raise RuntimeError("No actor instance has been defined yet?") - if actor.is_arbiter: + if actor.is_registrar: # we're already the arbiter # (likely a re-entrant call from the arbiter actor) - yield LocalPortal(actor, Channel((host, port))) + yield LocalPortal( + actor, + Channel((host, port)) + ) else: - async with _connect_chan(host, port) as chan: + async with ( + _connect_chan(host, port) as chan, + open_portal(chan) as regstr_ptl, + ): + yield regstr_ptl - async with open_portal(chan) as arb_portal: - yield arb_portal + +# TODO: deprecate and this remove _arbiter form! +@acm +async def get_arbiter(*args, **kwargs): + warnings.warn( + '`tractor.get_arbiter()` is now deprecated!\n' + 'Use `.get_registry()` instead!', + DeprecationWarning, + stacklevel=2, + ) + async with get_registry(*args, **kwargs) as to_yield: + yield to_yield @acm @@ -68,51 +98,80 @@ async def get_root( **kwargs, ) -> AsyncGenerator[Portal, None]: + # TODO: rename mailbox to `_root_maddr` when we finally + # add and impl libp2p multi-addrs? host, port = _runtime_vars['_root_mailbox'] assert host is not None - async with _connect_chan(host, port) as chan: - async with open_portal(chan, **kwargs) as portal: - yield portal + async with ( + _connect_chan(host, port) as chan, + open_portal(chan, **kwargs) as portal, + ): + yield portal @acm async def query_actor( name: str, - arbiter_sockaddr: Optional[tuple[str, int]] = None, + arbiter_sockaddr: tuple[str, int] | None = None, + regaddr: tuple[str, int] | None = None, -) -> AsyncGenerator[tuple[str, int], None]: +) -> AsyncGenerator[ + tuple[str, int] | None, + None, +]: ''' - Simple address lookup for a given actor name. + Make a transport address lookup for an actor name to a specific + registrar. - Returns the (socket) address or ``None``. + Returns the (socket) address or ``None`` if no entry under that + name exists for the given registrar listening @ `regaddr`. ''' - actor = current_actor() - async with get_arbiter( - *arbiter_sockaddr or actor._arb_addr - ) as arb_portal: + actor: Actor = current_actor() + if ( + name == 'registrar' + and actor.is_registrar + ): + raise RuntimeError( + 'The current actor IS the registry!?' + ) - sockaddr = await arb_portal.run_from_ns( + if arbiter_sockaddr is not None: + warnings.warn( + '`tractor.query_actor(regaddr=)` is deprecated.\n' + 'Use `registry_addrs: list[tuple]` instead!', + DeprecationWarning, + stacklevel=2, + ) + regaddr: list[tuple[str, int]] = arbiter_sockaddr + + reg_portal: Portal + regaddr: tuple[str, int] = regaddr or actor.reg_addrs[0] + async with get_registry(*regaddr) as reg_portal: + # TODO: return portals to all available actors - for now + # just the last one that registered + sockaddr: tuple[str, int] = await reg_portal.run_from_ns( 'self', 'find_actor', name=name, ) - - # TODO: return portals to all available actors - for now just - # the last one that registered - if name == 'arbiter' and actor.is_arbiter: - raise RuntimeError("The current actor is the arbiter") - - yield sockaddr if sockaddr else None + yield sockaddr @acm async def find_actor( name: str, - arbiter_sockaddr: tuple[str, int] | None = None + arbiter_sockaddr: tuple[str, int]|None = None, + registry_addrs: list[tuple[str, int]]|None = None, -) -> AsyncGenerator[Optional[Portal], None]: + only_first: bool = True, + raise_on_none: bool = False, + +) -> AsyncGenerator[ + Portal | list[Portal] | None, + None, +]: ''' Ask the arbiter to find actor(s) by name. @@ -120,24 +179,83 @@ async def find_actor( known to the arbiter. ''' - async with query_actor( - name=name, - arbiter_sockaddr=arbiter_sockaddr, - ) as sockaddr: + if arbiter_sockaddr is not None: + warnings.warn( + '`tractor.find_actor(arbiter_sockaddr=)` is deprecated.\n' + 'Use `registry_addrs: list[tuple]` instead!', + DeprecationWarning, + stacklevel=2, + ) + registry_addrs: list[tuple[str, int]] = [arbiter_sockaddr] - if sockaddr: - async with _connect_chan(*sockaddr) as chan: - async with open_portal(chan) as portal: - yield portal - else: + @acm + async def maybe_open_portal_from_reg_addr( + addr: tuple[str, int], + ): + async with query_actor( + name=name, + regaddr=addr, + ) as sockaddr: + if sockaddr: + async with _connect_chan(*sockaddr) as chan: + async with open_portal(chan) as portal: + yield portal + else: + yield None + + if not registry_addrs: + # XXX NOTE: make sure to dynamically read the value on + # every call since something may change it globally (eg. + # like in our discovery test suite)! + from . import _root + registry_addrs = ( + _runtime_vars['_registry_addrs'] + or + _root._default_lo_addrs + ) + + maybe_portals: list[ + AsyncContextManager[tuple[str, int]] + ] = list( + maybe_open_portal_from_reg_addr(addr) + for addr in registry_addrs + ) + + async with gather_contexts( + mngrs=maybe_portals, + ) as portals: + # log.runtime( + # 'Gathered portals:\n' + # f'{portals}' + # ) + # NOTE: `gather_contexts()` will return a + # `tuple[None, None, ..., None]` if no contact + # can be made with any regstrar at any of the + # N provided addrs! + if not any(portals): + if raise_on_none: + raise RuntimeError( + f'No actor "{name}" found registered @ {registry_addrs}' + ) yield None + return + + portals: list[Portal] = list(portals) + if only_first: + yield portals[0] + + else: + # TODO: currently this may return multiple portals + # given there are multi-homed or multiple registrars.. + # SO, we probably need de-duplication logic? + yield portals @acm async def wait_for_actor( name: str, arbiter_sockaddr: tuple[str, int] | None = None, - # registry_addr: tuple[str, int] | None = None, + registry_addr: tuple[str, int] | None = None, ) -> AsyncGenerator[Portal, None]: ''' @@ -146,17 +264,31 @@ async def wait_for_actor( A portal to the first registered actor is returned. ''' - actor = current_actor() + actor: Actor = current_actor() - async with get_arbiter( - *arbiter_sockaddr or actor._arb_addr, - ) as arb_portal: - sockaddrs = await arb_portal.run_from_ns( + if arbiter_sockaddr is not None: + warnings.warn( + '`tractor.wait_for_actor(arbiter_sockaddr=)` is deprecated.\n' + 'Use `registry_addr: tuple` instead!', + DeprecationWarning, + stacklevel=2, + ) + registry_addr: tuple[str, int] = arbiter_sockaddr + + # TODO: use `.trionics.gather_contexts()` like + # above in `find_actor()` as well? + reg_portal: Portal + regaddr: tuple[str, int] = registry_addr or actor.reg_addrs[0] + async with get_registry(*regaddr) as reg_portal: + sockaddrs = await reg_portal.run_from_ns( 'self', 'wait_for_actor', name=name, ) - sockaddr = sockaddrs[-1] + + # get latest registered addr by default? + # TODO: offer multi-portal yields in multi-homed case? + sockaddr: tuple[str, int] = sockaddrs[-1] async with _connect_chan(*sockaddr) as chan: async with open_portal(chan) as portal: diff --git a/tractor/_entry.py b/tractor/_entry.py index e8fb56db..e22a4f1f 100644 --- a/tractor/_entry.py +++ b/tractor/_entry.py @@ -20,6 +20,7 @@ Sub-process entry points. """ from __future__ import annotations from functools import partial +# import textwrap from typing import ( Any, TYPE_CHECKING, @@ -32,6 +33,7 @@ from .log import ( get_logger, ) from . import _state +from .devx import _debug from .to_asyncio import run_as_asyncio_guest from ._runtime import ( async_main, @@ -47,8 +49,8 @@ log = get_logger(__name__) def _mp_main( - actor: Actor, # type: ignore - accept_addr: tuple[str, int], + actor: Actor, + accept_addrs: list[tuple[str, int]], forkserver_info: tuple[Any, Any, Any, Any, Any], start_method: SpawnMethodKey, parent_addr: tuple[str, int] | None = None, @@ -77,8 +79,8 @@ def _mp_main( log.debug(f"parent_addr is {parent_addr}") trio_main = partial( async_main, - actor, - accept_addr, + actor=actor, + accept_addrs=accept_addrs, parent_addr=parent_addr ) try: @@ -91,12 +93,11 @@ def _mp_main( pass # handle it the same way trio does? finally: - log.info(f"Actor {actor.uid} terminated") + log.info(f"Subactor {actor.uid} terminated") def _trio_main( - - actor: Actor, # type: ignore + actor: Actor, *, parent_addr: tuple[str, int] | None = None, infect_asyncio: bool = False, @@ -106,33 +107,52 @@ 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}") + # __tracebackhide__: bool = True + _debug.hide_runtime_frames() _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 + 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 subactor:\n' + + + '>\n' # like a "started/play"-icon from super perspective + + + actor_info, + ) + try: if infect_asyncio: actor._infected_aio = True run_as_asyncio_guest(trio_main) else: trio.run(trio_main) - except KeyboardInterrupt: - log.cancel(f"Actor {actor.uid} received KBI") + except KeyboardInterrupt: + log.cancel( + 'Actor received KBI\n' + + + actor_info + ) finally: - log.info(f"Actor {actor.uid} terminated") + log.info( + 'Subactor terminated\n' + + + 'x\n' # like a "crossed-out/killed" from super perspective + + + actor_info + ) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 6da2e657..7164d6ab 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -14,22 +14,59 @@ # 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 +import sys +from types import ( + TracebackType, +) from typing import ( Any, + Callable, Type, + TYPE_CHECKING, ) +import textwrap import traceback -import exceptiongroup as eg import trio +from msgspec import ( + defstruct, + msgpack, + structs, + ValidationError, +) -from ._state import current_actor +from tractor._state import current_actor +from tractor.log import get_logger +from tractor.msg import ( + Error, + PayloadMsg, + MsgType, + MsgCodec, + MsgDec, + Stop, + types as msgtypes, +) +from tractor.msg.pretty_struct import ( + iter_fields, + Struct, + pformat as struct_format, +) + +if TYPE_CHECKING: + from ._context import Context + from .log import StackLevelAdapter + from ._stream import MsgStream + from ._ipc import Channel + +log = get_logger('tractor') _this_mod = importlib.import_module(__name__) @@ -38,32 +75,620 @@ class ActorFailure(Exception): "General actor failure" +class InternalError(RuntimeError): + ''' + Entirely unexpected internal machinery error indicating + a completely invalid state or interface. + + ''' + + +# NOTE: more or less should be close to these: +# 'boxed_type', +# 'src_type', +# 'src_uid', +# 'canceller', +# 'sender', +# TODO: format this better if we're going to include it. +# 'relay_path', +# +_ipcmsg_keys: list[str] = [ + fi.name + for fi, k, v + in iter_fields(Error) +] + +_body_fields: list[str] = list( + set(_ipcmsg_keys) + + # XXX NOTE: DON'T-SHOW-FIELDS + # - don't provide any extra useful info or, + # - are already shown as part of `.__repr__()` or, + # - are sub-type specific. + - { + 'src_type_str', + 'boxed_type_str', + 'tb_str', + 'relay_path', + 'cid', + 'message', + + # only ctxc should show it but `Error` does + # have it as an optional field. + 'canceller', + + # only for MTEs and generally only used + # when devving/testing/debugging. + '_msg_dict', + '_bad_msg', + } +) + + +def get_err_type(type_name: str) -> BaseException|None: + ''' + Look up an exception type by name from the set of locally + known namespaces: + + - `builtins` + - `tractor._exceptions` + - `trio` + + ''' + for ns in [ + builtins, + _this_mod, + trio, + ]: + if type_ref := getattr( + ns, + type_name, + False, + ): + return type_ref + + +def pack_from_raise( + local_err: ( + ContextCancelled + |StreamOverrun + |MsgTypeError + ), + cid: str, + hide_tb: bool = True, + + **rae_fields, + +) -> Error: + ''' + Raise the provided `RemoteActorError` subtype exception + instance locally to get a traceback and pack it into an IPC + `Error`-msg using `pack_error()` to extract the tb info. + + ''' + __tracebackhide__: bool = hide_tb + try: + raise local_err + except type(local_err) as local_err: + err_msg: dict[str, dict] = pack_error( + local_err, + cid=cid, + **rae_fields, + ) + return err_msg + + +# TODO: better compat with IPC msg structs? +# -[ ] rename to just `RemoteError` like in `mp.manager`? +# -[ ] make a `Struct`-subtype by using the .__post_init__()`? +# https://jcristharif.com/msgspec/structs.html#post-init-processing class RemoteActorError(Exception): - # TODO: local recontruction of remote exception deats - "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. + + ''' + reprol_fields: list[str] = [ + 'src_uid', + # 'relay_path', + ] + extra_body_fields: list[str] = [ + 'cid', + # NOTE: we only show this on relayed errors (aka + # "inceptions"). + 'relay_uid', + 'boxed_type', + ] + def __init__( self, message: str, - suberror_type: Type[BaseException] | None = None, - **msgdata + ipc_msg: Error|None = None, + boxed_type: Type[BaseException]|None = None, + + # NOTE: only provided by subtypes (ctxc and overruns) + # wishing to both manually instantiate and add field + # values defined on `Error` without having to construct an + # `Error()` before the exception is processed by + # `pack_error()`. + # + # TODO: a better way to support this without the extra + # private `._extra_msgdata`? + # -[ ] ctxc constructed inside `._rpc._invoke()` L:638 + # -[ ] overrun @ `._context.Context._deliver_msg()` L:1958 + **extra_msgdata, ) -> None: super().__init__(message) - self.type = suberror_type - self.msgdata = msgdata + # for manual display without having to muck with `Exception.args` + self._message: str = message + # 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._boxed_type: BaseException = boxed_type + self._src_type: BaseException|None = None + self._ipc_msg: Error|None = ipc_msg + self._extra_msgdata = extra_msgdata + + if ( + extra_msgdata + and + ipc_msg + ): + # XXX mutate the orig msg directly from + # manually provided input params. + for k, v in extra_msgdata.items(): + setattr( + self._ipc_msg, + k, + v, + ) + + # TODO: mask out eventually or place in `pack_error()` + # pre-`return` lines? + # sanity on inceptions + if boxed_type is RemoteActorError: + assert self.src_type_str != 'RemoteActorError' + assert self.src_uid not in self.relay_path + + # ensure type-str matches and round-tripping from that + # str results in same error type. + # + # TODO NOTE: this is currently exclusively for the + # `ContextCancelled(boxed_type=trio.Cancelled)` case as is + # used inside `._rpc._invoke()` atm though probably we + # should better emphasize that special (one off?) case + # either by customizing `ContextCancelled.__init__()` or + # through a special factor func? + elif boxed_type: + boxed_type_str: str = boxed_type.__name__ + if ( + ipc_msg + and + self._ipc_msg.boxed_type_str != boxed_type_str + ): + self._ipc_msg.boxed_type_str = boxed_type_str + assert self.boxed_type_str == self._ipc_msg.boxed_type_str + + # ensure any roundtripping evals to the input value + assert self.boxed_type is boxed_type @property - def src_actor_uid(self) -> tuple[str, str] | None: - return self.msgdata.get('src_actor_uid') + def message(self) -> str: + ''' + Be explicit, instead of trying to read it from the the parent + type's loosely defined `.args: tuple`: + + https://docs.python.org/3/library/exceptions.html#BaseException.args + + ''' + return self._message + + @property + def ipc_msg(self) -> Struct: + ''' + Re-render the underlying `._ipc_msg: MsgType` as + a `pretty_struct.Struct` for introspection such that the + returned value is a read-only copy of the original. + + ''' + if self._ipc_msg is None: + return None + + msg_type: MsgType = type(self._ipc_msg) + fields: dict[str, Any] = { + k: v for _, k, v in + iter_fields(self._ipc_msg) + } + return defstruct( + msg_type.__name__, + fields=fields.keys(), + bases=(msg_type, Struct), + )(**fields) + + @property + def msgdata(self) -> dict[str, Any]: + ''' + The (remote) error data provided by a merge of the + `._ipc_msg: Error` msg and any input `._extra_msgdata: dict` + (provided by subtypes via `.__init__()`). + + ''' + msgdata: dict = ( + structs.asdict(self._ipc_msg) + if self._ipc_msg + else {} + ) + return { + k: v for k, v in self._extra_msgdata.items() + } | msgdata + + @property + def src_type_str(self) -> str: + ''' + String-name of the source error's type. + + This should be the same as `.boxed_type_str` when unpacked + at the first relay/hop's receiving actor. + + ''' + return self._ipc_msg.src_type_str + + @property + def src_type(self) -> str: + ''' + Error type raised by original remote faulting actor. + + When the error has only been relayed a single actor-hop + this will be the same as the `.boxed_type`. + + ''' + if self._src_type is None: + self._src_type = get_err_type( + self._ipc_msg.src_type_str + ) + + return self._src_type + + @property + def boxed_type_str(self) -> str: + ''' + String-name of the (last hop's) boxed error type. + + ''' + bt: Type[BaseException] = self.boxed_type + if bt: + return str(bt.__name__) + + return '' + + @property + def boxed_type(self) -> Type[BaseException]: + ''' + Error type boxed by last actor IPC hop. + + ''' + if self._boxed_type is None: + self._boxed_type = get_err_type( + self._ipc_msg.boxed_type_str + ) + + return self._boxed_type + + @property + def relay_path(self) -> list[tuple]: + ''' + Return the list of actors which consecutively relayed + a boxed `RemoteActorError` the src error up until THIS + actor's hop. + + NOTE: a `list` field with the same name is expected to be + passed/updated in `.ipc_msg`. + + ''' + return self._ipc_msg.relay_path + + @property + def relay_uid(self) -> tuple[str, str]|None: + return tuple( + self._ipc_msg.relay_path[-1] + ) + + @property + def src_uid(self) -> tuple[str, str]|None: + if src_uid := ( + self._ipc_msg.src_uid + ): + return tuple(src_uid) + # TODO: use path lookup instead? + # return tuple( + # self._ipc_msg.relay_path[0] + # ) + + @property + def tb_str( + self, + indent: str = '', + ) -> str: + remote_tb: str = '' + + if self._ipc_msg: + remote_tb: str = self._ipc_msg.tb_str + else: + remote_tb = self.msgdata.get('tb_str') + + return textwrap.indent( + remote_tb or '', + prefix=indent, + ) + + def _mk_fields_str( + self, + fields: list[str], + end_char: str = '\n', + ) -> str: + _repr: str = '' + + for key in fields: + if ( + key == 'relay_uid' + and not self.is_inception() + ): + continue + + val: Any|None = ( + getattr(self, key, None) + or + getattr( + self._ipc_msg, + key, + None, + ) + ) + # TODO: for `.relay_path` on multiline? + # if not isinstance(val, str): + # val_str = pformat(val) + # else: + val_str: str = repr(val) + if val: + _repr += f'{key}={val_str}{end_char}' -class InternalActorError(RemoteActorError): - ''' - Remote internal ``tractor`` error indicating - failure of some primitive or machinery. + return _repr - ''' + def reprol(self) -> str: + ''' + Represent this error for "one line" display, like in + a field of our `Context.__repr__()` output. + + ''' + # TODO: use this matryoshka emjoi XD + # => 🪆 + reprol_str: str = ( + f'{type(self).__name__}' # type name + f'[{self.boxed_type_str}]' # parameterized by boxed type + ) + + _repr: str = self._mk_fields_str( + self.reprol_fields, + end_char=' ', + ) + if _repr: + reprol_str += '(' # init-style call + + return ( + reprol_str + + + _repr + ) + + def is_inception(self) -> bool: + ''' + Predicate which determines if the shuttled error type + is the same as the container error type; IOW is this + an "error within and error" which points to some original + source error that was relayed through multiple + actor hops. + + Ex. a relayed remote error will generally be some form of + `RemoteActorError[RemoteActorError]` with a `.src_type` which + is not of that same type. + + ''' + # if a single hop boxed error it was not relayed + # more then one hop directly from the src actor. + if ( + self.boxed_type + is + self.src_type + ): + return False + + return True + + def pformat( + self, + with_type_header: bool = True, + + ) -> str: + ''' + Format any boxed remote error by multi-line display of, + + - error's src or relay actor meta-data, + - remote runtime env's traceback, + + With optional control over the format of, + + - whether the boxed traceback is ascii-decorated with + a surrounding "box" annotating the embedded stack-trace. + - if the error's type name should be added as margins + around the field and tb content like: + + `> .. )>` + + - the placement of the `.message: str` (explicit equiv of + `.args[0]`), either placed below the `.tb_str` or in the + first line's header when the error is raised locally (since + the type name is already implicitly shown by python). + + ''' + header: str = '' + body: str = '' + message: str = '' + + # XXX when the currently raised exception is this instance, + # we do not ever use the "type header" style repr. + is_being_raised: bool = False + if ( + (exc := sys.exception()) + and + exc is self + ): + is_being_raised: bool = True + + with_type_header: bool = ( + with_type_header + and + not is_being_raised + ) + + # style + if with_type_header: + header: str = f'<{type(self).__name__}(' + + if message := self._message: + + # split off the first line so, if needed, it isn't + # indented the same like the "boxed content" which + # since there is no `.tb_str` is just the `.message`. + lines: list[str] = message.splitlines() + first: str = lines[0] + message: str = message.removeprefix(first) + + # with a type-style header we, + # - have no special message "first line" extraction/handling + # - place the message a space in from the header: + # `MsgTypeError( ..` + # ^-here + # - indent the `.message` inside the type body. + if with_type_header: + first = f' {first} )>' + + message: str = textwrap.indent( + message, + prefix=' '*2, + ) + message: str = first + message + + # IFF there is an embedded traceback-str we always + # draw the ascii-box around it. + if tb_str := self.tb_str: + fields: str = self._mk_fields_str( + _body_fields + + + self.extra_body_fields, + ) + from tractor.devx import ( + pformat_boxed_tb, + ) + body: str = pformat_boxed_tb( + tb_str=tb_str, + fields_str=fields, + field_prefix=' |_', + # ^- is so that it's placed like so, + # just after str: + return self.pformat( + with_type_header=False + ) + + def unwrap( + self, + ) -> BaseException: + ''' + Unpack the inner-most source error from it's original IPC + msg data. + + We attempt to reconstruct (as best as we can) the original + `Exception` from as it would have been raised in the + failing actor's remote env. + + ''' + src_type_ref: Type[BaseException] = self.src_type + if not src_type_ref: + raise TypeError( + 'Failed to lookup src error type:\n' + f'{self.src_type_str}' + ) + + # TODO: better tb insertion and all the fancier dunder + # metadata stuff as per `.__context__` etc. and friends: + # https://github.com/python-trio/trio/issues/611 + return src_type_ref(self.tb_str) + + # TODO: local recontruction of nested inception for a given + # "hop" / relay-node in this error's relay_path? + # => so would render a `RAE[RAE[RAE[Exception]]]` instance + # with all inner errors unpacked? + # -[ ] if this is useful shouldn't be too hard to impl right? + # def unbox(self) -> BaseException: + # ''' + # Unbox to the prior relays (aka last boxing actor's) + # inner error. + + # ''' + # if not self.relay_path: + # return self.unwrap() + + # # TODO.. + # # return self.boxed_type( + # # boxed_type=get_type_ref(.. + # raise NotImplementedError + + @property + def sender(self) -> tuple[str, str]|None: + if ( + (msg := self._ipc_msg) + and (value := msg.sender) + ): + return tuple(value) class ContextCancelled(RemoteActorError): @@ -72,12 +697,214 @@ class ContextCancelled(RemoteActorError): ``Portal.cancel_actor()`` or ``Context.cancel()``. ''' + reprol_fields: list[str] = [ + 'canceller', + ] + extra_body_fields: list[str] = [ + 'cid', + 'canceller', + ] @property - def canceller(self) -> tuple[str, str] | None: - value = self.msgdata.get('canceller') + 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: tuple[str, str]|None = self._ipc_msg.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}' + ) + + # TODO: to make `.__repr__()` work uniformly? + # src_actor_uid = canceller + + +class MsgTypeError( + RemoteActorError, +): + ''' + Equivalent of a runtime `TypeError` for IPC dialogs. + + Raise when any IPC wire-message is decoded to have invalid + field values (due to type) or for other `MsgCodec` related + violations such as having no extension-type for a field with + a custom type but no `enc/dec_hook()` support. + + Can be raised on the send or recv side of an IPC `Channel` + depending on the particular msg. + + Msgs which cause this to be raised on the `.send()` side (aka + in the "ctl" dialog phase) include: + - `Start` + - `Started` + - `Return` + + Those which cause it on on the `.recv()` side (aka the "nasty + streaming" dialog phase) are: + - `Yield` + - TODO: any embedded `.pld` type defined by user code? + + Normally the source of an error is re-raised from some + `.msg._codec` decode which itself raises in a backend interchange + lib (eg. a `msgspec.ValidationError`). + + ''' + reprol_fields: list[str] = [ + 'expected_msg_type', + ] + extra_body_fields: list[str] = [ + 'cid', + 'expected_msg', + ] + + @property + def bad_msg(self) -> PayloadMsg|None: + ''' + Ref to the the original invalid IPC shuttle msg which failed + to decode thus providing for the reason for this error. + + ''' + if ( + (_bad_msg := self.msgdata.get('_bad_msg')) + and + isinstance(_bad_msg, PayloadMsg) + ): + return _bad_msg + + elif bad_msg_dict := self.bad_msg_as_dict: + return msgtypes.from_dict_msg( + dict_msg=bad_msg_dict.copy(), + # use_pretty=True, + # ^-TODO-^ would luv to use this BUT then the + # `field_prefix` in `pformat_boxed_tb()` cucks it + # all up.. XD + ) + + return None + + @property + def bad_msg_as_dict(self) -> dict[str, Any]: + ''' + If the underlying IPC `MsgType` was received from a remote + actor but was unable to be decoded to a native `PayloadMsg` + (`Yield`|`Started`|`Return`) struct, the interchange backend + native format decoder can be used to stash a `dict` version + for introspection by the invalidating RPC task. + + Optionally when this error is constructed from + `.from_decode()` the caller can attempt to construct what + would have been the original `MsgType`-with-payload subtype + (i.e. an instance from the set of msgs in + `.msg.types._payload_msgs`) which failed validation. + + ''' + return self.msgdata.get('_bad_msg_as_dict') + + @property + def expected_msg_type(self) -> Type[MsgType]|None: + return type(self.bad_msg) + + @property + def cid(self) -> str: + # pull from required `.bad_msg` ref (or src dict) + if bad_msg := self.bad_msg: + return bad_msg.cid + + return self.msgdata['cid'] + + @classmethod + def from_decode( + cls, + message: str, + + bad_msg: PayloadMsg|None = None, + bad_msg_as_dict: dict|None = None, + + # if provided, expand and pack all RAE compat fields into the + # `._extra_msgdata` auxillary data `dict` internal to + # `RemoteActorError`. + **extra_msgdata, + + ) -> MsgTypeError: + ''' + Constuctor for easy creation from (presumably) catching + the backend interchange lib's underlying validation error + and passing context-specific meta-data to `_mk_msg_type_err()` + (which is normally the caller of this). + + ''' + if bad_msg_as_dict: + # NOTE: original "vanilla decode" of the msg-bytes + # is placed inside a value readable from + # `.msgdata['_msg_dict']` + extra_msgdata['_bad_msg_as_dict'] = bad_msg_as_dict + + # scrape out any underlying fields from the + # msg that failed validation. + for k, v in bad_msg_as_dict.items(): + if ( + # always skip a duplicate entry + # if already provided as an arg + k == '_bad_msg' and bad_msg + or + # skip anything not in the default msg-field set. + k not in _ipcmsg_keys + # k not in _body_fields + ): + continue + + extra_msgdata[k] = v + + + elif bad_msg: + if not isinstance(bad_msg, PayloadMsg): + raise TypeError( + 'The provided `bad_msg` is not a `PayloadMsg` type?\n\n' + f'{bad_msg}' + ) + extra_msgdata['_bad_msg'] = bad_msg + extra_msgdata['cid'] = bad_msg.cid + + extra_msgdata.setdefault('boxed_type', cls) + return cls( + message=message, + **extra_msgdata, + ) + + +class StreamOverrun( + RemoteActorError, + trio.TooSlowError, +): + reprol_fields: list[str] = [ + 'sender', + ] + ''' + This stream was overrun by its sender and can be optionally + handled by app code using `MsgStream.send()/.receive()`. + + ''' + class TransportClosed(trio.ClosedResourceError): "Underlying channel transport was closed prior to use" @@ -95,9 +922,6 @@ class NoRuntime(RuntimeError): "The root actor has not been initialized yet" -class StreamOverrun(trio.TooSlowError): - "This stream was overrun by sender" - class AsyncioCancelled(Exception): ''' @@ -107,90 +931,463 @@ class AsyncioCancelled(Exception): ''' +class MessagingError(Exception): + ''' + IPC related msg (typing), transaction (ordering) or dialog + handling error. + + ''' + def pack_error( - exc: BaseException, - tb=None, + exc: BaseException|RemoteActorError, -) -> dict[str, Any]: - """Create an "error message" for tranmission over - a channel (aka the wire). - """ - if tb: - tb_str = ''.join(traceback.format_tb(tb)) + cid: str|None = None, + src_uid: tuple[str, str]|None = None, + tb: TracebackType|None = None, + tb_str: str = '', + message: str = '', + +) -> Error: + ''' + 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 not tb_str: + tb_str: str = ( + ''.join(traceback.format_exception(exc)) + + # TODO: can we remove this since `exc` is required.. right? + or + # NOTE: this is just a shorthand for the "last error" as + # provided by `sys.exeception()`, see: + # - https://docs.python.org/3/library/traceback.html#traceback.print_exc + # - https://docs.python.org/3/library/traceback.html#traceback.format_exc + traceback.format_exc() + ) else: - tb_str = traceback.format_exc() + if tb_str[-2:] != '\n': + tb_str += '\n' - error_msg = { - 'tb_str': tb_str, - 'type_str': type(exc).__name__, - 'src_actor_uid': current_actor().uid, - } + # when caller provides a tb instance (say pulled from some other + # src error's `.__traceback__`) we use that as the "boxed" + # tb-string instead. + # https://docs.python.org/3/library/traceback.html#traceback.format_list + if tb: + tb_str: str = ''.join(traceback.format_tb(tb)) + tb_str - if isinstance(exc, ContextCancelled): + error_msg: dict[ # for IPC + str, + str | tuple[str, str] + ] = {} + our_uid: tuple = current_actor().uid + + if ( + isinstance(exc, RemoteActorError) + ): error_msg.update(exc.msgdata) - return {'error': error_msg} + # an onion/inception we need to pack as a nested and relayed + # remotely boxed error. + if ( + type(exc) is RemoteActorError + and (boxed := exc.boxed_type) + and boxed != RemoteActorError + ): + # sanity on source error (if needed when tweaking this) + assert (src_type := exc.src_type) != RemoteActorError + assert error_msg['src_type_str'] != 'RemoteActorError' + assert error_msg['src_type_str'] == src_type.__name__ + assert error_msg['src_uid'] != our_uid + + # set the boxed type to be another boxed type thus + # creating an "inception" when unpacked by + # `unpack_error()` in another actor who gets "relayed" + # this error Bo + # + # NOTE on WHY: since we are re-boxing and already + # boxed src error, we want to overwrite the original + # `boxed_type_str` and instead set it to the type of + # the input `exc` type. + error_msg['boxed_type_str'] = 'RemoteActorError' + + else: + error_msg['src_uid'] = src_uid or our_uid + error_msg['src_type_str'] = type(exc).__name__ + error_msg['boxed_type_str'] = type(exc).__name__ + + # XXX always append us the last relay in error propagation path + error_msg.setdefault( + 'relay_path', + [], + ).append(our_uid) + + # XXX NOTE XXX always ensure the traceback-str content is from + # the locally raised error (so, NOT the prior relay's boxed + # `._ipc_msg.tb_str`). + error_msg['tb_str'] = tb_str + error_msg['message'] = message or getattr(exc, 'message', '') + if cid is not None: + error_msg['cid'] = cid + + return Error(**error_msg) def unpack_error( + msg: Error, + chan: Channel, + box_type: RemoteActorError = RemoteActorError, - msg: dict[str, Any], - 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'] + if not isinstance(msg, Error): + return None - tb_str = error.get('tb_str', '') - message = f"{chan.uid}\n" + tb_str - type_name = error['type_str'] - suberror_type: Type[BaseException] = Exception + # try to lookup a suitable error type from the local runtime + # env then use it to construct a local instance. + # boxed_type_str: str = error_dict['boxed_type_str'] + boxed_type_str: str = msg.boxed_type_str + boxed_type: Type[BaseException] = get_err_type(boxed_type_str) - if type_name == 'ContextCancelled': - err_type = ContextCancelled - suberror_type = RemoteActorError + # retrieve the error's msg-encoded remotoe-env info + message: str = f'remote task raised a {msg.boxed_type_str!r}\n' - else: # try to lookup a suitable local error type - for ns in [ - builtins, - _this_mod, - eg, - trio, - ]: - try: - suberror_type = getattr(ns, type_name) - break - except AttributeError: - continue + # TODO: do we even really need these checks for RAEs? + if boxed_type_str in [ + 'ContextCancelled', + 'MsgTypeError', + ]: + box_type = { + 'ContextCancelled': ContextCancelled, + 'MsgTypeError': MsgTypeError, + }[boxed_type_str] + assert boxed_type is box_type - exc = err_type( + # TODO: already included by `_this_mod` in else loop right? + # + # we have an inception/onion-error so ensure + # we include the relay_path info and the + # original source error. + elif boxed_type_str == 'RemoteActorError': + assert boxed_type is RemoteActorError + assert len(msg.relay_path) >= 1 + + exc = box_type( message, - suberror_type=suberror_type, - - # unpack other fields into error type init - **msg['error'], + ipc_msg=msg, + tb_str=msg.tb_str, ) return exc -def is_multi_cancelled(exc: BaseException) -> bool: +def is_multi_cancelled( + exc: BaseException|BaseExceptionGroup +) -> 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, BaseExceptionGroup): return exc.subgroup( lambda exc: isinstance(exc, trio.Cancelled) ) is not None return False + + +def _raise_from_unexpected_msg( + ctx: Context, + msg: MsgType, + src_err: Exception, + log: StackLevelAdapter, # caller specific `log` obj + + expect_msg: Type[MsgType], + + # 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 (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 = hide_tb + + # an internal error should never get here + try: + cid: str = msg.cid + except AttributeError as src_err: + raise MessagingError( + f'IPC `Context` rx-ed msg without a ctx-id (cid)!?\n' + f'cid: {cid}\n\n' + + f'{pformat(msg)}\n' + ) from src_err + + # TODO: test that shows stream raising an expected error!!! + stream: MsgStream|None + _type: str = 'Context' + + # raise the error message in a boxed exception type! + if isinstance(msg, Error): + # match msg: + # case Error(): + exc: RemoteActorError = unpack_error( + msg, + ctx.chan, + ) + ctx._maybe_cancel_and_set_remote_error(exc) + raise exc from src_err + + # `MsgStream` termination msg. + # TODO: does it make more sense to pack + # the stream._eoc outside this in the calleer always? + # case Stop(): + elif stream := ctx._stream: + _type: str = 'MsgStream' + + if ( + stream._eoc + or + isinstance(msg, Stop) + ): + message: str = ( + f'Context[{cid}] stream was stopped by remote side\n' + f'cid: {cid}\n' + ) + log.debug(message) + + # 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! + + # XXX: this causes ``ReceiveChannel.__anext__()`` to + # raise a ``StopAsyncIteration`` **and** in our catch + # block below it will trigger ``.aclose()``. + eoc = trio.EndOfChannel( + f'Context stream ended due to msg:\n\n' + f'{pformat(msg)}\n' + ) + eoc.add_note(message) + + # 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 + + # in case there already is some underlying remote error + # that arrived which is probably the source of this stream + # closure + ctx.maybe_raise(from_src_exc=src_err) + raise eoc from src_err + + # TODO: our own transport/IPC-broke error subtype? + if stream._closed: + raise trio.ClosedResourceError('This stream was closed') + + # always re-raise the source error if no translation error case + # is activated above. + raise MessagingError( + f'{_type} was expecting a {expect_msg.__name__!r} message' + ' BUT received a non-error msg:\n\n' + f'{struct_format(msg)}' + ) from src_err + # ^-TODO-^ maybe `MsgDialogError` is better? + + +_raise_from_no_key_in_msg = _raise_from_unexpected_msg + + +def _mk_send_mte( + msg: Any|bytes|MsgType, + codec: MsgCodec|MsgDec, + + message: str|None = None, + verb_header: str = '', + + src_type_error: TypeError|None = None, + is_invalid_payload: bool = False, + + **mte_kwargs, + +) -> MsgTypeError: + ''' + Compose a `MsgTypeError` from a `Channel.send()`-side error, + normally raised witih a runtime IPC `Context`. + + ''' + if isinstance(codec, MsgDec): + raise RuntimeError( + '`codec` must be a `MsgCodec` for send-side errors?' + ) + + from tractor.devx import ( + pformat_caller_frame, + ) + # no src error from `msgspec.msgpack.Decoder.decode()` so + # prolly a manual type-check on our part. + if message is None: + tb_fmt: str = pformat_caller_frame(stack_limit=3) + message: str = ( + f'invalid msg -> {msg}: {type(msg)}\n\n' + f'{tb_fmt}\n' + f'Valid IPC msgs are:\n\n' + f'{codec.msg_spec_str}\n', + ) + elif src_type_error: + src_message: str = str(src_type_error) + patt: str = 'type ' + type_idx: int = src_message.find('type ') + invalid_type: str = src_message[type_idx + len(patt):].split()[0] + + enc_hook: Callable|None = codec.enc.enc_hook + if enc_hook is None: + message += ( + '\n\n' + + f"The current IPC-msg codec can't encode type `{invalid_type}` !\n" + f'Maybe a `msgpack.Encoder.enc_hook()` extension is needed?\n\n' + + f'Check the `msgspec` docs for ad-hoc type extending:\n' + '|_ https://jcristharif.com/msgspec/extending.html\n' + '|_ https://jcristharif.com/msgspec/extending.html#defining-a-custom-extension-messagepack-only\n' + ) + + msgtyperr = MsgTypeError( + message=message, + _bad_msg=msg, + ) + # ya, might be `None` + msgtyperr.__cause__ = src_type_error + return msgtyperr + + +def _mk_recv_mte( + msg: Any|bytes|MsgType, + codec: MsgCodec|MsgDec, + + message: str|None = None, + verb_header: str = '', + + src_validation_error: ValidationError|None = None, + is_invalid_payload: bool = False, + + **mte_kwargs, + +) -> MsgTypeError: + ''' + Compose a `MsgTypeError` from a + `Channel|Context|MsgStream.receive()`-side error, + normally raised witih a runtime IPC ctx or streaming + block. + + ''' + msg_dict: dict|None = None + bad_msg: PayloadMsg|None = None + + if is_invalid_payload: + msg_type: str = type(msg) + any_pld: Any = msgpack.decode(msg.pld) + message: str = ( + f'invalid `{msg_type.__qualname__}` msg payload\n\n' + f'value: `{any_pld!r}` does not match type-spec: ' + f'`{type(msg).__qualname__}.pld: {codec.pld_spec_str}`' + ) + bad_msg = msg + + else: + # decode the msg-bytes using the std msgpack + # interchange-prot (i.e. without any `msgspec.Struct` + # handling) so that we can determine what + # `.msg.types.PayloadMsg` is the culprit by reporting the + # received value. + msg: bytes + msg_dict: dict = msgpack.decode(msg) + msg_type_name: str = msg_dict['msg_type'] + msg_type = getattr(msgtypes, msg_type_name) + message: str = ( + f'invalid `{msg_type_name}` IPC msg\n\n' + ) + # XXX be "fancy" and see if we can determine the exact + # invalid field such that we can comprehensively report + # the specific field's type problem. + msgspec_msg: str = src_validation_error.args[0].rstrip('`') + msg, _, maybe_field = msgspec_msg.rpartition('$.') + obj = object() + if (field_val := msg_dict.get(maybe_field, obj)) is not obj: + field_name_expr: str = ( + f' |_{maybe_field}: {codec.pld_spec_str} = ' + ) + fmt_val_lines: list[str] = pformat(field_val).splitlines() + fmt_val: str = ( + f'{fmt_val_lines[0]}\n' + + + textwrap.indent( + '\n'.join(fmt_val_lines[1:]), + prefix=' '*len(field_name_expr), + ) + ) + message += ( + f'{msg.rstrip("`")}\n\n' + f'<{msg_type.__qualname__}(\n' + # f'{".".join([msg_type.__module__, msg_type.__qualname__])}\n' + f'{field_name_expr}{fmt_val}\n' + f')>' + ) + + if verb_header: + message = f'{verb_header} ' + message + + msgtyperr = MsgTypeError.from_decode( + message=message, + bad_msg=bad_msg, + bad_msg_as_dict=msg_dict, + boxed_type=type(src_validation_error), + + # NOTE: for pld-spec MTEs we set the `._ipc_msg` manually: + # - for the send-side `.started()` pld-validate + # case we actually raise inline so we don't need to + # set the it at all. + # - for recv side we set it inside `PldRx.decode_pld()` + # after a manual call to `pack_error()` since we + # actually want to emulate the `Error` from the mte we + # build here. So by default in that case, this is left + # as `None` here. + # ipc_msg=src_err_msg, + ) + msgtyperr.__cause__ = src_validation_error + return msgtyperr diff --git a/tractor/_ipc.py b/tractor/_ipc.py index ebfd261c..e5e3d10f 100644 --- a/tractor/_ipc.py +++ b/tractor/_ipc.py @@ -19,34 +19,50 @@ Inter-process comms abstractions """ from __future__ import annotations -import platform -import struct -import typing from collections.abc import ( AsyncGenerator, AsyncIterator, ) +from contextlib import ( + asynccontextmanager as acm, + contextmanager as cm, +) +import platform +from pprint import pformat +import struct +import typing from typing import ( Any, + Callable, runtime_checkable, - Optional, Protocol, Type, 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 ( + MsgTypeError, + pack_from_raise, + TransportClosed, + _mk_send_mte, + _mk_recv_mte, +) +from tractor.msg import ( + _ctxvar_MsgCodec, + _codec, + MsgCodec, + types as msgtypes, + pretty_struct, +) + log = get_logger(__name__) - _is_windows = platform.system() == 'Windows' -log = get_logger(__name__) def get_stream_addrs(stream: trio.SocketStream) -> tuple: @@ -59,6 +75,7 @@ def get_stream_addrs(stream: trio.SocketStream) -> tuple: ) +# TODO: this should be our `Union[*msgtypes.__spec__]` now right? MsgType = TypeVar("MsgType") # TODO: consider using a generic def and indexing with our eventual @@ -112,11 +129,28 @@ 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, prefix_size: int = 4, + # XXX optionally provided codec pair for `msgspec`: + # https://jcristharif.com/msgspec/extending.html#mapping-to-from-native-types + # + # TODO: define this as a `Codec` struct which can be + # overriden dynamically by the application/runtime. + codec: tuple[ + Callable[[Any], Any]|None, # coder + Callable[[type, Any], Any]|None, # decoder + ]|None = None, + ) -> None: self.stream = stream @@ -132,24 +166,36 @@ class MsgpackTCPStream(MsgTransport): # public i guess? self.drained: list[dict] = [] - self.recv_stream = BufferedReceiveStream(transport_stream=stream) + self.recv_stream = BufferedReceiveStream( + transport_stream=stream + ) self.prefix_size = prefix_size - # TODO: struct aware messaging coders - self.encode = msgspec.msgpack.Encoder().encode - self.decode = msgspec.msgpack.Decoder().decode # dict[str, Any]) + # allow for custom IPC msg interchange format + # dynamic override Bo + self._task = trio.lowlevel.current_task() + self._codec: MsgCodec = ( + codec + or + _codec._ctxvar_MsgCodec.get() + ) + # TODO: mask out before release? + # log.runtime( + # f'New {self} created with codec\n' + # f'codec: {self._codec}\n' + # ) async def _iter_packets(self) -> AsyncGenerator[dict, None]: - '''Yield packets from the underlying stream. + ''' + Yield `bytes`-blob decoded packets from the underlying TCP + stream using the current task's `MsgCodec`. ''' - import msgspec # noqa decodes_failed: int = 0 while True: try: - header = await self.recv_stream.receive_exactly(4) - + header: bytes = await self.recv_stream.receive_exactly(4) except ( ValueError, ConnectionResetError, @@ -171,12 +217,56 @@ class MsgpackTCPStream(MsgTransport): size, = struct.unpack(" None: + async def send( + self, + msg: msgtypes.MsgType, + + strict_types: bool = True, + # hide_tb: bool = False, + ) -> None: + ''' + Send a msgpack encoded py-object-blob-as-msg over TCP. + + If `strict_types == True` then a `MsgTypeError` will be raised on any + invalid msg type + + ''' + # __tracebackhide__: bool = hide_tb async with self._send_lock: - bytes_data: bytes = self.encode(msg) + # NOTE: lookup the `trio.Task.context`'s var for + # the current `MsgCodec`. + codec: MsgCodec = _ctxvar_MsgCodec.get() + + # TODO: mask out before release? + if self._codec.pld_spec != codec.pld_spec: + self._codec = codec + log.runtime( + f'Using new codec in {self}.send()\n' + f'codec: {self._codec}\n\n' + f'msg: {msg}\n' + ) + + if type(msg) not in msgtypes.__msg_types__: + if strict_types: + raise _mk_send_mte( + msg, + codec=codec, + ) + else: + log.warning( + 'Sending non-`Msg`-spec msg?\n\n' + f'{msg}\n' + ) + + try: + bytes_data: bytes = codec.encode(msg) + except TypeError as _err: + typerr = _err + msgtyperr: MsgTypeError = _mk_send_mte( + msg, + codec=codec, + message=( + f'IPC-msg-spec violation in\n\n' + f'{pretty_struct.Struct.pformat(msg)}' + ), + src_type_error=typerr, + ) + raise msgtyperr from typerr # supposedly the fastest says, # https://stackoverflow.com/a/54027962 size: bytes = struct.pack(" 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( @@ -307,37 +462,79 @@ 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, + + # XXX optionally provided codec pair for `msgspec`: + # https://jcristharif.com/msgspec/extending.html#mapping-to-from-native-types + codec: MsgCodec|None = None, ) -> MsgTransport: - type_key = type_key or self._transport_key - self.msgstream = get_msg_transport(type_key)(stream) - return self.msgstream + type_key = ( + type_key + or + self._transport_key + ) + # get transport type, then + self._transport = get_msg_transport( + type_key + # instantiate an instance of the msg-transport + )( + stream, + codec=codec, + ) + return self._transport + + # TODO: something simliar at the IPC-`Context` + # level so as to support + @cm + def apply_codec( + self, + codec: MsgCodec, + + ) -> None: + ''' + Temporarily override the underlying IPC msg codec for + dynamic enforcement of messaging schema. + + ''' + orig: MsgCodec = self._transport.codec + try: + self._transport.codec = codec + yield + finally: + self._transport.codec = orig 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, @@ -356,26 +553,46 @@ 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: + async def send( + self, + payload: Any, - log.transport(f"send `{item}`") # type: ignore - assert self.msgstream + # hide_tb: bool = False, - await self.msgstream.send(item) + ) -> None: + ''' + Send a coded msg-blob over the transport. + + ''' + # __tracebackhide__: bool = hide_tb + log.transport( + '=> send IPC msg:\n\n' + f'{pformat(payload)}\n' + ) # type: ignore + assert self._transport + await self._transport.send( + payload, + # hide_tb=hide_tb, + ) async def recv(self) -> Any: - assert self.msgstream - return await self.msgstream.recv() + assert self._transport + return await self._transport.recv() + # TODO: auto-reconnect features like 0mq/nanomsg? + # -[ ] implement it manually with nods to SC prot + # possibly on multiple transport backends? + # -> seems like that might be re-inventing scalability + # prots tho no? # try: - # return await self.msgstream.recv() + # return await self._transport.recv() # except trio.BrokenResourceError: # if self._autorecon: # await self._reconnect() @@ -388,8 +605,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): @@ -440,16 +657,38 @@ class Channel: Async iterate items from underlying stream. ''' - assert self.msgstream + assert self._transport while True: try: - async for item in self.msgstream: - yield item + async for msg in self._transport: + match msg: + # NOTE: if transport/interchange delivers + # a type error, we pack it with the far + # end peer `Actor.uid` and relay the + # `Error`-msg upward to the `._rpc` stack + # for normal RAE handling. + case MsgTypeError(): + yield pack_from_raise( + local_err=msg, + cid=msg.cid, + + # XXX we pack it here bc lower + # layers have no notion of an + # actor-id ;) + src_uid=self.uid, + ) + case _: + yield msg + + # TODO: if we were gonna do this it should be + # done up at the `MsgStream` layer! + # # sent = yield item # if sent is not None: # # optimization, passing None through all the # # time is pointless - # await self.msgstream.send(sent) + # await self._transport.send(sent) + except trio.BrokenResourceError: # if not self._autorecon: @@ -462,12 +701,14 @@ 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 +@acm async def _connect_chan( - host: str, port: int + host: str, + port: int + ) -> typing.AsyncGenerator[Channel, None]: ''' Create and connect a channel with disconnect on context manager @@ -477,4 +718,5 @@ async def _connect_chan( chan = Channel((host, port)) await chan.connect() yield chan - await chan.aclose() + with trio.CancelScope(shield=True): + await chan.aclose() diff --git a/tractor/_multiaddr.py b/tractor/_multiaddr.py new file mode 100644 index 00000000..e8713b40 --- /dev/null +++ b/tractor/_multiaddr.py @@ -0,0 +1,151 @@ +# 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 . + +''' +Multiaddress parser and utils according the spec(s) defined by +`libp2p` and used in dependent project such as `ipfs`: + +- https://docs.libp2p.io/concepts/fundamentals/addressing/ +- https://github.com/libp2p/specs/blob/master/addressing/README.md + +''' +from typing import Iterator + +from bidict import bidict + +# TODO: see if we can leverage libp2p ecosys projects instead of +# rolling our own (parser) impls of the above addressing specs: +# - https://github.com/libp2p/py-libp2p +# - https://docs.libp2p.io/concepts/nat/circuit-relay/#relay-addresses +# prots: bidict[int, str] = bidict({ +prots: bidict[int, str] = { + 'ipv4': 3, + 'ipv6': 3, + 'wg': 3, + + 'tcp': 4, + 'udp': 4, + + # TODO: support the next-gen shite Bo + # 'quic': 4, + # 'ssh': 7, # via rsyscall bootstrapping +} + +prot_params: dict[str, tuple[str]] = { + 'ipv4': ('addr',), + 'ipv6': ('addr',), + 'wg': ('addr', 'port', 'pubkey'), + + 'tcp': ('port',), + 'udp': ('port',), + + # 'quic': ('port',), + # 'ssh': ('port',), +} + + +def iter_prot_layers( + multiaddr: str, +) -> Iterator[ + tuple[ + int, + list[str] + ] +]: + ''' + Unpack a libp2p style "multiaddress" into multiple "segments" + for each "layer" of the protocoll stack (in OSI terms). + + ''' + tokens: list[str] = multiaddr.split('/') + root, tokens = tokens[0], tokens[1:] + assert not root # there is a root '/' on LHS + itokens = iter(tokens) + + prot: str | None = None + params: list[str] = [] + for token in itokens: + # every prot path should start with a known + # key-str. + if token in prots: + if prot is None: + prot: str = token + else: + yield prot, params + prot = token + + params = [] + + elif token not in prots: + params.append(token) + + else: + yield prot, params + + +def parse_maddr( + multiaddr: str, +) -> dict[str, str | int | dict]: + ''' + Parse a libp2p style "multiaddress" into its distinct protocol + segments where each segment is of the form: + + `..////../` + + and is loaded into a (order preserving) `layers: dict[str, + dict[str, Any]` which holds each protocol-layer-segment of the + original `str` path as a separate entry according to its approx + OSI "layer number". + + Any `paramN` in the path must be distinctly defined by a str-token in the + (module global) `prot_params` table. + + For eg. for wireguard which requires an address, port number and publickey + the protocol params are specified as the entry: + + 'wg': ('addr', 'port', 'pubkey'), + + and are thus parsed from a maddr in that order: + `'/wg/1.1.1.1/51820/'` + + ''' + layers: dict[str, str | int | dict] = {} + for ( + prot_key, + params, + ) in iter_prot_layers(multiaddr): + + layer: int = prots[prot_key] # OSI layer used for sorting + ep: dict[str, int | str] = {'layer': layer} + layers[prot_key] = ep + + # TODO; validation and resolving of names: + # - each param via a validator provided as part of the + # prot_params def? (also see `"port"` case below..) + # - do a resolv step that will check addrs against + # any loaded network.resolv: dict[str, str] + rparams: list = list(reversed(params)) + for key in prot_params[prot_key]: + val: str | int = rparams.pop() + + # TODO: UGHH, dunno what we should do for validation + # here, put it in the params spec somehow? + if key == 'port': + val = int(val) + + ep[key] = val + + return layers diff --git a/tractor/_portal.py b/tractor/_portal.py index 60293716..2c676e12 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -15,71 +15,70 @@ # 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 +from contextlib import asynccontextmanager as acm import importlib import inspect from typing import ( - Any, Optional, - Callable, AsyncGenerator, - Type, + Any, + Callable, + AsyncGenerator, + TYPE_CHECKING, ) from functools import partial from dataclasses import dataclass -from pprint import pformat import warnings import trio -from async_generator import asynccontextmanager from .trionics import maybe_open_nursery -from ._state import current_actor +from ._state import ( + current_actor, +) from ._ipc import Channel from .log import get_logger -from .msg import NamespacePath -from ._exceptions import ( - unpack_error, - NoResult, - ContextCancelled, +from .msg import ( + # Error, + PayloadMsg, + NamespacePath, + Return, +) +from ._exceptions import ( + # unpack_error, + NoResult, +) +from ._context import ( + Context, + open_context_from_portal, +) +from ._streaming import ( + MsgStream, ) -from ._context import Context -from ._streaming import MsgStream +if TYPE_CHECKING: + from ._runtime import Actor log = get_logger(__name__) -def _unwrap_msg( - msg: dict[str, Any], - channel: Channel - -) -> Any: - __tracebackhide__ = True - try: - return msg['return'] - except KeyError: - # internal error should never get here - assert msg.get('cid'), "Received internal error at portal?" - raise unpack_error(msg, channel) from None - - -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,22 +88,40 @@ 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 + def __init__( + self, + channel: Channel, + ) -> None: + + self.chan = channel # during the portal's lifetime - self._result_msg: Optional[dict] = None + self._final_result_pld: Any|None = None + self._final_result_msg: PayloadMsg|None = None # 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: Context | None = None + self._expect_result_ctx: Context|None = None self._streams: set[MsgStream] = set() - self.actor = current_actor() + self.actor: 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 + + # TODO: factor this out into an `ActorNursery` wrapper async def _submit_for_result( self, ns: str, @@ -112,32 +129,25 @@ class Portal: **kwargs ) -> None: - assert self._expect_result is None, \ - "A pending main result has already been submitted" + if self._expect_result_ctx is not None: + raise RuntimeError( + 'A pending main result has already been submitted' + ) - self._expect_result = await self.actor.start_remote_task( + self._expect_result_ctx: Context = await self.actor.start_remote_task( self.channel, - ns, - func, - kwargs + nsf=NamespacePath(f'{ns}:{func}'), + kwargs=kwargs, + portal=self, ) - async def _return_once( - self, - ctx: Context, - - ) -> dict[str, Any]: - - assert ctx._remote_func_type == 'asyncfunc' # single response - msg = await ctx._recv_chan.receive() - return msg - + # @api_frame async def result(self) -> Any: ''' Return the result(s) from the remote actor's "main" task. ''' - # __tracebackhide__ = True + __tracebackhide__ = True # Check for non-rpc errors slapped on the # channel for which we always raise exc = self.channel._exc @@ -145,7 +155,7 @@ class Portal: raise exc # not expecting a "main" result - if self._expect_result is None: + if self._expect_result_ctx is None: log.warning( f"Portal for {self.channel.uid} not expecting a final" " result?\nresult() should only be called if subactor" @@ -153,14 +163,24 @@ class Portal: return NoResult # expecting a "main" result - assert self._expect_result + assert self._expect_result_ctx - if self._result_msg is None: - self._result_msg = await self._return_once( - self._expect_result - ) + if self._final_result_msg is None: + try: + ( + self._final_result_msg, + self._final_result_pld, + ) = await self._expect_result_ctx._pld_rx.recv_msg_w_pld( + ipc=self._expect_result_ctx, + expect_msg=Return, + ) + except BaseException as err: + # TODO: wrap this into `@api_frame` optionally with + # some kinda filtering mechanism like log levels? + __tracebackhide__: bool = False + raise err - return _unwrap_msg(self._result_msg, self.channel) + return self._final_result_pld async def _cancel_streams(self): # terminate all locally running async generator @@ -191,33 +211,59 @@ 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(): - log.cancel("This channel is already closed can't cancel") + __runtimeframe__: int = 1 # noqa + + chan: Channel = self.channel + if not chan.connected(): + log.runtime( + 'This channel is already closed, skipping cancel request..' + ) return False + reminfo: str = ( + f'Portal.cancel_actor() => {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'Requesting runtime cancel for peer\n\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 + # 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 - - await self.run_from_ns('self', 'cancel') + cs.shield: bool = True + await self.run_from_ns( + 'self', + 'cancel', + ) return True if cs.cancelled_caught: - log.cancel(f"May have failed to cancel {self.channel.uid}") + # 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}' + ) # if we get here some weird cancellation case happened return False @@ -226,9 +272,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( @@ -247,27 +295,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! ''' - ctx = await self.actor.start_remote_task( - self.channel, - namespace_path, - function_name, - kwargs, + __runtimeframe__: int = 1 # noqa + nsf = NamespacePath( + f'{namespace_path}:{function_name}' + ) + ctx: Context = await self.actor.start_remote_task( + chan=self.channel, + nsf=nsf, + kwargs=kwargs, + portal=self, + ) + return await ctx._pld_rx.recv_pld( + ipc=ctx, + expect_msg=Return, ) - ctx._portal = self - msg = await self._return_once(ctx) - 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 @@ -277,6 +331,8 @@ class Portal: remote rpc task or a local async generator instance. ''' + __runtimeframe__: int = 1 # noqa + if isinstance(func, str): warnings.warn( "`Portal.run(namespace: str, funcname: str)` is now" @@ -286,8 +342,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 ( @@ -300,27 +357,34 @@ 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, + portal=self, ) - ctx._portal = self - return _unwrap_msg( - await self._return_once(ctx), - self.channel, + return await ctx._pld_rx.recv_pld( + ipc=ctx, + expect_msg=Return, ) - @asynccontextmanager + @acm async def open_stream_from( self, async_gen_func: Callable, # typing: ignore **kwargs, ) -> AsyncGenerator[MsgStream, None]: + ''' + Legacy one-way streaming API. + + TODO: re-impl on top `Portal.open_context()` + an async gen + around `Context.open_stream()`. + + ''' + __runtimeframe__: int = 1 # noqa if not inspect.isasyncgenfunction(async_gen_func): if not ( @@ -330,17 +394,12 @@ 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, + portal=self, ) - ctx._portal = self # ensure receive-only stream entrypoint assert ctx._remote_func_type == 'asyncgen' @@ -348,13 +407,14 @@ class Portal: try: # deliver receive only stream async with MsgStream( - ctx, ctx._recv_chan, - ) as rchan: - self._streams.add(rchan) - yield rchan + ctx=ctx, + rx_chan=ctx._rx_chan, + ) as stream: + self._streams.add(stream) + ctx._stream = stream + yield stream finally: - # cancel the far end task on consumer close # NOTE: this is a special case since we assume that if using # this ``.open_fream_from()`` api, the stream is one a one @@ -373,205 +433,14 @@ class Portal: # XXX: should this always be done? # await recv_chan.aclose() - self._streams.remove(rchan) + self._streams.remove(stream) - @asynccontextmanager - async def open_context( - - self, - func: Callable, - allow_overruns: bool = False, - **kwargs, - - ) -> AsyncGenerator[tuple[Context, Any], None]: - ''' - Open an inter-actor task context. - - 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``. - - ''' - # 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. - 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, - - # 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 = 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 - - except KeyError: - assert msg.get('cid'), ("Received internal error at context?") - - if msg.get('error'): - # raise kerr from unpack_error(msg, self.channel) - 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)}' - ) - - _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. - try: - 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: - # 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. - else: - log.debug(f'Context {ctx} cancelled gracefully') - - except ( - BaseException, - - # more specifically, we need to handle these but not - # sure it's worth being pedantic: - # Exception, - # trio.Cancelled, - # KeyboardInterrupt, - - ) as err: - etype = type(err) - - # 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() - except trio.BrokenResourceError: - log.warning( - 'IPC connection for context is broken?\n' - f'task:{cid}\n' - f'actor:{uid}' - ) - - raise - - else: - if ctx.chan.connected(): - log.info( - 'Waiting on final context-task result for\n' - f'task: {cid}\n' - f'actor: {uid}' - ) - result = await ctx.result() - log.runtime( - f'Context {fn_name} returned ' - 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 - # 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() - - if etype: - if ctx._cancel_called: - log.cancel( - f'Context {fn_name} cancelled by caller with\n{etype}' - ) - elif _err is not None: - log.cancel( - f'Context for task 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 - # ``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. - from ._debug import maybe_wait_for_debugger - await maybe_wait_for_debugger() - - # remove the context from runtime tracking - self.actor._contexts.pop( - (self.channel.uid, ctx.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 @@ -586,7 +455,12 @@ class LocalPortal: actor: 'Actor' # type: ignore # noqa channel: Channel - async def run_from_ns(self, ns: str, func_name: str, **kwargs) -> Any: + async def run_from_ns( + self, + ns: str, + func_name: str, + **kwargs, + ) -> Any: ''' Run a requested local function from a namespace path and return it's result. @@ -597,11 +471,11 @@ class LocalPortal: return await func(**kwargs) -@asynccontextmanager +@acm async def open_portal( channel: Channel, - nursery: Optional[trio.Nursery] = None, + tn: trio.Nursery|None = None, start_msg_loop: bool = True, shield: bool = False, @@ -609,15 +483,19 @@ async def open_portal( ''' Open a ``Portal`` through the provided ``channel``. - Spawns a background task to handle message processing (normally - done by the actor-runtime implicitly). + Spawns a background task to handle RPC processing, normally + done by the actor-runtime implicitly via a call to + `._rpc.process_messages()`. just after connection establishment. ''' actor = current_actor() assert actor - was_connected = False + was_connected: bool = False - async with maybe_open_nursery(nursery, shield=shield) as nursery: + async with maybe_open_nursery( + tn, + shield=shield, + ) as tn: if not channel.connected(): await channel.connect() @@ -626,10 +504,10 @@ 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( + msg_loop_cs = await tn.start( partial( process_messages, actor, @@ -646,12 +524,10 @@ async def open_portal( await portal.aclose() if was_connected: - # gracefully signal remote channel-msg loop - await channel.send(None) - # await channel.aclose() + await channel.aclose() # cancel background msg loop task - if msg_loop_cs: + if msg_loop_cs is not None: msg_loop_cs.cancel() - nursery.cancel_scope.cancel() + tn.cancel_scope.cancel() diff --git a/tractor/_root.py b/tractor/_root.py index a2d31586..3b8bd158 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -18,26 +18,27 @@ Root actor runtime ignition(s). ''' -from contextlib import asynccontextmanager +from contextlib import asynccontextmanager as acm from functools import partial import importlib import logging +import os import signal import sys -import os -import typing +from typing import Callable 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 +from .devx import _debug from . import _spawn from . import _state from . import log @@ -46,60 +47,117 @@ from ._exceptions import is_multi_cancelled # set at startup and after forks -_default_arbiter_host: str = '127.0.0.1' -_default_arbiter_port: int = 1616 +_default_host: str = '127.0.0.1' +_default_port: int = 1616 + +# default registry always on localhost +_default_lo_addrs: list[tuple[str, int]] = [( + _default_host, + _default_port, +)] logger = log.get_logger('tractor') -@asynccontextmanager +@acm async def open_root_actor( *, # defaults are above - arbiter_addr: tuple[str, int] | None = None, + registry_addrs: list[tuple[str, int]]|None = None, # defaults are above - registry_addr: tuple[str, int] | None = None, - - name: str | None = 'root', + arbiter_addr: tuple[str, int]|None = None, + + # binding addrs for the transport layer server + trans_bind_addrs: list[tuple[str, int]] = [(_default_host, _default_port)], + + name: str|None = 'root', # either the `multiprocessing` start method: # https://docs.python.org/3/library/multiprocessing.html#contexts-and-start-methods # OR `trio` (the new default). - start_method: _spawn.SpawnMethodKey | None = None, + start_method: _spawn.SpawnMethodKey|None = None, # enables the multi-process debugger support debug_mode: bool = False, + maybe_enable_greenback: bool = False, # `.pause_from_sync()/breakpoint()` support + enable_stack_on_sig: bool = False, # internal logging - loglevel: str | None = None, + loglevel: str|None = None, - enable_modules: list | None = None, - rpc_module_paths: list | None = None, + enable_modules: list|None = None, + rpc_module_paths: list|None = None, -) -> typing.Any: + # NOTE: allow caller to ensure that only one registry exists + # and that this call creates it. + ensure_registry: bool = False, + + hide_tb: bool = True, + +) -> Actor: ''' Runtime init entry point for ``tractor``. ''' + __tracebackhide__: bool = hide_tb + _debug.hide_runtime_frames() + + # TODO: stick this in a `@cm` defined in `devx._debug`? + # # Override the global debugger hook to make it play nice with # ``trio``, see much discussion in: # https://github.com/python-trio/trio/issues/1155#issuecomment-742964018 - builtin_bp_handler = sys.breakpointhook - orig_bp_path: str | None = os.environ.get('PYTHONBREAKPOINT', None) - os.environ['PYTHONBREAKPOINT'] = 'tractor._debug._set_trace' + builtin_bp_handler: Callable = sys.breakpointhook + orig_bp_path: str|None = os.environ.get( + 'PYTHONBREAKPOINT', + None, + ) + if ( + debug_mode + and maybe_enable_greenback + and await _debug.maybe_init_greenback( + raise_not_found=False, + ) + ): + os.environ['PYTHONBREAKPOINT'] = ( + 'tractor.devx._debug._sync_pause_from_builtin' + ) + _state._runtime_vars['use_greenback'] = True + + else: + # TODO: disable `breakpoint()` by default (without + # `greenback`) since it will break any multi-actor + # usage by a clobbered TTY's stdstreams! + def block_bps(*args, **kwargs): + raise RuntimeError( + 'Trying to use `breakpoint()` eh?\n\n' + 'Welp, `tractor` blocks `breakpoint()` built-in calls by default!\n' + 'If you need to use it please install `greenback` and set ' + '`debug_mode=True` when opening the runtime ' + '(either via `.open_nursery()` or `open_root_actor()`)\n' + ) + + sys.breakpointhook = block_bps + # lol ok, + # https://docs.python.org/3/library/sys.html#sys.breakpointhook + os.environ['PYTHONBREAKPOINT'] = "0" # attempt to retreive ``trio``'s sigint handler and stash it # on our debugger lock state. - _debug.Lock._trio_handler = signal.getsignal(signal.SIGINT) + _debug.DebugStatus._trio_handler = signal.getsignal(signal.SIGINT) # mark top most level process as root actor _state._runtime_vars['_is_root'] = True # caps based rpc list - enable_modules = enable_modules or [] + enable_modules = ( + enable_modules + or + [] + ) if rpc_module_paths: warnings.warn( @@ -115,29 +173,36 @@ async def open_root_actor( if arbiter_addr is not None: warnings.warn( - '`arbiter_addr` is now deprecated and has been renamed to' - '`registry_addr`.\nUse that instead..', + '`arbiter_addr` is now deprecated\n' + 'Use `registry_addrs: list[tuple]` instead..', DeprecationWarning, stacklevel=2, ) + registry_addrs = [arbiter_addr] - registry_addr = (host, port) = ( - registry_addr - or arbiter_addr - or ( - _default_arbiter_host, - _default_arbiter_port, - ) + registry_addrs: list[tuple[str, int]] = ( + registry_addrs + or + _default_lo_addrs ) + assert registry_addrs + + assert trans_bind_addrs - loglevel = (loglevel or log._default_loglevel).upper() + loglevel = ( + loglevel + or log._default_loglevel + ).upper() - if debug_mode and _spawn._spawn_method == 'trio': + if ( + debug_mode + and _spawn._spawn_method == 'trio' + ): _state._runtime_vars['_debug_mode'] = True - # expose internal debug module to every actor allowing - # for use of ``await tractor.breakpoint()`` - enable_modules.append('tractor._debug') + # expose internal debug module to every actor allowing for + # use of ``await tractor.pause()`` + enable_modules.append('tractor.devx._debug') # if debug mode get's enabled *at least* use that level of # logging for some informative console prompts. @@ -150,80 +215,149 @@ async def open_root_actor( ): loglevel = 'PDB' + elif debug_mode: raise RuntimeError( "Debug mode is only supported for the `trio` backend!" ) - log.get_console_log(loglevel) + assert loglevel + _log = log.get_console_log(loglevel) + assert _log - try: - # make a temporary connection to see if an arbiter exists, - # if one can't be made quickly we assume none exists. - arbiter_found = False + # TODO: factor this into `.devx._stackscope`!! + if ( + debug_mode + and + enable_stack_on_sig + ): + try: + logger.info('Enabling `stackscope` traces on SIGUSR1') + from .devx import enable_stack_on_sig + enable_stack_on_sig() + except ImportError: + logger.warning( + '`stackscope` not installed for use in debug mode!' + ) - # TODO: this connect-and-bail forces us to have to carefully - # rewrap TCP 104-connection-reset errors as EOF so as to avoid - # propagating cancel-causing errors to the channel-msg loop - # machinery. Likely it would be better to eventually have - # a "discovery" protocol with basic handshake instead. - with trio.move_on_after(1): - async with _connect_chan(host, port): - arbiter_found = True + # closed into below ping task-func + ponged_addrs: list[tuple[str, int]] = [] - except OSError: - # TODO: make this a "discovery" log level? - logger.warning(f"No actor registry found @ {host}:{port}") + async def ping_tpt_socket( + addr: tuple[str, int], + timeout: float = 1, + ) -> None: + ''' + Attempt temporary connection to see if a registry is + listening at the requested address by a tranport layer + ping. - # create a local actor and start up its main routine/task - if arbiter_found: + If a connection can't be made quickly we assume none no + server is listening at that addr. + + ''' + try: + # TODO: this connect-and-bail forces us to have to + # carefully rewrap TCP 104-connection-reset errors as + # EOF so as to avoid propagating cancel-causing errors + # to the channel-msg loop machinery. Likely it would + # be better to eventually have a "discovery" protocol + # with basic handshake instead? + with trio.move_on_after(timeout): + async with _connect_chan(*addr): + ponged_addrs.append(addr) + + except OSError: + # TODO: make this a "discovery" log level? + logger.warning(f'No actor registry found @ {addr}') + + async with trio.open_nursery() as tn: + for addr in registry_addrs: + tn.start_soon( + ping_tpt_socket, + tuple(addr), # TODO: just drop this requirement? + ) + + # Create a new local root-actor instance which IS NOT THE + # REGISTRAR + if ponged_addrs: + + if ensure_registry: + raise RuntimeError( + f'Failed to open `{name}`@{ponged_addrs}: ' + 'registry socket(s) already bound' + ) # we were able to connect to an arbiter - logger.info(f"Arbiter seems to exist @ {host}:{port}") + logger.info( + f'Registry(s) seem(s) to exist @ {ponged_addrs}' + ) actor = Actor( - name or 'anonymous', - arbiter_addr=registry_addr, + name=name or 'anonymous', + registry_addrs=ponged_addrs, loglevel=loglevel, enable_modules=enable_modules, ) - host, port = (host, 0) + # Start this local actor as the "registrar", aka a regular + # actor who manages the local registry of "mailboxes" of + # other process-tree-local sub-actors. else: - # start this local actor as the arbiter (aka a regular actor who - # manages the local registry of "mailboxes") - # Note that if the current actor is the arbiter it is desirable - # for it to stay up indefinitely until a re-election process has - # taken place - which is not implemented yet FYI). + # NOTE that if the current actor IS THE REGISTAR, the + # following init steps are taken: + # - the tranport layer server is bound to each (host, port) + # pair defined in provided registry_addrs, or the default. + trans_bind_addrs = registry_addrs + + # - it is normally desirable for any registrar to stay up + # indefinitely until either all registered (child/sub) + # actors are terminated (via SC supervision) or, + # a re-election process has taken place. + # NOTE: all of ^ which is not implemented yet - see: + # https://github.com/goodboy/tractor/issues/216 + # https://github.com/goodboy/tractor/pull/348 + # https://github.com/goodboy/tractor/issues/296 actor = Arbiter( - name or 'arbiter', - arbiter_addr=registry_addr, + name or 'registrar', + registry_addrs=registry_addrs, loglevel=loglevel, enable_modules=enable_modules, ) + # Start up main task set via core actor-runtime nurseries. try: # assign process-local actor _state._current_actor = actor # start local channel-server and fake the portal API # NOTE: this won't block since we provide the nursery - logger.info(f"Starting local {actor} @ {host}:{port}") + ml_addrs_str: str = '\n'.join( + f'@{addr}' for addr in trans_bind_addrs + ) + logger.info( + f'Starting local {actor.uid} on the following transport addrs:\n' + f'{ml_addrs_str}' + ) # start the actor runtime in a new task async with trio.open_nursery() as nursery: - # ``_runtime.async_main()`` creates an internal nursery and - # thus blocks here until the entire underlying actor tree has - # terminated thereby conducting structured concurrency. - + # ``_runtime.async_main()`` creates an internal nursery + # and blocks here until any underlying actor(-process) + # tree has terminated thereby conducting so called + # "end-to-end" structured concurrency throughout an + # entire hierarchical python sub-process set; all + # "actor runtime" primitives are SC-compat and thus all + # transitively spawned actors/processes must be as + # well. await nursery.start( partial( async_main, actor, - accept_addr=(host, port), + accept_addrs=trans_bind_addrs, parent_addr=None ) ) @@ -235,12 +369,20 @@ async def open_root_actor( BaseExceptionGroup, ) as err: - entered = await _debug._maybe_enter_pm(err) + import inspect + entered: bool = await _debug._maybe_enter_pm( + err, + api_frame=inspect.currentframe(), + ) - 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 + # ALWAYS re-raise any error bubbled up from the + # runtime! raise finally: @@ -253,20 +395,29 @@ async def open_root_actor( # for an in nurseries: # tempn.start_soon(an.exited.wait) - logger.cancel("Shutting down root actor") - await actor.cancel( - requesting_uid=actor.uid, + logger.info( + 'Closing down 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 - if orig_bp_path is not None: - os.environ['PYTHONBREAKPOINT'] = orig_bp_path - else: - # clear env back to having no entry - os.environ.pop('PYTHONBREAKPOINT') + # restore built-in `breakpoint()` hook state + if ( + debug_mode + and + maybe_enable_greenback + ): + if builtin_bp_handler is not None: + sys.breakpointhook = builtin_bp_handler + + if orig_bp_path is not None: + os.environ['PYTHONBREAKPOINT'] = orig_bp_path + + else: + # clear env back to having no entry + os.environ.pop('PYTHONBREAKPOINT', None) logger.runtime("Root actor terminated") @@ -276,10 +427,7 @@ def run_daemon( # runtime kwargs name: str | None = 'root', - registry_addr: tuple[str, int] = ( - _default_arbiter_host, - _default_arbiter_port, - ), + registry_addrs: list[tuple[str, int]] = _default_lo_addrs, start_method: str | None = None, debug_mode: bool = False, @@ -303,7 +451,7 @@ def run_daemon( async def _main(): async with open_root_actor( - registry_addr=registry_addr, + registry_addrs=registry_addrs, name=name, start_method=start_method, debug_mode=debug_mode, diff --git a/tractor/_rpc.py b/tractor/_rpc.py new file mode 100644 index 00000000..166ee96a --- /dev/null +++ b/tractor/_rpc.py @@ -0,0 +1,1247 @@ +# 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, + aclosing, +) +from functools import partial +import inspect +from pprint import pformat +import traceback +from typing import ( + Any, + Callable, + Coroutine, + TYPE_CHECKING, +) +import warnings + +import trio +from trio import ( + CancelScope, + Nursery, + TaskStatus, +) + +from ._ipc import Channel +from ._context import ( + Context, +) +from ._exceptions import ( + ContextCancelled, + RemoteActorError, + ModuleNotExposed, + MsgTypeError, + TransportClosed, + is_multi_cancelled, + pack_error, + unpack_error, +) +from .devx import ( + maybe_wait_for_debugger, + _debug, + add_div, +) +from . import _state +from .log import get_logger +from .msg import ( + current_codec, + MsgCodec, + PayloadT, + NamespacePath, + pretty_struct, + _ops as msgops, +) +from tractor.msg.types import ( + CancelAck, + Error, + MsgType, + Return, + Start, + StartAck, + Started, + Stop, + Yield, +) + +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, + return_msg_type: Return|CancelAck = Return, + + task_status: TaskStatus[ + Context | BaseException + ] = trio.TASK_STATUS_IGNORED, +): + __tracebackhide__: bool = True + + # TODO: can we unify this with the `context=True` impl below? + if inspect.isasyncgen(coro): + await chan.send( + StartAck( + cid=cid, + functype='asyncgen', + ) + ) + # 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( + cid=cid, + pld=item, + ) + ) + + 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(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( + StartAck( + cid=cid, + functype='asyncgen', + ) + ) + # XXX: the async-func may spawn further tasks which push + # back values like an async-generator would but must + # manualy construct the response dict-packet-responses as + # above + with cancel_scope as cs: + ctx._scope = cs + task_status.started(ctx) + 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(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: + ack = StartAck( + cid=cid, + functype='asyncfunc', + ) + await chan.send(ack) + except ( + trio.ClosedResourceError, + trio.BrokenResourceError, + BrokenPipeError, + ) as ipc_err: + failed_resp = True + if is_rpc: + raise ipc_err + else: + log.exception( + f'Failed to ack runtime RPC request\n\n' + f'{func} x=> {ctx.chan}\n\n' + f'{ack}\n' + ) + + 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: + ret_msg = return_msg_type( + cid=cid, + pld=result, + ) + await chan.send(ret_msg) + except ( + BrokenPipeError, + trio.BrokenResourceError, + ): + log.warning( + 'Failed to send RPC result?\n' + f'|_{func}@{actor.uid}() -> {ret_msg}\n\n' + f'x=> peer: {chan.uid}\n' + ) + +@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: + # NOTE: we normally always hide this frame in call-stack tracebacks + # if the crash originated from an RPC task (since normally the + # user is only going to care about their own code not this + # internal runtime frame) and we DID NOT + # fail due to an IPC transport error! + __tracebackhide__: bool = hide_tb + + # TODO: a debug nursery when in debug mode! + # async with maybe_open_debugger_nursery() as debug_tn: + # => see matching comment in side `._debug._pause()` + rpc_err: BaseException|None = None + 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: + rpc_err = err + + # TODO: maybe we'll want different "levels" of debugging + # eventualy such as ('app', 'supervisory', 'runtime') ? + # + # -[ ] this if check is duplicate with `._maybe_enter_pm()`.. + if not is_multi_cancelled(err): + 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 + ) + ) + ): + # 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. + log.debug( + 'RPC task crashed, attempting to enter debugger\n' + f'|_{ctx}' + ) + entered_debug = await _debug._maybe_enter_pm( + err, + api_frame=inspect.currentframe(), + ) + if not entered_debug: + # if we prolly should have entered the REPL but + # didn't, maybe there was an internal error in + # the above code and we do want to show this + # frame! + if _state.debug_mode(): + __tracebackhide__: bool = False + + log.exception( + 'RPC task crashed\n' + f'|_{ctx}' + ) + + # ALWAYS try to ship RPC errors back to parent/caller task + 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, + ) + + # if the ctx cs is NOT allocated, the error is likely from + # above `coro` invocation machinery NOT from inside the + # `coro` itself, i.e. err is NOT a user application error. + 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. + # since RPC tasks are scheduled inside a flat + # `Actor._service_n`, we add "handles" to each such that + # they can be individually ccancelled. + finally: + + # if the error is not from user code and instead a failure + # of a runtime RPC or transport failure we do prolly want to + # show this frame + if ( + rpc_err + and ( + not is_rpc + or + not chan.connected() + ) + ): + __tracebackhide__: bool = False + + try: + ctx: Context + func: Callable + is_complete: trio.Event + ( + ctx, + func, + is_complete, + ) = actor._rpc_tasks.pop( + (chan, ctx.cid) + ) + is_complete.set() + + except KeyError: + # If we're cancelled before the task returns then the + # cancel scope will not have been inserted yet + if is_rpc: + log.warning( + 'RPC task likely errored or cancelled before start?\n' + f'|_{ctx._task}\n' + f' >> {ctx.repr_rpc}\n' + ) + # TODO: remove this right? rn the only non-`is_rpc` cases + # are cancellation methods and according the RPC loop eps + # for thoses below, nothing is ever registered in + # `Actor._rpc_tasks` for those cases.. but should we? + # + # -[ ] maybe we should have an equiv `Actor._runtime_rpc_tasks`? + # 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, + return_msg_type: Return|CancelAck = Return, + + 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() + and + _state._runtime_vars['use_greenback'] + ): + # XXX for .pause_from_sync()` usage we need to make sure + # `greenback` is boostrapped in the subactor! + await _debug.maybe_init_greenback() + + # TODO: possibly a specially formatted traceback + # (not sure what typing is for this..)? + # tb: TracebackType = None + + cancel_scope = CancelScope() + cs: CancelScope|None = None # ref when activated + ctx = actor.get_context( + chan=chan, + cid=cid, + nsf=NamespacePath.from_ref(func), + + # NOTE: no portal passed bc this is the "child"-side + + # 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_ep_func: bool = False + + # set the current IPC ctx var for this RPC task + _state._ctxvar_Context.set(ctx) + + # 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 + + + # handle decorated ``@tractor.context`` async function + # - pull out any typed-pld-spec info and apply (below) + # - (TODO) store func-ref meta data for API-frame-info logging + elif ( + ctx_meta := getattr( + func, + '_tractor_context_meta', + False, + ) + ): + # kwargs['ctx'] = ctx + # set the required `tractor.Context` typed input argument to + # the allocated RPC task context. + kwargs[ctx_meta['ctx_var_name']] = ctx + context_ep_func = 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: impl all these cases in terms of the `Context` one! + if not context_ep_func: + await _invoke_non_context( + actor, + cancel_scope, + ctx, + cid, + chan, + func, + coro, + kwargs, + treat_as_gen, + is_rpc, + return_msg_type, + task_status, + ) + # XXX below fallthrough is ONLY for `@context` eps + 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 is the most general and + # "least sugary" type of RPC with support for + # bi-dir streaming B) + # + # the concurrency relation is simlar to a task nursery + # wherein a "parent" task (the one that enters + # `trio.open_nursery()` in some actor "opens" (via + # `Portal.open_context()`) an IPC ctx to another peer + # (which is maybe a sub-) actor who then schedules (aka + # `trio.Nursery.start()`s) a new "child" task to execute + # the `@context` annotated func; that is this func we're + # running directly below! + # ------ - ------ + # + # StartAck: respond immediately with endpoint info + await chan.send( + StartAck( + cid=cid, + functype='context', + ) + ) + + # TODO: should we also use an `.open_context()` equiv + # for this child side by factoring the impl from + # `Portal.open_context()` into a common helper? + # + # NOTE: there are many different ctx state details + # in a child 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, + msgops.maybe_limit_plds( + ctx=ctx, + spec=ctx_meta.get('pld_spec'), + dec_hook=ctx_meta.get('dec_hook'), + ), + ): + ctx._scope_nursery = tn + ctx._scope = tn.cancel_scope + task_status.started(ctx) + + # TODO: better `trionics` tooling: + # -[ ] should would be nice to have our `TaskMngr` + # nursery here! + # -[ ] payload value checking like we do with + # `.started()` such that the debbuger can engage + # here in the child task instead of waiting for the + # parent to crash with it's own MTE.. + res: Any|PayloadT = await coro + return_msg: Return|CancelAck = return_msg_type( + cid=cid, + pld=res, + ) + # set and shuttle final result to "parent"-side task. + ctx._result = res + await chan.send(return_msg) + + # NOTE: this happens IFF `ctx._scope.cancel()` is + # called by any of, + # - *this* child 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. + 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 + explain: str = f'{ctx.side!r}-side task 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 + explain += '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. + # + # TODO: determine if the ctx peer task was the + # exact task which cancelled, vs. some other + # task in the same actor. + elif canceller == ctx.chan.uid: + explain += f'its {ctx.peer_side!r}-side peer' + + else: + explain += 'a remote peer' + + explain += ( + add_div(message=explain) + + + f'<= canceller: {canceller}\n' + f'=> cancellee: {our_uid}\n' + # TODO: better repr for ctx tasks.. + f' |_{ctx.side!r} {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 child (or callee) + # task, so relay this cancel signal to the + # other side. + ctxc = ContextCancelled( + message=explain, + boxed_type=trio.Cancelled, + canceller=canceller, + ) + raise ctxc + + # XXX: do we ever trigger this block any more? + except ( + BaseExceptionGroup, + trio.Cancelled, + BaseException, + + ) as scope_error: + if ( + isinstance(scope_error, RuntimeError) + and scope_error.args + and 'Cancel scope stack corrupted' in scope_error.args[0] + ): + log.exception('Cancel scope stack corrupted!?\n') + # _debug.mk_pdb().set_trace() + + # always set this (child) side's exception as the + # local error on the context + ctx._local_error: BaseException = scope_error + # ^-TODO-^ question, + # does this matter other then for + # consistentcy/testing? + # |_ no user code should be in this scope at this point + # AND we already set this in the block below? + + # 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, + )) + + logmeth: Callable = log.runtime + merr: Exception|None = ctx.maybe_error + message: str = 'IPC context terminated ' + descr_str: str = ( + f'after having {ctx.repr_state!r}\n' + ) + if merr: + + logmeth: Callable = log.error + if isinstance(merr, ContextCancelled): + logmeth: Callable = log.runtime + + if not isinstance(merr, RemoteActorError): + tb_str: str = ''.join(traceback.format_exception(merr)) + descr_str += ( + f'\n{merr!r}\n' # needed? + f'{tb_str}\n' + ) + else: + descr_str += f'\n{merr!r}\n' + else: + descr_str += f'\nand final result {ctx.outcome!r}\n' + + logmeth( + message + + + descr_str + ) + + +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( + msg: Error = pack_error( + err, + cid=cid, + + # TODO: special tb fmting for ctxc cases? + # tb=tb, + ) + 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, + ): + log.critical( + 'IPC transport failure -> ' + f'failed to ship error to {remote_descr}!\n\n' + f'X=> {channel.uid}\n\n' + + # TODO: use `.msg.preetty_struct` for this! + f'{msg}\n' + ) + except BaseException: + log.exception( + 'Errored while attempting error shipment?' + ) + __tracebackhide__: bool = False + raise + + +async def process_messages( + actor: Actor, + chan: Channel, + shield: bool = False, + task_status: TaskStatus[CancelScope] = trio.TASK_STATUS_IGNORED, + +) -> ( + bool, # chan diconnected + MsgType, # last msg +): + ''' + 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()` ). + + ''' + assert actor._service_n # runtime state sanity + + # TODO: once `trio` get's an "obvious way" for req/resp we + # should use it? + # -[ ] existing GH https://github.com/python-trio/trio/issues/467 + # -[ ] for other transports (like QUIC) we can possibly just + # entirely avoid the feeder mem-chans since each msg will be + # delivered with a ctx-id already? + # + # |_ for ex, from `aioquic` which exposed "stream ids": + # - https://github.com/aiortc/aioquic/blob/main/src/aioquic/quic/connection.py#L1175 + # - https://github.com/aiortc/aioquic/blob/main/src/aioquic/quic/connection.py#L659 + nursery_cancelled_before_task: bool = False + msg: MsgType|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: + log.transport( # type: ignore + f'IPC msg from peer\n' + f'<= {chan.uid}\n\n' + + # TODO: avoid fmting depending on loglevel for perf? + # -[ ] specifically `pretty_struct.pformat()` sub-call..? + # - how to only log-level-aware actually call this? + # -[ ] use `.msg.pretty_struct` here now instead! + # f'{pretty_struct.pformat(msg)}\n' + f'{msg}\n' + ) + + match msg: + # msg for an ongoing IPC ctx session, deliver msg to + # local task. + case ( + StartAck(cid=cid) + | Started(cid=cid) + | Yield(cid=cid) + | Stop(cid=cid) + | Return(cid=cid) + | CancelAck(cid=cid) + + # `.cid` indicates RPC-ctx-task scoped + | Error(cid=cid) + + # recv-side `MsgType` decode violation + | MsgTypeError(cid=cid) + ): + # deliver response to local caller/waiter + # via its per-remote-context memory channel. + await actor._deliver_ctx_payload( + chan, + cid, + msg, + ) + + # `Actor`(-internal) runtime cancel requests + case Start( + ns='self', + func='cancel', + cid=cid, + kwargs=kwargs, + ): + kwargs |= {'req_chan': chan} + + # XXX NOTE XXX don't start entire actor + # runtime cancellation if this actor is + # currently in debug mode! + pdb_complete: trio.Event|None = _debug.DebugStatus.repl_release + 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, + actor.cancel, + kwargs, + is_rpc=False, + return_msg_type=CancelAck, + ) + + log.runtime( + 'Cancelling IPC transport msg-loop with peer:\n' + f'|_{chan}\n' + ) + loop_cs.cancel() + break + + case Start( + ns='self', + func='_cancel_task', + cid=cid, + kwargs=kwargs, + ): + target_cid: str = kwargs['cid'] + kwargs |= { + 'requesting_uid': chan.uid, + 'ipc_msg': msg, + + # XXX NOTE! ONLY the rpc-task-owning + # parent IPC channel should be able to + # cancel it! + 'parent_chan': chan, + } + try: + await _invoke( + actor, + cid, + chan, + actor._cancel_task, + kwargs, + is_rpc=False, + return_msg_type=CancelAck, + ) + 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' + ) + + # the "MAIN" RPC endpoint to schedule-a-`trio.Task` + # ------ - ------ + # -[x] discard un-authed msgs as per, + # + case Start( + cid=cid, + ns=ns, + func=funcname, + kwargs=kwargs, # type-spec this? see `msg.types` + uid=actorid, + ): + start_status: str = ( + 'Handling RPC `Start` request\n' + f'<= peer: {actorid}\n\n' + f' |_{chan}\n' + f' |_cid: {cid}\n\n' + # f' |_{ns}.{funcname}({kwargs})\n' + f'>> {actor.uid}\n' + f' |_{actor}\n' + f' -> nsp: `{ns}.{funcname}({kwargs})`\n' + + # f' |_{ns}.{funcname}({kwargs})\n\n' + + # f'{pretty_struct.pformat(msg)}\n' + ) + + # runtime-internal endpoint: `Actor.` + # only registry methods exist now yah, + # like ``.register_actor()`` etc. ? + if ns == 'self': + func: Callable = getattr(actor, funcname) + + # application RPC endpoint + else: + try: + func: Callable = actor._get_rpc_func( + ns, + funcname, + ) + except ( + ModuleNotExposed, + AttributeError, + ) as err: + # always complain to requester + # client about un-enabled modules + err_msg: dict[str, dict] = pack_error( + err, + cid=cid, + ) + await chan.send(err_msg) + continue + + start_status += ( + f' -> func: {func}\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? + start_status += ' -> scheduling new task..\n' + log.runtime(start_status) + 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( + start_status + + + ' -> task for RPC failed?\n\n' + f'{err}' + ) + continue + + else: + # mark our global state with 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(), + ) + + # runtime-scoped remote (internal) error + # (^- bc no `Error.cid` -^) + # + # NOTE: 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). Raise error inline and + # mark the channel as "globally errored" for + # all downstream consuming primitives. + case Error(): + chan._exc: Exception = unpack_error( + msg, + chan=chan, + ) + raise chan._exc + + # unknown/invalid msg type? + case _: + codec: MsgCodec = current_codec() + message: str = ( + f'Unhandled IPC msg for codec?\n\n' + f'|_{codec}\n\n' + f'{msg}\n' + ) + log.exception(message) + raise RuntimeError(message) + + log.transport( + 'Waiting on next IPC msg from\n' + f'peer: {chan.uid}\n' + f'|_{chan}\n' + ) + + # END-OF `async for`: + # IPC disconnected via `trio.EndOfChannel`, likely + # due to a (graceful) `Channel.aclose()`. + log.runtime( + f'channel for {chan.uid} disconnected, cancelling RPC tasks\n' + f'|_{chan}\n' + ) + 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: maybe add a teardown handshake? and, + # -[ ] don't show this msg if it's an ephemeral discovery ep call? + # -[ ] figure out how this will break with other transports? + log.runtime( + f'IPC channel closed abruptly\n' + f'<=x peer: {chan.uid}\n' + f' |_{chan.raddr}\n' + ) + + # transport **WAS** disconnected + return (True, msg) + + 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 + if msg is None: + message: str = 'Exiting IPC msg loop without receiving a msg?' + else: + message: str = ( + 'Exiting IPC msg loop with final msg\n\n' + f'<= peer: {chan.uid}\n' + f' |_{chan}\n\n' + f'{pretty_struct.pformat(msg)}' + ) + + log.runtime(message) + + # transport **WAS NOT** disconnected + return (False, msg) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 08ddabc4..3cf35ff9 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -14,36 +14,63 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . -""" -Actor primitives and helpers +''' +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. -""" +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: + +- 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. + +''' 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 -import inspect +from pprint import pformat import signal import sys from typing import ( Any, Callable, - Union, TYPE_CHECKING, ) import uuid from types import ModuleType import os -from contextlib import ExitStack 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, + Nursery, + TaskStatus, +) +from tractor.msg import ( + pretty_struct, + NamespacePath, + types as msgtypes, + MsgType, +) from ._ipc import Channel from ._context import ( mk_context, @@ -51,18 +78,25 @@ from ._context import ( ) from .log import get_logger from ._exceptions import ( - pack_error, - unpack_error, - ModuleNotExposed, - is_multi_cancelled, ContextCancelled, + ModuleNotExposed, + MsgTypeError, + unpack_error, TransportClosed, ) -from . import _debug -from ._discovery import get_arbiter +from .devx import ( + # pause, + maybe_wait_for_debugger, + _debug, +) +from ._discovery import get_registry from ._portal import Portal from . import _state from . import _mp_fixup_main +from ._rpc import ( + process_messages, + try_ship_error_to_remote, +) if TYPE_CHECKING: @@ -72,371 +106,10 @@ if TYPE_CHECKING: log = get_logger('tractor') -async def _invoke( - - actor: 'Actor', - cid: str, - chan: Channel, - func: Callable, - kwargs: dict[str, Any], - - is_rpc: bool = True, - task_status: TaskStatus[ - Union[Context, BaseException] - ] = trio.TASK_STATUS_IGNORED, -): - ''' - Invoke local func and deliver result(s) over provided channel. - - This is the core "RPC task" starting machinery. - - ''' - __tracebackhide__ = 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() - # activated cancel scope ref - cs: trio.CancelScope | None = None - - 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): - # handle decorated ``@tractor.stream`` async functions - sig = inspect.signature(func) - params = sig.parameters - - # compat with old api - kwargs['ctx'] = ctx - - 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 - - treat_as_gen = True - - 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 - try: - if not ( - inspect.isasyncgenfunction(func) or - inspect.iscoroutinefunction(func) - ): - raise TypeError(f'{func} must be an async function!') - - try: - coro = func(**kwargs) - except TypeError: - raise - - 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 - 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}) - - elif context: - # context func with support for bi-dir streaming - 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) - res = await coro - 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 - - # 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 = actor._contexts.pop((chan.uid, cid)) - if ctx: - log.runtime( - f'Context entrypoint {func} was terminated:\n{ctx}' - ) - - if ctx.cancelled_caught: - - # 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__ - cs: trio.CancelScope = ctx._scope - if cs.cancel_called: - canceller = ctx._cancelled_remote - # await _debug.breakpoint() - - # 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, - ) - - else: - # regular async function - try: - await chan.send({'functype': 'asyncfunc', 'cid': cid}) - except trio.BrokenResourceError: - failed_resp = True - if is_rpc: - raise - else: - log.warning( - f'Failed to respond to non-rpc request: {func}' - ) - - with cancel_scope as cs: - ctx._scope = cs - task_status.started(ctx) - result = await coro - 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} - ) - - except ( - Exception, - BaseExceptionGroup, - ) as err: - - 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: 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 = pack_error(err, tb=tb) - err_msg['cid'] = cid - - 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} !?" - ) - - # 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(f'{func.__name__}({kwargs}) failed?') - - finally: - if not actor._rpc_tasks: - log.runtime("All RPC tasks have completed") - actor._ongoing_rpc_tasks.set() - - -def _get_mod_abspath(module): +def _get_mod_abspath(module: ModuleType) -> str: return os.path.abspath(module.__file__) -async def try_ship_error_to_parent( - channel: Channel, - err: Union[Exception, BaseExceptionGroup], - -) -> None: - with trio.CancelScope(shield=True): - try: - # internal error so ship to parent without cid - await channel.send(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. - log.critical( - f"Failed to ship error to parent " - f"{channel.uid}, channel was closed" - ) - - class Actor: ''' The fundamental "runtime" concurrency primitive. @@ -448,43 +121,52 @@ 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 # https://github.com/goodboy/tractor/issues/216 is_arbiter: bool = False + + @property + def is_registrar(self) -> bool: + return self.is_arbiter + 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 + _spawn_spec: msgtypes.SpawnSpec|None = None # syncs for setup/teardown sequences - _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 + _server_down: trio.Event|None = None # if started on ``asycio`` running ``trio`` in guest mode _infected_aio: bool = False + # TODO: nursery tracking like `trio` does? + # _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() @@ -494,10 +176,14 @@ class Actor: name: str, *, enable_modules: list[str] = [], - uid: str | None = None, - loglevel: str | None = None, - arbiter_addr: tuple[str, int] | None = None, - spawn_method: str | None = None + uid: str|None = None, + loglevel: str|None = None, + registry_addrs: list[tuple[str, int]]|None = None, + spawn_method: str|None = None, + + # TODO: remove! + arbiter_addr: tuple[str, int]|None = None, + ) -> None: ''' This constructor is called in the parent actor **before** the spawning @@ -505,10 +191,13 @@ 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 + self._cancel_called_by_remote: tuple[str, tuple]|None = None self._cancel_called: bool = False # retreive and store parent `__main__` data which @@ -516,29 +205,32 @@ class Actor: self._parent_main_data = _mp_fixup_main._mp_figure_out_main() # always include debugging tools module - enable_modules.append('tractor._debug') + enable_modules.append('tractor.devx._debug') - mods = {} + self.enable_modules: dict[str, str] = {} for name in enable_modules: - mod = importlib.import_module(name) - mods[name] = _get_mod_abspath(mod) + mod: ModuleType = importlib.import_module(name) + self.enable_modules[name] = _get_mod_abspath(mod) - self.enable_modules = mods self._mods: dict[str, ModuleType] = {} - self.loglevel = loglevel + self.loglevel: str = loglevel - self._arb_addr: tuple[str, int] | None = ( - str(arbiter_addr[0]), - int(arbiter_addr[1]) - ) if arbiter_addr else None + if arbiter_addr is not None: + warnings.warn( + '`Actor(arbiter_addr=)` is now deprecated.\n' + 'Use `registry_addrs: list[tuple]` instead.', + DeprecationWarning, + stacklevel=2, + ) + registry_addrs: list[tuple[str, int]] = [arbiter_addr] # marked by the process spawning backend at startup # will be None for the parent most process started manually # by the user (currently called the "arbiter") - self._spawn_method = spawn_method + self._spawn_method: str = spawn_method self._peers: defaultdict = defaultdict(list) - self._peer_connected: dict = {} + self._peer_connected: dict[tuple[str, str], trio.Event] = {} self._no_more_peers = trio.Event() self._no_more_peers.set() self._ongoing_rpc_tasks = trio.Event() @@ -552,39 +244,91 @@ class Actor: # map {actor uids -> Context} self._contexts: dict[ - tuple[tuple[str, str], str], + tuple[ + tuple[str, str], # .uid + str, # .cid + str, # .side + ], Context ] = {} self._listeners: list[trio.abc.Listener] = [] - self._parent_chan: Channel | None = None - self._forkserver_info: tuple | None = None + self._parent_chan: Channel|None = None + self._forkserver_info: tuple|None = None + + # track each child/sub-actor in it's locally + # supervising nursery self._actoruid2nursery: dict[ - tuple[str, str], - ActorNursery | None, - ] = {} # type: ignore # noqa + tuple[str, str], # sub-`Actor.uid` + ActorNursery|None, + ] = {} + + # when provided, init the registry addresses property from + # input via the validator. + self._reg_addrs: list[tuple[str, int]] = [] + if registry_addrs: + self.reg_addrs: list[tuple[str, int]] = registry_addrs + _state._runtime_vars['_registry_addrs'] = registry_addrs + + @property + def reg_addrs(self) -> list[tuple[str, int]]: + ''' + List of (socket) addresses for all known (and contactable) + registry actors. + + ''' + return self._reg_addrs + + @reg_addrs.setter + def reg_addrs( + self, + addrs: list[tuple[str, int]], + ) -> None: + if not addrs: + log.warning( + 'Empty registry address list is invalid:\n' + f'{addrs}' + ) + return + + # always sanity check the input list since it's critical + # that addrs are correct for discovery sys operation. + for addr in addrs: + if not isinstance(addr, tuple): + raise ValueError( + 'Expected `Actor.reg_addrs: list[tuple[str, int]]`\n' + f'Got {addrs}' + ) + + self._reg_addrs = addrs async def wait_for_peer( - self, uid: tuple[str, str] + 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 sub-)actor with + a `uid` using a `trio.Event` for sync. ''' - log.runtime(f"Waiting for peer {uid} to connect") + log.debug(f'Waiting for peer {uid!r} to connect') event = self._peer_connected.setdefault(uid, trio.Event()) await event.wait() - log.runtime(f"{uid} successfully connected back to us") + log.debug(f'{uid!r} 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). + Load explicitly enabled python modules from local fs after + process 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 manually (presuming it exists). ''' try: @@ -597,23 +341,35 @@ class Actor: _mp_fixup_main._fixup_main_from_path( parent_data['init_main_from_path']) + status: str = 'Attempting to import enabled modules:\n' for modpath, filepath in self.enable_modules.items(): # XXX append the allowed module to the python path which # should allow for relative (at least downward) imports. sys.path.append(os.path.dirname(filepath)) - log.runtime(f"Attempting to import {modpath}@{filepath}") - mod = importlib.import_module(modpath) + status += ( + f'|_{modpath!r} -> {filepath!r}\n' + ) + mod: ModuleType = importlib.import_module(modpath) self._mods[modpath] = mod if modpath == '__main__': self._mods['__mp_main__'] = mod + log.runtime(status) + 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): + ''' + 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: @@ -637,25 +393,43 @@ class Actor: raise mne + # TODO: maybe change to mod-func and rename for implied + # multi-transport semantics? async def _stream_handler( - self, 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 IPC connections on a specific + transport 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_status: str = '' + + # TODO: remove this branch since can never happen? + # NOTE: `.uid` is only set after first contact + if their_uid: + con_status = ( + 'IPC Re-connection from already known peer?\n' + ) + else: + con_status = ( + 'New inbound IPC connection <=\n' + ) + + con_status += ( + f'|_{chan}\n' + # f' |_@{chan.raddr}\n\n' + # ^-TODO-^ remove since alfready in chan.__repr__()? + ) # 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. @@ -670,49 +444,77 @@ 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.runtime( + con_status + + + ' -> But failed to handshake? Ignoring..\n' + ) return - # channel tracking - event = self._peer_connected.pop(uid, None) + con_status += ( + 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_status += ( + ' -> 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() - chans = self._peers[uid] + else: + con_status += ( + f' -> Registered IPC chan for peer actor {uid}@{chan.raddr}\n' + ) # type: ignore - # TODO: re-use channels for new connections instead - # of always new ones; will require changing all the - # discovery funcs - if chans: - log.runtime( - f"already have channel(s) for {uid}:{chans}?" - ) + 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"Registered {chan} for {uid}") # type: ignore # append new channel - self._peers[uid].append(chan) + # TODO: can we just use list-ref directly? + chans.append(chan) - local_nursery: ActorNursery | None = None # noqa - disconnected: bool = False + con_status += ' -> Entering RPC msg loop..\n' + log.runtime(con_status) # Begin channel management - respond to remote requests and # process received reponses. + disconnected: bool = False + last_msg: MsgType try: - disconnected = await process_messages(self, chan) - - except ( - trio.Cancelled, - ): - log.cancel(f"Msg loop was cancelled for {chan}") + ( + disconnected, + last_msg, + ) = await process_messages( + self, + chan, + ) + except trio.Cancelled: + log.cancel( + 'IPC transport msg loop was cancelled for \n' + f'|_{chan}\n' + ) 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 @@ -720,29 +522,55 @@ class Actor: # moving on with closing our own side. if ( local_nursery + and ( + self._cancel_called + or + chan._cancel_called + ) + # + # ^-TODO-^ along with this is there another condition + # that we should filter with to avoid entering this + # waiting block needlessly? + # -[ ] maybe `and local_nursery.cancelled` and/or + # only if the `._children` table is empty or has + # only `Portal`s with .chan._cancel_called == + # True` as per what we had below; the MAIN DIFF + # BEING that just bc one `Portal.cancel_actor()` + # was called, doesn't mean the whole actor-nurse + # is gonna exit any time soon right!? + # + # or + # all(chan._cancel_called for chan in chans) + ): - 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 # remote peer side since we presume that any channel - # which is mapped to a sub-actor (i.e. it's managed by - # one of our local nurseries) has a message is sent to - # the peer likely by this actor (which is now in - # a cancelled condition) when the local runtime here is - # now cancelled while (presumably) in the middle of msg - # 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 + # which is mapped to a sub-actor (i.e. it's managed + # by local actor-nursery) has a message that is sent + # to the peer likely by this actor (which may be in + # a shutdown sequence due to cancellation) when the + # local runtime here is now cancelled while + # (presumably) in the middle of msg loop processing. + chan_info: str = ( + f'{chan.uid}\n' + f'|_{chan}\n' + f' |_{chan.transport}\n\n' + ) + with trio.move_on_after(0.5) as drain_cs: + drain_cs.shield = True - log.runtime( - f'Draining lingering msgs from stream {chan.msgstream}' - ) + # 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(): - async for msg in chan.msgstream.drain(): # try to deliver any lingering msgs # before we destroy the channel. # This accomplishes deterministic @@ -750,46 +578,143 @@ 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}') - cid = msg.get('cid') + log.warning( + 'Draining msg from disconnected peer\n' + f'{chan_info}' + f'{pformat(msg)}\n' + ) + # cid: str|None = msg.get('cid') + cid: str|None = msg.cid if cid: # deliver response to local caller/waiter - await self._push_result(chan, cid, msg) + await self._deliver_ctx_payload( + chan, + cid, + msg, + ) + if drain_cs.cancelled_caught: + log.warning( + 'Timed out waiting on IPC transport channel to drain?\n' + f'{chan_info}' + ) - await local_nursery.exited.wait() + # XXX NOTE XXX when no explicit call to + # `open_root_actor()` was made by the application + # (normally we implicitly make that call inside + # the first `.open_nursery()` in root-actor + # user/app code), we can assume that either we + # are NOT the root actor or are root but the + # runtime was started manually. and thus DO have + # to wait for the nursery-enterer to exit before + # shutting down the local runtime to avoid + # clobbering any ongoing subactor + # teardown/debugging/graceful-cancel. + # + # see matching note inside `._supervise.open_nursery()` + # + # TODO: should we have a separate cs + timeout + # block here? + if ( + # XXX SO either, + # - not root OR, + # - is root but `open_root_actor()` was + # entered manually (in which case we do + # the equiv wait there using the + # `devx._debug` sub-sys APIs). + not local_nursery._implicit_runtime_started + ): + log.runtime( + 'Waiting on local actor nursery to exit..\n' + f'|_{local_nursery}\n' + ) + with trio.move_on_after(0.5) as an_exit_cs: + an_exit_cs.shield = True + await local_nursery.exited.wait() + + # TODO: currently this is always triggering for every + # sub-daemon spawned from the `piker.services._mngr`? + # -[ ] how do we ensure that the IPC is supposed to + # be long lived and isn't just a register? + # |_ in the register case how can we signal that the + # ephemeral msg loop was intentional? + if ( + # not local_nursery._implicit_runtime_started + # and + an_exit_cs.cancelled_caught + ): + log.warning( + 'Timed out waiting on local actor-nursery to exit?\n' + f'{local_nursery}\n' + f' |_{pformat(local_nursery._children)}\n' + ) + # await _debug.pause() if disconnected: # if the transport died and this actor is still - # registered within a local nursery, we report that the - # IPC layer may have failed unexpectedly since it may be - # the cause of other downstream errors. - entry = local_nursery._children.get(uid) + # registered within a local nursery, we report + # that the IPC layer may have failed + # unexpectedly since it may be the cause of + # other downstream errors. + entry: tuple|None = 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?' + if ( + (poll := getattr(proc, 'poll', None)) + and + poll() is None # proc still alive + ): + # TODO: change log level based on + # detecting whether chan was created for + # ephemeral `.register_actor()` request! + # -[ ] also, that should be avoidable by + # re-using any existing chan from the + # `._discovery.get_registry()` call as + # well.. + log.runtime( + f'Peer IPC broke but subproc is alive?\n\n' + + f'<=x {chan.uid}@{chan.raddr}\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}") - chans = self._peers.get(chan.uid) + # drop ref to channel so it can be gc-ed and disconnected + con_teardown_status: str = ( + f'IPC channel disconnected:\n' + f'<=x uid: {chan.uid}\n' + f' |_{pformat(chan)}\n\n' + ) chans.remove(chan) + # TODO: do we need to be this pedantic? if not chans: - log.runtime(f"No more channels for {chan.uid}") + con_teardown_status += ( + 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' + ) + + con_teardown_status += ( + f'-> Remaining IPC {len(self._peers)} peers: {peers_str}\n' + ) # No more channels to other actors (at all) registered # as connected. if not self._peers: - log.runtime("Signalling no more peer channel connections") + con_teardown_status += ( + 'Signalling no more peer channel connections' + ) self._no_more_peers.set() # NOTE: block this actor from acquiring the @@ -800,85 +725,161 @@ 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 + # + # XXX => YES IT DOES, when i was testing ctl-c + # from broken debug TTY locking due to + # msg-spec races on application using RunVar... + if ( + (ctx_in_debug := pdb_lock.ctx_in_debug) + and + (pdb_user_uid := ctx_in_debug.chan.uid) + and + local_nursery + ): + entry: tuple|None = local_nursery._children.get( + tuple(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\n' + 'a DISCONNECTED child still has the debug ' + 'lock!\n\n' + # f'root uid: {self.uid}\n' + f'last disconnected child uid: {uid}\n' + f'locking child uid: {pdb_user_uid}\n' + ) + await 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 if ( - db_cs + (db_cs := pdb_lock.get_locking_task_cs()) and not db_cs.cancel_called + and uid == pdb_user_uid ): - log.warning( + log.critical( f'STALE DEBUG LOCK DETECTED FOR {uid}' ) # TODO: figure out why this breaks tests.. 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. + log.runtime(con_teardown_status) + # finally block closure - 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() - - except trio.BrokenResourceError: - log.runtime(f"Channel {chan.uid} was already closed") - - async def _push_result( + # TODO: rename to `._deliver_payload()` since this handles + # more then just `result` msgs now obvi XD + async def _deliver_ctx_payload( self, chan: Channel, cid: str, - msg: dict[str, Any], - ) -> None: + msg: MsgType|MsgTypeError, + + ) -> 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 = 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, + + # TODO: how to determine this tho? + # side, + )] 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\n' + # XXX don't need right since it's always in msg? + # f'=> cid: {cid}\n\n' + + f'{pretty_struct.pformat(msg)}\n' + ) return + # if isinstance(msg, MsgTypeError): + # return await ctx._deliver_bad_msg() + return await ctx._deliver_msg(msg) def get_context( self, chan: Channel, cid: str, + nsf: NamespacePath, - msg_buffer_size: int | None = None, + # 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, ) -> 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. ''' - 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)] + ctx = self._contexts[( + actor_uid, + cid, + # side, + )] + log.debug( + 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 @@ -887,70 +888,129 @@ class Actor: state.max_buffer_size = msg_buffer_size except KeyError: + log.debug( + f'Allocate 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, ) - self._contexts[(actor_uid, cid)] = ctx + self._contexts[( + actor_uid, + cid, + # side, + )] = ctx return ctx async def start_remote_task( self, chan: Channel, - ns: str, - func: str, + nsf: NamespacePath, kwargs: dict, - msg_buffer_size: int | None = None, + + # determines `Context.side: str` + portal: Portal|None = None, + + # IPC channel config + msg_buffer_size: int|None = None, allow_overruns: bool = False, + load_nsf: bool = False, + ack_timeout: float = float('inf'), ) -> 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, - cid, + chan=chan, + cid=cid, + nsf=nsf, + + # side='caller', msg_buffer_size=msg_buffer_size, allow_overruns=allow_overruns, ) - log.runtime(f"Sending cmd to {chan.uid}: {ns}.{func}({kwargs})") - await chan.send( - {'cmd': (ns, func, kwargs, self.uid, cid)} + ctx._portal = portal + + 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() + + msg = msgtypes.Start( + ns=ns, + func=func, + kwargs=kwargs, + uid=self.uid, + cid=cid, ) + log.runtime( + 'Sending RPC `Start`\n\n' + f'=> peer: {chan.uid}\n' + f' |_ {ns}.{func}({kwargs})\n\n' - # Wait on first response msg and validate; this should be - # immediate. - first_msg = await ctx._recv_chan.receive() - functype = first_msg.get('functype') + f'{pretty_struct.pformat(msg)}' + ) + await chan.send(msg) - if 'error' in first_msg: + # NOTE wait on first `StartAck` response msg and validate; + # this should be immediate and does not (yet) wait for the + # remote child task to sync via `Context.started()`. + with trio.fail_after(ack_timeout): + first_msg: msgtypes.StartAck = await ctx._rx_chan.receive() + try: + functype: str = first_msg.functype + except AttributeError: raise unpack_error(first_msg, chan) - elif functype not in ( + if functype not in ( 'asyncfunc', 'asyncgen', 'context', ): - raise ValueError(f"{first_msg} is an invalid response packet?") + raise ValueError( + f'Invalid `StartAck.functype: str = {first_msg!r}` ??' + ) ctx._remote_func_type = functype return ctx async def _from_parent( self, - parent_addr: tuple[str, int] | None, - ) -> tuple[Channel, tuple[str, int] | None]: + parent_addr: 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 @@ -960,87 +1020,134 @@ class Actor: ) await chan.connect() + # TODO: move this into a `Channel.handshake()`? # Initial handshake: swap names. await self._do_handshake(chan) - accept_addr: tuple[str, int] | None = None + accept_addrs: list[tuple[str, int]]|None = None if self._spawn_method == "trio": - # Receive runtime state from our parent - parent_data: dict[str, Any] - parent_data = await chan.recv() + + # Receive post-spawn runtime state from our parent. + spawnspec: msgtypes.SpawnSpec = await chan.recv() + self._spawn_spec = spawnspec + log.runtime( - "Received state from parent:\n" - f"{parent_data}" + 'Received runtime spec 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'{pretty_struct.pformat(spawnspec)}\n' ) - accept_addr = ( - parent_data.pop('bind_host'), - parent_data.pop('bind_port'), - ) - rvs = parent_data.pop('_runtime_vars') - log.runtime(f"Runtime vars are: {rvs}") + accept_addrs: list[tuple[str, int]] = spawnspec.bind_addrs + + # TODO: another `Struct` for rtvs.. + rvs: dict[str, Any] = spawnspec._runtime_vars + if rvs['_debug_mode']: + try: + # TODO: maybe return some status msgs upward + # to that we can emit them in `con_status` + # instead? + log.devx( + 'Enabling `stackscope` traces on SIGUSR1' + ) + from .devx import enable_stack_on_sig + enable_stack_on_sig() + except ImportError: + log.warning( + '`stackscope` not installed for use in debug mode!' + ) + rvs['_is_root'] = False _state._runtime_vars.update(rvs) - for attr, value in parent_data.items(): + # XXX: ``msgspec`` doesn't support serializing tuples + # so just cash manually here since it's what our + # internals expect. + # + self.reg_addrs = [ + # TODO: we don't really NEED these as tuples? + # so we can probably drop this casting since + # apparently in python lists are "more + # efficient"? + tuple(val) + for val in spawnspec.reg_addrs + ] - if attr == '_arb_addr': - # XXX: ``msgspec`` doesn't support serializing tuples - # so just cash manually here since it's what our - # internals expect. - value = tuple(value) if value else None - self._arb_addr = value + # TODO: better then monkey patching.. + # -[ ] maybe read the actual f#$-in `._spawn_spec` XD + for _, attr, value in pretty_struct.iter_fields( + spawnspec, + ): + setattr(self, attr, value) - else: - setattr(self, attr, value) - - return chan, accept_addr + return ( + chan, + accept_addrs, + ) 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 spawning parent actor!?\n' + f'x=> {parent_addr}\n' + f'|_{self}\n\n' + ) + await self.cancel(req_chan=None) # self cancel raise 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, - accept_port: int = 0, - task_status: TaskStatus[trio.Nursery] = trio.TASK_STATUS_IGNORED, + listen_sockaddrs: list[tuple[str, int]]|None = None, + + task_status: TaskStatus[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. ''' + if listen_sockaddrs is None: + listen_sockaddrs = [(None, 0)] + self._server_down = trio.Event() try: async with trio.open_nursery() as server_n: - listeners: list[trio.abc.Listener] = await server_n.start( - partial( - trio.serve_tcp, - self._stream_handler, - # new connections will stay alive even if this server - # is cancelled - handler_nursery=handler_nursery, - port=accept_port, - host=accept_host, + + for host, port in listen_sockaddrs: + listeners: list[trio.abc.Listener] = await server_n.start( + partial( + trio.serve_tcp, + + handler=self._stream_handler, + port=port, + host=host, + + # NOTE: configured such that new + # connections will stay alive even if + # this server is cancelled! + handler_nursery=handler_nursery, + ) ) - ) - sockets: list[trio.socket] = [ - getattr(listener, 'socket', 'unknown socket') - for listener in listeners - ] - log.runtime( - f'Started tcp server(s) on {sockets}') - self._listeners.extend(listeners) + sockets: list[trio.socket] = [ + getattr(listener, 'socket', 'unknown socket') + for listener in listeners + ] + log.runtime( + 'Started TCP server(s)\n' + f'|_{sockets}\n' + ) + self._listeners.extend(listeners) + task_status.started(server_n) + finally: # signal the server is down since nursery above terminated self._server_down.set() @@ -1049,46 +1156,95 @@ 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 - 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 + termination of its containing OS 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} is trying to cancel") + ( + 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, + log.runtime, + ) + # TODO: just use the new `Context.repr_rpc: str` (and + # other) repr fields instead of doing this all manual.. + msg: str = ( + f'Runtime cancel request from {requester_type}:\n\n' + f'<= .cancel(): {requesting_uid}\n\n' + ) + + # TODO: what happens here when we self-cancel tho? self._cancel_called_by_remote: tuple = requesting_uid 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 - dbcs = _debug.Lock._debugger_request_cs - if dbcs is not None: - log.cancel("Cancelling active debugger request") - dbcs.cancel() + debug_req = _debug.DebugStatus + lock_req_ctx: Context = debug_req.req_ctx + if ( + lock_req_ctx + and + lock_req_ctx.has_outcome + ): + msg += ( + '-> Cancelling active debugger request..\n' + f'|_{_debug.Lock.repr()}\n\n' + f'|_{lock_req_ctx}\n\n' + ) + # lock_req_ctx._scope.cancel() + # TODO: wrap this in a method-API.. + debug_req.req_cs.cancel() - # kill all ongoing tasks - await self.cancel_rpc_tasks(requesting_uid=requesting_uid) + # self-cancel **all** ongoing RPC tasks + await self.cancel_rpc_tasks( + req_uid=requesting_uid, + parent_chan=None, + ) # stop channel server self.cancel_server() @@ -1096,13 +1252,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_meth(msg) self._cancel_complete.set() return True @@ -1117,162 +1274,325 @@ class Actor: async def _cancel_task( self, cid: str, - chan: Channel, + parent_chan: Channel, + requesting_uid: tuple[str, str]|None, + + ipc_msg: dict|None|bool = False, - requesting_uid: tuple[str, str] | None = None, ) -> 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. ''' - # right now this is only implicitly called by - # streaming IPC but it should be called - # to cancel any remotely spawned task + + # 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 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)] - scope = ctx._scope + ( + ctx, + func, + is_complete, + ) = self._rpc_tasks[( + parent_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: + # - child returns a result before cancel-msg/ctxc-raised + # - child self raises ctxc before parent send request, + # - child errors prior to cancel req. + log.runtime( + 'Cancel request for invalid RPC task.\n' + 'The task likely already completed or was never started!\n\n' + f'<= canceller: {requesting_uid}\n' + f'=> {cid}@{parent_chan.uid}\n' + f' |_{parent_chan}\n' + ) return True log.cancel( - f"Cancelling task:\ncid: {cid}\nfunc: {func}\n" - f"peer: {chan.uid}\n") + 'Cancel request for RPC task\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' + # 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: {cid}\n' + # f' >> {ctx._nsf}()\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 + + # 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: + log.error('Do not cancel a cancel!?') 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? + # await ctx.cancel() 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\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\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: ''' - 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 = self._rpc_tasks + tasks: dict = self._rpc_tasks + if not tasks: + log.runtime( + 'Actor has no cancellable RPC tasks?\n' + f'<= canceller: {req_uid}\n' + ) + return + + # TODO: seriously factor this into some helper funcs XD + tasks_str: str = '' + for (ctx, func, _) in tasks.values(): + + # 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' + ) + + descr: str = ( + 'all' if not parent_chan + else + "IPC channel's " + ) + rent_chan_repr: str = ( + f' |_{parent_chan}\n\n' + if parent_chan + else '' + ) + log.cancel( + f'Cancelling {descr} RPC tasks\n\n' + f'<= canceller: {req_uid}\n' + f'{rent_chan_repr}' + f'=> cancellee: {self.uid}\n' + f' |_{self}.cancel_rpc_tasks()\n' + f' |_tasks: {len(tasks)}\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 + + # 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, + ) + if tasks: - log.cancel(f"Cancelling all {len(tasks)} rpc tasks:\n{tasks} ") - for ( - (chan, cid), - (ctx, func, is_complete), - ) in tasks.copy().items(): - if only_chan is not None: - if only_chan != chan: - continue - - # TODO: this should really done in a nursery batch - if func != self._cancel_task: - await self._cancel_task( - cid, - chan, - requesting_uid=requesting_uid, - ) - log.cancel( - f"Waiting for remaining rpc tasks to complete {tasks}") - await self._ongoing_rpc_tasks.wait() + 'Waiting for remaining rpc tasks to complete\n' + f'|_{tasks_str}' + ) + await self._ongoing_rpc_tasks.wait() - def cancel_server(self) -> None: + def cancel_server(self) -> bool: ''' - 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: - log.runtime("Shutting down channel server") + # TODO: obvi a different server type when we eventually + # support some others XD + server_prot: str = 'TCP' + log.runtime( + f'Cancelling {server_prot} server' + ) self._server_n.cancel_scope.cancel() + return True + + return False @property - def accept_addr(self) -> tuple[str, int] | None: + def accept_addrs(self) -> list[tuple[str, int]]: ''' - Primary address to which the channel server is bound. + All addresses to which the IPC-transport-channel server + binds and listens for new connections. ''' # throws OSError on failure - return self._listeners[0].socket.getsockname() # type: ignore + return [ + listener.socket.getsockname() + for listener in self._listeners + ] # type: ignore + + @property + def accept_addr(self) -> tuple[str, int]: + ''' + Primary address to which the IPC transport server is + bound. + + ''' + # throws OSError on failure + return self.accept_addrs[0] 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?" return Portal(self._parent_chan) - def get_chans(self, uid: tuple[str, str]) -> list[Channel]: + 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] + # TODO: move to `Channel.handshake(uid)` async def _do_handshake( self, chan: Channel - ) -> tuple[str, str]: + ) -> msgtypes.Aid: ''' - Exchange (name, UUIDs) identifiers as the first communication step. + Exchange `(name, UUIDs)` identifiers as the first + communication step with any (peer) remote `Actor`. - 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() - uid: tuple[str, str] = (str(value[0]), str(value[1])) + name, uuid = self.uid + await chan.send( + msgtypes.Aid( + name=name, + uuid=uuid, + ) + ) + aid: msgtypes.Aid = await chan.recv() + chan.aid = aid + + uid: tuple[str, str] = ( + # str(value[0]), + # str(value[1]) + aid.name, + aid.uuid, + ) 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") + chan.uid = uid 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 async def async_main( actor: Actor, - accept_addr: tuple[str, int] | None = None, + accept_addrs: tuple[str, int]|None = None, # XXX: currently ``parent_addr`` is only needed for the # ``multiprocessing`` backend (which pickles state sent to @@ -1281,42 +1601,45 @@ 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: tuple[str, int] | None = None, + parent_addr: tuple[str, int]|None = None, task_status: TaskStatus[None] = trio.TASK_STATUS_IGNORED, ) -> 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 - # on our debugger lock state. - _debug.Lock._trio_handler = signal.getsignal(signal.SIGINT) + # on our debugger state. + _debug.DebugStatus._trio_handler = signal.getsignal(signal.SIGINT) - registered_with_arbiter = False + is_registered: bool = False try: # establish primary connection with immediate parent - actor._parent_chan = None + actor._parent_chan: Channel|None = None if parent_addr is not None: - actor._parent_chan, accept_addr_rent = await actor._from_parent( - parent_addr) + ( + actor._parent_chan, + set_accept_addr_says_rent, + ) = await actor._from_parent(parent_addr) - # either it's passed in because we're not a child - # or because we're running in mp mode - 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() + # either it's passed in because we're not a child or + # because we're running in mp mode + if ( + set_accept_addr_says_rent + and + set_accept_addr_says_rent is not None + ): + accept_addrs = set_accept_addr_says_rent # The "root" nursery ensures the channel with the immediate # parent is kept alive as a resilient service until @@ -1334,38 +1657,97 @@ 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 - assert accept_addr - host, port = accept_addr + assert accept_addrs - actor._server_n = await service_nursery.start( - partial( - actor._serve_forever, - service_nursery, - accept_host=host, - accept_port=port + try: + # TODO: why is this not with the root nursery? + actor._server_n = await service_nursery.start( + partial( + actor._serve_forever, + service_nursery, + listen_sockaddrs=accept_addrs, + ) ) - ) - accept_addr = actor.accept_addr + except OSError as oserr: + # NOTE: always allow runtime hackers to debug + # tranport address bind errors - normally it's + # something silly like the wrong socket-address + # passed via a config or CLI Bo + entered_debug = await _debug._maybe_enter_pm( + oserr, + ) + if entered_debug: + log.runtime('Exited debug REPL..') + raise + + accept_addrs: list[tuple[str, int]] = actor.accept_addrs + + # NOTE: only set the loopback addr for the + # process-tree-global "root" mailbox since + # all sub-actors should be able to speak to + # their root actor over that channel. if _state._runtime_vars['_is_root']: - _state._runtime_vars['_root_mailbox'] = accept_addr + for addr in accept_addrs: + host, _ = addr + # TODO: generic 'lo' detector predicate + if '127.0.0.1' in host: + _state._runtime_vars['_root_mailbox'] = addr # Register with the arbiter if we're told its addr - log.runtime(f"Registering {actor} for role `{actor.name}`") - assert isinstance(actor._arb_addr, tuple) + log.runtime( + f'Registering `{actor.name}` ->\n' + f'{pformat(accept_addrs)}' + ) - async with get_arbiter(*actor._arb_addr) as arb_portal: - await arb_portal.run_from_ns( - 'self', - 'register_actor', - uid=actor.uid, - sockaddr=accept_addr, - ) + # TODO: ideally we don't fan out to all registrars + # if addresses point to the same actor.. + # So we need a way to detect that? maybe iterate + # only on unique actor uids? + for addr in actor.reg_addrs: + try: + assert isinstance(addr, tuple) + assert addr[1] # non-zero after bind + except AssertionError: + await _debug.pause() - registered_with_arbiter = True + async with get_registry(*addr) as reg_portal: + for accept_addr in accept_addrs: + + if not accept_addr[1]: + await _debug.pause() + + assert accept_addr[1] + + await reg_portal.run_from_ns( + 'self', + 'register_actor', + uid=actor.uid, + sockaddr=accept_addr, + ) + + is_registered: bool = True # init steps complete task_status.started() @@ -1383,7 +1765,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" @@ -1395,22 +1780,27 @@ async def async_main( log.runtime("Closing all actor lifetime contexts") actor.lifetime_stack.close() - if not registered_with_arbiter: + if not is_registered: # TODO: I guess we could try to connect back # to the parent through a channel and engage a debugger # once we have that all working with std streams locking? log.exception( f"Actor errored and failed to register with arbiter " - f"@ {actor._arb_addr}?") + f"@ {actor.reg_addrs[0]}?") log.error( - "\n\n\t^^^ THIS IS PROBABLY A TRACTOR BUGGGGG!!! ^^^\n" - "\tCALMLY CALL THE AUTHORITIES AND HIDE YOUR CHILDREN.\n\n" - "\tYOUR PARENT CODE IS GOING TO KEEP WORKING FINE!!!\n" - "\tTHIS IS HOW RELIABlE SYSTEMS ARE SUPPOSED TO WORK!?!?\n" + "\n\n\t^^^ THIS IS PROBABLY AN INTERNAL `tractor` BUG! ^^^\n\n" + "\t>> CALMLY CALL THE AUTHORITIES AND HIDE YOUR CHILDREN <<\n\n" + "\tIf this is a sub-actor hopefully its parent will keep running " + "correctly presuming this error was safely ignored..\n\n" + "\tPLEASE REPORT THIS TRACEBACK IN A BUG REPORT: " + "https://github.com/goodboy/tractor/issues\n" ) if actor._parent_chan: - await try_ship_error_to_parent(actor._parent_chan, err) + await try_ship_error_to_remote( + actor._parent_chan, + err, + ) # always! match err: @@ -1424,11 +1814,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, @@ -1436,34 +1828,38 @@ 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() - - # Unregister actor from the arbiter + # Unregister actor from the registry-sys / registrar. if ( - registered_with_arbiter - and not actor.is_arbiter + is_registered + and not actor.is_registrar ): - failed = False - assert isinstance(actor._arb_addr, tuple) - with trio.move_on_after(0.5) as cs: - cs.shield = True - try: - async with get_arbiter(*actor._arb_addr) as arb_portal: - await arb_portal.run_from_ns( - 'self', - 'unregister_actor', - uid=actor.uid - ) - except OSError: + failed: bool = False + for addr in actor.reg_addrs: + assert isinstance(addr, tuple) + with trio.move_on_after(0.5) as cs: + cs.shield = True + try: + async with get_registry( + *addr, + ) as reg_portal: + await reg_portal.run_from_ns( + 'self', + 'unregister_actor', + uid=actor.uid + ) + except OSError: + failed = True + if cs.cancelled_caught: failed = True - if cs.cancelled_caught: - failed = True - if failed: - log.warning( - f"Failed to unregister {actor.name} from arbiter") + + if failed: + log.warning( + f'Failed to unregister {actor.name} from ' + f'registar @ {addr}' + ) # Ensure all peers (actors connected to us as clients) are finished if not actor._no_more_peers.is_set(): @@ -1472,301 +1868,46 @@ 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") log.runtime("Runtime completed") -async def process_messages( - actor: Actor, - chan: Channel, - shield: bool = False, - task_status: TaskStatus[trio.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 https://github.com/python-trio/trio/issues/467 gets - # worked out we'll likely want to use that! - msg: dict | None = None - nursery_cancelled_before_task: bool = False - - log.runtime(f"Entering msg loop for {chan} from {chan.uid}") - try: - with trio.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 - - log.cancel( - f"Channel to {chan.uid} terminated?\n" - "Cancelling all associated tasks..") - - for (channel, cid) in actor._rpc_tasks.copy(): - 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}") - - 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'] - 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( - f"Processing request from {actorid}\n" - f"{ns}.{funcname}({kwargs})") - - if ns == 'self': - if funcname == 'cancel': - func = 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 - if pdb_complete: - await pdb_complete.wait() - - # 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()`` - log.cancel( - "Actor runtime for was remotely cancelled " - f"by {chan.uid}" - ) - await _invoke( - 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): - 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, - ) - except BaseException: - log.exception("failed to cancel task?") - - continue - else: - # normally registry methods, eg. - # ``.register_actor()`` etc. - func = 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 = pack_error(err) - err_msg['cid'] = cid - 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 - 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(ctx, Exception): - log.warning( - f"Task for RPC func {func} failed with" - 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)] = ( - ctx, - func, - trio.Event(), - ) - - log.runtime( - f"Waiting on next msg for {chan} from {chan.uid}") - - # end of async for, channel disconnect vis - # ``trio.EndOfChannel`` - log.runtime( - f"{chan} for {chan.uid} disconnected, cancelling tasks" - ) - await actor.cancel_rpc_tasks(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. - log.runtime( - f'channel from {chan.uid} closed abruptly:\n' - f'-> {chan.raddr}\n' - ) - - # transport **was** disconnected - return True - - except ( - Exception, - BaseExceptionGroup, - ) as err: - if nursery_cancelled_before_task: - sn = actor._service_n - assert sn and sn.cancel_scope.cancel_called - 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_parent(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( - f"Exiting msg loop for {chan} from {chan.uid} " - f"with last msg:\n{msg}" - ) - - # 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 - access to a top level nursery. + A special registrar actor who can contact all other actors + within its immediate process tree and possibly keeps a registry + of others meant to be discoverable in a distributed + application. Normally the registrar is also the "root actor" + and thus always has access to the top-most-level actor + (process) nursery. - The arbiter is by default the first actor spawned on each host - and is responsible for keeping track of all other actors for - coordination purposes. If a new main process is launched and an - arbiter is already running that arbiter will be used. + By default, the registrar is always initialized when and if no + other registrar socket addrs have been specified to runtime + init entry-points (such as `open_root_actor()` or + `open_nursery()`). Any time a new main process is launched (and + thus thus a new root actor created) and, no existing registrar + can be contacted at the provided `registry_addr`, then a new + one is always created; however, if one can be reached it is + used. + + Normally a distributed app requires at least registrar per + logical host where for that given "host space" (aka localhost + IPC domain of addresses) it is responsible for making all other + host (local address) bound actors *discoverable* to external + actor trees running on remote hosts. ''' is_arbiter = True - def __init__(self, *args, **kwargs) -> None: + def __init__( + self, + *args, + **kwargs, + ) -> None: self._registry: dict[ tuple[str, str], @@ -1786,7 +1927,7 @@ class Arbiter(Actor): self, name: str, - ) -> tuple[str, int] | None: + ) -> tuple[str, int]|None: for uid, sockaddr in self._registry.items(): if name in uid: @@ -1808,7 +1949,10 @@ class Arbiter(Actor): # unpacker since we have tuples as keys (not this makes the # arbiter suscetible to hashdos): # https://github.com/msgpack/msgpack-python#major-breaking-changes-in-msgpack-10 - return {'.'.join(key): val for key, val in self._registry.items()} + return { + '.'.join(key): val + for key, val in self._registry.items() + } async def wait_for_actor( self, @@ -1825,8 +1969,13 @@ class Arbiter(Actor): sockaddrs: list[tuple[str, int]] = [] sockaddr: tuple[str, int] - for (aname, _), sockaddr in self._registry.items(): - if name == aname: + mailbox_info: str = 'Actor registry contact infos:\n' + for uid, sockaddr in self._registry.items(): + mailbox_info += ( + f'|_uid: {uid}\n' + f'|_sockaddr: {sockaddr}\n\n' + ) + if name == uid[0]: sockaddrs.append(sockaddr) if not sockaddrs: @@ -1838,6 +1987,7 @@ class Arbiter(Actor): if not isinstance(uid, trio.Event): sockaddrs.append(self._registry[uid]) + log.runtime(mailbox_info) return sockaddrs async def register_actor( @@ -1846,8 +1996,15 @@ class Arbiter(Actor): sockaddr: tuple[str, int] ) -> None: - uid = name, _ = (str(uid[0]), str(uid[1])) - self._registry[uid] = (str(sockaddr[0]), int(sockaddr[1])) + uid = name, hash = (str(uid[0]), str(uid[1])) + addr = (host, port) = ( + str(sockaddr[0]), + int(sockaddr[1]), + ) + if port == 0: + await _debug.pause() + assert port # should never be 0-dynamic-os-alloc + self._registry[uid] = addr # pop and signal all waiter events events = self._waiters.pop(name, []) @@ -1862,4 +2019,6 @@ class Arbiter(Actor): ) -> None: uid = (str(uid[0]), str(uid[1])) - self._registry.pop(uid) + entry: tuple = self._registry.pop(uid, None) + if entry is None: + log.warning(f'Request to de-register {uid} failed?') diff --git a/tractor/_shm.py b/tractor/_shm.py new file mode 100644 index 00000000..da6d8ddb --- /dev/null +++ b/tractor/_shm.py @@ -0,0 +1,833 @@ +# 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 . + +""" +SC friendly shared memory management geared at real-time +processing. + +Support for ``numpy`` compatible array-buffers is provided but is +considered optional within the context of this runtime-library. + +""" +from __future__ import annotations +from sys import byteorder +import time +from typing import Optional +from multiprocessing import shared_memory as shm +from multiprocessing.shared_memory import ( + SharedMemory, + ShareableList, +) + +from msgspec import Struct +import tractor + +from .log import get_logger + + +_USE_POSIX = getattr(shm, '_USE_POSIX', False) +if _USE_POSIX: + from _posixshmem import shm_unlink + + +try: + import numpy as np + from numpy.lib import recfunctions as rfn + # import nptyping +except ImportError: + pass + + +log = get_logger(__name__) + + +def disable_mantracker(): + ''' + Disable all ``multiprocessing``` "resource tracking" machinery since + it's an absolute multi-threaded mess of non-SC madness. + + ''' + from multiprocessing import resource_tracker as mantracker + + # Tell the "resource tracker" thing to fuck off. + class ManTracker(mantracker.ResourceTracker): + def register(self, name, rtype): + pass + + def unregister(self, name, rtype): + pass + + def ensure_running(self): + pass + + # "know your land and know your prey" + # https://www.dailymotion.com/video/x6ozzco + mantracker._resource_tracker = ManTracker() + mantracker.register = mantracker._resource_tracker.register + mantracker.ensure_running = mantracker._resource_tracker.ensure_running + mantracker.unregister = mantracker._resource_tracker.unregister + mantracker.getfd = mantracker._resource_tracker.getfd + + +disable_mantracker() + + +class SharedInt: + ''' + Wrapper around a single entry shared memory array which + holds an ``int`` value used as an index counter. + + ''' + def __init__( + self, + shm: SharedMemory, + ) -> None: + self._shm = shm + + @property + def value(self) -> int: + return int.from_bytes(self._shm.buf, byteorder) + + @value.setter + def value(self, value) -> None: + self._shm.buf[:] = value.to_bytes(self._shm.size, byteorder) + + def destroy(self) -> None: + if _USE_POSIX: + # We manually unlink to bypass all the "resource tracker" + # nonsense meant for non-SC systems. + name = self._shm.name + try: + shm_unlink(name) + except FileNotFoundError: + # might be a teardown race here? + log.warning(f'Shm for {name} already unlinked?') + + +class NDToken(Struct, frozen=True): + ''' + Internal represenation of a shared memory ``numpy`` array "token" + which can be used to key and load a system (OS) wide shm entry + and correctly read the array by type signature. + + This type is msg safe. + + ''' + shm_name: str # this servers as a "key" value + shm_first_index_name: str + shm_last_index_name: str + dtype_descr: tuple + size: int # in struct-array index / row terms + + # TODO: use nptyping here on dtypes + @property + def dtype(self) -> list[tuple[str, str, tuple[int, ...]]]: + return np.dtype( + list( + map(tuple, self.dtype_descr) + ) + ).descr + + def as_msg(self): + return self.to_dict() + + @classmethod + def from_msg(cls, msg: dict) -> NDToken: + if isinstance(msg, NDToken): + return msg + + # TODO: native struct decoding + # return _token_dec.decode(msg) + + msg['dtype_descr'] = tuple(map(tuple, msg['dtype_descr'])) + return NDToken(**msg) + + +# _token_dec = msgspec.msgpack.Decoder(NDToken) + +# TODO: this api? +# _known_tokens = tractor.ActorVar('_shm_tokens', {}) +# _known_tokens = tractor.ContextStack('_known_tokens', ) +# _known_tokens = trio.RunVar('shms', {}) + +# TODO: this should maybe be provided via +# a `.trionics.maybe_open_context()` wrapper factory? +# process-local store of keys to tokens +_known_tokens: dict[str, NDToken] = {} + + +def get_shm_token(key: str) -> NDToken | None: + ''' + Convenience func to check if a token + for the provided key is known by this process. + + Returns either the ``numpy`` token or a string for a shared list. + + ''' + return _known_tokens.get(key) + + +def _make_token( + key: str, + size: int, + dtype: np.dtype, + +) -> NDToken: + ''' + Create a serializable token that can be used + to access a shared array. + + ''' + return NDToken( + shm_name=key, + shm_first_index_name=key + "_first", + shm_last_index_name=key + "_last", + dtype_descr=tuple(np.dtype(dtype).descr), + size=size, + ) + + +class ShmArray: + ''' + A shared memory ``numpy.ndarray`` API. + + An underlying shared memory buffer is allocated based on + a user specified ``numpy.ndarray``. This fixed size array + can be read and written to by pushing data both onto the "front" + or "back" of a set index range. The indexes for the "first" and + "last" index are themselves stored in shared memory (accessed via + ``SharedInt`` interfaces) values such that multiple processes can + interact with the same array using a synchronized-index. + + ''' + def __init__( + self, + shmarr: np.ndarray, + first: SharedInt, + last: SharedInt, + shm: SharedMemory, + # readonly: bool = True, + ) -> None: + self._array = shmarr + + # indexes for first and last indices corresponding + # to fille data + self._first = first + self._last = last + + self._len = len(shmarr) + self._shm = shm + self._post_init: bool = False + + # pushing data does not write the index (aka primary key) + self._write_fields: list[str] | None = None + dtype = shmarr.dtype + if dtype.fields: + self._write_fields = list(shmarr.dtype.fields.keys())[1:] + + # TODO: ringbuf api? + + @property + def _token(self) -> NDToken: + return NDToken( + shm_name=self._shm.name, + shm_first_index_name=self._first._shm.name, + shm_last_index_name=self._last._shm.name, + dtype_descr=tuple(self._array.dtype.descr), + size=self._len, + ) + + @property + def token(self) -> dict: + """Shared memory token that can be serialized and used by + another process to attach to this array. + """ + return self._token.as_msg() + + @property + def index(self) -> int: + return self._last.value % self._len + + @property + def array(self) -> np.ndarray: + ''' + Return an up-to-date ``np.ndarray`` view of the + so-far-written data to the underlying shm buffer. + + ''' + a = self._array[self._first.value:self._last.value] + + # first, last = self._first.value, self._last.value + # a = self._array[first:last] + + # TODO: eventually comment this once we've not seen it in the + # wild in a long time.. + # XXX: race where first/last indexes cause a reader + # to load an empty array.. + if len(a) == 0 and self._post_init: + raise RuntimeError('Empty array race condition hit!?') + # breakpoint() + + return a + + def ustruct( + self, + fields: Optional[list[str]] = None, + + # type that all field values will be cast to + # in the returned view. + common_dtype: np.dtype = float, + + ) -> np.ndarray: + + array = self._array + + if fields: + selection = array[fields] + # fcount = len(fields) + else: + selection = array + # fcount = len(array.dtype.fields) + + # XXX: manual ``.view()`` attempt that also doesn't work. + # uview = selection.view( + # dtype=' np.ndarray: + ''' + Return the last ``length``'s worth of ("row") entries from the + array. + + ''' + return self.array[-length:] + + def push( + self, + data: np.ndarray, + + field_map: Optional[dict[str, str]] = None, + prepend: bool = False, + update_first: bool = True, + start: int | None = None, + + ) -> int: + ''' + Ring buffer like "push" to append data + into the buffer and return updated "last" index. + + NB: no actual ring logic yet to give a "loop around" on overflow + condition, lel. + + ''' + length = len(data) + + if prepend: + index = (start or self._first.value) - length + + if index < 0: + raise ValueError( + f'Array size of {self._len} was overrun during prepend.\n' + f'You have passed {abs(index)} too many datums.' + ) + + else: + index = start if start is not None else self._last.value + + end = index + length + + if field_map: + src_names, dst_names = zip(*field_map.items()) + else: + dst_names = src_names = self._write_fields + + try: + self._array[ + list(dst_names) + ][index:end] = data[list(src_names)][:] + + # NOTE: there was a race here between updating + # the first and last indices and when the next reader + # tries to access ``.array`` (which due to the index + # overlap will be empty). Pretty sure we've fixed it now + # but leaving this here as a reminder. + if ( + prepend + and update_first + and length + ): + assert index < self._first.value + + if ( + index < self._first.value + and update_first + ): + assert prepend, 'prepend=True not passed but index decreased?' + self._first.value = index + + elif not prepend: + self._last.value = end + + self._post_init = True + return end + + except ValueError as err: + if field_map: + raise + + # should raise if diff detected + self.diff_err_fields(data) + raise err + + def diff_err_fields( + self, + data: np.ndarray, + ) -> None: + # reraise with any field discrepancy + our_fields, their_fields = ( + set(self._array.dtype.fields), + set(data.dtype.fields), + ) + + only_in_ours = our_fields - their_fields + only_in_theirs = their_fields - our_fields + + if only_in_ours: + raise TypeError( + f"Input array is missing field(s): {only_in_ours}" + ) + elif only_in_theirs: + raise TypeError( + f"Input array has unknown field(s): {only_in_theirs}" + ) + + # TODO: support "silent" prepends that don't update ._first.value? + def prepend( + self, + data: np.ndarray, + ) -> int: + end = self.push(data, prepend=True) + assert end + + def close(self) -> None: + self._first._shm.close() + self._last._shm.close() + self._shm.close() + + def destroy(self) -> None: + if _USE_POSIX: + # We manually unlink to bypass all the "resource tracker" + # nonsense meant for non-SC systems. + shm_unlink(self._shm.name) + + self._first.destroy() + self._last.destroy() + + def flush(self) -> None: + # TODO: flush to storage backend like markestore? + ... + + +def open_shm_ndarray( + size: int, + key: str | None = None, + dtype: np.dtype | None = None, + append_start_index: int | None = None, + readonly: bool = False, + +) -> ShmArray: + ''' + Open a memory shared ``numpy`` using the standard library. + + This call unlinks (aka permanently destroys) the buffer on teardown + and thus should be used from the parent-most accessor (process). + + ''' + # create new shared mem segment for which we + # have write permission + a = np.zeros(size, dtype=dtype) + a['index'] = np.arange(len(a)) + + shm = SharedMemory( + name=key, + create=True, + size=a.nbytes + ) + array = np.ndarray( + a.shape, + dtype=a.dtype, + buffer=shm.buf + ) + array[:] = a[:] + array.setflags(write=int(not readonly)) + + token = _make_token( + key=key, + size=size, + dtype=dtype, + ) + + # create single entry arrays for storing an first and last indices + first = SharedInt( + shm=SharedMemory( + name=token.shm_first_index_name, + create=True, + size=4, # std int + ) + ) + + last = SharedInt( + shm=SharedMemory( + name=token.shm_last_index_name, + create=True, + size=4, # std int + ) + ) + + # Start the "real-time" append-updated (or "pushed-to") section + # after some start index: ``append_start_index``. This allows appending + # from a start point in the array which isn't the 0 index and looks + # something like, + # ------------------------- + # | | i + # _________________________ + # <-------------> <-------> + # history real-time + # + # Once fully "prepended", the history section will leave the + # ``ShmArray._start.value: int = 0`` and the yet-to-be written + # real-time section will start at ``ShmArray.index: int``. + + # this sets the index to nearly 2/3rds into the the length of + # the buffer leaving at least a "days worth of second samples" + # for the real-time section. + if append_start_index is None: + append_start_index = round(size * 0.616) + + last.value = first.value = append_start_index + + shmarr = ShmArray( + array, + first, + last, + shm, + ) + + assert shmarr._token == token + _known_tokens[key] = shmarr.token + + # "unlink" created shm on process teardown by + # pushing teardown calls onto actor context stack + stack = tractor.current_actor().lifetime_stack + stack.callback(shmarr.close) + stack.callback(shmarr.destroy) + + return shmarr + + +def attach_shm_ndarray( + token: tuple[str, str, tuple[str, str]], + readonly: bool = True, + +) -> ShmArray: + ''' + Attach to an existing shared memory array previously + created by another process using ``open_shared_array``. + + No new shared mem is allocated but wrapper types for read/write + access are constructed. + + ''' + token = NDToken.from_msg(token) + key = token.shm_name + + if key in _known_tokens: + assert NDToken.from_msg(_known_tokens[key]) == token, "WTF" + + # XXX: ugh, looks like due to the ``shm_open()`` C api we can't + # actually place files in a subdir, see discussion here: + # https://stackoverflow.com/a/11103289 + + # attach to array buffer and view as per dtype + _err: Optional[Exception] = None + for _ in range(3): + try: + shm = SharedMemory( + name=key, + create=False, + ) + break + except OSError as oserr: + _err = oserr + time.sleep(0.1) + else: + if _err: + raise _err + + shmarr = np.ndarray( + (token.size,), + dtype=token.dtype, + buffer=shm.buf + ) + shmarr.setflags(write=int(not readonly)) + + first = SharedInt( + shm=SharedMemory( + name=token.shm_first_index_name, + create=False, + size=4, # std int + ), + ) + last = SharedInt( + shm=SharedMemory( + name=token.shm_last_index_name, + create=False, + size=4, # std int + ), + ) + + # make sure we can read + first.value + + sha = ShmArray( + shmarr, + first, + last, + shm, + ) + # read test + sha.array + + # Stash key -> token knowledge for future queries + # via `maybe_opepn_shm_array()` but only after we know + # we can attach. + if key not in _known_tokens: + _known_tokens[key] = token + + # "close" attached shm on actor teardown + tractor.current_actor().lifetime_stack.callback(sha.close) + + return sha + + +def maybe_open_shm_ndarray( + key: str, # unique identifier for segment + size: int, + dtype: np.dtype | None = None, + append_start_index: int = 0, + readonly: bool = True, + +) -> tuple[ShmArray, bool]: + ''' + Attempt to attach to a shared memory block using a "key" lookup + to registered blocks in the users overall "system" registry + (presumes you don't have the block's explicit token). + + This function is meant to solve the problem of discovering whether + a shared array token has been allocated or discovered by the actor + running in **this** process. Systems where multiple actors may seek + to access a common block can use this function to attempt to acquire + a token as discovered by the actors who have previously stored + a "key" -> ``NDToken`` map in an actor local (aka python global) + variable. + + If you know the explicit ``NDToken`` for your memory segment instead + use ``attach_shm_array``. + + ''' + try: + # see if we already know this key + token = _known_tokens[key] + return ( + attach_shm_ndarray( + token=token, + readonly=readonly, + ), + False, # not newly opened + ) + except KeyError: + log.warning(f"Could not find {key} in shms cache") + if dtype: + token = _make_token( + key, + size=size, + dtype=dtype, + ) + else: + + try: + return ( + attach_shm_ndarray( + token=token, + readonly=readonly, + ), + False, + ) + except FileNotFoundError: + log.warning(f"Could not attach to shm with token {token}") + + # This actor does not know about memory + # associated with the provided "key". + # Attempt to open a block and expect + # to fail if a block has been allocated + # on the OS by someone else. + return ( + open_shm_ndarray( + key=key, + size=size, + dtype=dtype, + append_start_index=append_start_index, + readonly=readonly, + ), + True, + ) + + +class ShmList(ShareableList): + ''' + Carbon copy of ``.shared_memory.ShareableList`` with a few + enhancements: + + - readonly mode via instance var flag `._readonly: bool` + - ``.__getitem__()`` accepts ``slice`` inputs + - exposes the underlying buffer "name" as a ``.key: str`` + + ''' + def __init__( + self, + sequence: list | None = None, + *, + name: str | None = None, + readonly: bool = True + + ) -> None: + self._readonly = readonly + self._key = name + return super().__init__( + sequence=sequence, + name=name, + ) + + @property + def key(self) -> str: + return self._key + + @property + def readonly(self) -> bool: + return self._readonly + + def __setitem__( + self, + position, + value, + + ) -> None: + + # mimick ``numpy`` error + if self._readonly: + raise ValueError('assignment destination is read-only') + + return super().__setitem__(position, value) + + def __getitem__( + self, + indexish, + ) -> list: + + # NOTE: this is a non-writeable view (copy?) of the buffer + # in a new list instance. + if isinstance(indexish, slice): + return list(self)[indexish] + + return super().__getitem__(indexish) + + # TODO: should we offer a `.array` and `.push()` equivalent + # to the `ShmArray`? + # currently we have the following limitations: + # - can't write slices of input using traditional slice-assign + # syntax due to the ``ShareableList.__setitem__()`` implementation. + # - ``list(shmlist)`` returns a non-mutable copy instead of + # a writeable view which would be handier numpy-style ops. + + +def open_shm_list( + key: str, + sequence: list | None = None, + size: int = int(2 ** 10), + dtype: float | int | bool | str | bytes | None = float, + readonly: bool = True, + +) -> ShmList: + + if sequence is None: + default = { + float: 0., + int: 0, + bool: True, + str: 'doggy', + None: None, + }[dtype] + sequence = [default] * size + + shml = ShmList( + sequence=sequence, + name=key, + readonly=readonly, + ) + + # "close" attached shm on actor teardown + try: + actor = tractor.current_actor() + actor.lifetime_stack.callback(shml.shm.close) + actor.lifetime_stack.callback(shml.shm.unlink) + except RuntimeError: + log.warning('tractor runtime not active, skipping teardown steps') + + return shml + + +def attach_shm_list( + key: str, + readonly: bool = False, + +) -> ShmList: + + return ShmList( + name=key, + readonly=readonly, + ) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index db465421..aeb7a7c9 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -31,25 +31,28 @@ from typing import ( TYPE_CHECKING, ) -from exceptiongroup import BaseExceptionGroup import trio -from trio_typing import TaskStatus +from trio import TaskStatus -from ._debug import ( +from tractor.devx 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, + _runtime_vars, +) +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 +from tractor.msg.types import ( + SpawnSpec, ) -from .log import get_logger -from ._portal import Portal -from ._runtime import Actor -from ._entry import _mp_main -from ._exceptions import ActorFailure if TYPE_CHECKING: @@ -140,11 +143,13 @@ async def exhaust_portal( ''' __tracebackhide__ = True try: - log.debug(f"Waiting on final result from {actor.uid}") + log.debug( + f'Waiting on final result from {actor.uid}' + ) # 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 +157,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,41 +185,74 @@ 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 - log.warning( - f"Cancelling {portal.channel.uid} after error {result}" + errors[actor.uid]: Exception = result + 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( - f"Cancelling {portal.channel.uid} gracefully " - f"after result {result}") + 'Cancelling subactor gracefully:\n\n' + f'Portal.cancel_actor() => {portal.channel.uid}\n\n' + f'result: {result}\n' + ) # cancel the process now that we have a final result await portal.cancel_actor() -async def do_hard_kill( +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 + # 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. + + ''' + 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 @@ -215,6 +263,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() @@ -230,16 +281,25 @@ 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}") + # TODO: toss in the skynet-logo face as ascii art? + log.critical( + # 'Well, the #ZOMBIE_LORD_IS_HERE# to collect\n' + '#T-800 deployed to collect zombie B0\n' + f'|\n' + f'|_{proc}\n' + ) proc.kill() -async def soft_wait( - +async def soft_kill( proc: ProcessType, wait_func: Callable[ [ProcessType], @@ -248,15 +308,39 @@ 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. - uid = portal.channel.uid + ''' + 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(f'Soft waiting on actor:\n{uid}') + log.cancel( + 'Soft killing sub-actor via `Portal.cancel_actor()`\n' + f'|_{proc}\n' + ) + # wait on sub-proc to signal termination await wait_func(proc) + except trio.Cancelled: + with trio.CancelScope(shield=True): + await maybe_wait_for_debugger( + child_in_debug=_runtime_vars.get( + '_debug_mode', False + ), + header_msg=( + 'Delaying `soft_kill()` subproc reaper while debugger locked..\n' + ), + # TODO: need a diff value then default? + # poll_steps=9999999, + ) + # if cancelled during a soft wait, cancel the child # actor before entering the hard reap sequence # below. This means we try to do a graceful teardown @@ -267,22 +351,29 @@ 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-cancel" request: if we detect that the + underlying sub-process exited prior to + a `Portal.cancel_actor()` call completing . ''' 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 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 @@ -294,7 +385,7 @@ async def new_proc( errors: dict[tuple[str, str], Exception], # passed through to actor main - bind_addr: tuple[str, int], + bind_addrs: list[tuple[str, int]], parent_addr: tuple[str, int], _runtime_vars: dict[str, Any], # serialized and sent to _child @@ -306,7 +397,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 @@ -316,7 +407,7 @@ async def new_proc( actor_nursery, subactor, errors, - bind_addr, + bind_addrs, parent_addr, _runtime_vars, # run time vars infect_asyncio=infect_asyncio, @@ -331,7 +422,7 @@ async def trio_proc( errors: dict[tuple[str, str], Exception], # passed through to actor main - bind_addr: tuple[str, int], + bind_addrs: list[tuple[str, int]], parent_addr: tuple[str, int], _runtime_vars: dict[str, Any], # serialized and sent to _child *, @@ -374,19 +465,21 @@ 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}") + proc: trio.Process = await trio.lowlevel.open_process(spawn_cmd) + log.runtime( + 'Started new child\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 @@ -415,18 +508,20 @@ async def trio_proc( portal, ) - # send additional init params - await chan.send({ - "_parent_main_data": subactor._parent_main_data, - "enable_modules": subactor.enable_modules, - "_arb_addr": subactor._arb_addr, - "bind_host": bind_addr[0], - "bind_port": bind_addr[1], - "_runtime_vars": _runtime_vars, - }) + # send a "spawning specification" which configures the + # initial runtime state of the child. + await chan.send( + SpawnSpec( + _parent_main_data=subactor._parent_main_data, + enable_modules=subactor.enable_modules, + reg_addrs=subactor.reg_addrs, + bind_addrs=bind_addrs, + _runtime_vars=_runtime_vars, + ) + ) # track subactor in current nursery - curr_actor = current_actor() + curr_actor: Actor = current_actor() curr_actor._actoruid2nursery[subactor.uid] = actor_nursery # resume caller at next checkpoint now that child is up @@ -448,7 +543,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 @@ -457,8 +552,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: @@ -467,8 +563,8 @@ async def trio_proc( # killing the process too early. if proc: log.cancel(f'Hard reap sequence starting for {subactor.uid}') - with trio.CancelScope(shield=True): + with trio.CancelScope(shield=True): # don't clobber an ongoing pdb if cancelled_during_spawn: # Try again to avoid TTY clobbering. @@ -476,22 +572,40 @@ 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), - ) + await maybe_wait_for_debugger( + child_in_debug=_runtime_vars.get( + '_debug_mode', False + ), + header_msg=( + 'Delaying subproc reaper while debugger locked..\n' + ), + + # 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}") - await do_hard_kill(proc) + await hard_kill(proc) log.debug(f"Joined {proc}") else: @@ -509,7 +623,7 @@ async def mp_proc( subactor: Actor, errors: dict[tuple[str, str], Exception], # passed through to actor main - bind_addr: tuple[str, int], + bind_addrs: list[tuple[str, int]], parent_addr: tuple[str, int], _runtime_vars: dict[str, Any], # serialized and sent to _child *, @@ -567,7 +681,7 @@ async def mp_proc( target=_mp_main, args=( subactor, - bind_addr, + bind_addrs, fs_info, _spawn_method, parent_addr, @@ -635,7 +749,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 diff --git a/tractor/_state.py b/tractor/_state.py index f94c3ebb..8c5cca14 100644 --- a/tractor/_state.py +++ b/tractor/_state.py @@ -18,27 +18,80 @@ Per process state """ +from __future__ import annotations +from contextvars import ( + ContextVar, +) from typing import ( - Optional, Any, + TYPE_CHECKING, ) -_current_actor: Optional['Actor'] = None # type: ignore # noqa +from trio.lowlevel import current_task + +if TYPE_CHECKING: + from ._runtime import Actor + from ._context import Context + + +_current_actor: Actor|None = None # type: ignore # noqa +_last_actor_terminated: Actor|None = None + +# TODO: mk this a `msgspec.Struct`! _runtime_vars: dict[str, Any] = { '_debug_mode': False, '_is_root': False, - '_root_mailbox': (None, None) + '_root_mailbox': (None, None), + '_registry_addrs': [], + + # for `breakpoint()` support + 'use_greenback': False, } -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?\n' + 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' + '\nDid you forget to call one of,\n' + '- `tractor.open_root_actor()`\n' + '- `tractor.open_nursery()`\n' + ) + + raise NoRuntime(msg) return _current_actor @@ -63,3 +116,26 @@ def debug_mode() -> bool: def is_root_process() -> bool: return _runtime_vars['_is_root'] + + +_ctxvar_Context: ContextVar[Context] = ContextVar( + 'ipc_context', + default=None, +) + + +def current_ipc_ctx( + error_on_not_set: bool = False, +) -> Context|None: + ctx: Context = _ctxvar_Context.get() + + if ( + not ctx + and error_on_not_set + ): + from ._exceptions import InternalError + raise InternalError( + 'No IPC context has been allocated for this task yet?\n' + f'|_{current_task()}\n' + ) + return ctx diff --git a/tractor/_streaming.py b/tractor/_streaming.py index 3045b835..314a93b8 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -21,10 +21,12 @@ 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, + AsyncGenerator, Callable, AsyncIterator, TYPE_CHECKING, @@ -34,16 +36,25 @@ import warnings import trio from ._exceptions import ( - unpack_error, + # _raise_from_no_key_in_msg, + ContextCancelled, ) from .log import get_logger from .trionics import ( broadcast_receiver, BroadcastReceiver, ) +from tractor.msg import ( + # Return, + # Stop, + MsgType, + Yield, +) if TYPE_CHECKING: + from ._runtime import Actor from ._context import Context + from ._ipc import Channel log = get_logger(__name__) @@ -54,14 +65,13 @@ log = get_logger(__name__) # messages? class ReceiveChannel(AsyncResource, Generic[ReceiveType]): # - use __slots__ on ``Context``? - class MsgStream(trio.abc.Channel): ''' A bidirectional message stream for receiving logically sequenced - values over an inter-actor IPC ``Channel``. + values over an inter-actor IPC `Channel`. This is the type returned to a local task which entered either - ``Portal.open_stream_from()`` or ``Context.open_stream()``. + `Portal.open_stream_from()` or `Context.open_stream()`. Termination rules: @@ -77,7 +87,7 @@ class MsgStream(trio.abc.Channel): self, ctx: Context, # typing: ignore # noqa rx_chan: trio.MemoryReceiveChannel, - _broadcaster: BroadcastReceiver | None = None, + _broadcaster: BroadcastReceiver|None = None, ) -> None: self._ctx = ctx @@ -85,122 +95,245 @@ 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): - msg = self._rx_chan.receive_nowait() - return msg['yield'] - - async def receive(self): - '''Async receive a single msg from the IPC transport, the next - in sequence for this stream. + @property + def ctx(self) -> Context: + ''' + This stream's IPC `Context` ref. ''' + return self._ctx + + @property + def chan(self) -> Channel: + ''' + Ref to the containing `Context`'s transport `Channel`. + + ''' + return self._ctx.chan + + # TODO: could we make this a direct method bind to `PldRx`? + # -> receive_nowait = PldRx.recv_pld + # |_ means latter would have to accept `MsgStream`-as-`self`? + # => should be fine as long as, + # -[ ] both define `._rx_chan` + # -[ ] .ctx is bound into `PldRx` using a `@cm`? + # + # delegate directly to underlying mem channel + def receive_nowait( + self, + expect_msg: MsgType = Yield, + ): + ctx: Context = self._ctx + return ctx._pld_rx.recv_pld_nowait( + ipc=self, + expect_msg=expect_msg, + ) + + async def receive( + self, + + hide_tb: bool = False, + ): + ''' + 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): + # + # 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: - raise trio.EndOfChannel + raise self._eoc if self._closed: - raise trio.ClosedResourceError('This stream was closed') + raise self._closed + src_err: Exception|None = None # orig tb try: - 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 + ctx: Context = self._ctx + return await ctx._pld_rx.recv_pld(ipc=self) + # XXX: the stream terminates on either of: + # - via `self._rx_chan.receive()` raising after manual closure + # by the rpc-runtime OR, + # - via a received `{'stop': ...}` msg from remote side. + # |_ NOTE: previously this was triggered by calling + # ``._rx_chan.aclose()`` on the send side of the channel inside + # `Actor._deliver_ctx_payload()`, but now the 'stop' message handling + # has been put just above inside `_raise_from_no_key_in_msg()`. except ( - trio.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: - - # 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. + trio.EndOfChannel, + ) as eoc: + src_err = eoc + self._eoc = eoc # 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. + # 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 + # 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?' + ) + 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: + # from .devx import pause + # await pause() + 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 + ctx.maybe_raise( + raise_ctxc_from_self_call=True, + from_src_exc=src_err, + ) + + # 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 + + async def aclose(self) -> list[Exception|dict]: ''' 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: - log.cancel(f"{self} is already closed") + # 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 + return [] - self._eoc = True + ctx: Context = self._ctx + drained: list[Exception|dict] = [] + while not drained: + try: + maybe_final_msg = self.receive_nowait( + # allow_msgs=[Yield, Return], + expect_msg=Yield, + ) + if maybe_final_msg: + log.debug( + 'Drained un-processed stream msg:\n' + f'{pformat(maybe_final_msg)}' + ) + # 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) + 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' + f'{pformat(ctxc.msgdata)}' + ) + break # NOTE: this is super subtle IPC messaging stuff: # Relay stop iteration to far end **iff** we're @@ -231,26 +364,39 @@ 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`` ??!? + if not self._eoc: + message: str = ( + f'Stream self-closed by {self._ctx.side!r}-side before EoC\n' + f'|_{self}\n' + ) + log.cancel(message) + self._eoc = trio.EndOfChannel(message) - # 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() @@ -279,6 +425,26 @@ 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 + + @property + def closed(self) -> bool: + + 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' + 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( @@ -308,6 +474,9 @@ class MsgStream(trio.abc.Channel): self, # use memory channel size by default self._rx_chan._state.max_buffer_size, # type: ignore + + # TODO: can remove this kwarg right since + # by default behaviour is to do this anyway? receive_afunc=self.receive, ) @@ -334,19 +503,260 @@ 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 + + # raise any alreay known error immediately + self._ctx.maybe_raise() + if self._eoc: + raise self._eoc if self._closed: - raise trio.ClosedResourceError('This stream was already closed') + raise self._closed + + try: + await self._ctx.chan.send( + payload=Yield( + cid=self._ctx.cid, + pld=data, + ), + ) + except ( + trio.ClosedResourceError, + trio.BrokenResourceError, + BrokenPipeError, + ) as trans_err: + if hide_tb: + raise type(trans_err)( + *trans_err.args + ) from trans_err + else: + raise + + # TODO: msg capability context api1 + # @acm + # async def enable_msg_caps( + # self, + # msg_subtypes: Union[ + # list[list[Struct]], + # Protocol, # hypothetical type that wraps a msg set + # ], + # ) -> tuple[Callable, Callable]: # payload enc, dec pair + # ... + + +@acm +async def open_stream_from_ctx( + ctx: Context, + allow_overruns: bool|None = False, + msg_buffer_size: int|None = None, + +) -> AsyncGenerator[MsgStream, None]: + ''' + Open a `MsgStream`, a bi-directional msg transport dialog + connected to the cross-actor peer task for an IPC `Context`. + + This context manager must be entered in both the "parent" (task + which entered `Portal.open_context()`) and "child" (RPC task + which is decorated by `@context`) tasks for the stream to + logically be considered "open"; if one side begins sending to an + un-opened peer, depending on policy config, msgs will either be + queued until the other side opens and/or a `StreamOverrun` will + (eventually) be raised. + + ------ - ------ + + Runtime semantics design: + + A `MsgStream` session adheres to "one-shot use" semantics, + meaning if you close the scope it **can not** be "re-opened". + + Instead you must re-establish a new surrounding RPC `Context` + (RTC: remote task context?) using `Portal.open_context()`. + + In the future this *design choice* may need to be changed but + currently there seems to be no obvious reason to support such + semantics.. + + - "pausing a stream" can be supported with a message implemented + by the `tractor` application dev. + + - any remote error will normally require a restart of the entire + `trio.Task`'s scope due to the nature of `trio`'s cancellation + (`CancelScope`) system and semantics (level triggered). + + ''' + actor: Actor = ctx._actor + + # 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 ctx._cancel_called: + + # XXX NOTE: ALWAYS RAISE any remote error here even if + # 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! + ctx.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 ctx._remote_error: + # raise ctx._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'{ctx}' + ) + + if ( + not ctx._portal + and not ctx._started_called + ): + raise RuntimeError( + 'Context.started()` must be called before opening a stream' + ) + + # NOTE: in one way streaming this only happens on the + # parent-ctx-task side (on the side that calls + # `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: Context = actor.get_context( + chan=ctx.chan, + cid=ctx.cid, + nsf=ctx._nsf, + # side=ctx.side, + + msg_buffer_size=msg_buffer_size, + allow_overruns=allow_overruns, + ) + ctx._allow_overruns: bool = allow_overruns + assert ctx is ctx + + # 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._rx_chan._closed: + raise trio.ClosedResourceError( + 'The underlying channel for this stream was already closed!\n' + ) + + # 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=ctx, + rx_chan=ctx._rx_chan, + ) as stream: + + # NOTE: we track all existing streams per portal for + # the purposes of attempting graceful closes on runtime + # cancel requests. + if ctx._portal: + ctx._portal._streams.add(stream) + + try: + ctx._stream_opened: bool = True + ctx._stream = stream + + # XXX: do we need this? + # ensure we aren't cancelled before yielding the stream + # 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 + # ``Actor._deliver_ctx_payload()`` 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() + + # 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 + + log.warning( + 'Stream was terminated by EoC\n\n' + # NOTE: won't show the error but + # does show txt followed by IPC msg. + f'{str(eoc)}\n' + ) + + finally: + if ctx._portal: + try: + ctx._portal._streams.remove(stream) + except KeyError: + log.warning( + f'Stream was already destroyed?\n' + f'actor: {ctx.chan.uid}\n' + f'ctx id: {ctx.cid}' + ) - await self._ctx.chan.send({'yield': data, 'cid': self._ctx.cid}) def stream(func: Callable) -> Callable: @@ -356,7 +766,7 @@ def stream(func: Callable) -> Callable: ''' # 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 + func._tractor_stream_function: bool = True # type: ignore sig = inspect.signature(func) params = sig.parameters diff --git a/tractor/_supervise.py b/tractor/_supervise.py index 7f77784b..8f3574bb 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -21,22 +21,22 @@ from contextlib import asynccontextmanager as acm from functools import partial import inspect -from typing import ( - Optional, - TYPE_CHECKING, -) +from pprint import pformat +from typing import TYPE_CHECKING import typing import warnings -from exceptiongroup import BaseExceptionGroup import trio -from ._debug import maybe_wait_for_debugger +from .devx._debug import maybe_wait_for_debugger 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 @@ -84,6 +84,7 @@ class ActorNursery: ria_nursery: trio.Nursery, da_nursery: trio.Nursery, errors: dict[tuple[str, str], BaseException], + ) -> None: # self.supervisor = supervisor # TODO self._actor: Actor = actor @@ -94,7 +95,7 @@ class ActorNursery: tuple[ Actor, trio.Process | mp.Process, - Optional[Portal], + Portal | None, ] ] = {} # portals spawned with ``run_in_actor()`` are @@ -105,25 +106,42 @@ class ActorNursery: self._at_least_one_child_in_debug: bool = False self.errors = errors self.exited = trio.Event() + self._scope_error: BaseException|None = None + + # 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, + *, - bind_addr: tuple[str, int] = _default_bind_addr, - rpc_module_paths: list[str] | None = None, - enable_modules: list[str] | None = None, - loglevel: str | None = None, # set log level per subactor - nursery: trio.Nursery | None = None, - debug_mode: Optional[bool] | None = None, + + bind_addrs: list[tuple[str, int]] = [_default_bind_addr], + rpc_module_paths: list[str]|None = None, + enable_modules: list[str]|None = None, + loglevel: str|None = None, # set log level per subactor + nursery: trio.Nursery|None = None, + debug_mode: bool|None = None, infect_asyncio: bool = False, + ) -> Portal: ''' Start a (daemon) actor: an process that has no designated "main task" besides the runtime. ''' - loglevel = loglevel or self._actor.loglevel or get_loglevel() + __runtimeframe__: int = 1 # noqa + loglevel: str = ( + loglevel + or self._actor.loglevel + or get_loglevel() + ) # configure and pass runtime state _rtv = _state._runtime_vars.copy() @@ -150,14 +168,16 @@ class ActorNursery: # modules allowed to invoked funcs from enable_modules=enable_modules, loglevel=loglevel, - arbiter_addr=current_actor()._arb_addr, + + # verbatim relay this actor's registrar addresses + registry_addrs=current_actor().reg_addrs, ) parent_addr = self._actor.accept_addr assert parent_addr # 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 @@ -167,21 +187,28 @@ class ActorNursery: self, subactor, self.errors, - bind_addr, + bind_addrs, parent_addr, _rtv, # run time vars infect_asyncio=infect_asyncio, ) ) + # TODO: DEPRECATE THIS: + # -[ ] impl instead as a hilevel wrapper on + # top of a `@context` style invocation. + # |_ dynamic @context decoration on child side + # |_ implicit `Portal.open_context() as (ctx, first):` + # and `return first` on parent side. + # -[ ] use @api_frame on the wrapper async def run_in_actor( self, fn: typing.Callable, *, - name: Optional[str] = None, - bind_addr: tuple[str, int] = _default_bind_addr, + name: str | None = None, + bind_addrs: tuple[str, int] = [_default_bind_addr], rpc_module_paths: list[str] | None = None, enable_modules: list[str] | None = None, loglevel: str | None = None, # set log level per subactor @@ -190,25 +217,28 @@ 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__ + + ''' + __runtimeframe__: int = 1 # noqa + mod_path: str = fn.__module__ if name is None: # use the explicit function name if not provided name = fn.__name__ - portal = await self.start_actor( + portal: Portal = await self.start_actor( name, enable_modules=[mod_path] + ( enable_modules or rpc_module_paths or [] ), - bind_addr=bind_addr, + bind_addrs=bind_addrs, loglevel=loglevel, # use the run_in_actor nursery nursery=self._ria_nursery, @@ -232,21 +262,39 @@ class ActorNursery: ) return portal - async def cancel(self, hard_kill: bool = False) -> None: - """Cancel this nursery by instructing each subactor to cancel + # @api_frame + 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. If ``hard_killl`` is set to ``True`` then kill the processes directly without any far end graceful ``trio`` cancellation. - """ + + ''' + __runtimeframe__: int = 1 # noqa self.cancelled = True - log.cancel(f"Cancelling nursery in {self._actor.uid}") + # 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: - 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 children.values(): # TODO: are we ever even going to use this or # is the spawning backend responsible for such @@ -258,12 +306,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 @@ -280,14 +329,24 @@ 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) + 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!") - for subactor, proc, portal in self._children.values(): + f'Failed to cancel {self}?\n' + 'Hard killing underlying subprocess tree!\n' + ) + subactor: Actor + proc: trio.Process + portal: Portal + for ( + subactor, + proc, + portal, + ) in children.values(): log.warning(f"Hard killing process {proc}") proc.terminate() @@ -301,8 +360,11 @@ async def _open_and_supervise_one_cancels_all_nursery( ) -> typing.AsyncGenerator[ActorNursery, None]: - # TODO: yay or nay? - __tracebackhide__ = True + # normally don't need to show user by default + __tracebackhide__: bool = True + + outer_err: BaseException|None = None + inner_err: BaseException|None = None # the collection of errors retreived from spawned sub-actors errors: dict[tuple[str, str], BaseException] = {} @@ -312,7 +374,7 @@ async def _open_and_supervise_one_cancels_all_nursery( # handling errors that are generated by the inner nursery in # a supervisor strategy **before** blocking indefinitely to wait for # actors spawned in "daemon mode" (aka started using - # ``ActorNursery.start_actor()``). + # `ActorNursery.start_actor()`). # errors from this daemon actor nursery bubble up to caller async with trio.open_nursery() as da_nursery: @@ -327,7 +389,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, @@ -336,18 +398,19 @@ 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: + except BaseException as _inner_err: + inner_err = _inner_err errors[actor.uid] = inner_err # If we error in the root but the debugger is @@ -357,37 +420,60 @@ 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 + # 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} " - f"errored with") + 'Nursery 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() + await an.cancel() # ria_nursery scope end @@ -402,24 +488,30 @@ async def _open_and_supervise_one_cancels_all_nursery( Exception, BaseExceptionGroup, trio.Cancelled + ) as _outer_err: + outer_err = _outer_err - ) as err: + an._scope_error = outer_err or inner_err # 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 # ".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 anursery._children: + if an._children: + log.cancel( + 'Actor-nursery cancelling due error type:\n' + f'{outer_err}\n' + ) with trio.CancelScope(shield=True): - await anursery.cancel() + await an.cancel() raise + finally: # No errors were raised while awaiting ".run_in_actor()" # actors but those actors may have returned remote errors as @@ -428,9 +520,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: @@ -441,11 +533,19 @@ async def _open_and_supervise_one_cancels_all_nursery( else: raise list(errors.values())[0] + # show frame on any (likely) internal error + if ( + not an.cancelled + and an._scope_error + ): + __tracebackhide__: bool = False + # da_nursery scope end - nursery checkpoint # final exit @acm +# @api_frame async def open_nursery( **kwargs, @@ -465,19 +565,21 @@ async def open_nursery( which cancellation scopes correspond to each spawned subactor set. ''' - implicit_runtime = False - - actor = current_actor(err_on_no_runtime=False) - + __tracebackhide__: bool = True + implicit_runtime: bool = False + actor: Actor = current_actor(err_on_no_runtime=False) + an: ActorNursery|None = None 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!") # 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() @@ -485,24 +587,50 @@ async def open_nursery( try: async with _open_and_supervise_one_cancels_all_nursery( actor - ) as anursery: - yield anursery + ) 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: - anursery.exited.set() + # 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 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") + # show frame on any internal runtime-scope error + if ( + an + and not an.cancelled + and an._scope_error + ): + __tracebackhide__: bool = False + + msg: str = ( + 'Actor-nursery exited\n' + f'|_{an}\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) diff --git a/tractor/_testing/__init__.py b/tractor/_testing/__init__.py new file mode 100644 index 00000000..fd79fe20 --- /dev/null +++ b/tractor/_testing/__init__.py @@ -0,0 +1,77 @@ +# 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 +) +from .fault_simulation import ( + break_ipc as break_ipc, +) + + +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/fault_simulation.py b/tractor/_testing/fault_simulation.py new file mode 100644 index 00000000..fbd97bf5 --- /dev/null +++ b/tractor/_testing/fault_simulation.py @@ -0,0 +1,92 @@ +# 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 tractor import ( + MsgStream, +) + +async def break_ipc( + stream: MsgStream, + method: str|None = None, + pre_close: bool = False, + + def_method: str = 'socket_close', + +) -> 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 + print( + '#################################\n' + 'Simulating CHILD-side IPC BREAK!\n' + f'method: {method}\n' + f'pre `.aclose()`: {pre_close}\n' + '#################################\n' + ) + + match method: + case 'socket_close': + await stream._ctx.chan.transport.stream.aclose() + + case 'socket_eof': + # NOTE: `trio` does the following underneath this + # call in `src/trio/_highlevel_socket.py`: + # `Stream.socket.shutdown(tsocket.SHUT_WR)` + await stream._ctx.chan.transport.stream.send_eof() + + # TODO: remove since now this will be invalid with our + # new typed msg spec? + # 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}' + ) 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 diff --git a/tractor/devx/__init__.py b/tractor/devx/__init__.py new file mode 100644 index 00000000..ab9d2d1a --- /dev/null +++ b/tractor/devx/__init__.py @@ -0,0 +1,42 @@ +# 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 . + +""" +Runtime "developer experience" utils and addons to aid our +(advanced) users and core devs in building distributed applications +and working with/on the actor runtime. + +""" +from ._debug import ( + maybe_wait_for_debugger as maybe_wait_for_debugger, + acquire_debug_lock as acquire_debug_lock, + breakpoint as breakpoint, + pause as pause, + pause_from_sync as pause_from_sync, + shield_sigint_handler as shield_sigint_handler, + open_crash_handler as open_crash_handler, + maybe_open_crash_handler as maybe_open_crash_handler, + post_mortem as post_mortem, + mk_pdb as mk_pdb, +) +from ._stackscope import ( + enable_stack_on_sig as enable_stack_on_sig, +) +from .pformat import ( + add_div as add_div, + pformat_caller_frame as pformat_caller_frame, + pformat_boxed_tb as pformat_boxed_tb, +) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py new file mode 100644 index 00000000..ccf57d62 --- /dev/null +++ b/tractor/devx/_debug.py @@ -0,0 +1,2830 @@ +# 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 +# . + +""" +Multi-core debugging for da peeps! + +""" +from __future__ import annotations +import bdb +from contextlib import ( + asynccontextmanager as acm, + contextmanager as cm, + nullcontext, + _GeneratorContextManager, + _AsyncGeneratorContextManager, +) +from functools import ( + partial, + cached_property, +) +import inspect +import os +import signal +import sys +import textwrap +import threading +import traceback +from typing import ( + Any, + Callable, + AsyncIterator, + AsyncGenerator, + TypeAlias, + TYPE_CHECKING, +) +from types import ( + FunctionType, + FrameType, + ModuleType, + TracebackType, + CodeType, +) + +from msgspec import Struct +import pdbp +import sniffio +import trio +from trio import CancelScope +from trio.lowlevel import ( + current_task, +) +from trio import ( + TaskStatus, +) +import tractor +from tractor.log import get_logger +from tractor._context import Context +from tractor._state import ( + current_actor, + is_root_process, + debug_mode, + current_ipc_ctx, +) +# from .pformat import ( +# pformat_caller_frame, +# pformat_cs, +# ) + +if TYPE_CHECKING: + from trio.lowlevel import Task + from threading import Thread + from tractor._ipc import Channel + from tractor._runtime import ( + Actor, + ) + from tractor.msg import ( + _codec, + ) + +log = get_logger(__name__) + +# TODO: refine the internal impl and APIs in this module! +# +# -[ ] rework `._pause()` and it's branch-cases for root vs. +# subactor: +# -[ ] `._pause_from_root()` + `_pause_from_subactor()`? +# -[ ] do the de-factor based on bg-thread usage in +# `.pause_from_sync()` & `_pause_from_bg_root_thread()`. +# -[ ] drop `debug_func == None` case which is confusing af.. +# -[ ] factor out `_enter_repl_sync()` into a util func for calling +# the `_set_trace()` / `_post_mortem()` APIs? +# +# -[ ] figure out if we need `acquire_debug_lock()` and/or re-implement +# it as part of the `.pause_from_sync()` rework per above? +# +# -[ ] pair the `._pause_from_subactor()` impl with a "debug nursery" +# that's dynamically allocated inside the `._rpc` task thus +# avoiding the `._service_n.start()` usage for the IPC request? +# -[ ] see the TODO inside `._rpc._errors_relayed_via_ipc()` +# +# -[ ] impl a `open_debug_request()` which encaps all +# `request_root_stdio_lock()` task scheduling deats +# + `DebugStatus` state mgmt; which should prolly be re-branded as +# a `DebugRequest` type anyway AND with suppoort for bg-thread +# (from root actor) usage? +# +# -[ ] handle the `xonsh` case for bg-root-threads in the SIGINT +# handler! +# -[ ] do we need to do the same for subactors? +# -[ ] make the failing tests finally pass XD +# +# -[ ] simplify `maybe_wait_for_debugger()` to be a root-task only +# API? +# -[ ] currently it's implemented as that so might as well make it +# formal? + + +def hide_runtime_frames() -> dict[FunctionType, CodeType]: + ''' + Hide call-stack frames for various std-lib and `trio`-API primitives + such that the tracebacks presented from our runtime are as minimized + as possible, particularly from inside a `PdbREPL`. + + ''' + # XXX HACKZONE XXX + # hide exit stack frames on nurseries and cancel-scopes! + # |_ so avoid seeing it when the `pdbp` REPL is first engaged from + # inside a `trio.open_nursery()` scope (with no line after it + # in before the block end??). + # + # TODO: FINALLY got this workin originally with + # `@pdbp.hideframe` around the `wrapper()` def embedded inside + # `_ki_protection_decoratior()`.. which is in the module: + # /home/goodboy/.virtualenvs/tractor311/lib/python3.11/site-packages/trio/_core/_ki.py + # + # -[ ] make an issue and patch for `trio` core? maybe linked + # to the long outstanding `pdb` one below? + # |_ it's funny that there's frame hiding throughout `._run.py` + # but not where it matters on the below exit funcs.. + # + # -[ ] provide a patchset for the lonstanding + # |_ https://github.com/python-trio/trio/issues/1155 + # + # -[ ] make a linked issue to ^ and propose allowing all the + # `._core._run` code to have their `__tracebackhide__` value + # configurable by a `RunVar` to allow getting scheduler frames + # if desired through configuration? + # + # -[ ] maybe dig into the core `pdb` issue why the extra frame is shown + # at all? + # + funcs: list[FunctionType] = [ + trio._core._run.NurseryManager.__aexit__, + trio._core._run.CancelScope.__exit__, + _GeneratorContextManager.__exit__, + _AsyncGeneratorContextManager.__aexit__, + _AsyncGeneratorContextManager.__aenter__, + trio.Event.wait, + ] + func_list_str: str = textwrap.indent( + "\n".join(f.__qualname__ for f in funcs), + prefix=' |_ ', + ) + log.devx( + 'Hiding the following runtime frames by default:\n' + f'{func_list_str}\n' + ) + + codes: dict[FunctionType, CodeType] = {} + for ref in funcs: + # stash a pre-modified version of each ref's code-obj + # so it can be reverted later if needed. + codes[ref] = ref.__code__ + pdbp.hideframe(ref) + # + # pdbp.hideframe(trio._core._run.NurseryManager.__aexit__) + # pdbp.hideframe(trio._core._run.CancelScope.__exit__) + # pdbp.hideframe(_GeneratorContextManager.__exit__) + # pdbp.hideframe(_AsyncGeneratorContextManager.__aexit__) + # pdbp.hideframe(_AsyncGeneratorContextManager.__aenter__) + # pdbp.hideframe(trio.Event.wait) + return codes + + +class LockStatus( + Struct, + tag=True, + tag_field='msg_type', +): + subactor_uid: tuple[str, str] + cid: str + locked: bool + + +class LockRelease( + Struct, + tag=True, + tag_field='msg_type', +): + subactor_uid: tuple[str, str] + cid: str + + +__pld_spec__: TypeAlias = LockStatus|LockRelease + + +# TODO: instantiate this only in root from factory +# so as to allow runtime errors from subactors. +class Lock: + ''' + Actor-tree-global debug lock state, exists only in a root process. + + Mostly to avoid a lot of global declarations for now XD. + + ''' + @staticmethod + def get_locking_task_cs() -> CancelScope|None: + if not is_root_process(): + raise RuntimeError( + '`Lock.locking_task_cs` is invalid in subactors!' + ) + + if ctx := Lock.ctx_in_debug: + return ctx._scope + + return None + + # TODO: once we convert to singleton-per-actor-style + # @property + # def stats(cls) -> trio.LockStatistics: + # return cls._debug_lock.statistics() + + # @property + # def owner(cls) -> Task: + # return cls._debug_lock.statistics().owner + + # ROOT ONLY + # ------ - ------- + # the root-actor-ONLY singletons for, + # + # - the uid of the actor who's task is using a REPL + # - a literal task-lock, + # - a shielded-cancel-scope around the acquiring task*, + # - a broadcast event to signal no-actor using a REPL in tree, + # - a filter list to block subs-by-uid from locking. + # + # * in case it needs to be manually cancelled in root due to + # a stale lock condition (eg. IPC failure with the locking + # child + ctx_in_debug: Context|None = None + req_handler_finished: trio.Event|None = None + + _owned_by_root: bool = False + _debug_lock: trio.StrictFIFOLock = trio.StrictFIFOLock() + _blocked: set[ + tuple[str, str] # `Actor.uid` for per actor + |str # Context.cid for per task + ] = set() + + @classmethod + def repr(cls) -> str: + lock_stats: trio.LockStatistics = cls._debug_lock.statistics() + req: trio.Event|None = cls.req_handler_finished + fields: str = ( + f'|_ ._blocked: {cls._blocked}\n' + f'|_ ._debug_lock: {cls._debug_lock}\n' + f' {lock_stats}\n\n' + + f'|_ .ctx_in_debug: {cls.ctx_in_debug}\n' + f'|_ .req_handler_finished: {req}\n' + ) + if req: + req_stats: trio.EventStatistics = req.statistics() + fields += f' {req_stats}\n' + + body: str = textwrap.indent( + fields, + prefix=' ', + ) + return ( + f'<{cls.__name__}(\n' + f'{body}' + ')>\n\n' + ) + + @classmethod + @pdbp.hideframe + def release( + cls, + force: bool = False, + raise_on_thread: bool = True, + + ) -> bool: + ''' + Release the actor-tree global TTY stdio lock (only) from the + `trio.run()`-main-thread. + + ''' + we_released: bool = False + ctx_in_debug: Context|None = cls.ctx_in_debug + repl_task: Task|Thread|None = DebugStatus.repl_task + if not DebugStatus.is_main_trio_thread(): + thread: threading.Thread = threading.current_thread() + message: str = ( + '`Lock.release()` can not be called from a non-main-`trio` thread!\n' + f'{thread}\n' + ) + if raise_on_thread: + raise RuntimeError(message) + + log.devx(message) + return False + + task: Task = current_task() + + # sanity check that if we're the root actor + # the lock is marked as such. + # note the pre-release value may be diff the the + # post-release task. + if repl_task is task: + assert cls._owned_by_root + message: str = ( + 'TTY lock held by root-actor on behalf of local task\n' + f'|_{repl_task}\n' + ) + else: + assert DebugStatus.repl_task is not task + + message: str = ( + 'TTY lock was NOT released on behalf of caller\n' + f'|_{task}\n' + ) + + try: + lock: trio.StrictFIFOLock = cls._debug_lock + owner: Task = lock.statistics().owner + if ( + (lock.locked() or force) + # ^-TODO-NOTE-^ should we just remove this, since the + # RTE case above will always happen when you force + # from the wrong task? + + and (owner is task) + # ^-NOTE-^ if we do NOT ensure this, `trio` will + # raise a RTE when a non-owner tries to releasee the + # lock. + # + # Further we need to be extra pedantic about the + # correct task, greenback-spawned-task and/or thread + # being set to the `.repl_task` such that the above + # condition matches and we actually release the lock. + # This is particular of note from `.pause_from_sync()`! + + ): + cls._debug_lock.release() + we_released: bool = True + if repl_task: + message: str = ( + 'Lock released on behalf of root-actor-local REPL owner\n' + f'|_{repl_task}\n' + ) + else: + message: str = ( + 'TTY lock released by us on behalf of remote peer?\n' + f'|_ctx_in_debug: {ctx_in_debug}\n\n' + ) + # mk_pdb().set_trace() + # elif owner: + + except RuntimeError as rte: + log.exception( + 'Failed to release `Lock._debug_lock: trio.FIFOLock`?\n' + ) + raise rte + + finally: + # IFF there are no more requesting tasks queued up fire, the + # "tty-unlocked" event thereby alerting any monitors of the lock that + # we are now back in the "tty unlocked" state. This is basically + # and edge triggered signal around an empty queue of sub-actor + # tasks that may have tried to acquire the lock. + lock_stats: trio.LockStatistics = cls._debug_lock.statistics() + req_handler_finished: trio.Event|None = Lock.req_handler_finished + if ( + not lock_stats.owner + and req_handler_finished is None + ): + message += ( + '-> No new task holds the TTY lock!\n\n' + f'{Lock.repr()}\n' + ) + + elif ( + req_handler_finished # new IPC ctx debug request active + and + lock.locked() # someone has the lock + ): + behalf_of_task = ( + ctx_in_debug + or + repl_task + ) + message += ( + f'\nA non-caller task still owns this lock on behalf of ' + f'{behalf_of_task}\n' + f'|_{lock_stats.owner}\n' + ) + + if ( + we_released + and + ctx_in_debug + ): + cls.ctx_in_debug = None # unset + + # post-release value (should be diff then value above!) + repl_task: Task|Thread|None = DebugStatus.repl_task + if ( + cls._owned_by_root + and + we_released + ): + cls._owned_by_root = False + + if task is not repl_task: + message += ( + 'Lock released by root actor on behalf of bg thread\n' + f'|_{repl_task}\n' + ) + + log.devx(message) + + return we_released + + @classmethod + @acm + async def acquire_for_ctx( + cls, + ctx: Context, + + ) -> AsyncIterator[trio.StrictFIFOLock]: + ''' + Acquire a root-actor local FIFO lock which tracks mutex access of + the process tree's global debugger breakpoint. + + This lock avoids tty clobbering (by preventing multiple processes + reading from stdstreams) and ensures multi-actor, sequential access + to the ``pdb`` repl. + + ''' + if not is_root_process(): + raise RuntimeError('Only callable by a root actor task!') + + # subactor_uid: tuple[str, str] = ctx.chan.uid + we_acquired: bool = False + log.runtime( + f'Attempting to acquire TTY lock for sub-actor\n' + f'{ctx}' + ) + try: + pre_msg: str = ( + f'Entering lock checkpoint for sub-actor\n' + f'{ctx}' + ) + stats = cls._debug_lock.statistics() + if owner := stats.owner: + pre_msg += ( + f'\n' + f'`Lock` already held by local task?\n' + f'{owner}\n\n' + # f'On behalf of task: {cls.remote_task_in_debug!r}\n' + f'On behalf of IPC ctx\n' + f'{ctx}' + ) + log.runtime(pre_msg) + + # NOTE: if the surrounding cancel scope from the + # `lock_stdio_for_peer()` caller is cancelled, this line should + # unblock and NOT leave us in some kind of + # a "child-locked-TTY-but-child-is-uncontactable-over-IPC" + # condition. + await cls._debug_lock.acquire() + cls.ctx_in_debug = ctx + we_acquired = True + + log.runtime( + f'TTY lock acquired for sub-actor\n' + f'{ctx}' + ) + + # NOTE: critical section: this yield is unshielded! + # + # IF we received a cancel during the shielded lock entry of some + # next-in-queue requesting task, then the resumption here will + # result in that ``trio.Cancelled`` being raised to our caller + # (likely from `lock_stdio_for_peer()` below)! In + # this case the ``finally:`` below should trigger and the + # surrounding caller side context should cancel normally + # relaying back to the caller. + + yield cls._debug_lock + + finally: + message :str = 'Exiting `Lock.acquire_for_ctx()` on behalf of sub-actor\n' + if we_acquired: + cls.release() + message += '-> TTY lock released by child\n' + + else: + message += '-> TTY lock never acquired by child??\n' + + log.runtime( + f'{message}\n' + f'{ctx}' + ) + + +@tractor.context( + # enable the locking msgspec + pld_spec=__pld_spec__, +) +async def lock_stdio_for_peer( + ctx: Context, + subactor_task_uid: tuple[str, int], + +) -> LockStatus|LockRelease: + ''' + Lock the TTY in the root process of an actor tree in a new + inter-actor-context-task such that the ``pdbp`` debugger console + can be mutex-allocated to the calling sub-actor for REPL control + without interference by other processes / threads. + + NOTE: this task must be invoked in the root process of the actor + tree. It is meant to be invoked as an rpc-task and should be + highly reliable at releasing the mutex complete! + + ''' + subactor_uid: tuple[str, str] = ctx.chan.uid + + # mark the tty lock as being in use so that the runtime + # can try to avoid clobbering any connection from a child + # that's currently relying on it. + we_finished = Lock.req_handler_finished = trio.Event() + try: + if ctx.cid in Lock._blocked: + raise RuntimeError( + f'Double lock request!?\n' + f'The same remote task already has an active request for TTY lock ??\n\n' + f'subactor uid: {subactor_uid}\n\n' + + 'This might be mean that the requesting task ' + 'in `request_root_stdio_lock()` may have crashed?\n' + 'Consider that an internal bug exists given the TTY ' + '`Lock`ing IPC dialog..\n' + ) + + root_task_name: str = current_task().name + if tuple(subactor_uid) in Lock._blocked: + log.warning( + f'Subactor is blocked from acquiring debug lock..\n' + f'subactor_uid: {subactor_uid}\n' + f'remote task: {subactor_task_uid}\n' + ) + ctx._enter_debugger_on_cancel: bool = False + message: str = ( + f'Debug lock blocked for {subactor_uid}\n' + 'Cancelling debug request!\n' + ) + log.cancel(message) + await ctx.cancel() + raise DebugRequestError(message) + + log.devx( + 'Subactor attempting to acquire TTY lock\n' + f'root task: {root_task_name}\n' + f'subactor_uid: {subactor_uid}\n' + f'remote task: {subactor_task_uid}\n' + ) + DebugStatus.shield_sigint() + Lock._blocked.add(ctx.cid) + + # NOTE: we use the IPC ctx's cancel scope directly in order to + # ensure that on any transport failure, or cancellation request + # from the child we expect + # `Context._maybe_cancel_and_set_remote_error()` to cancel this + # scope despite the shielding we apply below. + debug_lock_cs: CancelScope = ctx._scope + + async with Lock.acquire_for_ctx(ctx=ctx): + debug_lock_cs.shield = True + + log.devx( + 'Subactor acquired debugger request lock!\n' + f'root task: {root_task_name}\n' + f'subactor_uid: {subactor_uid}\n' + f'remote task: {subactor_task_uid}\n\n' + + 'Sending `ctx.started(LockStatus)`..\n' + + ) + + # indicate to child that we've locked stdio + await ctx.started( + LockStatus( + subactor_uid=subactor_uid, + cid=ctx.cid, + locked=True, + ) + ) + + log.devx( + f'Actor {subactor_uid} acquired `Lock` via debugger request' + ) + + # wait for unlock pdb by child + async with ctx.open_stream() as stream: + release_msg: LockRelease = await stream.receive() + + # TODO: security around only releasing if + # these match? + log.devx( + f'TTY lock released requested\n\n' + f'{release_msg}\n' + ) + assert release_msg.cid == ctx.cid + assert release_msg.subactor_uid == tuple(subactor_uid) + + log.devx( + f'Actor {subactor_uid} released TTY lock' + ) + + return LockStatus( + subactor_uid=subactor_uid, + cid=ctx.cid, + locked=False, + ) + + except BaseException as req_err: + message: str = ( + f'On behalf of remote peer {subactor_task_uid!r}@{ctx.chan.uid!r}\n\n' + 'Forcing `Lock.release()` for req-ctx since likely an ' + 'internal error!\n\n' + f'{ctx}' + ) + if isinstance(req_err, trio.Cancelled): + message = ( + 'Cancelled during root TTY-lock dialog\n' + + + message + ) + else: + message = ( + 'Errored during root TTY-lock dialog\n' + + + message + ) + + log.exception(message) + Lock.release() #force=True) + raise + + finally: + Lock._blocked.remove(ctx.cid) + + # wakeup any waiters since the lock was (presumably) + # released, possibly only temporarily. + we_finished.set() + DebugStatus.unshield_sigint() + + +# TODO: rename to ReplState or somethin? +# DebugRequest, make it a singleton instance? +class DebugStatus: + ''' + Singleton-state for debugging machinery in a subactor. + + Composes conc primitives for syncing with a root actor to + acquire the tree-global (TTY) `Lock` such that only ever one + actor's task can have the REPL active at a given time. + + Methods to shield the process' `SIGINT` handler are used + whenever a local task is an active REPL. + + ''' + # XXX local ref to the `pdbp.Pbp` instance, ONLY set in the + # actor-process that currently has activated a REPL i.e. it + # should be `None` (unset) in any other actor-process that does + # not yet have the `Lock` acquired via a root-actor debugger + # request. + repl: PdbREPL|None = None + + # TODO: yet again this looks like a task outcome where we need + # to sync to the completion of one task (and get its result) + # being used everywhere for syncing.. + # -[ ] see if we can get our proto oco task-mngr to work for + # this? + repl_task: Task|None = None + repl_release: trio.Event|None = None + + req_task: Task|None = None + req_ctx: Context|None = None + req_cs: CancelScope|None = None + req_finished: trio.Event|None = None + req_err: BaseException|None = None + + lock_status: LockStatus|None = None + + _orig_sigint_handler: Callable|None = None + _trio_handler: ( + Callable[[int, FrameType|None], Any] + |int + | None + ) = None + + @classmethod + def repr(cls) -> str: + fields: str = ( + f'repl: {cls.repl}\n' + f'repl_task: {cls.repl_task}\n' + f'repl_release: {cls.repl_release}\n' + f'req_ctx: {cls.req_ctx}\n' + ) + body: str = textwrap.indent( + fields, + prefix=' |_', + ) + return ( + f'<{cls.__name__}(\n' + f'{body}' + ')>' + ) + + # TODO: how do you get this to work on a non-inited class? + # __repr__ = classmethod(repr) + # __str__ = classmethod(repr) + + @classmethod + def shield_sigint(cls): + ''' + Shield out SIGINT handling (which by default triggers + `Task` cancellation) in subactors when a `pdb` REPL + is active. + + Avoids cancellation of the current actor (task) when the user + mistakenly sends ctl-c or via a recevied signal (from an + external request). Explicit runtime cancel requests are + allowed until the current REPL-session (the blocking call + `Pdb.interaction()`) exits, normally via the 'continue' or + 'quit' command - at which point the orig SIGINT handler is + restored via `.unshield_sigint()` below. + + Impl notes: + ----------- + - we prefer that `trio`'s default handler is always used when + SIGINT is unshielded (hence disabling the `pdb.Pdb` + defaults in `mk_pdb()`) such that reliable KBI cancellation + is always enforced. + + - we always detect whether we're running from a non-main + thread, in which case schedule the SIGINT shielding override + to in the main thread as per, + + https://docs.python.org/3/library/signal.html#signals-and-threads + + ''' + # + # XXX detect whether we're running from a non-main thread + # in which case schedule the SIGINT shielding override + # to in the main thread. + # https://docs.python.org/3/library/signal.html#signals-and-threads + if not cls.is_main_trio_thread(): + cls._orig_sigint_handler: Callable = trio.from_thread.run_sync( + signal.signal, + signal.SIGINT, + shield_sigint_handler, + ) + + else: + cls._orig_sigint_handler = signal.signal( + signal.SIGINT, + shield_sigint_handler, + ) + + @classmethod + @pdbp.hideframe # XXX NOTE XXX see below in `.pause_from_sync()` + def unshield_sigint(cls): + ''' + Un-shield SIGINT for REPL-active (su)bactor. + + See details in `.shield_sigint()`. + + ''' + # always restore ``trio``'s sigint handler. see notes below in + # the pdb factory about the nightmare that is that code swapping + # out the handler when the repl activates... + if not cls.is_main_trio_thread(): + trio.from_thread.run_sync( + signal.signal, + signal.SIGINT, + cls._trio_handler, + ) + else: + trio_h: Callable = cls._trio_handler + # XXX should never really happen XXX + if not trio_h: + mk_pdb().set_trace() + + signal.signal( + signal.SIGINT, + cls._trio_handler, + ) + + cls._orig_sigint_handler = None + + @classmethod + def is_main_trio_thread(cls) -> bool: + ''' + Check if we're the "main" thread (as in the first one + started by cpython) AND that it is ALSO the thread that + called `trio.run()` and not some thread spawned with + `trio.to_thread.run_sync()`. + + ''' + is_trio_main = ( + # TODO: since this is private, @oremanj says + # we should just copy the impl for now.. + (is_main_thread := trio._util.is_main_thread()) + and + (async_lib := sniffio.current_async_library()) == 'trio' + ) + if ( + not is_trio_main + and is_main_thread + ): + log.warning( + f'Current async-lib detected by `sniffio`: {async_lib}\n' + ) + return is_trio_main + # XXX apparently unreliable..see ^ + # ( + # threading.current_thread() + # is not threading.main_thread() + # ) + + @classmethod + def cancel(cls) -> bool: + if (req_cs := cls.req_cs): + req_cs.cancel() + return True + + return False + + @classmethod + @pdbp.hideframe + def release( + cls, + cancel_req_task: bool = False, + ): + repl_release: trio.Event = cls.repl_release + try: + # sometimes the task might already be terminated in + # which case this call will raise an RTE? + if ( + repl_release is not None + ): + if cls.is_main_trio_thread(): + repl_release.set() + else: + # XXX NOTE ONLY used for bg root-actor sync + # threads, see `.pause_from_sync()`. + trio.from_thread.run_sync( + repl_release.set + ) + finally: + # if req_ctx := cls.req_ctx: + # req_ctx._scope.cancel() + if cancel_req_task: + cancelled: bool = cls.cancel() + if not cancelled: + log.warning( + 'Failed to cancel request task!?\n' + f'{cls.repl_task}\n' + ) + + # actor-local state, irrelevant for non-root. + cls.repl_task = None + cls.repl = None + + # restore original sigint handler + cls.unshield_sigint() + + +class TractorConfig(pdbp.DefaultConfig): + ''' + Custom `pdbp` config which tries to use the best tradeoff + between pretty and minimal. + + ''' + use_pygments: bool = True + sticky_by_default: bool = False + enable_hidden_frames: bool = True + + # much thanks @mdmintz for the hot tip! + # fixes line spacing issue when resizing terminal B) + truncate_long_lines: bool = False + + # ------ - ------ + # our own custom config vars mostly + # for syncing with the actor tree's singleton + # TTY `Lock`. + + +class PdbREPL(pdbp.Pdb): + ''' + Add teardown hooks and local state describing any + ongoing TTY `Lock` request dialog. + + ''' + # override the pdbp config with our coolio one + # NOTE: this is only loaded when no `~/.pdbrc` exists + # so we should prolly pass it into the .__init__() instead? + # i dunno, see the `DefaultFactory` and `pdb.Pdb` impls. + DefaultConfig = TractorConfig + + status = DebugStatus + + # NOTE: see details in stdlib's `bdb.py` + # def user_exception(self, frame, exc_info): + # ''' + # Called when we stop on an exception. + # ''' + # log.warning( + # 'Exception during REPL sesh\n\n' + # f'{frame}\n\n' + # f'{exc_info}\n\n' + # ) + + # NOTE: this actually hooks but i don't see anyway to detect + # if an error was caught.. this is why currently we just always + # call `DebugStatus.release` inside `_post_mortem()`. + # def preloop(self): + # print('IN PRELOOP') + # super().preloop() + + # TODO: cleaner re-wrapping of all this? + # -[ ] figure out how to disallow recursive .set_trace() entry + # since that'll cause deadlock for us. + # -[ ] maybe a `@cm` to call `super().()`? + # -[ ] look at hooking into the `pp` hook specially with our + # own set of pretty-printers? + # * `.pretty_struct.Struct.pformat()` + # * `.pformat(MsgType.pld)` + # * `.pformat(Error.tb_str)`? + # * .. maybe more? + # + def set_continue(self): + try: + super().set_continue() + finally: + # NOTE: for subactors the stdio lock is released via the + # allocated RPC locker task, so for root we have to do it + # manually. + if ( + is_root_process() + and + Lock._debug_lock.locked() + and + DebugStatus.is_main_trio_thread() + ): + # Lock.release(raise_on_thread=False) + Lock.release() + + # XXX after `Lock.release()` for root local repl usage + DebugStatus.release() + + def set_quit(self): + try: + super().set_quit() + finally: + if ( + is_root_process() + and + Lock._debug_lock.locked() + and + DebugStatus.is_main_trio_thread() + ): + # Lock.release(raise_on_thread=False) + Lock.release() + + # XXX after `Lock.release()` for root local repl usage + DebugStatus.release() + + # XXX NOTE: we only override this because apparently the stdlib pdb + # bois likes to touch the SIGINT handler as much as i like to touch + # my d$%&. + def _cmdloop(self): + self.cmdloop() + + @cached_property + def shname(self) -> str | None: + ''' + Attempt to return the login shell name with a special check for + the infamous `xonsh` since it seems to have some issues much + different from std shells when it comes to flushing the prompt? + + ''' + # SUPER HACKY and only really works if `xonsh` is not used + # before spawning further sub-shells.. + shpath = os.getenv('SHELL', None) + + if shpath: + if ( + os.getenv('XONSH_LOGIN', default=False) + or 'xonsh' in shpath + ): + return 'xonsh' + + return os.path.basename(shpath) + + return None + + +async def request_root_stdio_lock( + actor_uid: tuple[str, str], + task_uid: tuple[str, int], + + shield: bool = False, + task_status: TaskStatus[CancelScope] = trio.TASK_STATUS_IGNORED, +): + ''' + Connect to the root actor for this actor's process tree and + RPC-invoke a task which acquires the std-streams global `Lock`: + a process-tree-global mutex which prevents multiple actors from + entering `PdbREPL.interaction()` at the same time such that the + parent TTY's stdio is never "clobbered" by simultaneous + reads/writes. + + The actual `Lock` singleton instance exists ONLY in the root + actor's memory space and does nothing more then manage + process-tree global state, + namely a `._debug_lock: trio.FIFOLock`. + + The actual `PdbREPL` interaction/operation is completely isolated + to each sub-actor (process) with the root's `Lock` providing the + multi-process mutex-syncing mechanism to avoid parallel REPL + usage within an actor tree. + + ''' + log.devx( + 'Initing stdio-lock request task with root actor' + ) + # TODO: likely we can implement this mutex more generally as + # a `._sync.Lock`? + # -[ ] simply add the wrapping needed for the debugger specifics? + # - the `__pld_spec__` impl and maybe better APIs for the client + # vs. server side state tracking? (`Lock` + `DebugStatus`) + # -[ ] for eg. `mp` has a multi-proc lock via the manager + # - https://docs.python.org/3.8/library/multiprocessing.html#synchronization-primitives + # -[ ] technically we need a `RLock` since re-acquire should be a noop + # - https://docs.python.org/3.8/library/multiprocessing.html#multiprocessing.RLock + DebugStatus.req_finished = trio.Event() + DebugStatus.req_task = current_task() + try: + from tractor._discovery import get_root + # NOTE: we need this to ensure that this task exits + # BEFORE the REPl instance raises an error like + # `bdb.BdbQuit` directly, OW you get a trio cs stack + # corruption! + # Further, the since this task is spawned inside the + # `Context._scope_nursery: trio.Nursery`, once an RPC + # task errors that cs is cancel_called and so if we want + # to debug the TPC task that failed we need to shield + # against that expected `.cancel()` call and instead + # expect all of the `PdbREPL`.set_[continue/quit/]()` + # methods to unblock this task by setting the + # `.repl_release: # trio.Event`. + with trio.CancelScope(shield=shield) as req_cs: + # XXX: was orig for debugging cs stack corruption.. + # log.devx( + # 'Request cancel-scope is:\n\n' + # f'{pformat_cs(req_cs, var_name="req_cs")}\n\n' + # ) + DebugStatus.req_cs = req_cs + req_ctx: Context|None = None + try: + # TODO: merge into single async with ? + async with get_root() as portal: + async with portal.open_context( + lock_stdio_for_peer, + subactor_task_uid=task_uid, + + # NOTE: set it here in the locker request task bc it's + # possible for multiple such requests for the lock in any + # single sub-actor AND there will be a race between when the + # root locking task delivers the `Started(pld=LockStatus)` + # and when the REPL is actually entered by the requesting + # application task who called + # `.pause()`/`.post_mortem()`. + # + # SO, applying the pld-spec here means it is only applied to + # this IPC-ctx request task, NOT any other task(s) + # including the one that actually enters the REPL. This + # is oc desired bc ow the debugged task will msg-type-error. + # pld_spec=__pld_spec__, + + ) as (req_ctx, status): + + DebugStatus.req_ctx = req_ctx + log.devx( + 'Subactor locked TTY with msg\n\n' + f'{status}\n' + ) + + # try: + assert status.subactor_uid == actor_uid + assert status.cid + # except AttributeError: + # log.exception('failed pldspec asserts!') + # mk_pdb().set_trace() + # raise + + # set last rxed lock dialog status. + DebugStatus.lock_status = status + + async with req_ctx.open_stream() as stream: + task_status.started(req_ctx) + + # wait for local task to exit + # `PdbREPL.interaction()`, normally via + # a `DebugStatus.release()`call, and + # then unblock us here. + await DebugStatus.repl_release.wait() + await stream.send( + LockRelease( + subactor_uid=actor_uid, + cid=status.cid, + ) + ) + + # sync with child-side root locker task + # completion + status: LockStatus = await req_ctx.result() + assert not status.locked + DebugStatus.lock_status = status + + log.devx( + 'TTY lock was released for subactor with msg\n\n' + f'{status}\n\n' + f'Exitting {req_ctx.side!r}-side of locking req_ctx\n' + ) + + except ( + tractor.ContextCancelled, + trio.Cancelled, + ): + log.cancel( + 'Debug lock request was CANCELLED?\n\n' + f'{req_ctx}\n' + # f'{pformat_cs(req_cs, var_name="req_cs")}\n\n' + # f'{pformat_cs(req_ctx._scope, var_name="req_ctx._scope")}\n\n' + ) + raise + + except ( + BaseException, + ) as ctx_err: + message: str = ( + 'Failed during debug request dialog with root actor?\n\n' + ) + + if req_ctx: + message += ( + f'{req_ctx}\n' + f'Cancelling IPC ctx!\n' + ) + await req_ctx.cancel() + + else: + message += 'Failed during `Portal.open_context()` ?\n' + + log.exception(message) + ctx_err.add_note(message) + raise ctx_err + + + except ( + tractor.ContextCancelled, + trio.Cancelled, + ): + log.cancel( + 'Debug lock request CANCELLED?\n' + f'{req_ctx}\n' + ) + raise + + except BaseException as req_err: + # log.error('Failed to request root stdio-lock?') + DebugStatus.req_err = req_err + DebugStatus.release() + + # TODO: how to dev a test that ensures we actually drop + # into THIS internal frame on any internal error in the above + # code? + # -[ ] eg. on failed pld_dec assert above we should be able + # to REPL pm it. + # -[ ]FURTHER, after we 'continue', we should be able to + # ctl-c out of the currently hanging task! + raise DebugRequestError( + 'Failed to lock stdio from subactor IPC ctx!\n\n' + + f'req_ctx: {DebugStatus.req_ctx}\n' + ) from req_err + + finally: + log.devx('Exiting debugger TTY lock request func from child') + # signal request task exit + DebugStatus.req_finished.set() + DebugStatus.req_task = None + + +def mk_pdb() -> PdbREPL: + ''' + Deliver a new `PdbREPL`: a multi-process safe `pdbp.Pdb`-variant + using the magic of `tractor`'s SC-safe IPC. + + B) + + Our `pdb.Pdb` subtype accomplishes multi-process safe debugging + by: + + - mutexing access to the root process' std-streams (& thus parent + process TTY) via an IPC managed `Lock` singleton per + actor-process tree. + + - temporarily overriding any subactor's SIGINT handler to shield + during live REPL sessions in sub-actors such that cancellation + is never (mistakenly) triggered by a ctrl-c and instead only by + explicit runtime API requests or after the + `pdb.Pdb.interaction()` call has returned. + + FURTHER, the `pdbp.Pdb` instance is configured to be `trio` + "compatible" from a SIGINT handling perspective; we mask out + the default `pdb` handler and instead apply `trio`s default + which mostly addresses all issues described in: + + - https://github.com/python-trio/trio/issues/1155 + + The instance returned from this factory should always be + preferred over the default `pdb[p].set_trace()` whenever using + a `pdb` REPL inside a `trio` based runtime. + + ''' + pdb = PdbREPL() + + # XXX: These are the important flags mentioned in + # https://github.com/python-trio/trio/issues/1155 + # which resolve the traceback spews to console. + pdb.allow_kbdint = True + pdb.nosigint = True + return pdb + + +def any_connected_locker_child() -> bool: + ''' + Predicate to determine if a reported child subactor in debug + is actually connected. + + Useful to detect stale `Lock` requests after IPC failure. + + ''' + actor: Actor = current_actor() + + if not is_root_process(): + raise RuntimeError('This is a root-actor only API!') + + if ( + (ctx := Lock.ctx_in_debug) + and + (uid_in_debug := ctx.chan.uid) + ): + chans: list[tractor.Channel] = actor._peers.get( + tuple(uid_in_debug) + ) + if chans: + return any( + chan.connected() + for chan in chans + ) + + return False + + +def shield_sigint_handler( + signum: int, + frame: 'frame', # type: ignore # noqa + *args, + +) -> None: + ''' + Specialized, debugger-aware SIGINT handler. + + In childred we always ignore/shield for SIGINT to avoid + deadlocks since cancellation should always be managed by the + supervising parent actor. The root actor-proces is always + cancelled on ctrl-c. + + ''' + __tracebackhide__: bool = True + actor: Actor = current_actor() + + def do_cancel(): + # If we haven't tried to cancel the runtime then do that instead + # of raising a KBI (which may non-gracefully destroy + # a ``trio.run()``). + if not actor._cancel_called: + actor.cancel_soon() + + # If the runtime is already cancelled it likely means the user + # hit ctrl-c again because teardown didn't fully take place in + # which case we do the "hard" raising of a local KBI. + else: + raise KeyboardInterrupt + + # only set in the actor actually running the REPL + repl: PdbREPL|None = DebugStatus.repl + + # TODO: maybe we should flatten out all these cases using + # a match/case? + # + # root actor branch that reports whether or not a child + # has locked debugger. + if is_root_process(): + # try to see if the supposed (sub)actor in debug still + # has an active connection to *this* actor, and if not + # it's likely they aren't using the TTY lock / debugger + # and we should propagate SIGINT normally. + any_connected: bool = any_connected_locker_child() + # if not any_connected: + # return do_cancel() + + problem = ( + f'root {actor.uid} handling SIGINT\n' + f'any_connected: {any_connected}\n\n' + + f'{Lock.repr()}\n' + ) + + if ( + (ctx := Lock.ctx_in_debug) + and + (uid_in_debug := ctx.chan.uid) # "someone" is (ostensibly) using debug `Lock` + ): + name_in_debug: str = uid_in_debug[0] + assert not repl + # if not repl: # but it's NOT us, the root actor. + # sanity: since no repl ref is set, we def shouldn't + # be the lock owner! + assert name_in_debug != 'root' + + # IDEAL CASE: child has REPL as expected + if any_connected: # there are subactors we can contact + # XXX: only if there is an existing connection to the + # (sub-)actor in debug do we ignore SIGINT in this + # parent! Otherwise we may hang waiting for an actor + # which has already terminated to unlock. + # + # NOTE: don't emit this with `.pdb()` level in + # root without a higher level. + log.runtime( + f'Ignoring SIGINT while debug REPL in use by child ' + f'{uid_in_debug}\n' + ) + problem = None + + else: + problem += ( + '\n' + f'A `pdb` REPL is SUPPOSEDLY in use by child {uid_in_debug}\n' + f'BUT, no child actors are IPC contactable!?!?\n' + ) + + # IDEAL CASE: root has REPL as expected + else: + # root actor still has this SIGINT handler active without + # an actor using the `Lock` (a bug state) ?? + # => so immediately cancel any stale lock cs and revert + # the handler! + if not repl: + # TODO: WHEN should we revert back to ``trio`` + # handler if this one is stale? + # -[ ] maybe after a counts work of ctl-c mashes? + # -[ ] use a state var like `stale_handler: bool`? + problem += ( + '\n' + 'No subactor is using a `pdb` REPL according `Lock.ctx_in_debug`?\n' + 'BUT, the root should be using it, WHY this handler ??\n' + ) + else: + log.pdb( + 'Ignoring SIGINT while pdb REPL in use by root actor..\n' + ) + problem = None + + # XXX if one is set it means we ARE NOT operating an ideal + # case where a child subactor or us (the root) has the + # lock without any other detected problems. + if problem: + + # detect, report and maybe clear a stale lock request + # cancel scope. + lock_cs: trio.CancelScope = Lock.get_locking_task_cs() + maybe_stale_lock_cs: bool = ( + lock_cs is not None + and not lock_cs.cancel_called + ) + if maybe_stale_lock_cs: + problem += ( + '\n' + 'Stale `Lock.ctx_in_debug._scope: CancelScope` detected?\n' + f'{Lock.ctx_in_debug}\n\n' + + '-> Calling ctx._scope.cancel()!\n' + ) + lock_cs.cancel() + + # TODO: wen do we actually want/need this, see above. + # DebugStatus.unshield_sigint() + log.warning(problem) + + # child actor that has locked the debugger + elif not is_root_process(): + log.debug( + f'Subactor {actor.uid} handling SIGINT\n\n' + f'{Lock.repr()}\n' + ) + + rent_chan: Channel = actor._parent_chan + if ( + rent_chan is None + or + not rent_chan.connected() + ): + log.warning( + 'This sub-actor thinks it is debugging ' + 'but it has no connection to its parent ??\n' + f'{actor.uid}\n' + 'Allowing SIGINT propagation..' + ) + DebugStatus.unshield_sigint() + # do_cancel() + + repl_task: str|None = DebugStatus.repl_task + req_task: str|None = DebugStatus.req_task + if ( + repl_task + and + repl + ): + log.pdb( + f'Ignoring SIGINT while local task using debug REPL\n' + f'|_{repl_task}\n' + f' |_{repl}\n' + ) + elif req_task: + log.pdb( + f'Ignoring SIGINT while debug request task is open\n' + f'|_{req_task}\n' + ) + else: + msg: str = ( + 'SIGINT shield handler still active BUT, \n\n' + ) + if repl_task is None: + msg += ( + '- No local task claims to be in debug?\n' + ) + + if repl is None: + msg += ( + '- No local REPL is currently active?\n' + ) + + if req_task is None: + msg += ( + '- No debug request task is active?\n' + ) + + log.warning( + msg + + + 'Reverting handler to `trio` default!\n' + ) + DebugStatus.unshield_sigint() + + # XXX ensure that the reverted-to-handler actually is + # able to rx what should have been **this** KBI ;) + do_cancel() + + # TODO: how to handle the case of an intermediary-child actor + # that **is not** marked in debug mode? See oustanding issue: + # https://github.com/goodboy/tractor/issues/320 + # elif debug_mode(): + + # NOTE: currently (at least on ``fancycompleter`` 0.9.2) + # it looks to be that the last command that was run (eg. ll) + # will be repeated by default. + + # maybe redraw/print last REPL output to console since + # we want to alert the user that more input is expect since + # nothing has been done dur to ignoring sigint. + if ( + repl # only when current actor has a REPL engaged + ): + # XXX: yah, mega hack, but how else do we catch this madness XD + if repl.shname == 'xonsh': + repl.stdout.write(repl.prompt) + + repl.stdout.flush() + + # TODO: make this work like sticky mode where if there is output + # detected as written to the tty we redraw this part underneath + # and erase the past draw of this same bit above? + # repl.sticky = True + # repl._print_if_sticky() + + # also see these links for an approach from ``ptk``: + # https://github.com/goodboy/tractor/issues/130#issuecomment-663752040 + # https://github.com/prompt-toolkit/python-prompt-toolkit/blob/c2c6af8a0308f9e5d7c0e28cb8a02963fe0ce07a/prompt_toolkit/patch_stdout.py + + # XXX only for tracing this handler + log.devx('exiting SIGINT') + + +_pause_msg: str = 'Attaching to pdb REPL in actor' + + +class DebugRequestError(RuntimeError): + ''' + Failed to request stdio lock from root actor! + + ''' + + +_repl_fail_msg: str = ( + 'Failed to REPl via `_pause()` ' +) + + +async def _pause( + + debug_func: Callable|partial|None, + + # NOTE: must be passed in the `.pause_from_sync()` case! + repl: PdbREPL|None = None, + + # TODO: allow caller to pause despite task cancellation, + # exactly the same as wrapping with: + # with CancelScope(shield=True): + # await pause() + # => the REMAINING ISSUE is that the scope's .__exit__() frame + # is always show in the debugger on entry.. and there seems to + # be no way to override it?.. + # + shield: bool = False, + hide_tb: bool = True, + called_from_sync: bool = False, + called_from_bg_thread: bool = False, + task_status: TaskStatus[ + tuple[Task, PdbREPL], + trio.Event + ] = trio.TASK_STATUS_IGNORED, + **debug_func_kwargs, + +) -> tuple[PdbREPL, Task]|None: + ''' + Inner impl for `pause()` to avoid the `trio.CancelScope.__exit__()` + stack frame when not shielded (since apparently i can't figure out + how to hide it using the normal mechanisms..) + + Hopefully we won't need this in the long run. + + ''' + __tracebackhide__: bool = hide_tb + actor: Actor = current_actor() + try: + task: Task = current_task() + except RuntimeError as rte: + log.exception('Failed to get current task?') + if actor.is_infected_aio(): + raise RuntimeError( + '`tractor.pause[_from_sync]()` not yet supported ' + 'for infected `asyncio` mode!' + ) from rte + + raise + + if debug_func is not None: + debug_func = partial(debug_func) + + # XXX NOTE XXX set it here to avoid ctl-c from cancelling a debug + # request from a subactor BEFORE the REPL is entered by that + # process. + if not repl: + DebugStatus.shield_sigint() + repl: PdbREPL = repl or mk_pdb() + + # TODO: move this into a `open_debug_request()` @acm? + # -[ ] prolly makes the most sense to do the request + # task spawn as part of an `@acm` api which delivers the + # `DebugRequest` instance and ensures encapsing all the + # pld-spec and debug-nursery? + # -[ ] maybe make this a `PdbREPL` method or mod func? + # -[ ] factor out better, main reason for it is common logic for + # both root and sub repl entry + def _enter_repl_sync( + debug_func: partial[None], + ) -> None: + __tracebackhide__: bool = hide_tb + debug_func_name: str = ( + debug_func.func.__name__ if debug_func else 'None' + ) + + # TODO: do we want to support using this **just** for the + # locking / common code (prolly to help address #320)? + task_status.started((task, repl)) + try: + if debug_func: + # block here one (at the appropriate frame *up*) where + # ``breakpoint()`` was awaited and begin handling stdio. + log.devx( + 'Entering sync world of the `pdb` REPL for task..\n' + f'{repl}\n' + f' |_{task}\n' + ) + + # set local task on process-global state to avoid + # recurrent entries/requests from the same + # actor-local task. + DebugStatus.repl_task = task + DebugStatus.repl = repl + + # invoke the low-level REPL activation routine which itself + # should call into a `Pdb.set_trace()` of some sort. + debug_func( + repl=repl, + hide_tb=hide_tb, + **debug_func_kwargs, + ) + + # TODO: maybe invert this logic and instead + # do `assert debug_func is None` when + # `called_from_sync`? + else: + if ( + called_from_sync + and + not DebugStatus.is_main_trio_thread() + ): + assert called_from_bg_thread + assert DebugStatus.repl_task is not task + + return (task, repl) + + except trio.Cancelled: + log.exception( + 'Cancelled during invoke of internal\n\n' + f'`debug_func = {debug_func_name}`\n' + ) + # XXX NOTE: DON'T release lock yet + raise + + except BaseException: + __tracebackhide__: bool = False + log.exception( + 'Failed to invoke internal\n\n' + f'`debug_func = {debug_func_name}`\n' + ) + # NOTE: OW this is ONLY called from the + # `.set_continue/next` hooks! + DebugStatus.release(cancel_req_task=True) + + raise + + log.devx( + 'Entering `._pause()` for requesting task\n' + f'|_{task}\n' + ) + + # TODO: this should be created as part of `DebugRequest()` init + # which should instead be a one-shot-use singleton much like + # the `PdbREPL`. + repl_task: Thread|Task|None = DebugStatus.repl_task + if ( + not DebugStatus.repl_release + or + DebugStatus.repl_release.is_set() + ): + log.devx( + 'Setting new `DebugStatus.repl_release: trio.Event` for requesting task\n' + f'|_{task}\n' + ) + DebugStatus.repl_release = trio.Event() + else: + log.devx( + 'Already an existing actor-local REPL user task\n' + f'|_{repl_task}\n' + ) + + # ^-NOTE-^ this must be created BEFORE scheduling any subactor + # debug-req task since it needs to wait on it just after + # `.started()`-ing back its wrapping `.req_cs: CancelScope`. + + repl_err: BaseException|None = None + try: + if is_root_process(): + # we also wait in the root-parent for any child that + # may have the tty locked prior + # TODO: wait, what about multiple root tasks (with bg + # threads) acquiring it though? + ctx: Context|None = Lock.ctx_in_debug + repl_task: Task|None = DebugStatus.repl_task + if ( + ctx is None + and + repl_task is task + # and + # DebugStatus.repl + # ^-NOTE-^ matches for multi-threaded case as well? + ): + # re-entrant root process already has it: noop. + log.warning( + f'This root actor task is already within an active REPL session\n' + f'Ignoring this recurrent`tractor.pause()` entry\n\n' + f'|_{task}\n' + # TODO: use `._frame_stack` scanner to find the @api_frame + ) + with trio.CancelScope(shield=shield): + await trio.lowlevel.checkpoint() + return repl, task + + # elif repl_task: + # log.warning( + # f'This root actor has another task already in REPL\n' + # f'Waitin for the other task to complete..\n\n' + # f'|_{task}\n' + # # TODO: use `._frame_stack` scanner to find the @api_frame + # ) + # with trio.CancelScope(shield=shield): + # await DebugStatus.repl_release.wait() + # await trio.sleep(0.1) + + # must shield here to avoid hitting a `Cancelled` and + # a child getting stuck bc we clobbered the tty + with trio.CancelScope(shield=shield): + ctx_line = '`Lock` in this root actor task' + acq_prefix: str = 'shield-' if shield else '' + if ( + Lock._debug_lock.locked() + ): + if ctx: + ctx_line: str = ( + 'active `Lock` owned by ctx\n\n' + f'{ctx}' + ) + elif Lock._owned_by_root: + ctx_line: str = ( + 'Already owned by root-task `Lock`\n\n' + f'repl_task: {DebugStatus.repl_task}\n' + f'repl: {DebugStatus.repl}\n' + ) + else: + ctx_line: str = ( + '**STALE `Lock`** held by unknown root/remote task ' + 'with no request ctx !?!?' + ) + + log.devx( + f'attempting to {acq_prefix}acquire ' + f'{ctx_line}' + ) + await Lock._debug_lock.acquire() + Lock._owned_by_root = True + # else: + + # if ( + # not called_from_bg_thread + # and not called_from_sync + # ): + # log.devx( + # f'attempting to {acq_prefix}acquire ' + # f'{ctx_line}' + # ) + + # XXX: since we need to enter pdb synchronously below, + # and we don't want to block the thread that starts + # stepping through the application thread, we later + # must `Lock._debug_lock.release()` manually from + # some `PdbREPL` completion callback(`.set_[continue/exit]()`). + # + # So, when `._pause()` is called from a (bg/non-trio) + # thread, special provisions are needed and we need + # to do the `.acquire()`/`.release()` calls from + # a common `trio.task` (due to internal impl of + # `FIFOLock`). Thus we do not acquire here and + # instead expect `.pause_from_sync()` to take care of + # this detail depending on the caller's (threading) + # usage. + # + # NOTE that this special case is ONLY required when + # using `.pause_from_sync()` from the root actor + # since OW a subactor will instead make an IPC + # request (in the branch below) to acquire the + # `Lock`-mutex and a common root-actor RPC task will + # take care of `._debug_lock` mgmt! + + # enter REPL from root, no TTY locking IPC ctx necessary + # since we can acquire the `Lock._debug_lock` directly in + # thread. + return _enter_repl_sync(debug_func) + + # TODO: need a more robust check for the "root" actor + elif ( + not is_root_process() + and actor._parent_chan # a connected child + ): + repl_task: Task|None = DebugStatus.repl_task + req_task: Task|None = DebugStatus.req_task + if req_task: + log.warning( + f'Already an ongoing repl request?\n' + f'|_{req_task}\n\n' + + f'REPL task is\n' + f'|_{repl_task}\n\n' + + ) + # Recurrent entry case. + # this task already has the lock and is likely + # recurrently entering a `.pause()`-point either bc, + # - someone is hacking on runtime internals and put + # one inside code that get's called on the way to + # this code, + # - a legit app task uses the 'next' command while in + # a REPL sesh, and actually enters another + # `.pause()` (in a loop or something). + # + # XXX Any other cose is likely a bug. + if ( + repl_task + ): + if repl_task is task: + log.warning( + f'{task.name}@{actor.uid} already has TTY lock\n' + f'ignoring..' + ) + with trio.CancelScope(shield=shield): + await trio.lowlevel.checkpoint() + return + + else: + # if **this** actor is already in debug REPL we want + # to maintain actor-local-task mutex access, so block + # here waiting for the control to be released - this + # -> allows for recursive entries to `tractor.pause()` + log.warning( + f'{task}@{actor.uid} already has TTY lock\n' + f'waiting for release..' + ) + with trio.CancelScope(shield=shield): + await DebugStatus.repl_release.wait() + await trio.sleep(0.1) + + elif ( + req_task + ): + log.warning( + 'Local task already has active debug request\n' + f'|_{task}\n\n' + + 'Waiting for previous request to complete..\n' + ) + with trio.CancelScope(shield=shield): + await DebugStatus.req_finished.wait() + + # this **must** be awaited by the caller and is done using the + # root nursery so that the debugger can continue to run without + # being restricted by the scope of a new task nursery. + + # TODO: if we want to debug a trio.Cancelled triggered exception + # we have to figure out how to avoid having the service nursery + # cancel on this task start? I *think* this works below: + # ```python + # actor._service_n.cancel_scope.shield = shield + # ``` + # but not entirely sure if that's a sane way to implement it? + + # NOTE currently we spawn the lock request task inside this + # subactor's global `Actor._service_n` so that the + # lifetime of the lock-request can outlive the current + # `._pause()` scope while the user steps through their + # application code and when they finally exit the + # session, via 'continue' or 'quit' cmds, the `PdbREPL` + # will manually call `DebugStatus.release()` to release + # the lock session with the root actor. + # + # TODO: ideally we can add a tighter scope for this + # request task likely by conditionally opening a "debug + # nursery" inside `_errors_relayed_via_ipc()`, see the + # todo in tht module, but + # -[ ] it needs to be outside the normal crash handling + # `_maybe_enter_debugger()` block-call. + # -[ ] we probably only need to allocate the nursery when + # we detect the runtime is already in debug mode. + # + curr_ctx: Context = current_ipc_ctx() + # req_ctx: Context = await curr_ctx._debug_tn.start( + log.devx( + 'Starting request task\n' + f'|_{task}\n' + ) + with trio.CancelScope(shield=shield): + req_ctx: Context = await actor._service_n.start( + partial( + request_root_stdio_lock, + actor_uid=actor.uid, + task_uid=(task.name, id(task)), # task uuid (effectively) + shield=shield, + ) + ) + # XXX sanity, our locker task should be the one which + # entered a new IPC ctx with the root actor, NOT the one + # that exists around the task calling into `._pause()`. + assert ( + req_ctx + is + DebugStatus.req_ctx + is not + curr_ctx + ) + + # enter REPL + return _enter_repl_sync(debug_func) + + # TODO: prolly factor this plus the similar block from + # `_enter_repl_sync()` into a common @cm? + except BaseException as pause_err: + if isinstance(pause_err, bdb.BdbQuit): + log.devx( + 'REPL for pdb was quit!\n' + ) + + # when the actor is mid-runtime cancellation the + # `Actor._service_n` might get closed before we can spawn + # the request task, so just ignore expected RTE. + elif ( + isinstance(pause_err, RuntimeError) + and + actor._cancel_called + ): + # service nursery won't be usable and we + # don't want to lock up the root either way since + # we're in (the midst of) cancellation. + log.warning( + 'Service nursery likely closed due to actor-runtime cancellation..\n' + 'Ignoring failed debugger lock request task spawn..\n' + ) + return + + else: + log.exception( + _repl_fail_msg + + + f'on behalf of {repl_task} ??\n' + ) + + DebugStatus.release(cancel_req_task=True) + + # sanity checks for ^ on request/status teardown + assert DebugStatus.repl is None + assert DebugStatus.repl_task is None + + # sanity, for when hackin on all this? + if not isinstance(pause_err, trio.Cancelled): + req_ctx: Context = DebugStatus.req_ctx + if req_ctx: + # XXX, bc the child-task in root might cancel it? + # assert req_ctx._scope.cancel_called + assert req_ctx.maybe_error + + raise + + finally: + # set in finally block of func.. this can be synced-to + # eventually with a debug_nursery somehow? + # assert DebugStatus.req_task is None + + # always show frame when request fails due to internal + # failure in the above code (including an `BdbQuit`). + if ( + DebugStatus.req_err + or + repl_err + ): + __tracebackhide__: bool = False + + +def _set_trace( + repl: PdbREPL, # passed by `_pause()` + hide_tb: bool, + + # partial-ed in by `.pause()` + api_frame: FrameType, + + # optionally passed in to provide support for + # `pause_from_sync()` where + actor: tractor.Actor|None = None, + task: Task|Thread|None = None, +): + __tracebackhide__: bool = hide_tb + actor: tractor.Actor = actor or current_actor() + task: Task|Thread = task or current_task() + + # else: + # TODO: maybe print the actor supervion tree up to the + # root here? Bo + log.pdb( + f'{_pause_msg}\n' + '|\n' + # TODO: more compact pformating? + # -[ ] make an `Actor.__repr()__` + # -[ ] should we use `log.pformat_task_uid()`? + f'|_ {task} @ {actor.uid}\n' + ) + # presuming the caller passed in the "api frame" + # (the last frame before user code - like `.pause()`) + # then we only step up one frame to where the user + # called our API. + caller_frame: FrameType = api_frame.f_back # type: ignore + + # pretend this frame is the caller frame to show + # the entire call-stack all the way down to here. + if not hide_tb: + caller_frame: FrameType = inspect.currentframe() + + # engage ze REPL + # B~() + repl.set_trace(frame=caller_frame) + + +async def pause( + *, + hide_tb: bool = True, + api_frame: FrameType|None = None, + + # TODO: figure out how to still make this work: + # -[ ] pass it direct to `_pause()`? + # -[ ] use it to set the `debug_nursery.cancel_scope.shield` + shield: bool = False, + **_pause_kwargs, + +) -> None: + ''' + A pause point (more commonly known as a "breakpoint") interrupt + instruction for engaging a blocking debugger instance to + conduct manual console-based-REPL-interaction from within + `tractor`'s async runtime, normally from some single-threaded + and currently executing actor-hosted-`trio`-task in some + (remote) process. + + NOTE: we use the semantics "pause" since it better encompasses + the entirety of the necessary global-runtime-state-mutation any + actor-task must access and lock in order to get full isolated + control over the process tree's root TTY: + https://en.wikipedia.org/wiki/Breakpoint + + ''' + __tracebackhide__: bool = hide_tb + + # always start 1 level up from THIS in user code since normally + # `tractor.pause()` is called explicitly by use-app code thus + # making it the highest up @api_frame. + api_frame: FrameType = api_frame or inspect.currentframe() + + # XXX TODO: this was causing cs-stack corruption in trio due to + # usage within the `Context._scope_nursery` (which won't work + # based on scoping of it versus call to `_maybe_enter_debugger()` + # from `._rpc._invoke()`) + # with trio.CancelScope( + # shield=shield, + # ) as cs: + # NOTE: so the caller can always manually cancel even + # if shielded! + # task_status.started(cs) + # log.critical( + # '`.pause() cancel-scope is:\n\n' + # f'{pformat_cs(cs, var_name="pause_cs")}\n\n' + # ) + await _pause( + debug_func=partial( + _set_trace, + api_frame=api_frame, + ), + shield=shield, + **_pause_kwargs + ) + # XXX avoid cs stack corruption when `PdbREPL.interaction()` + # raises `BdbQuit`. + # await DebugStatus.req_finished.wait() + + +_gb_mod: None|ModuleType|False = None + + +def maybe_import_greenback( + raise_not_found: bool = True, + force_reload: bool = False, + +) -> ModuleType|False: + # be cached-fast on module-already-inited + global _gb_mod + + if _gb_mod is False: + return False + + elif ( + _gb_mod is not None + and not force_reload + ): + return _gb_mod + + try: + import greenback + _gb_mod = greenback + return greenback + + except ModuleNotFoundError as mnf: + log.debug( + '`greenback` is not installed.\n' + 'No sync debug support!\n' + ) + _gb_mod = False + + if raise_not_found: + raise RuntimeError( + 'The `greenback` lib is required to use `tractor.pause_from_sync()`!\n' + 'https://github.com/oremanj/greenback\n' + ) from mnf + + return False + + +async def maybe_init_greenback( + **kwargs, +) -> None|ModuleType: + + if mod := maybe_import_greenback(**kwargs): + await mod.ensure_portal() + log.devx( + '`greenback` portal opened!\n' + 'Sync debug support activated!\n' + ) + return mod + + return None + + + +async def _pause_from_bg_root_thread( + behalf_of_thread: Thread, + repl: PdbREPL, + hide_tb: bool, + task_status: TaskStatus[Task] = trio.TASK_STATUS_IGNORED, + **_pause_kwargs, +): + ''' + Acquire the `Lock._debug_lock` from a bg (only need for + root-actor) non-`trio` thread (started via a call to + `.to_thread.run_sync()` in some actor) by scheduling this func in + the actor's service (TODO eventually a special debug_mode) + nursery. This task acquires the lock then `.started()`s the + `DebugStatus.repl_release: trio.Event` waits for the `PdbREPL` to + set it, then terminates very much the same way as + `request_root_stdio_lock()` uses an IPC `Context` from a subactor + to do the same from a remote process. + + This task is normally only required to be scheduled for the + special cases of a bg sync thread running in the root actor; see + the only usage inside `.pause_from_sync()`. + + ''' + global Lock + # TODO: unify this copied code with where it was + # from in `maybe_wait_for_debugger()` + # if ( + # Lock.req_handler_finished is not None + # and not Lock.req_handler_finished.is_set() + # and (in_debug := Lock.ctx_in_debug) + # ): + # log.devx( + # '\nRoot is waiting on tty lock to release from\n\n' + # # f'{caller_frame_info}\n' + # ) + # with trio.CancelScope(shield=True): + # await Lock.req_handler_finished.wait() + + # log.pdb( + # f'Subactor released debug lock\n' + # f'|_{in_debug}\n' + # ) + task: Task = current_task() + + # Manually acquire since otherwise on release we'll + # get a RTE raised by `trio` due to ownership.. + log.devx( + 'Trying to acquire `Lock` on behalf of bg thread\n' + f'|_{behalf_of_thread}\n' + ) + # DebugStatus.repl_task = behalf_of_thread + out = await _pause( + debug_func=None, + repl=repl, + hide_tb=hide_tb, + called_from_sync=True, + called_from_bg_thread=True, + **_pause_kwargs + ) + lock: trio.FIFOLock = Lock._debug_lock + stats: trio.LockStatistics= lock.statistics() + assert stats.owner is task + assert Lock._owned_by_root + assert DebugStatus.repl_release + + # TODO: do we actually need this? + # originally i was trying to solve wy this was + # unblocking too soon in a thread but it was actually + # that we weren't setting our own `repl_release` below.. + while stats.owner is not task: + log.devx( + 'Trying to acquire `._debug_lock` from {stats.owner} for\n' + f'|_{behalf_of_thread}\n' + ) + await lock.acquire() + break + + # XXX NOTE XXX super important dawg.. + # set our own event since the current one might + # have already been overriden and then set when the + # last REPL mutex holder exits their sesh! + # => we do NOT want to override any existing one + # and we want to ensure we set our own ONLY AFTER we have + # acquired the `._debug_lock` + repl_release = DebugStatus.repl_release = trio.Event() + + # unblock caller thread delivering this bg task + log.devx( + 'Unblocking root-bg-thread since we acquired lock via `._pause()`\n' + f'|_{behalf_of_thread}\n' + ) + task_status.started(out) + DebugStatus.shield_sigint() + + # wait for bg thread to exit REPL sesh. + try: + await repl_release.wait() + finally: + log.devx( + 'releasing lock from bg root thread task!\n' + f'|_ {behalf_of_thread}\n' + ) + Lock.release() + + +def pause_from_sync( + hide_tb: bool = True, + called_from_builtin: bool = False, + api_frame: FrameType|None = None, + + # proxy to `._pause()`, for ex: + # shield: bool = False, + # api_frame: FrameType|None = None, + **_pause_kwargs, + +) -> None: + ''' + Pause a `tractor` scheduled task or thread from sync (non-async + function) code. + + When `greenback` is installed we remap python's builtin + `breakpoint()` hook to this runtime-aware version which takes + care of all bg-thread detection and appropriate synchronization + with the root actor's `Lock` to avoid mult-thread/process REPL + clobbering Bo + + ''' + __tracebackhide__: bool = hide_tb + try: + actor: tractor.Actor = current_actor( + err_on_no_runtime=False, + ) + message: str = ( + f'{actor.uid} task called `tractor.pause_from_sync()`\n\n' + ) + if not actor: + raise RuntimeError( + 'Not inside the `tractor`-runtime?\n' + '`tractor.pause_from_sync()` is not functional without a wrapping\n' + '- `async with tractor.open_nursery()` or,\n' + '- `async with tractor.open_root_actor()`\n' + ) + + # TODO: once supported, remove this AND the one + # inside `._pause()`! + if actor.is_infected_aio(): + raise RuntimeError( + '`tractor.pause[_from_sync]()` not yet supported ' + 'for infected `asyncio` mode!' + ) + + DebugStatus.shield_sigint() + repl: PdbREPL = mk_pdb() + + # message += f'-> created local REPL {repl}\n' + is_root: bool = is_root_process() + + # TODO: we could also check for a non-`.to_thread` context + # using `trio.from_thread.check_cancelled()` (says + # oremanj) wherein we get the following outputs: + # + # `RuntimeError`: non-`.to_thread` spawned thread + # noop: non-cancelled `.to_thread` + # `trio.Cancelled`: cancelled `.to_thread` + + # when called from a (bg) thread, run an async task in a new + # thread which will call `._pause()` manually with special + # handling for root-actor caller usage. + if not DebugStatus.is_main_trio_thread(): + thread: threading.Thread = threading.current_thread() + repl_owner = thread + + # TODO: make root-actor bg thread usage work! + if is_root: + message += ( + f'-> called from a root-actor bg {thread}\n' + f'-> scheduling `._pause_from_sync_thread()`..\n' + ) + bg_task, repl = trio.from_thread.run( + afn=partial( + actor._service_n.start, + partial( + _pause_from_bg_root_thread, + behalf_of_thread=thread, + repl=repl, + hide_tb=hide_tb, + **_pause_kwargs, + ), + ) + ) + message += ( + f'-> `._pause_from_sync_thread()` started bg task {bg_task}\n' + ) + else: + message += f'-> called from a bg {thread}\n' + # NOTE: since this is a subactor, `._pause()` will + # internally issue a debug request via + # `request_root_stdio_lock()` and we don't need to + # worry about all the special considerations as with + # the root-actor per above. + bg_task, repl = trio.from_thread.run( + afn=partial( + _pause, + debug_func=None, + repl=repl, + hide_tb=hide_tb, + + # XXX to prevent `._pause()` for setting + # `DebugStatus.repl_task` to the gb task! + called_from_sync=True, + called_from_bg_thread=True, + + **_pause_kwargs + ), + ) + assert bg_task is not DebugStatus.repl_task + + else: # we are presumably the `trio.run()` + main thread + # raises on not-found by default + greenback: ModuleType = maybe_import_greenback() + message += f'-> imported {greenback}\n' + repl_owner: Task = current_task() + message += '-> calling `greenback.await_(_pause(debug_func=None))` from sync caller..\n' + out = greenback.await_( + _pause( + debug_func=None, + repl=repl, + hide_tb=hide_tb, + called_from_sync=True, + **_pause_kwargs, + ) + ) + if out: + bg_task, repl = out + assert repl is repl + assert bg_task is repl_owner + + # NOTE: normally set inside `_enter_repl_sync()` + DebugStatus.repl_task: str = repl_owner + + # TODO: ensure we aggressively make the user aware about + # entering the global `breakpoint()` built-in from sync + # code? + message += ( + f'-> successfully scheduled `._pause()` in `trio` thread on behalf of {bg_task}\n' + f'-> Entering REPL via `tractor._set_trace()` from caller {repl_owner}\n' + ) + log.devx(message) + + DebugStatus.repl = repl + _set_trace( + api_frame=api_frame or inspect.currentframe(), + repl=repl, + hide_tb=hide_tb, + actor=actor, + task=repl_owner, + ) + # LEGACY NOTE on next LOC's frame showing weirdness.. + # + # XXX NOTE XXX no other LOC can be here without it + # showing up in the REPL's last stack frame !?! + # -[ ] tried to use `@pdbp.hideframe` decoration but + # still doesn't work + except BaseException as err: + __tracebackhide__: bool = False + raise err + + +def _sync_pause_from_builtin( + *args, + called_from_builtin=True, + **kwargs, +) -> None: + ''' + Proxy call `.pause_from_sync()` but indicate the caller is the + `breakpoint()` built-in. + + Note: this assigned to `os.environ['PYTHONBREAKPOINT']` inside `._root` + + ''' + pause_from_sync( + *args, + called_from_builtin=True, + api_frame=inspect.currentframe(), + **kwargs, + ) + + +# NOTE prefer a new "pause" semantic since it better describes +# "pausing the actor's runtime" for this particular +# paralell task to do debugging in a REPL. +async def breakpoint( + hide_tb: bool = True, + **kwargs, +): + log.warning( + '`tractor.breakpoint()` is deprecated!\n' + 'Please use `tractor.pause()` instead!\n' + ) + __tracebackhide__: bool = hide_tb + await pause( + api_frame=inspect.currentframe(), + **kwargs, + ) + + +_crash_msg: str = ( + 'Attaching to pdb REPL in crashed actor' +) + + +def _post_mortem( + # provided and passed by `_pause()` + repl: PdbREPL, + + # XXX all `partial`-ed in by `post_mortem()` below! + tb: TracebackType, + api_frame: FrameType, + + shield: bool = False, + hide_tb: bool = False, + +) -> None: + ''' + Enter the ``pdbpp`` port mortem entrypoint using our custom + debugger instance. + + ''' + __tracebackhide__: bool = hide_tb + actor: tractor.Actor = current_actor() + + # TODO: print the actor supervion tree up to the root + # here! Bo + log.pdb( + f'{_crash_msg}\n' + '|\n' + # f'|_ {current_task()}\n' + f'|_ {current_task()} @ {actor.uid}\n' + + # f'|_ @{actor.uid}\n' + # TODO: make an `Actor.__repr()__` + # f'|_ {current_task()} @ {actor.name}\n' + ) + + # NOTE only replacing this from `pdbp.xpm()` to add the + # `end=''` to the print XD + print(traceback.format_exc(), end='') + + caller_frame: FrameType = api_frame.f_back + + # NOTE: see the impl details of followings to understand usage: + # - `pdbp.post_mortem()` + # - `pdbp.xps()` + # - `bdb.interaction()` + repl.reset() + repl.interaction( + frame=caller_frame, + # frame=None, + traceback=tb, + ) + # XXX NOTE XXX: absolutely required to avoid hangs! + # Since we presume the post-mortem was enaged to a task-ending + # error, we MUST release the local REPL request so that not other + # local task nor the root remains blocked! + DebugStatus.release() + + +async def post_mortem( + *, + tb: TracebackType|None = None, + api_frame: FrameType|None = None, + hide_tb: bool = False, + + # TODO: support shield here just like in `pause()`? + # shield: bool = False, + + **_pause_kwargs, + +) -> None: + ''' + `tractor`'s builtin async equivalient of `pdb.post_mortem()` + which can be used inside exception handlers. + + It's also used for the crash handler when `debug_mode == True` ;) + + ''' + __tracebackhide__: bool = hide_tb + + tb: TracebackType = tb or sys.exc_info()[2] + + # TODO: do upward stack scan for highest @api_frame and + # use its parent frame as the expected user-app code + # interact point. + api_frame: FrameType = api_frame or inspect.currentframe() + + await _pause( + debug_func=partial( + _post_mortem, + api_frame=api_frame, + tb=tb, + ), + hide_tb=hide_tb, + **_pause_kwargs + ) + + +async def _maybe_enter_pm( + err: BaseException, + *, + tb: TracebackType|None = None, + api_frame: FrameType|None = None, + hide_tb: bool = False, +): + from tractor._exceptions import is_multi_cancelled + if ( + debug_mode() + + # NOTE: don't enter debug mode recursively after quitting pdb + # Iow, don't re-enter the repl if the `quit` command was issued + # by the user. + and not isinstance(err, bdb.BdbQuit) + + # XXX: if the error is the likely result of runtime-wide + # cancellation, we don't want to enter the debugger since + # there's races between when the parent actor has killed all + # comms and when the child tries to contact said parent to + # acquire the tty lock. + + # Really we just want to mostly avoid catching KBIs here so there + # might be a simpler check we can do? + and not is_multi_cancelled(err) + ): + api_frame: FrameType = api_frame or inspect.currentframe() + tb: TracebackType = tb or sys.exc_info()[2] + await post_mortem( + api_frame=api_frame, + tb=tb, + ) + return True + + else: + return False + + +@acm +async def acquire_debug_lock( + subactor_uid: tuple[str, str], +) -> AsyncGenerator[ + trio.CancelScope|None, + tuple, +]: + ''' + Request to acquire the TTY `Lock` in the root actor, release on exit. + + This helper is for actor's who don't actually need to acquired + the debugger but want to wait until the lock is free in the + process-tree root such that they don't clobber an ongoing pdb + REPL session in some peer or child! + + ''' + if not debug_mode(): + yield None + return + + async with trio.open_nursery() as n: + ctx: Context = await n.start( + request_root_stdio_lock, + subactor_uid, + ) + yield ctx + ctx.cancel() + + +async def maybe_wait_for_debugger( + poll_steps: int = 2, + poll_delay: float = 0.1, + child_in_debug: bool = False, + + header_msg: str = '', + _ll: str = 'devx', + +) -> bool: # was locked and we polled? + + if ( + not debug_mode() + and not child_in_debug + ): + return False + + logmeth: Callable = getattr(log, _ll) + + msg: str = header_msg + if ( + is_root_process() + ): + # If we error in the root but the debugger is + # engaged we don't want to prematurely kill (and + # thus clobber access to) the local tty since it + # will make the pdb repl unusable. + # Instead try to wait for pdb to be released before + # tearing down. + ctx_in_debug: Context|None = Lock.ctx_in_debug + in_debug: tuple[str, str]|None = ( + ctx_in_debug.chan.uid + if ctx_in_debug + else None + ) + if in_debug == current_actor().uid: + log.debug( + msg + + + 'Root already owns the TTY LOCK' + ) + return True + + elif in_debug: + msg += ( + f'Debug `Lock` in use by subactor\n|\n|_{in_debug}\n' + ) + # TODO: could this make things more deterministic? + # wait to see if a sub-actor task will be + # scheduled and grab the tty lock on the next + # tick? + # XXX => but it doesn't seem to work.. + # await trio.testing.wait_all_tasks_blocked(cushion=0) + else: + logmeth( + msg + + + 'Root immediately acquired debug TTY LOCK' + ) + return False + + for istep in range(poll_steps): + if ( + Lock.req_handler_finished is not None + and not Lock.req_handler_finished.is_set() + and in_debug is not None + ): + # caller_frame_info: str = pformat_caller_frame() + logmeth( + msg + + + '\nRoot is waiting on tty lock to release from\n\n' + # f'{caller_frame_info}\n' + ) + + if not any_connected_locker_child(): + Lock.get_locking_task_cs().cancel() + + with trio.CancelScope(shield=True): + await Lock.req_handler_finished.wait() + + log.devx( + f'Subactor released debug lock\n' + f'|_{in_debug}\n' + ) + break + + # is no subactor locking debugger currently? + if ( + in_debug is None + and ( + Lock.req_handler_finished is None + or Lock.req_handler_finished.is_set() + ) + ): + logmeth( + msg + + + 'Root acquired tty lock!' + ) + break + + else: + logmeth( + 'Root polling for debug:\n' + f'poll step: {istep}\n' + f'poll delya: {poll_delay}\n\n' + f'{Lock.repr()}\n' + ) + with CancelScope(shield=True): + await trio.sleep(poll_delay) + continue + + return True + + # else: + # # TODO: non-root call for #320? + # this_uid: tuple[str, str] = current_actor().uid + # async with acquire_debug_lock( + # subactor_uid=this_uid, + # ): + # pass + return False + +# TODO: better naming and what additionals? +# - [ ] optional runtime plugging? +# - [ ] detection for sync vs. async code? +# - [ ] specialized REPL entry when in distributed mode? +# - [x] allow ignoring kbi Bo +@cm +def open_crash_handler( + catch: set[BaseException] = { + Exception, + BaseException, + }, + ignore: set[BaseException] = { + KeyboardInterrupt, + }, +): + ''' + Generic "post mortem" crash handler using `pdbp` REPL debugger. + + We expose this as a CLI framework addon to both `click` and + `typer` users so they can quickly wrap cmd endpoints which get + automatically wrapped to use the runtime's `debug_mode: bool` + AND `pdbp.pm()` around any code that is PRE-runtime entry + - any sync code which runs BEFORE the main call to + `trio.run()`. + + ''' + try: + yield + except tuple(catch) as err: + + if type(err) not in ignore: + pdbp.xpm() + + raise + + +@cm +def maybe_open_crash_handler( + pdb: bool = False, +): + ''' + Same as `open_crash_handler()` but with bool input flag + to allow conditional handling. + + Normally this is used with CLI endpoints such that if the --pdb + flag is passed the pdb REPL is engaed on any crashes B) + ''' + rtctx = nullcontext + if pdb: + rtctx = open_crash_handler + + with rtctx(): + yield diff --git a/tractor/devx/_frame_stack.py b/tractor/devx/_frame_stack.py new file mode 100644 index 00000000..89a9e849 --- /dev/null +++ b/tractor/devx/_frame_stack.py @@ -0,0 +1,288 @@ +# 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 . + +''' +Tools for code-object annotation, introspection and mutation +as it pertains to improving the grok-ability of our runtime! + +''' +from __future__ import annotations +from functools import partial +import inspect +from types import ( + FrameType, + FunctionType, + MethodType, + # CodeType, +) +from typing import ( + Any, + Callable, + Type, +) + +from tractor.msg import ( + pretty_struct, + NamespacePath, +) +import wrapt + + +# TODO: yeah, i don't love this and we should prolly just +# write a decorator that actually keeps a stupid ref to the func +# obj.. +def get_class_from_frame(fr: FrameType) -> ( + FunctionType + |MethodType +): + ''' + Attempt to get the function (or method) reference + from a given `FrameType`. + + Verbatim from an SO: + https://stackoverflow.com/a/2220759 + + ''' + args, _, _, value_dict = inspect.getargvalues(fr) + + # we check the first parameter for the frame function is + # named 'self' + if ( + len(args) + and + # TODO: other cases for `@classmethod` etc..?) + args[0] == 'self' + ): + # in that case, 'self' will be referenced in value_dict + instance: object = value_dict.get('self') + if instance: + # return its class + return getattr( + instance, + '__class__', + None, + ) + + # return None otherwise + return None + + +def get_ns_and_func_from_frame( + frame: FrameType, +) -> Callable: + ''' + Return the corresponding function object reference from + a `FrameType`, and return it and it's parent namespace `dict`. + + ''' + ns: dict[str, Any] + + # for a method, go up a frame and lookup the name in locals() + if '.' in (qualname := frame.f_code.co_qualname): + cls_name, _, func_name = qualname.partition('.') + ns = frame.f_back.f_locals[cls_name].__dict__ + + else: + func_name: str = frame.f_code.co_name + ns = frame.f_globals + + return ( + ns, + ns[func_name], + ) + + +def func_ref_from_frame( + frame: FrameType, +) -> Callable: + func_name: str = frame.f_code.co_name + try: + return frame.f_globals[func_name] + except KeyError: + cls: Type|None = get_class_from_frame(frame) + if cls: + return getattr( + cls, + func_name, + ) + + +class CallerInfo(pretty_struct.Struct): + # https://docs.python.org/dev/reference/datamodel.html#frame-objects + # https://docs.python.org/dev/library/inspect.html#the-interpreter-stack + _api_frame: FrameType + + @property + def api_frame(self) -> FrameType: + try: + self._api_frame.clear() + except RuntimeError: + # log.warning( + print( + f'Frame {self._api_frame} for {self.api_func} is still active!' + ) + + return self._api_frame + + _api_func: Callable + + @property + def api_func(self) -> Callable: + return self._api_func + + _caller_frames_up: int|None = 1 + _caller_frame: FrameType|None = None # cached after first stack scan + + @property + def api_nsp(self) -> NamespacePath|None: + func: FunctionType = self.api_func + if func: + return NamespacePath.from_ref(func) + + return '' + + @property + def caller_frame(self) -> FrameType: + + # if not already cached, scan up stack explicitly by + # configured count. + if not self._caller_frame: + if self._caller_frames_up: + for _ in range(self._caller_frames_up): + caller_frame: FrameType|None = self.api_frame.f_back + + if not caller_frame: + raise ValueError( + 'No frame exists {self._caller_frames_up} up from\n' + f'{self.api_frame} @ {self.api_nsp}\n' + ) + + self._caller_frame = caller_frame + + return self._caller_frame + + @property + def caller_nsp(self) -> NamespacePath|None: + func: FunctionType = self.api_func + if func: + return NamespacePath.from_ref(func) + + return '' + + +def find_caller_info( + dunder_var: str = '__runtimeframe__', + iframes:int = 1, + check_frame_depth: bool = True, + +) -> CallerInfo|None: + ''' + Scan up the callstack for a frame with a `dunder_var: str` variable + and return the `iframes` frames above it. + + By default we scan for a `__runtimeframe__` scope var which + denotes a `tractor` API above which (one frame up) is "user + app code" which "called into" the `tractor` method or func. + + TODO: ex with `Portal.open_context()` + + ''' + # TODO: use this instead? + # https://docs.python.org/3/library/inspect.html#inspect.getouterframes + frames: list[inspect.FrameInfo] = inspect.stack() + for fi in frames: + assert ( + fi.function + == + fi.frame.f_code.co_name + ) + this_frame: FrameType = fi.frame + dunder_val: int|None = this_frame.f_locals.get(dunder_var) + if dunder_val: + go_up_iframes: int = ( + dunder_val # could be 0 or `True` i guess? + or + iframes + ) + rt_frame: FrameType = fi.frame + call_frame = rt_frame + for i in range(go_up_iframes): + call_frame = call_frame.f_back + + return CallerInfo( + _api_frame=rt_frame, + _api_func=func_ref_from_frame(rt_frame), + _caller_frames_up=go_up_iframes, + ) + + return None + + +_frame2callerinfo_cache: dict[FrameType, CallerInfo] = {} + + +# TODO: -[x] move all this into new `.devx._code`! +# -[ ] consider rename to _callstack? +# -[ ] prolly create a `@runtime_api` dec? +# |_ @api_frame seems better? +# -[ ] ^- make it capture and/or accept buncha optional +# meta-data like a fancier version of `@pdbp.hideframe`. +# +def api_frame( + wrapped: Callable|None = None, + *, + caller_frames_up: int = 1, + +) -> Callable: + + # handle the decorator called WITHOUT () case, + # i.e. just @api_frame, NOT @api_frame(extra=) + if wrapped is None: + return partial( + api_frame, + caller_frames_up=caller_frames_up, + ) + + @wrapt.decorator + async def wrapper( + wrapped: Callable, + instance: object, + args: tuple, + kwargs: dict, + ): + # maybe cache the API frame for this call + global _frame2callerinfo_cache + this_frame: FrameType = inspect.currentframe() + api_frame: FrameType = this_frame.f_back + + if not _frame2callerinfo_cache.get(api_frame): + _frame2callerinfo_cache[api_frame] = CallerInfo( + _api_frame=api_frame, + _api_func=wrapped, + _caller_frames_up=caller_frames_up, + ) + + return wrapped(*args, **kwargs) + + # annotate the function as a "api function", meaning it is + # a function for which the function above it in the call stack should be + # non-`tractor` code aka "user code". + # + # in the global frame cache for easy lookup from a given + # func-instance + wrapped._call_infos: dict[FrameType, CallerInfo] = _frame2callerinfo_cache + wrapped.__api_func__: bool = True + return wrapper(wrapped) diff --git a/tractor/devx/_stackscope.py b/tractor/devx/_stackscope.py new file mode 100644 index 00000000..e8e97d1a --- /dev/null +++ b/tractor/devx/_stackscope.py @@ -0,0 +1,137 @@ +# tractor: structured concurrent "actors". +# Copyright 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 +import multiprocessing as mp +from signal import ( + signal, + SIGUSR1, +) +import traceback +from typing import TYPE_CHECKING + +import trio +from tractor import ( + _state, + log as logmod, +) + +log = logmod.get_logger(__name__) + + +if TYPE_CHECKING: + from tractor._spawn import ProcessType + from tractor import ( + Actor, + ActorNursery, + ) + + +@trio.lowlevel.disable_ki_protection +def dump_task_tree() -> None: + import stackscope + from tractor.log import get_console_log + + tree_str: str = str( + stackscope.extract( + trio.lowlevel.current_root_task(), + recurse_child_tasks=True + ) + ) + log = get_console_log( + name=__name__, + level='cancel', + ) + actor: Actor = _state.current_actor() + log.devx( + f'Dumping `stackscope` tree for actor\n' + f'{actor.name}: {actor}\n' + f' |_{mp.current_process()}\n\n' + f'{tree_str}\n' + ) + # import logging + # try: + # with open("/dev/tty", "w") as tty: + # tty.write(tree_str) + # except BaseException: + # logging.getLogger( + # "task_tree" + # ).exception("Error printing task tree") + + +def signal_handler( + sig: int, + frame: object, + + relay_to_subs: bool = True, + +) -> None: + try: + trio.lowlevel.current_trio_token( + ).run_sync_soon(dump_task_tree) + except RuntimeError: + # not in async context -- print a normal traceback + traceback.print_stack() + + if not relay_to_subs: + return + + an: ActorNursery + for an in _state.current_actor()._actoruid2nursery.values(): + + subproc: ProcessType + subactor: Actor + for subactor, subproc, _ in an._children.values(): + log.devx( + f'Relaying `SIGUSR1`[{sig}] to sub-actor\n' + f'{subactor}\n' + f' |_{subproc}\n' + ) + + if isinstance(subproc, trio.Process): + subproc.send_signal(sig) + + elif isinstance(subproc, mp.Process): + subproc._send_signal(sig) + + +def enable_stack_on_sig( + sig: int = SIGUSR1 +) -> None: + ''' + Enable `stackscope` tracing on reception of a signal; by + default this is SIGUSR1. + + ''' + signal( + sig, + signal_handler, + ) + # NOTE: not the above can be triggered from + # a (xonsh) shell using: + # kill -SIGUSR1 @$(pgrep -f '') + # + # for example if you were looking to trace a `pytest` run + # kill -SIGUSR1 @$(pgrep -f 'pytest') diff --git a/tractor/devx/cli.py b/tractor/devx/cli.py new file mode 100644 index 00000000..c44f9686 --- /dev/null +++ b/tractor/devx/cli.py @@ -0,0 +1,129 @@ +# 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 . + +""" +CLI framework extensions for hacking on the actor runtime. + +Currently popular frameworks supported are: + + - `typer` via the `@callback` API + +""" +from __future__ import annotations +from typing import ( + Any, + Callable, +) +from typing_extensions import Annotated + +import typer + + +_runtime_vars: dict[str, Any] = {} + + +def load_runtime_vars( + ctx: typer.Context, + callback: Callable, + pdb: bool = False, # --pdb + ll: Annotated[ + str, + typer.Option( + '--loglevel', + '-l', + help='BigD logging level', + ), + ] = 'cancel', # -l info +): + ''' + Maybe engage crash handling with `pdbp` when code inside + a `typer` CLI endpoint cmd raises. + + To use this callback simply take your `app = typer.Typer()` instance + and decorate this function with it like so: + + .. code:: python + + from tractor.devx import cli + + app = typer.Typer() + + # manual decoration to hook into `click`'s context system! + cli.load_runtime_vars = app.callback( + invoke_without_command=True, + ) + + And then you can use the now augmented `click` CLI context as so, + + .. code:: python + + @app.command( + context_settings={ + "allow_extra_args": True, + "ignore_unknown_options": True, + } + ) + def my_cli_cmd( + ctx: typer.Context, + ): + rtvars: dict = ctx.runtime_vars + pdb: bool = rtvars['pdb'] + + with tractor.devx.cli.maybe_open_crash_handler(pdb=pdb): + trio.run( + partial( + my_tractor_main_task_func, + debug_mode=pdb, + loglevel=rtvars['ll'], + ) + ) + + which will enable log level and debug mode globally for the entire + `tractor` + `trio` runtime thereafter! + + Bo + + ''' + global _runtime_vars + _runtime_vars |= { + 'pdb': pdb, + 'll': ll, + } + + ctx.runtime_vars: dict[str, Any] = _runtime_vars + print( + f'`typer` sub-cmd: {ctx.invoked_subcommand}\n' + f'`tractor` runtime vars: {_runtime_vars}' + ) + + # XXX NOTE XXX: hackzone.. if no sub-cmd is specified (the + # default if the user just invokes `bigd`) then we simply + # invoke the sole `_bigd()` cmd passing in the "parent" + # typer.Context directly to that call since we're treating it + # as a "non sub-command" or wtv.. + # TODO: ideally typer would have some kinda built-in way to get + # this behaviour without having to construct and manually + # invoke our own cmd.. + if ( + ctx.invoked_subcommand is None + or ctx.invoked_subcommand == callback.__name__ + ): + cmd: typer.core.TyperCommand = typer.core.TyperCommand( + name='bigd', + callback=callback, + ) + ctx.params = {'ctx': ctx} + cmd.invoke(ctx) diff --git a/tractor/devx/pformat.py b/tractor/devx/pformat.py new file mode 100644 index 00000000..5fe9bc62 --- /dev/null +++ b/tractor/devx/pformat.py @@ -0,0 +1,168 @@ +# 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 . + +''' +Pretty formatters for use throughout the code base. +Mostly handy for logging and exception message content. + +''' +import textwrap +import traceback + +from trio import CancelScope + + +def add_div( + message: str, + div_str: str = '------ - ------', + +) -> str: + ''' + Add a "divider string" to the input `message` with + a little math to center it underneath. + + ''' + div_offset: int = ( + round(len(message)/2)+1 + - + round(len(div_str)/2)+1 + ) + div_str: str = ( + '\n' + ' '*div_offset + f'{div_str}\n' + ) + return div_str + + +def pformat_boxed_tb( + tb_str: str, + fields_str: str|None = None, + field_prefix: str = ' |_', + + tb_box_indent: int|None = None, + tb_body_indent: int = 1, + +) -> str: + ''' + Create a "boxed" looking traceback string. + + Useful for emphasizing traceback text content as being an + embedded attribute of some other object (like + a `RemoteActorError` or other boxing remote error shuttle + container). + + Any other parent/container "fields" can be passed in the + `fields_str` input along with other prefix/indent settings. + + ''' + if ( + fields_str + and + field_prefix + ): + fields: str = textwrap.indent( + fields_str, + prefix=field_prefix, + ) + else: + fields = fields_str or '' + + tb_body = tb_str + if tb_body_indent: + tb_body: str = textwrap.indent( + tb_str, + prefix=tb_body_indent * ' ', + ) + + tb_box: str = ( + f'|\n' + f' ------ - ------\n' + f'{tb_body}' + f' ------ - ------\n' + f'_|\n' + ) + tb_box_indent: str = ( + tb_box_indent + or + 1 + + # (len(field_prefix)) + # ? ^-TODO-^ ? if you wanted another indent level + ) + if tb_box_indent > 0: + tb_box: str = textwrap.indent( + tb_box, + prefix=tb_box_indent * ' ', + ) + + return ( + fields + + + tb_box + ) + + +def pformat_caller_frame( + stack_limit: int = 1, + box_tb: bool = True, +) -> str: + ''' + Capture and return the traceback text content from + `stack_limit` call frames up. + + ''' + tb_str: str = ( + '\n'.join( + traceback.format_stack(limit=stack_limit) + ) + ) + if box_tb: + tb_str: str = pformat_boxed_tb( + tb_str=tb_str, + field_prefix=' ', + indent='', + ) + return tb_str + + +def pformat_cs( + cs: CancelScope, + var_name: str = 'cs', + field_prefix: str = ' |_', +) -> str: + ''' + Pretty format info about a `trio.CancelScope` including most + of its public state and `._cancel_status`. + + The output can be modified to show a "var name" for the + instance as a field prefix, just a simple str before each + line more or less. + + ''' + + fields: str = textwrap.indent( + ( + f'cancel_called = {cs.cancel_called}\n' + f'cancelled_caught = {cs.cancelled_caught}\n' + f'_cancel_status = {cs._cancel_status}\n' + f'shield = {cs.shield}\n' + ), + prefix=field_prefix, + ) + return ( + f'{var_name}: {cs}\n' + + + fields + ) 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/log.py b/tractor/log.py index 1ea99315..edb058e3 100644 --- a/tractor/log.py +++ b/tractor/log.py @@ -21,6 +21,11 @@ Log like a forester! from collections.abc import Mapping import sys import logging +from logging import ( + LoggerAdapter, + Logger, + StreamHandler, +) import colorlog # type: ignore import trio @@ -48,17 +53,19 @@ LOG_FORMAT = ( DATE_FORMAT = '%b %d %H:%M:%S' -LEVELS = { +# FYI, ERROR is 40 +CUSTOM_LEVELS: dict[str, int] = { 'TRANSPORT': 5, 'RUNTIME': 15, - 'CANCEL': 16, + 'DEVX': 17, + 'CANCEL': 18, 'PDB': 500, } - STD_PALETTE = { 'CRITICAL': 'red', 'ERROR': 'red', 'PDB': 'white', + 'DEVX': 'cyan', 'WARNING': 'yellow', 'INFO': 'green', 'CANCEL': 'yellow', @@ -75,7 +82,7 @@ BOLD_PALETTE = { # TODO: this isn't showing the correct '{filename}' # as it did before.. -class StackLevelAdapter(logging.LoggerAdapter): +class StackLevelAdapter(LoggerAdapter): def transport( self, @@ -83,7 +90,8 @@ class StackLevelAdapter(logging.LoggerAdapter): ) -> None: ''' - IPC level msg-ing. + IPC transport level msg IO; generally anything below + `._ipc.Channel` and friends. ''' return self.log(5, msg) @@ -99,29 +107,65 @@ class StackLevelAdapter(logging.LoggerAdapter): msg: str, ) -> None: ''' - Cancellation logging, mostly for runtime reporting. + Cancellation sequencing, mostly for runtime reporting. ''' - return self.log(16, msg) + return self.log( + level=22, + msg=msg, + # stacklevel=4, + ) def pdb( self, msg: str, ) -> None: ''' - Debugger logging. + `pdb`-REPL (debugger) related statuses. ''' return self.log(500, msg) - def log(self, level, msg, *args, **kwargs): - """ + def devx( + self, + msg: str, + ) -> None: + ''' + "Developer experience" sub-sys statuses. + + ''' + return self.log(17, msg) + + 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 CUSTOM_LEVELS.values() + ): + 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 +178,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, @@ -151,8 +198,30 @@ class StackLevelAdapter(logging.LoggerAdapter): ) +# TODO IDEAs: +# -[ ] move to `.devx.pformat`? +# -[ ] do per task-name and actor-name color coding +# -[ ] unique color per task-id and actor-uuid +def pformat_task_uid( + id_part: str = 'tail' +): + ''' + Return `str`-ified unique for a `trio.Task` via a combo of its + `.name: str` and `id()` truncated output. + + ''' + task: trio.Task = trio.lowlevel.current_task() + tid: str = str(id(task)) + if id_part == 'tail': + tid_part: str = tid[-6:] + else: + tid_part: str = tid[:6] + + return f'{task.name}[{tid_part}]' + + _conc_name_getters = { - 'task': lambda: trio.lowlevel.current_task().name, + 'task': pformat_task_uid, 'actor': lambda: current_actor(), 'actor_name': lambda: current_actor().name, 'actor_uid': lambda: current_actor().uid[1][:6], @@ -160,7 +229,10 @@ _conc_name_getters = { class ActorContextInfo(Mapping): - "Dyanmic lookup for local actor and task names" + ''' + Dyanmic lookup for local actor and task names. + + ''' _context_keys = ( 'task', 'actor', @@ -191,17 +263,42 @@ def get_logger( '''Return the package log or a sub-logger for ``name`` if provided. ''' + log: 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 @@ -209,7 +306,7 @@ def get_logger( logger = StackLevelAdapter(log, ActorContextInfo()) # additional levels - for name, val in LEVELS.items(): + for name, val in CUSTOM_LEVELS.items(): logging.addLevelName(val, name) # ensure customs levels exist as methods @@ -221,7 +318,7 @@ def get_logger( def get_console_log( level: str | None = None, **kwargs, -) -> logging.LoggerAdapter: +) -> LoggerAdapter: '''Get the package logger and enable a handler which writes to stderr. Yeah yeah, i know we can use ``DictConfig``. You do it. @@ -232,13 +329,21 @@ 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() + handler = StreamHandler() formatter = colorlog.ColoredFormatter( LOG_FORMAT, datefmt=DATE_FORMAT, @@ -254,3 +359,23 @@ def get_console_log( def get_loglevel() -> str: return _default_loglevel + + +# global module logger for tractor itself +log = get_logger('tractor') + + +def at_least_level( + log: Logger|LoggerAdapter, + level: int|str, +) -> bool: + ''' + Predicate to test if a given level is active. + + ''' + if isinstance(level, str): + level: int = CUSTOM_LEVELS[level.upper()] + + if log.getEffectiveLevel() <= level: + return True + return False diff --git a/tractor/msg.py b/tractor/msg.py deleted file mode 100644 index 9af3ccd7..00000000 --- a/tractor/msg.py +++ /dev/null @@ -1,80 +0,0 @@ -# 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. - -''' - -# TODO: integration with our ``enable_modules: list[str]`` caps sys. - -# ``pkgutil.resolve_name()`` internally uses -# ``importlib.import_module()`` which can be filtered by inserting -# a ``MetaPathFinder`` into ``sys.meta_path`` (which we could do before -# entering the ``_runtime.process_messages()`` loop). -# - https://github.com/python/cpython/blob/main/Lib/pkgutil.py#L645 -# - https://stackoverflow.com/questions/1350466/preventing-python-code-from-importing-certain-modules -# - https://stackoverflow.com/a/63320902 -# - https://docs.python.org/3/library/sys.html#sys.meta_path - -# the new "Implicit Namespace Packages" might be relevant? -# - https://www.python.org/dev/peps/pep-0420/ - -# add implicit serialized message type support so that paths can be -# handed directly to IPC primitives such as streams and `Portal.run()` -# calls: -# - via ``msgspec``: -# - https://jcristharif.com/msgspec/api.html#struct -# - https://jcristharif.com/msgspec/extending.html -# via ``msgpack-python``: -# - https://github.com/msgpack/msgpack-python#packingunpacking-of-custom-data-type - -from __future__ import annotations -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. - - ''' - _ref: object = None - - def load_ref(self) -> object: - if self._ref is None: - self._ref = resolve_name(self) - return self._ref - - def to_tuple( - self, - - ) -> tuple[str, str]: - ref = self.load_ref() - return ref.__module__, getattr(ref, '__name__', '') - - @classmethod - def from_ref( - cls, - ref, - - ) -> NamespacePath: - return cls(':'.join( - (ref.__module__, - getattr(ref, '__name__', '')) - )) diff --git a/tractor/msg/__init__.py b/tractor/msg/__init__.py new file mode 100644 index 00000000..44586f2d --- /dev/null +++ b/tractor/msg/__init__.py @@ -0,0 +1,73 @@ +# 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 typing import ( + TypeAlias, +) +from .ptr import ( + NamespacePath as NamespacePath, +) +from .pretty_struct import ( + Struct as Struct, +) +from ._codec import ( + _def_msgspec_codec as _def_msgspec_codec, + _ctxvar_MsgCodec as _ctxvar_MsgCodec, + + apply_codec as apply_codec, + mk_codec as mk_codec, + MsgCodec as MsgCodec, + MsgDec as MsgDec, + current_codec as current_codec, +) +# currently can't bc circular with `._context` +# from ._ops import ( +# PldRx as PldRx, +# _drain_to_final_msg as _drain_to_final_msg, +# ) + +from .types import ( + PayloadMsg as PayloadMsg, + + Aid as Aid, + SpawnSpec as SpawnSpec, + + Start as Start, + StartAck as StartAck, + + Started as Started, + Yield as Yield, + Stop as Stop, + Return as Return, + CancelAck as CancelAck, + + Error as Error, + + # type-var for `.pld` field + PayloadT as PayloadT, + + # full msg class set from above as list + __msg_types__ as __msg_types__, + + # type-alias for union of all msgs + MsgType as MsgType, +) + +__msg_spec__: TypeAlias = MsgType diff --git a/tractor/msg/_codec.py b/tractor/msg/_codec.py new file mode 100644 index 00000000..c1301bd2 --- /dev/null +++ b/tractor/msg/_codec.py @@ -0,0 +1,660 @@ +# 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 . + +''' +IPC msg interchange codec management. + +Supported backend libs: +- `msgspec.msgpack` + +ToDo: backends we prolly should offer: + +- see project/lib list throughout GH issue discussion comments: + https://github.com/goodboy/tractor/issues/196 + +- `capnproto`: https://capnproto.org/rpc.html + - https://capnproto.org/language.html#language-reference + +''' +from __future__ import annotations +from contextlib import ( + contextmanager as cm, +) +from contextvars import ( + ContextVar, + Token, +) +import textwrap +from typing import ( + Any, + Callable, + Type, + TYPE_CHECKING, + Union, +) +from types import ModuleType + +import msgspec +from msgspec import ( + msgpack, + Raw, +) +# TODO: see notes below from @mikenerone.. +# from tricycle import TreeVar + +from tractor.msg.pretty_struct import Struct +from tractor.msg.types import ( + mk_msg_spec, + MsgType, +) +from tractor.log import get_logger + +if TYPE_CHECKING: + from tractor._context import Context + +log = get_logger(__name__) + + +# TODO: unify with `MsgCodec` by making `._dec` part this? +class MsgDec(Struct): + ''' + An IPC msg (payload) decoder. + + Normally used to decode only a payload: `MsgType.pld: + PayloadT` field before delivery to IPC consumer code. + + ''' + _dec: msgpack.Decoder + + @property + def dec(self) -> msgpack.Decoder: + return self._dec + + def __repr__(self) -> str: + + speclines: str = self.spec_str + + # in multi-typed spec case we stick the list + # all on newlines after the |__pld_spec__:, + # OW it's prolly single type spec-value + # so just leave it on same line. + if '\n' in speclines: + speclines: str = '\n' + textwrap.indent( + speclines, + prefix=' '*3, + ) + + body: str = textwrap.indent( + f'|_dec_hook: {self.dec.dec_hook}\n' + f'|__pld_spec__: {speclines}\n', + prefix=' '*2, + ) + return ( + f'<{type(self).__name__}(\n' + f'{body}' + ')>' + ) + + # struct type unions + # https://jcristharif.com/msgspec/structs.html#tagged-unions + # + # ^-TODO-^: make a wrapper type for this such that alt + # backends can be represented easily without a `Union` needed, + # AND so that we have better support for wire transport. + # + # -[ ] maybe `FieldSpec` is a good name since msg-spec + # better applies to a `MsgType[FieldSpec]`? + # + # -[ ] both as part of the `.open_context()` call AND as part of the + # immediate ack-reponse (see similar below) + # we should do spec matching and fail if anything is awry? + # + # -[ ] eventually spec should be generated/parsed from the + # type-annots as # desired in GH issue: + # https://github.com/goodboy/tractor/issues/365 + # + # -[ ] semantics of the mismatch case + # - when caller-callee specs we should raise + # a `MsgTypeError` or `MsgSpecError` or similar? + # + # -[ ] wrapper types for both spec types such that we can easily + # IPC transport them? + # - `TypeSpec: Union[Type]` + # * also a `.__contains__()` for doing `None in + # TypeSpec[None|int]` since rn you need to do it on + # `.__args__` for unions.. + # - `MsgSpec: Union[MsgType] + # + # -[ ] auto-genning this from new (in 3.12) type parameter lists Bo + # |_ https://docs.python.org/3/reference/compound_stmts.html#type-params + # |_ historical pep 695: https://peps.python.org/pep-0695/ + # |_ full lang spec: https://typing.readthedocs.io/en/latest/spec/ + # |_ on annotation scopes: + # https://docs.python.org/3/reference/executionmodel.html#annotation-scopes + # |_ 3.13 will have subscriptable funcs Bo + # https://peps.python.org/pep-0718/ + @property + def spec(self) -> Union[Type[Struct]]: + # NOTE: defined and applied inside `mk_codec()` + return self._dec.type + + # no difference, as compared to a `MsgCodec` which defines the + # `MsgType.pld: PayloadT` part of its spec separately + pld_spec = spec + + # TODO: would get moved into `FieldSpec.__str__()` right? + @property + def spec_str(self) -> str: + return pformat_msgspec( + codec=self, + join_char='|', + ) + + pld_spec_str = spec_str + + def decode( + self, + raw: Raw|bytes, + ) -> Any: + return self._dec.decode(raw) + + @property + def hook(self) -> Callable|None: + return self._dec.dec_hook + + +def mk_dec( + spec: Union[Type[Struct]]|Any = Any, + dec_hook: Callable|None = None, + +) -> MsgDec: + + return MsgDec( + _dec=msgpack.Decoder( + type=spec, # like `MsgType[Any]` + dec_hook=dec_hook, + ) + ) + + +def mk_msgspec_table( + dec: msgpack.Decoder, + msg: MsgType|None = None, + +) -> dict[str, MsgType]|str: + ''' + Fill out a `dict` of `MsgType`s keyed by name + for a given input `msgspec.msgpack.Decoder` + as defined by its `.type: Union[Type]` setting. + + If `msg` is provided, only deliver a `dict` with a single + entry for that type. + + ''' + msgspec: Union[Type]|Type = dec.type + + if not (msgtypes := getattr(msgspec, '__args__', False)): + msgtypes = [msgspec] + + msgt_table: dict[str, MsgType] = { + msgt: str(msgt.__name__) + for msgt in msgtypes + } + if msg: + msgt: MsgType = type(msg) + str_repr: str = msgt_table[msgt] + return {msgt: str_repr} + + return msgt_table + + +def pformat_msgspec( + codec: MsgCodec|MsgDec, + msg: MsgType|None = None, + join_char: str = '\n', + +) -> str: + dec: msgpack.Decoder = getattr(codec, 'dec', codec) + return join_char.join( + mk_msgspec_table( + dec=dec, + msg=msg, + ).values() + ) + +# TODO: overall IPC msg-spec features (i.e. in this mod)! +# +# -[ ] API changes towards being interchange lib agnostic! +# -[ ] capnproto has pre-compiled schema for eg.. +# * https://capnproto.org/language.html +# * http://capnproto.github.io/pycapnp/quickstart.html +# * https://github.com/capnproto/pycapnp/blob/master/examples/addressbook.capnp +# +# -[ ] struct aware messaging coders as per: +# -[x] https://github.com/goodboy/tractor/issues/36 +# -[ ] https://github.com/goodboy/tractor/issues/196 +# -[ ] https://github.com/goodboy/tractor/issues/365 +# +class MsgCodec(Struct): + ''' + A IPC msg interchange format lib's encoder + decoder pair. + + Pretty much nothing more then delegation to underlying + `msgspec..Encoder/Decoder`s for now. + + ''' + _enc: msgpack.Encoder + _dec: msgpack.Decoder + _pld_spec: Type[Struct]|Raw|Any + + def __repr__(self) -> str: + speclines: str = textwrap.indent( + pformat_msgspec(codec=self), + prefix=' '*3, + ) + body: str = textwrap.indent( + f'|_lib = {self.lib.__name__!r}\n' + f'|_enc_hook: {self.enc.enc_hook}\n' + f'|_dec_hook: {self.dec.dec_hook}\n' + f'|_pld_spec: {self.pld_spec_str}\n' + # f'|\n' + f'|__msg_spec__:\n' + f'{speclines}\n', + prefix=' '*2, + ) + return ( + f'<{type(self).__name__}(\n' + f'{body}' + ')>' + ) + + @property + def pld_spec(self) -> Type[Struct]|Raw|Any: + return self._pld_spec + + @property + def pld_spec_str(self) -> str: + + # TODO: could also use match: instead? + spec: Union[Type]|Type = self.pld_spec + + # `typing.Union` case + if getattr(spec, '__args__', False): + return str(spec) + + # just a single type + else: + return spec.__name__ + + # struct type unions + # https://jcristharif.com/msgspec/structs.html#tagged-unions + @property + def msg_spec(self) -> Union[Type[Struct]]: + # NOTE: defined and applied inside `mk_codec()` + return self._dec.type + + # TODO: some way to make `pretty_struct.Struct` use this + # wrapped field over the `.msg_spec` one? + @property + def msg_spec_str(self) -> str: + return pformat_msgspec(self.msg_spec) + + lib: ModuleType = msgspec + + # TODO: use `functools.cached_property` for these ? + # https://docs.python.org/3/library/functools.html#functools.cached_property + @property + def enc(self) -> msgpack.Encoder: + return self._enc + + # TODO: reusing encode buffer for perf? + # https://jcristharif.com/msgspec/perf-tips.html#reusing-an-output-buffer + _buf: bytearray = bytearray() + + def encode( + self, + py_obj: Any, + + use_buf: bool = False, + # ^-XXX-^ uhh why am i getting this? + # |_BufferError: Existing exports of data: object cannot be re-sized + + ) -> bytes: + ''' + Encode input python objects to `msgpack` bytes for + transfer on a tranport protocol connection. + + When `use_buf == True` use the output buffer optimization: + https://jcristharif.com/msgspec/perf-tips.html#reusing-an-output-buffer + + ''' + if use_buf: + self._enc.encode_into(py_obj, self._buf) + return self._buf + else: + return self._enc.encode(py_obj) + + @property + def dec(self) -> msgpack.Decoder: + return self._dec + + def decode( + self, + msg: bytes, + ) -> Any: + ''' + Decode received `msgpack` bytes into a local python object + with special `msgspec.Struct` (or other type) handling + determined by the + + ''' + # https://jcristharif.com/msgspec/usage.html#typed-decoding + return self._dec.decode(msg) + + +# [x] TODO: a sub-decoder system as well? => No! +# +# -[x] do we still want to try and support the sub-decoder with +# `.Raw` technique in the case that the `Generic` approach gives +# future grief? +# => NO, since we went with the `PldRx` approach instead B) +# +# IF however you want to see the code that was staged for this +# from wayyy back, see the pure removal commit. + + +def mk_codec( + # struct type unions set for `Decoder` + # https://jcristharif.com/msgspec/structs.html#tagged-unions + ipc_pld_spec: Union[Type[Struct]]|Any = Any, + + # TODO: offering a per-msg(-field) type-spec such that + # the fields can be dynamically NOT decoded and left as `Raw` + # values which are later loaded by a sub-decoder specified + # by `tag_field: str` value key? + # payload_msg_specs: dict[ + # str, # tag_field value as sub-decoder key + # Union[Type[Struct]] # `MsgType.pld` type spec + # ]|None = None, + + libname: str = 'msgspec', + + # proxy as `Struct(**kwargs)` for ad-hoc type extensions + # https://jcristharif.com/msgspec/extending.html#mapping-to-from-native-types + # ------ - ------ + dec_hook: Callable|None = None, + enc_hook: Callable|None = None, + # ------ - ------ + # + # Encoder: + # write_buffer_size=write_buffer_size, + # + # Decoder: + # ext_hook: ext_hook_sig + +) -> MsgCodec: + ''' + Convenience factory for creating codecs eventually meant + to be interchange lib agnostic (i.e. once we support more then just + `msgspec` ;). + + ''' + # (manually) generate a msg-payload-spec for all relevant + # god-boxing-msg subtypes, parameterizing the `PayloadMsg.pld: PayloadT` + # for the decoder such that all sub-type msgs in our SCIPP + # will automatically decode to a type-"limited" payload (`Struct`) + # object (set). + ( + ipc_msg_spec, + msg_types, + ) = mk_msg_spec( + payload_type_union=ipc_pld_spec, + ) + assert len(ipc_msg_spec.__args__) == len(msg_types) + assert ipc_msg_spec + + # TODO: use this shim instead? + # bc.. unification, err somethin? + # dec: MsgDec = mk_dec( + # spec=ipc_msg_spec, + # dec_hook=dec_hook, + # ) + + dec = msgpack.Decoder( + type=ipc_msg_spec, + dec_hook=dec_hook, + ) + enc = msgpack.Encoder( + enc_hook=enc_hook, + ) + + codec = MsgCodec( + _enc=enc, + _dec=dec, + _pld_spec=ipc_pld_spec, + ) + + # sanity on expected backend support + assert codec.lib.__name__ == libname + + return codec + + +# instance of the default `msgspec.msgpack` codec settings, i.e. +# no custom structs, hooks or other special types. +_def_msgspec_codec: MsgCodec = mk_codec(ipc_pld_spec=Any) + +# The built-in IPC `Msg` spec. +# Our composing "shuttle" protocol which allows `tractor`-app code +# to use any `msgspec` supported type as the `PayloadMsg.pld` payload, +# https://jcristharif.com/msgspec/supported-types.html +# +_def_tractor_codec: MsgCodec = mk_codec( + # TODO: use this for debug mode locking prot? + # ipc_pld_spec=Any, + ipc_pld_spec=Raw, +) +# TODO: IDEALLY provides for per-`trio.Task` specificity of the +# IPC msging codec used by the transport layer when doing +# `Channel.send()/.recv()` of wire data. + +# ContextVar-TODO: DIDN'T WORK, kept resetting in every new task to default!? +# _ctxvar_MsgCodec: ContextVar[MsgCodec] = ContextVar( + +# TreeVar-TODO: DIDN'T WORK, kept resetting in every new embedded nursery +# even though it's supposed to inherit from a parent context ??? +# +# _ctxvar_MsgCodec: TreeVar[MsgCodec] = TreeVar( +# +# ^-NOTE-^: for this to work see the mods by @mikenerone from `trio` gitter: +# +# 22:02:54 even for regular contextvars, all you have to do is: +# `task: Task = trio.lowlevel.current_task()` +# `task.parent_nursery.parent_task.context.run(my_ctx_var.set, new_value)` +# +# From a comment in his prop code he couldn't share outright: +# 1. For every TreeVar set in the current task (which covers what +# we need from SynchronizerFacade), walk up the tree until the +# root or finding one where the TreeVar is already set, setting +# it in all of the contexts along the way. +# 2. For each of those, we also forcibly set the values that are +# pending for child nurseries that have not yet accessed the +# TreeVar. +# 3. We similarly set the pending values for the child nurseries +# of the *current* task. +# +_ctxvar_MsgCodec: ContextVar[MsgCodec] = ContextVar( + 'msgspec_codec', + default=_def_tractor_codec, +) + + +@cm +def apply_codec( + codec: MsgCodec, + + ctx: Context|None = None, + +) -> MsgCodec: + ''' + Dynamically apply a `MsgCodec` to the current task's runtime + context such that all (of a certain class of payload + containing i.e. `MsgType.pld: PayloadT`) IPC msgs are + processed with it for that task. + + Uses a `contextvars.ContextVar` to ensure the scope of any + codec setting matches the current `Context` or + `._rpc.process_messages()` feeder task's prior setting without + mutating any surrounding scope. + + When a `ctx` is supplied, only mod its `Context.pld_codec`. + + matches the `@cm` block and DOES NOT change to the original + (default) value in new tasks (as it does for `ContextVar`). + + ''' + __tracebackhide__: bool = True + + if ctx is not None: + var: ContextVar = ctx._var_pld_codec + else: + # use IPC channel-connection "global" codec + var: ContextVar = _ctxvar_MsgCodec + + orig: MsgCodec = var.get() + + assert orig is not codec + if codec.pld_spec is None: + breakpoint() + + log.info( + 'Applying new msg-spec codec\n\n' + f'{codec}\n' + ) + token: Token = var.set(codec) + + # ?TODO? for TreeVar approach which copies from the + # cancel-scope of the prior value, NOT the prior task + # See the docs: + # - https://tricycle.readthedocs.io/en/latest/reference.html#tree-variables + # - https://github.com/oremanj/tricycle/blob/master/tricycle/_tests/test_tree_var.py + # ^- see docs for @cm `.being()` API + # with _ctxvar_MsgCodec.being(codec): + # new = _ctxvar_MsgCodec.get() + # assert new is codec + # yield codec + + try: + yield var.get() + finally: + var.reset(token) + log.info( + 'Reverted to last msg-spec codec\n\n' + f'{orig}\n' + ) + assert var.get() is orig + + +def current_codec() -> MsgCodec: + ''' + Return the current `trio.Task.context`'s value + for `msgspec_codec` used by `Channel.send/.recv()` + for wire serialization. + + ''' + return _ctxvar_MsgCodec.get() + + +@cm +def limit_msg_spec( + payload_spec: Union[Type[Struct]], + + # TODO: don't need this approach right? + # -> related to the `MsgCodec._payload_decs` stuff above.. + # tagged_structs: list[Struct]|None = None, + + **codec_kwargs, + +) -> MsgCodec: + ''' + Apply a `MsgCodec` that will natively decode the SC-msg set's + `PayloadMsg.pld: Union[Type[Struct]]` payload fields using + tagged-unions of `msgspec.Struct`s from the `payload_types` + for all IPC contexts in use by the current `trio.Task`. + + ''' + __tracebackhide__: bool = True + curr_codec: MsgCodec = current_codec() + msgspec_codec: MsgCodec = mk_codec( + ipc_pld_spec=payload_spec, + **codec_kwargs, + ) + with apply_codec(msgspec_codec) as applied_codec: + assert applied_codec is msgspec_codec + yield msgspec_codec + + assert curr_codec is current_codec() + + +# XXX: msgspec won't allow this with non-struct custom types +# like `NamespacePath`!@! +# @cm +# def extend_msg_spec( +# payload_spec: Union[Type[Struct]], + +# ) -> MsgCodec: +# ''' +# Extend the current `MsgCodec.pld_spec` (type set) by extending +# the payload spec to **include** the types specified by +# `payload_spec`. + +# ''' +# codec: MsgCodec = current_codec() +# pld_spec: Union[Type] = codec.pld_spec +# extended_spec: Union[Type] = pld_spec|payload_spec + +# with limit_msg_spec(payload_types=extended_spec) as ext_codec: +# # import pdbp; pdbp.set_trace() +# assert ext_codec.pld_spec == extended_spec +# yield ext_codec + + +# TODO: make something similar to this inside `._codec` such that +# user can just pass a type table of some sort? +# -[ ] we would need to decode all msgs to `pretty_struct.Struct` +# and then call `.to_dict()` on them? +# -[x] we're going to need to re-impl all the stuff changed in the +# runtime port such that it can handle dicts or `Msg`s? +# +# def mk_dict_msg_codec_hooks() -> tuple[Callable, Callable]: +# ''' +# Deliver a `enc_hook()`/`dec_hook()` pair which does +# manual convertion from our above native `Msg` set +# to `dict` equivalent (wire msgs) in order to keep legacy compat +# with the original runtime implementation. +# +# Note: this is is/was primarly used while moving the core +# runtime over to using native `Msg`-struct types wherein we +# start with the send side emitting without loading +# a typed-decoder and then later flipping the switch over to +# load to the native struct types once all runtime usage has +# been adjusted appropriately. +# +# ''' +# return ( +# # enc_to_dict, +# dec_from_dict, +# ) diff --git a/tractor/msg/_ops.py b/tractor/msg/_ops.py new file mode 100644 index 00000000..91c0ddea --- /dev/null +++ b/tractor/msg/_ops.py @@ -0,0 +1,804 @@ +# 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 . + +''' +Near-application abstractions for `MsgType.pld: PayloadT|Raw` +delivery, filtering and type checking as well as generic +operational helpers for processing transaction flows. + +''' +from __future__ import annotations +from contextlib import ( + asynccontextmanager as acm, + contextmanager as cm, +) +from typing import ( + Any, + Callable, + Type, + TYPE_CHECKING, + Union, +) +# ------ - ------ +from msgspec import ( + msgpack, + Raw, + Struct, + ValidationError, +) +import trio +# ------ - ------ +from tractor.log import get_logger +from tractor._exceptions import ( + MessagingError, + InternalError, + _raise_from_unexpected_msg, + MsgTypeError, + _mk_recv_mte, + pack_error, +) +from tractor._state import current_ipc_ctx +from ._codec import ( + mk_dec, + MsgDec, + MsgCodec, + current_codec, +) +from .types import ( + CancelAck, + Error, + MsgType, + PayloadT, + Return, + Started, + Stop, + Yield, + pretty_struct, +) + + +if TYPE_CHECKING: + from tractor._context import Context + from tractor._streaming import MsgStream + + +log = get_logger(__name__) + + +_def_any_pldec: MsgDec[Any] = mk_dec() + + +class PldRx(Struct): + ''' + A "msg payload receiver". + + The pairing of a "feeder" `trio.abc.ReceiveChannel` and an + interchange-specific (eg. msgpack) payload field decoder. The + validation/type-filtering rules are runtime mutable and allow + type constraining the set of `MsgType.pld: Raw|PayloadT` + values at runtime, per IPC task-context. + + This abstraction, being just below "user application code", + allows for the equivalent of our `MsgCodec` (used for + typer-filtering IPC dialog protocol msgs against a msg-spec) + but with granular control around payload delivery (i.e. the + data-values user code actually sees and uses (the blobs that + are "shuttled" by the wrapping dialog prot) such that invalid + `.pld: Raw` can be decoded and handled by IPC-primitive user + code (i.e. that operates on `Context` and `Msgstream` APIs) + without knowledge of the lower level `Channel`/`MsgTransport` + primitives nor the `MsgCodec` in use. Further, lazily decoding + payload blobs allows for topical (and maybe intentionally + "partial") encryption of msg field subsets. + + ''' + # TODO: better to bind it here? + # _rx_mc: trio.MemoryReceiveChannel + _pld_dec: MsgDec + _ctx: Context|None = None + _ipc: Context|MsgStream|None = None + + @property + def pld_dec(self) -> MsgDec: + return self._pld_dec + + # TODO: a better name? + # -[ ] when would this be used as it avoids needingn to pass the + # ipc prim to every method + @cm + def wraps_ipc( + self, + ipc_prim: Context|MsgStream, + + ) -> PldRx: + ''' + Apply this payload receiver to an IPC primitive type, one + of `Context` or `MsgStream`. + + ''' + self._ipc = ipc_prim + try: + yield self + finally: + self._ipc = None + + @cm + def limit_plds( + self, + spec: Union[Type[Struct]], + **dec_kwargs, + + ) -> MsgDec: + ''' + Type-limit the loadable msg payloads via an applied + `MsgDec` given an input spec, revert to prior decoder on + exit. + + ''' + orig_dec: MsgDec = self._pld_dec + limit_dec: MsgDec = mk_dec( + spec=spec, + **dec_kwargs, + ) + try: + self._pld_dec = limit_dec + yield limit_dec + finally: + self._pld_dec = orig_dec + + @property + def dec(self) -> msgpack.Decoder: + return self._pld_dec.dec + + def recv_pld_nowait( + self, + # TODO: make this `MsgStream` compat as well, see above^ + # ipc_prim: Context|MsgStream, + ipc: Context|MsgStream, + + ipc_msg: MsgType|None = None, + expect_msg: Type[MsgType]|None = None, + hide_tb: bool = False, + **dec_pld_kwargs, + + ) -> Any|Raw: + __tracebackhide__: bool = hide_tb + + msg: MsgType = ( + ipc_msg + or + + # sync-rx msg from underlying IPC feeder (mem-)chan + ipc._rx_chan.receive_nowait() + ) + return self.decode_pld( + msg, + ipc=ipc, + expect_msg=expect_msg, + hide_tb=hide_tb, + **dec_pld_kwargs, + ) + + async def recv_pld( + self, + ipc: Context|MsgStream, + ipc_msg: MsgType|None = None, + expect_msg: Type[MsgType]|None = None, + hide_tb: bool = True, + + **dec_pld_kwargs, + + ) -> Any|Raw: + ''' + Receive a `MsgType`, then decode and return its `.pld` field. + + ''' + __tracebackhide__: bool = hide_tb + msg: MsgType = ( + ipc_msg + or + # async-rx msg from underlying IPC feeder (mem-)chan + await ipc._rx_chan.receive() + ) + return self.decode_pld( + msg=msg, + ipc=ipc, + expect_msg=expect_msg, + **dec_pld_kwargs, + ) + + def decode_pld( + self, + msg: MsgType, + ipc: Context|MsgStream, + expect_msg: Type[MsgType]|None, + + raise_error: bool = True, + hide_tb: bool = True, + + # XXX for special (default?) case of send side call with + # `Context.started(validate_pld_spec=True)` + is_started_send_side: bool = False, + + ) -> PayloadT|Raw: + ''' + Decode a msg's payload field: `MsgType.pld: PayloadT|Raw` and + return the value or raise an appropriate error. + + ''' + __tracebackhide__: bool = hide_tb + src_err: BaseException|None = None + match msg: + # payload-data shuttle msg; deliver the `.pld` value + # directly to IPC (primitive) client-consumer code. + case ( + Started(pld=pld) # sync phase + |Yield(pld=pld) # streaming phase + |Return(pld=pld) # termination phase + ): + try: + pld: PayloadT = self._pld_dec.decode(pld) + log.runtime( + 'Decoded msg payload\n\n' + f'{msg}\n' + f'where payload decoded as\n' + f'|_pld={pld!r}\n' + ) + return pld + + # XXX pld-value type failure + except ValidationError as valerr: + # pack mgterr into error-msg for + # reraise below; ensure remote-actor-err + # info is displayed nicely? + mte: MsgTypeError = _mk_recv_mte( + msg=msg, + codec=self.pld_dec, + src_validation_error=valerr, + is_invalid_payload=True, + expected_msg=expect_msg, + ) + # NOTE: just raise the MTE inline instead of all + # the pack-unpack-repack non-sense when this is + # a "send side" validation error. + if is_started_send_side: + raise mte + + # NOTE: the `.message` is automatically + # transferred into the message as long as we + # define it as a `Error.message` field. + err_msg: Error = pack_error( + exc=mte, + cid=msg.cid, + src_uid=( + ipc.chan.uid + if not is_started_send_side + else ipc._actor.uid + ), + ) + mte._ipc_msg = err_msg + + # XXX override the `msg` passed to + # `_raise_from_unexpected_msg()` (below) so so + # that we're effectively able to use that same + # func to unpack and raise an "emulated remote + # `Error`" of this local MTE. + msg = err_msg + # XXX NOTE: so when the `_raise_from_unexpected_msg()` + # raises the boxed `err_msg` from above it raises + # it from the above caught interchange-lib + # validation error. + src_err = valerr + + # a runtime-internal RPC endpoint response. + # always passthrough since (internal) runtime + # responses are generally never exposed to consumer + # code. + case CancelAck( + pld=bool(cancelled) + ): + return cancelled + + case Error(): + src_err = MessagingError( + 'IPC ctx dialog terminated without `Return`-ing a result\n' + f'Instead it raised {msg.boxed_type_str!r}!' + ) + # XXX NOTE XXX another super subtle runtime-y thing.. + # + # - when user code (transitively) calls into this + # func (usually via a `Context/MsgStream` API) we + # generally want errors to propagate immediately + # and directly so that the user can define how it + # wants to handle them. + # + # HOWEVER, + # + # - for certain runtime calling cases, we don't want to + # directly raise since the calling code might have + # special logic around whether to raise the error + # or supress it silently (eg. a `ContextCancelled` + # received from the far end which was requested by + # this side, aka a self-cancel). + # + # SO, we offer a flag to control this. + if not raise_error: + return src_err + + case Stop(cid=cid): + ctx: Context = getattr(ipc, 'ctx', ipc) + message: str = ( + f'{ctx.side!r}-side of ctx received stream-`Stop` from ' + f'{ctx.peer_side!r} peer ?\n' + f'|_cid: {cid}\n\n' + + f'{pretty_struct.pformat(msg)}\n' + ) + if ctx._stream is None: + explain: str = ( + f'BUT, no `MsgStream` (was) open(ed) on this ' + f'{ctx.side!r}-side of the IPC ctx?\n' + f'Maybe check your code for streaming phase race conditions?\n' + ) + log.warning( + message + + + explain + ) + # let caller decide what to do when only one + # side opened a stream, don't raise. + return msg + + else: + explain: str = ( + 'Received a `Stop` when it should NEVER be possible!?!?\n' + ) + # TODO: this is constructed inside + # `_raise_from_unexpected_msg()` but maybe we + # should pass it in? + # src_err = trio.EndOfChannel(explain) + src_err = None + + case _: + src_err = InternalError( + 'Unknown IPC msg ??\n\n' + f'{msg}\n' + ) + + # TODO: maybe use the new `.add_note()` from 3.11? + # |_https://docs.python.org/3.11/library/exceptions.html#BaseException.add_note + # + # fallthrough and raise from `src_err` + try: + _raise_from_unexpected_msg( + ctx=getattr(ipc, 'ctx', ipc), + msg=msg, + src_err=src_err, + log=log, + expect_msg=expect_msg, + hide_tb=hide_tb, + ) + except UnboundLocalError: + # XXX if there's an internal lookup error in the above + # code (prolly on `src_err`) we want to show this frame + # in the tb! + __tracebackhide__: bool = False + raise + + dec_msg = decode_pld + + async def recv_msg_w_pld( + self, + ipc: Context|MsgStream, + expect_msg: MsgType, + + # NOTE: generally speaking only for handling `Stop`-msgs that + # arrive during a call to `drain_to_final_msg()` above! + passthrough_non_pld_msgs: bool = True, + hide_tb: bool = True, + **kwargs, + + ) -> tuple[MsgType, PayloadT]: + ''' + Retrieve the next avail IPC msg, decode it's payload, and return + the pair of refs. + + ''' + __tracebackhide__: bool = hide_tb + msg: MsgType = await ipc._rx_chan.receive() + + if passthrough_non_pld_msgs: + match msg: + case Stop(): + return msg, None + + # TODO: is there some way we can inject the decoded + # payload into an existing output buffer for the original + # msg instance? + pld: PayloadT = self.decode_pld( + msg, + ipc=ipc, + expect_msg=expect_msg, + hide_tb=hide_tb, + **kwargs, + ) + return msg, pld + + +@cm +def limit_plds( + spec: Union[Type[Struct]], + **dec_kwargs, + +) -> MsgDec: + ''' + Apply a `MsgCodec` that will natively decode the SC-msg set's + `PayloadMsg.pld: Union[Type[Struct]]` payload fields using + tagged-unions of `msgspec.Struct`s from the `payload_types` + for all IPC contexts in use by the current `trio.Task`. + + ''' + __tracebackhide__: bool = True + try: + curr_ctx: Context = current_ipc_ctx() + rx: PldRx = curr_ctx._pld_rx + orig_pldec: MsgDec = rx.pld_dec + + with rx.limit_plds( + spec=spec, + **dec_kwargs, + ) as pldec: + log.runtime( + 'Applying payload-decoder\n\n' + f'{pldec}\n' + ) + yield pldec + finally: + log.runtime( + 'Reverted to previous payload-decoder\n\n' + f'{orig_pldec}\n' + ) + # sanity on orig settings + assert rx.pld_dec is orig_pldec + + +@acm +async def maybe_limit_plds( + ctx: Context, + spec: Union[Type[Struct]]|None = None, + dec_hook: Callable|None = None, + **kwargs, + +) -> MsgDec|None: + ''' + Async compat maybe-payload type limiter. + + Mostly for use inside other internal `@acm`s such that a separate + indent block isn't needed when an async one is already being + used. + + ''' + if ( + spec is None + and + dec_hook is None + ): + yield None + return + + # sanity on scoping + curr_ctx: Context = current_ipc_ctx() + assert ctx is curr_ctx + + with ctx._pld_rx.limit_plds( + spec=spec, + dec_hook=dec_hook, + **kwargs, + ) as msgdec: + yield msgdec + + curr_ctx: Context = current_ipc_ctx() + assert ctx is curr_ctx + + +async def drain_to_final_msg( + ctx: Context, + + hide_tb: bool = True, + msg_limit: int = 6, + +) -> tuple[ + Return|None, + list[MsgType] +]: + ''' + Drain IPC msgs delivered to the underlying IPC primitive's + rx-mem-chan (eg. `Context._rx_chan`) from the runtime in + search for a final result or error. + + 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. + + ''' + __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[MsgType] = [] + result_msg: Return|Error|None = None + while not ( + ctx.maybe_error + and not ctx._final_result_is_set() + ): + try: + # receive all msgs, scanning for either a final result + # or error; the underlying call should never raise any + # remote error directly! + msg, pld = await ctx._pld_rx.recv_msg_w_pld( + ipc=ctx, + expect_msg=Return, + raise_error=False, + hide_tb=hide_tb, + ) + # ^-TODO-^ some bad ideas? + # -[ ] wrap final outcome .receive() 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._rx_chan.receive() + # if res_cs.cancelled_caught: + # + # -[ ] make sure pause points work here for REPLing + # the runtime itself; i.e. ensure there's no hangs! + # |_from tractor.devx._debug import pause + # await pause() + + + # NOTE: we get here if the far end was + # `ContextCancelled` in 2 cases: + # 1. we requested the cancellation and thus + # SHOULD NOT raise that far end error, + # 2. WE DID NOT REQUEST that cancel and thus + # SHOULD RAISE HERE! + except trio.Cancelled as taskc: + + # CASE 2: mask the local cancelled-error(s) + # only when we are sure the remote error is + # the source cause of this local task's + # cancellation. + ctx.maybe_raise( + # TODO: when use this/ + # from_src_exc=taskc, + ) + + # CASE 1: we DID request the cancel we simply + # continue to bubble up as normal. + raise taskc + + match msg: + + # final result arrived! + case Return(): + log.runtime( + 'Context delivered final draining msg:\n' + f'{pretty_struct.pformat(msg)}' + ) + ctx._result: Any = pld + result_msg = msg + break + + # far end task is still streaming to us so discard + # and report depending on local ctx state. + case Yield(): + pre_result_drained.append(msg) + if ( + (ctx._stream.closed + and (reason := 'stream was already closed') + ) + or (ctx.cancel_acked + and (reason := 'ctx cancelled other side') + ) + or (ctx._cancel_called + and (reason := 'ctx called `.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'{pretty_struct.pformat(msg)}\n' + ) + break + + # 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'{pretty_struct.pformat(msg)}\n' + ) + continue + + # stream terminated, but no result yet.. + # + # 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? + # Stop() + case Stop(): + pre_result_drained.append(msg) + log.cancel( + 'Remote stream terminated due to "stop" msg:\n\n' + f'{pretty_struct.pformat(msg)}\n' + ) + continue + + # remote error msg, likely already handled inside + # `Context._deliver_msg()` + case Error(): + # TODO: can we replace this with `ctx.maybe_raise()`? + # -[ ] 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: + assert msg is ctx._cancel_msg + # NOTE: this solved a super duper 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.. + # + # XXX will raise if necessary but ow break + # from loop presuming any supressed error + # (ctxc) should terminate 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, + ) + result_msg = msg + break # OOOOOF, yeah obvi we need this.. + + else: + # bubble the original src key error + raise + + # XXX should pretty much never get here unless someone + # overrides the default `MsgType` spec. + case _: + pre_result_drained.append(msg) + # It's definitely an internal error if any other + # msg type without a`'cid'` field arrives here! + if not msg.cid: + raise InternalError( + 'Unexpected cid-missing msg?\n\n' + f'{msg}\n' + ) + + raise RuntimeError('Unknown msg type: {msg}') + + else: + log.cancel( + 'Skipping `MsgStream` drain since final outcome is set\n\n' + f'{ctx.outcome}\n' + ) + + return ( + result_msg, + pre_result_drained, + ) + + +def validate_payload_msg( + pld_msg: Started|Yield|Return, + pld_value: PayloadT, + ipc: Context|MsgStream, + + raise_mte: bool = True, + strict_pld_parity: bool = False, + hide_tb: bool = True, + +) -> MsgTypeError|None: + ''' + Validate a `PayloadMsg.pld` value with the current + IPC ctx's `PldRx` and raise an appropriate `MsgTypeError` + on failure. + + ''' + __tracebackhide__: bool = hide_tb + codec: MsgCodec = current_codec() + msg_bytes: bytes = codec.encode(pld_msg) + try: + roundtripped: Started = codec.decode(msg_bytes) + ctx: Context = getattr(ipc, 'ctx', ipc) + pld: PayloadT = ctx.pld_rx.decode_pld( + msg=roundtripped, + ipc=ipc, + expect_msg=Started, + hide_tb=hide_tb, + is_started_send_side=True, + ) + if ( + strict_pld_parity + and + pld != pld_value + ): + # TODO: make that one a mod func too.. + diff = pretty_struct.Struct.__sub__( + roundtripped, + pld_msg, + ) + complaint: str = ( + 'Started value does not match after roundtrip?\n\n' + f'{diff}' + ) + raise ValidationError(complaint) + + # raise any msg type error NO MATTER WHAT! + except ValidationError as verr: + try: + mte: MsgTypeError = _mk_recv_mte( + msg=roundtripped, + codec=codec, + src_validation_error=verr, + verb_header='Trying to send ', + is_invalid_payload=True, + ) + except BaseException: + __tracebackhide__: bool = False + raise + + if not raise_mte: + return mte + + raise mte from verr diff --git a/tractor/msg/pretty_struct.py b/tractor/msg/pretty_struct.py new file mode 100644 index 00000000..f27fb89c --- /dev/null +++ b/tractor/msg/pretty_struct.py @@ -0,0 +1,275 @@ +# 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 . + +''' +Prettified version of `msgspec.Struct` for easier console grokin. + +''' +from __future__ import annotations +from collections import UserList +from typing import ( + Any, + Iterator, +) + +from msgspec import ( + msgpack, + Struct as _Struct, + structs, +) +from pprint import ( + saferepr, +) + +# 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): + ''' + 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 + + +def iter_fields(struct: Struct) -> Iterator[ + tuple[ + structs.FieldIinfo, + str, + Any, + ] +]: + ''' + Iterate over all non-@property fields of this struct. + + ''' + fi: structs.FieldInfo + for fi in structs.fields(struct): + key: str = fi.name + val: Any = getattr(struct, key) + yield ( + fi, + key, + val, + ) + + +def pformat( + struct: Struct, + 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 + struct.__class__.__qualname__ + qtn: str = struct.__class__.__qualname__ + + obj_str: str = '' # accumulator + fi: structs.FieldInfo + k: str + v: Any + for fi, k, v in iter_fields(struct): + + # 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) + + # TODO: LOLOL use `textwrap.indent()` instead dawwwwwg! + obj_str += (field_ws + f'{k}: {typ_name} = {val_str},\n') + + return ( + f'{qtn}(\n' + f'{obj_str}' + f'{ws})' + ) + + +class Struct( + _Struct, + + # https://jcristharif.com/msgspec/structs.html#tagged-unions + # tag='pikerstruct', + # tag=True, +): + ''' + A "human friendlier" (aka repl buddy) struct subtype. + + ''' + 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 iter_fields(self): + sin_props[k] = asdict[k] + + return sin_props + + pformat = pformat + # __str__ = __repr__ = pformat + # TODO: use a pprint.PrettyPrinter instance around ONLY rendering + # inside a known tty? + # def __repr__(self) -> str: + # ... + __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 diff --git a/tractor/msg/ptr.py b/tractor/msg/ptr.py new file mode 100644 index 00000000..abe5406e --- /dev/null +++ b/tractor/msg/ptr.py @@ -0,0 +1,139 @@ +# 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 . + +''' +IPC-compat cross-mem-boundary object pointer. + +''' + +# TODO: integration with our ``enable_modules: list[str]`` caps sys. + +# ``pkgutil.resolve_name()`` internally uses +# ``importlib.import_module()`` which can be filtered by inserting +# a ``MetaPathFinder`` into ``sys.meta_path`` (which we could do before +# entering the ``_runtime.process_messages()`` loop). +# - https://github.com/python/cpython/blob/main/Lib/pkgutil.py#L645 +# - https://stackoverflow.com/questions/1350466/preventing-python-code-from-importing-certain-modules +# - https://stackoverflow.com/a/63320902 +# - https://docs.python.org/3/library/sys.html#sys.meta_path + +# the new "Implicit Namespace Packages" might be relevant? +# - https://www.python.org/dev/peps/pep-0420/ + +# add implicit serialized message type support so that paths can be +# handed directly to IPC primitives such as streams and `Portal.run()` +# calls: +# - via ``msgspec``: +# - https://jcristharif.com/msgspec/api.html#struct +# - https://jcristharif.com/msgspec/extending.html +# via ``msgpack-python``: +# - https://github.com/msgpack/msgpack-python#packingunpacking-of-custom-data-type + +from __future__ import annotations +from inspect import ( + isfunction, + ismethod, +) +from pkgutil import resolve_name + + +class NamespacePath(str): + ''' + 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 + + # 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) + return self._ref + + @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. + + 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 isfunction(ref): + name: str = getattr(ref, '__name__') + mod_name: str = ref.__module__ + + elif ismethod(ref): + # build out the path manually i guess..? + # TODO: better way? + name: str = '.'.join([ + type(ref.__self__).__name__, + ref.__func__.__name__, + ]) + mod_name: str = ref.__self__.__module__ + + else: # object or other? + # isinstance(ref, object) + # and not isfunction(ref) + name: str = type(ref).__name__ + mod_name: str = ref.__module__ + + # TODO: return static value direactly? + # + # fully qualified namespace path, tuple. + fqnp: tuple[str, str] = ( + mod_name, + name, + ) + return fqnp + + @classmethod + def from_ref( + cls, + ref: type|object, + + ) -> NamespacePath: + + fqnp: tuple[str, str] = cls._mk_fqnp(ref) + return cls(':'.join(fqnp)) + + 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() + ) diff --git a/tractor/msg/types.py b/tractor/msg/types.py new file mode 100644 index 00000000..0904411f --- /dev/null +++ b/tractor/msg/types.py @@ -0,0 +1,730 @@ +# 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 . + +''' +Define our strictly typed IPC message spec for the SCIPP: + +that is, + +the "Structurred-Concurrency-Inter-Process-(dialog)-(un)Protocol". + +''' +from __future__ import annotations +import types +from typing import ( + Any, + Generic, + Literal, + Type, + TypeVar, + TypeAlias, + Union, +) + +from msgspec import ( + defstruct, + # field, + Raw, + Struct, + # UNSET, + # UnsetType, +) + +from tractor.msg import ( + pretty_struct, +) +from tractor.log import get_logger + + +log = get_logger('tractor.msgspec') + +# type variable for the boxed payload field `.pld` +PayloadT = TypeVar('PayloadT') + + +class PayloadMsg( + Struct, + Generic[PayloadT], + + # https://jcristharif.com/msgspec/structs.html#tagged-unions + tag=True, + tag_field='msg_type', + + # https://jcristharif.com/msgspec/structs.html#field-ordering + # kw_only=True, + + # https://jcristharif.com/msgspec/structs.html#equality-and-order + # order=True, + + # https://jcristharif.com/msgspec/structs.html#encoding-decoding-as-arrays + # as_array=True, +): + ''' + An abstract payload boxing/shuttling IPC msg type. + + Boxes data-values passed to/from user code + + (i.e. any values passed by `tractor` application code using any of + + |_ `._streaming.MsgStream.send/receive()` + |_ `._context.Context.started/result()` + |_ `._ipc.Channel.send/recv()` + + aka our "IPC primitive APIs") + + as message "payloads" set to the `.pld` field and uses + `msgspec`'s "tagged unions" feature to support a subset of our + "SC-transitive shuttle protocol" specification with + a `msgspec.Struct` inheritance tree. + + ''' + cid: str # call/context-id + # ^-TODO-^: more explicit type? + # -[ ] use UNSET here? + # https://jcristharif.com/msgspec/supported-types.html#unset + # + # -[ ] `uuid.UUID` which has multi-protocol support + # https://jcristharif.com/msgspec/supported-types.html#uuid + + # The msg's "payload" (spelled without vowels): + # https://en.wikipedia.org/wiki/Payload_(computing) + pld: Raw + + # ^-NOTE-^ inherited from any `PayloadMsg` (and maybe type + # overriden via the `._ops.limit_plds()` API), but by default is + # parameterized to be `Any`. + # + # XXX this `Union` must strictly NOT contain `Any` if + # a limited msg-type-spec is intended, such that when + # creating and applying a new `MsgCodec` its + # `.decoder: Decoder` is configured with a `Union[Type[Struct]]` which + # restricts the allowed payload content (this `.pld` field) + # by type system defined loading constraints B) + # + # TODO: could also be set to `msgspec.Raw` if the sub-decoders + # approach is preferred over the generic parameterization + # approach as take by `mk_msg_spec()` below. + + +# TODO: complete rename +Msg = PayloadMsg + + +class Aid( + Struct, + tag=True, + tag_field='msg_type', +): + ''' + Actor-identity msg. + + Initial contact exchange enabling an actor "mailbox handshake" + delivering the peer identity (and maybe eventually contact) + info. + + Used by discovery protocol to register actors as well as + conduct the initial comms (capability) filtering. + + ''' + name: str + uuid: str + # TODO: use built-in support for UUIDs? + # -[ ] `uuid.UUID` which has multi-protocol support + # https://jcristharif.com/msgspec/supported-types.html#uuid + + +class SpawnSpec( + pretty_struct.Struct, + tag=True, + tag_field='msg_type', +): + ''' + Initial runtime spec handed down from a spawning parent to its + child subactor immediately following first contact via an + `Aid` msg. + + ''' + # TODO: similar to the `Start` kwargs spec needed below, we need + # a hard `Struct` def for all of these fields! + _parent_main_data: dict + _runtime_vars: dict[str, Any] + + # module import capability + enable_modules: dict[str, str] + + # TODO: not just sockaddr pairs? + # -[ ] abstract into a `TransportAddr` type? + reg_addrs: list[tuple[str, int]] + bind_addrs: list[tuple[str, int]] + + +# TODO: caps based RPC support in the payload? +# +# -[ ] integration with our ``enable_modules: list[str]`` caps sys. +# ``pkgutil.resolve_name()`` internally uses +# ``importlib.import_module()`` which can be filtered by +# inserting a ``MetaPathFinder`` into ``sys.meta_path`` (which +# we could do before entering the ``Actor._process_messages()`` +# loop)? +# - https://github.com/python/cpython/blob/main/Lib/pkgutil.py#L645 +# - https://stackoverflow.com/questions/1350466/preventing-python-code-from-importing-certain-modules +# - https://stackoverflow.com/a/63320902 +# - https://docs.python.org/3/library/sys.html#sys.meta_path +# +# -[ ] can we combine .ns + .func into a native `NamespacePath` field? +# +# -[ ] better name, like `Call/TaskInput`? +# +# -[ ] XXX a debugger lock msg transaction with payloads like, +# child -> `.pld: DebugLock` -> root +# child <- `.pld: DebugLocked` <- root +# child -> `.pld: DebugRelease` -> root +# +# WHY => when a pld spec is provided it might not allow for +# debug mode msgs as they currently are (using plain old `pld. +# str` payloads) so we only when debug_mode=True we need to +# union in this debugger payload set? +# +# mk_msg_spec( +# MyPldSpec, +# debug_mode=True, +# ) -> ( +# Union[MyPldSpec] +# | Union[DebugLock, DebugLocked, DebugRelease] +# ) + +# class Params( +# Struct, +# Generic[PayloadT], +# ): +# spec: PayloadT|ParamSpec +# inputs: InputsT|dict[str, Any] + + # TODO: for eg. we could stringently check the target + # task-func's type sig and enforce it? + # as an example for an IPTC, + # @tractor.context + # async def send_back_nsp( + # ctx: Context, + # expect_debug: bool, + # pld_spec_str: str, + # add_hooks: bool, + # started_msg_dict: dict, + # ) -> : + + # TODO: figure out which of the `typing` feats we want to + # support: + # - plain ol `ParamSpec`: + # https://docs.python.org/3/library/typing.html#typing.ParamSpec + # - new in 3.12 type parameter lists Bo + # |_ https://docs.python.org/3/reference/compound_stmts.html#type-params + # |_ historical pep 695: https://peps.python.org/pep-0695/ + # |_ full lang spec: https://typing.readthedocs.io/en/latest/spec/ + # |_ on annotation scopes: + # https://docs.python.org/3/reference/executionmodel.html#annotation-scopes + # spec: ParamSpec[ + # expect_debug: bool, + # pld_spec_str: str, + # add_hooks: bool, + # started_msg_dict: dict, + # ] + + +# TODO: possibly sub-type for runtime method requests? +# -[ ] `Runtime(Start)` with a `.ns: str = 'self' or +# we can just enforce any such method as having a strict +# ns for calling funcs, namely the `Actor` instance? +class Start( + Struct, + tag=True, + tag_field='msg_type', +): + ''' + Initial request to remotely schedule an RPC `trio.Task` via + `Actor.start_remote_task()`. + + It is called by all the following public APIs: + + - `ActorNursery.run_in_actor()` + + - `Portal.run()` + `|_.run_from_ns()` + `|_.open_stream_from()` + `|_._submit_for_result()` + + - `Context.open_context()` + + ''' + cid: str + + ns: str + func: str + + # TODO: make this a sub-struct which can be further + # type-limited, maybe `Inputs`? + # => SEE ABOVE <= + kwargs: dict[str, Any] + uid: tuple[str, str] # (calling) actor-id + + # TODO: enforcing a msg-spec in terms `Msg.pld` + # parameterizable msgs to be used in the appls IPC dialog. + # => SEE `._codec.MsgDec` for more <= + pld_spec: str = str(Any) + + +class StartAck( + Struct, + tag=True, + tag_field='msg_type', +): + ''' + Init response to a `Cmd` request indicating the far + end's RPC spec, namely its callable "type". + + ''' + cid: str + # TODO: maybe better names for all these? + # -[ ] obvi ^ would need sync with `._rpc` + functype: Literal[ + 'asyncfunc', + 'asyncgen', + 'context', # TODO: the only one eventually? + ] + + # import typing + # eval(str(Any), {}, {'typing': typing}) + # started_spec: str = str(Any) + # return_spec + + +class Started( + PayloadMsg, + Generic[PayloadT], +): + ''' + Packet to shuttle the "first value" delivered by + `Context.started(value: Any)` from a `@tractor.context` + decorated IPC endpoint. + + ''' + pld: PayloadT|Raw + + +# TODO: cancel request dedicated msg? +# -[ ] instead of using our existing `Start`? +# +# class Cancel: +# cid: str + + +class Yield( + PayloadMsg, + Generic[PayloadT], +): + ''' + Per IPC transmission of a value from `await MsgStream.send()`. + + ''' + pld: PayloadT|Raw + + +class Stop( + Struct, + tag=True, + tag_field='msg_type', +): + ''' + Stream termination signal much like an IPC version + of `StopAsyncIteration`. + + ''' + cid: str + # TODO: do we want to support a payload on stop? + # pld: UnsetType = UNSET + + +# TODO: is `Result` or `Out[come]` a better name? +class Return( + PayloadMsg, + Generic[PayloadT], +): + ''' + Final `return ` from a remotely scheduled + func-as-`trio.Task`. + + ''' + pld: PayloadT|Raw + + +class CancelAck( + PayloadMsg, + Generic[PayloadT], +): + ''' + Deliver the `bool` return-value from a cancellation `Actor` + method scheduled via and prior RPC request. + + - `Actor.cancel()` + `|_.cancel_soon()` + `|_.cancel_rpc_tasks()` + `|_._cancel_task()` + `|_.cancel_server()` + + RPCs to these methods must **always** be able to deliver a result + despite the currently configured IPC msg spec such that graceful + cancellation is always functional in the runtime. + + ''' + pld: bool + + +# TODO: unify this with `._exceptions.RemoteActorError` +# such that we can have a msg which is both raisable and +# IPC-wire ready? +# B~o +class Error( + Struct, + tag=True, + tag_field='msg_type', + + # TODO may omit defaults? + # https://jcristharif.com/msgspec/structs.html#omitting-default-values + # omit_defaults=True, +): + ''' + A pkt that wraps `RemoteActorError`s for relay and raising. + + Fields are 1-to-1 meta-data as needed originally by + `RemoteActorError.msgdata: dict` but now are defined here. + + Note: this msg shuttles `ContextCancelled` and `StreamOverrun` + as well is used to rewrap any `MsgTypeError` for relay-reponse + to bad `Yield.pld` senders during an IPC ctx's streaming dialog + phase. + + ''' + src_uid: tuple[str, str] + src_type_str: str + boxed_type_str: str + relay_path: list[tuple[str, str]] + + # normally either both are provided or just + # a message for certain special cases where + # we pack a message for a locally raised + # mte or ctxc. + message: str|None = None + tb_str: str = '' + + # TODO: only optionally include sub-type specfic fields? + # -[ ] use UNSET or don't include them via `omit_defaults` (see + # inheritance-line options above) + # + # `ContextCancelled` reports the src cancelling `Actor.uid` + canceller: tuple[str, str]|None = None + + # `StreamOverrun`-specific src `Actor.uid` + sender: tuple[str, str]|None = None + + # `MsgTypeError` meta-data + cid: str|None = None + # when the receiver side fails to decode a delivered + # `PayloadMsg`-subtype; one and/or both the msg-struct instance + # and `Any`-decoded to `dict` of the msg are set and relayed + # (back to the sender) for introspection. + _bad_msg: Started|Yield|Return|None = None + _bad_msg_as_dict: dict|None = None + + +def from_dict_msg( + dict_msg: dict, + + msgT: MsgType|None = None, + tag_field: str = 'msg_type', + use_pretty: bool = False, + +) -> MsgType: + ''' + Helper to build a specific `MsgType` struct from a "vanilla" + decoded `dict`-ified equivalent of the msg: i.e. if the + `msgpack.Decoder.type == Any`, the default when using + `msgspec.msgpack` and not "typed decoding" using + `msgspec.Struct`. + + ''' + msg_type_tag_field: str = ( + msgT.__struct_config__.tag_field + if msgT is not None + else tag_field + ) + # XXX ensure tag field is removed + msgT_name: str = dict_msg.pop(msg_type_tag_field) + msgT: MsgType = _msg_table[msgT_name] + if use_pretty: + msgT = defstruct( + name=msgT_name, + fields=[ + (key, fi.type) + for fi, key, _ + in pretty_struct.iter_fields(msgT) + ], + bases=( + pretty_struct.Struct, + msgT, + ), + ) + return msgT(**dict_msg) + +# TODO: should be make a set of cancel msgs? +# -[ ] a version of `ContextCancelled`? +# |_ and/or with a scope field? +# -[ ] or, a full `ActorCancelled`? +# +# class Cancelled(MsgType): +# cid: str +# +# -[ ] what about overruns? +# +# class Overrun(MsgType): +# cid: str + +_runtime_msgs: list[Struct] = [ + + # identity handshake on first IPC `Channel` contact. + Aid, + + # parent-to-child spawn specification passed as 2nd msg after + # handshake ONLY after child connects back to parent. + SpawnSpec, + + # inter-actor RPC initiation + Start, # schedule remote task-as-func + StartAck, # ack the schedule request + + # emission from `MsgStream.aclose()` + Stop, + + # `Return` sub-type that we always accept from + # runtime-internal cancel endpoints + CancelAck, + + # box remote errors, normally subtypes + # of `RemoteActorError`. + Error, +] + +# the no-outcome-yet IAC (inter-actor-communication) sub-set which +# can be `PayloadMsg.pld` payload field type-limited by application code +# using `apply_codec()` and `limit_msg_spec()`. +_payload_msgs: list[PayloadMsg] = [ + # first from `Context.started()` + Started, + + # any sent via `MsgStream.send()` + Yield, + + # the final value returned from a `@context` decorated + # IPC endpoint. + Return, +] + +# built-in SC shuttle protocol msg type set in +# approx order of the IPC txn-state spaces. +__msg_types__: list[MsgType] = ( + _runtime_msgs + + + _payload_msgs +) + + +_msg_table: dict[str, MsgType] = { + msgT.__name__: msgT + for msgT in __msg_types__ +} + +# TODO: use new type declaration syntax for msg-type-spec +# https://docs.python.org/3/library/typing.html#type-aliases +# https://docs.python.org/3/reference/simple_stmts.html#type +MsgType: TypeAlias = Union[*__msg_types__] + + +def mk_msg_spec( + payload_type_union: Union[Type] = Any, + + spec_build_method: Literal[ + 'indexed_generics', # works + 'defstruct', + 'types_new_class', + + ] = 'indexed_generics', + +) -> tuple[ + Union[MsgType], + list[MsgType], +]: + ''' + Create a payload-(data-)type-parameterized IPC message specification. + + Allows generating IPC msg types from the above builtin set + with a payload (field) restricted data-type, the `Msg.pld: PayloadT`. + + This allows runtime-task contexts to use the python type system + to limit/filter payload values as determined by the input + `payload_type_union: Union[Type]`. + + Notes: originally multiple approaches for constructing the + type-union passed to `msgspec` were attempted as selected via the + `spec_build_method`, but it turns out only the defaul method + 'indexed_generics' seems to work reliably in all use cases. As + such, the others will likely be removed in the near future. + + ''' + submsg_types: list[MsgType] = Msg.__subclasses__() + bases: tuple = ( + # XXX NOTE XXX the below generic-parameterization seems to + # be THE ONLY way to get this to work correctly in terms + # of getting ValidationError on a roundtrip? + Msg[payload_type_union], + Generic[PayloadT], + ) + defstruct_bases: tuple = ( + Msg, # [payload_type_union], + # Generic[PayloadT], + # ^-XXX-^: not allowed? lul.. + ) + ipc_msg_types: list[Msg] = [] + + idx_msg_types: list[Msg] = [] + defs_msg_types: list[Msg] = [] + nc_msg_types: list[Msg] = [] + + for msgtype in __msg_types__: + + # for the NON-payload (user api) type specify-able + # msgs types, we simply aggregate the def as is + # for inclusion in the output type `Union`. + if msgtype not in _payload_msgs: + ipc_msg_types.append(msgtype) + continue + + # check inheritance sanity + assert msgtype in submsg_types + + # TODO: wait why do we need the dynamic version here? + # XXX ANSWER XXX -> BC INHERITANCE.. don't work w generics.. + # + # NOTE previously bc msgtypes WERE NOT inheritting + # directly the `Generic[PayloadT]` type, the manual method + # of generic-paraming with `.__class_getitem__()` wasn't + # working.. + # + # XXX but bc i changed that to make every subtype inherit + # it, this manual "indexed parameterization" method seems + # to work? + # + # -[x] paraming the `PayloadT` values via `Generic[T]` + # does work it seems but WITHOUT inheritance of generics + # + # -[-] is there a way to get it to work at module level + # just using inheritance or maybe a metaclass? + # => thot that `defstruct` might work, but NOPE, see + # below.. + # + idxed_msg_type: Msg = msgtype[payload_type_union] + idx_msg_types.append(idxed_msg_type) + + # TODO: WHY do we need to dynamically generate the + # subtype-msgs here to ensure the `.pld` parameterization + # propagates as well as works at all in terms of the + # `msgpack.Decoder()`..? + # + # dynamically create the payload type-spec-limited msg set. + newclass_msgtype: Type = types.new_class( + name=msgtype.__name__, + bases=bases, + kwds={}, + ) + nc_msg_types.append( + newclass_msgtype[payload_type_union] + ) + + # with `msgspec.structs.defstruct` + # XXX ALSO DOESN'T WORK + defstruct_msgtype = defstruct( + name=msgtype.__name__, + fields=[ + ('cid', str), + + # XXX doesn't seem to work.. + # ('pld', PayloadT), + + ('pld', payload_type_union), + ], + bases=defstruct_bases, + ) + defs_msg_types.append(defstruct_msgtype) + + # assert index_paramed_msg_type == manual_paramed_msg_subtype + + # paramed_msg_type = manual_paramed_msg_subtype + + # ipc_payload_msgs_type_union |= index_paramed_msg_type + + idx_spec: Union[Type[Msg]] = Union[*idx_msg_types] + def_spec: Union[Type[Msg]] = Union[*defs_msg_types] + nc_spec: Union[Type[Msg]] = Union[*nc_msg_types] + + specs: dict[str, Union[Type[Msg]]] = { + 'indexed_generics': idx_spec, + 'defstruct': def_spec, + 'types_new_class': nc_spec, + } + msgtypes_table: dict[str, list[Msg]] = { + 'indexed_generics': idx_msg_types, + 'defstruct': defs_msg_types, + 'types_new_class': nc_msg_types, + } + + # XXX lol apparently type unions can't ever + # be equal eh? + # TODO: grok the diff here better.. + # + # assert ( + # idx_spec + # == + # nc_spec + # == + # def_spec + # ) + # breakpoint() + + pld_spec: Union[Type] = specs[spec_build_method] + runtime_spec: Union[Type] = Union[*ipc_msg_types] + ipc_spec = pld_spec | runtime_spec + log.runtime( + 'Generating new IPC msg-spec\n' + f'{ipc_spec}\n' + ) + assert ( + ipc_spec + and + ipc_spec is not Any + ) + return ( + ipc_spec, + msgtypes_table[spec_build_method] + + + ipc_msg_types, + ) diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index be3ac8d3..d1451b4c 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -28,16 +28,19 @@ from typing import ( Callable, AsyncIterator, Awaitable, - Optional, ) import trio from outcome import Error -from .log import get_logger -from ._state import current_actor -from ._exceptions import AsyncioCancelled -from .trionics._broadcast import ( +from tractor.log import get_logger +from tractor._state import ( + current_actor, + debug_mode, +) +from tractor.devx import _debug +from tractor._exceptions import AsyncioCancelled +from tractor.trionics._broadcast import ( broadcast_receiver, BroadcastReceiver, ) @@ -65,9 +68,9 @@ class LinkedTaskChannel(trio.abc.Channel): _trio_exited: bool = False # set after ``asyncio.create_task()`` - _aio_task: Optional[asyncio.Task] = None - _aio_err: Optional[BaseException] = None - _broadcaster: Optional[BroadcastReceiver] = None + _aio_task: asyncio.Task|None = None + _aio_err: BaseException|None = None + _broadcaster: BroadcastReceiver|None = None async def aclose(self) -> None: await self._from_aio.aclose() @@ -159,7 +162,9 @@ def _run_asyncio_task( ''' __tracebackhide__ = True if not current_actor().is_infected_aio(): - raise RuntimeError("`infect_asyncio` mode is not enabled!?") + raise RuntimeError( + "`infect_asyncio` mode is not enabled!?" + ) # ITC (inter task comms), these channel/queue names are mostly from # ``asyncio``'s perspective. @@ -188,7 +193,7 @@ def _run_asyncio_task( cancel_scope = trio.CancelScope() aio_task_complete = trio.Event() - aio_err: Optional[BaseException] = None + aio_err: BaseException|None = None chan = LinkedTaskChannel( aio_q, # asyncio.Queue @@ -217,7 +222,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 @@ -247,7 +259,7 @@ def _run_asyncio_task( if not inspect.isawaitable(coro): raise TypeError(f"No support for invoking {coro}") - task = asyncio.create_task( + task: asyncio.Task = asyncio.create_task( wait_on_coro_final_result( to_trio, coro, @@ -256,6 +268,18 @@ def _run_asyncio_task( ) chan._aio_task = task + # XXX TODO XXX get this actually workin.. XD + # maybe setup `greenback` for `asyncio`-side task REPLing + if ( + debug_mode() + and + (greenback := _debug.maybe_import_greenback( + force_reload=True, + raise_not_found=False, + )) + ): + greenback.bestow_portal(task) + def cancel_trio(task: asyncio.Task) -> None: ''' Cancel the calling ``trio`` task on error. @@ -263,7 +287,7 @@ def _run_asyncio_task( ''' nonlocal chan aio_err = chan._aio_err - task_err: Optional[BaseException] = None + task_err: BaseException|None = None # only to avoid ``asyncio`` complaining about uncaptured # task exceptions @@ -272,12 +296,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 +324,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 @@ -329,11 +367,11 @@ async def translate_aio_errors( ''' trio_task = trio.lowlevel.current_task() - aio_err: Optional[BaseException] = None + aio_err: BaseException|None = None # TODO: make thisi a channel method? def maybe_raise_aio_err( - err: Optional[Exception] = None + err: Exception|None = None ) -> None: aio_err = chan._aio_err if ( @@ -511,6 +549,16 @@ def run_as_asyncio_guest( loop = asyncio.get_running_loop() trio_done_fut = asyncio.Future() + if debug_mode(): + # XXX make it obvi we know this isn't supported yet! + log.error( + 'Attempting to enter unsupported `greenback` init ' + 'from `asyncio` task..' + ) + await _debug.maybe_init_greenback( + force_reload=True, + ) + def trio_done_callback(main_outcome): if isinstance(main_outcome, Error): @@ -529,14 +577,18 @@ def run_as_asyncio_guest( log.runtime(f"trio_main finished: {main_outcome!r}") # start the infection: run trio on the asyncio loop in "guest mode" - log.info(f"Infecting asyncio process with {trio_main}") + log.runtime( + 'Infecting `asyncio`-process with a `trio` guest-run of\n\n' + f'{trio_main!r}\n\n' + f'{trio_done_callback}\n' + ) trio.lowlevel.start_guest_run( trio_main, run_sync_soon_threadsafe=loop.call_soon_threadsafe, done_callback=trio_done_callback, ) - # ``.unwrap()`` will raise here on error + # NOTE `.unwrap()` will raise on error return (await trio_done_fut).unwrap() # might as well if it's installed. 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', -] 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: diff --git a/tractor/trionics/_mngrs.py b/tractor/trionics/_mngrs.py index 89db895b..08e70ad2 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__) @@ -70,6 +69,7 @@ async def _enter_and_wait( unwrapped: dict[int, T], all_entered: trio.Event, parent_exit: trio.Event, + seed: int, ) -> None: ''' @@ -80,7 +80,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 +94,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 +113,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 +129,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 +142,7 @@ async def gather_contexts( unwrapped, all_entered, parent_exit, + seed, ) # deliver control once all managers have started up @@ -168,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: @@ -209,6 +224,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. @@ -216,6 +232,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 @@ -254,8 +271,11 @@ async def maybe_open_context( yield False, yielded else: - log.info(f'Reusing _Cached resource for {ctx_key}') _Cache.users += 1 + log.runtime( + f'Reusing resource for `_Cache` user {_Cache.users}\n\n' + f'{ctx_key!r} -> {yielded!r}\n' + ) lock.release() yield True, yielded @@ -275,4 +295,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?' + )