diff --git a/examples/advanced_faults/ipc_failure_during_stream.py b/examples/advanced_faults/ipc_failure_during_stream.py index 9dca92b1..60b28c3e 100644 --- a/examples/advanced_faults/ipc_failure_during_stream.py +++ b/examples/advanced_faults/ipc_failure_during_stream.py @@ -21,75 +21,12 @@ import trio import pytest -async def break_ipc( - stream: MsgStream, - method: str|None = None, - pre_close: bool = False, - - def_method: str = 'eof', - -) -> None: - ''' - XXX: close the channel right after an error is raised - purposely breaking the IPC transport to make sure the parent - doesn't get stuck in debug or hang on the connection join. - this more or less simulates an infinite msg-receive hang on - the other end. - - ''' - # close channel via IPC prot msging before - # any transport breakage - if pre_close: - await stream.aclose() - - method: str = method or def_method - print( - '#################################\n' - 'Simulating CHILD-side IPC BREAK!\n' - f'method: {method}\n' - f'pre `.aclose()`: {pre_close}\n' - '#################################\n' - ) - - match method: - case 'trans_aclose': - await stream._ctx.chan.transport.stream.aclose() - - case 'eof': - await stream._ctx.chan.transport.stream.send_eof() - - case 'msg': - await stream._ctx.chan.send(None) - - # TODO: the actual real-world simulated cases like - # transport layer hangs and/or lower layer 2-gens type - # scenarios.. - # - # -[ ] already have some issues for this general testing - # area: - # - https://github.com/goodboy/tractor/issues/97 - # - https://github.com/goodboy/tractor/issues/124 - # - PR from @guille: - # https://github.com/goodboy/tractor/pull/149 - # case 'hang': - # TODO: framework research: - # - # - https://github.com/GuoTengda1993/pynetem - # - https://github.com/shopify/toxiproxy - # - https://manpages.ubuntu.com/manpages/trusty/man1/wirefilter.1.html - - case _: - raise RuntimeError( - f'IPC break method unsupported: {method}' - ) - - async def break_ipc_then_error( stream: MsgStream, break_ipc_with: str|None = None, pre_close: bool = False, ): - await break_ipc( + await _testing.break_ipc( stream=stream, method=break_ipc_with, pre_close=pre_close, @@ -121,6 +58,7 @@ async def recv_and_spawn_net_killers( Receive stream msgs and spawn some IPC killers mid-stream. ''' + broke_ipc: bool = False await ctx.started() async with ( ctx.open_stream() as stream, @@ -128,13 +66,17 @@ 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 ): + broke_ipc = True n.start_soon( iter_ipc_stream, stream, @@ -242,14 +184,13 @@ async def main( # 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}') + # 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( @@ -262,6 +203,13 @@ async def main( # 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: 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/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/sync_bp.py b/examples/debugging/sync_bp.py index efa4e405..137710fc 100644 --- a/examples/debugging/sync_bp.py +++ b/examples/debugging/sync_bp.py @@ -1,13 +1,21 @@ +from functools import partial +import time + import trio import tractor def sync_pause( - use_builtin: bool = True, + 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=False) + breakpoint(hide_tb=hide_tb) else: tractor.pause_from_sync() @@ -25,44 +33,114 @@ async def start_n_sync_pause( # sync to parent-side task await ctx.started() - print(f'entering SYNC PAUSE in {actor.uid}') + print(f'Entering `sync_pause()` in subactor: {actor.uid}\n') sync_pause() - print(f'back from SYNC PAUSE in {actor.uid}') + 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, - ) as an: + 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, - loglevel='cancel', ) # TODO: 3 sub-actor usage cases: + # -[x] via a `.open_context()` # -[ ] via a `.run_in_actor()` call # -[ ] via a `.run()` - # -[ ] via a `.open_context()` - # + # -[ ] via a `.to_thread.run_sync()` in subactor async with p.open_context( start_n_sync_pause, ) as (ctx, first): assert first is None - await tractor.pause() - sync_pause() + # 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! - # TODO: make this work!! - await trio.to_thread.run_sync( - sync_pause, - abandon_on_cancel=False, - ) + # 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() diff --git a/examples/full_fledged_streaming_service.py b/examples/full_fledged_streaming_service.py index c93df242..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,7 +73,7 @@ async def aggregate(seed): print("FINISHED ITERATING in aggregator") - await nursery.cancel() + await an.cancel() print("WAITING on `ActorNursery` to finish") print("AGGREGATOR COMPLETE!") @@ -75,18 +88,21 @@ async def main() -> list[int]: ''' # yes, a nursery which spawns `trio`-"actors" B) - nursery: tractor.ActorNursery - async with tractor.open_nursery() as nursery: + an: ActorNursery + async with tractor.open_nursery( + loglevel='cancel', + debug_mode=True, + ) as an: seed = int(1e3) pre_start = time.time() - portal: tractor.Portal = await nursery.start_actor( + portal: Portal = await an.start_actor( name='aggregator', enable_modules=[__name__], ) - stream: tractor.MsgStream + stream: MsgStream async with portal.open_stream_from( aggregate, seed=seed, @@ -95,11 +111,12 @@ async def main() -> list[int]: 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/service_discovery.py b/examples/service_discovery.py index 858f7f12..a0f37b88 100644 --- a/examples/service_discovery.py +++ b/examples/service_discovery.py @@ -9,7 +9,7 @@ async def main(service_name): async with tractor.open_nursery() as an: await an.start_actor(service_name) - async with tractor.get_arbiter('127.0.0.1', 1616) as portal: + async with tractor.get_registry('127.0.0.1', 1616) as portal: print(f"Arbiter is listening on {portal.channel}") async with tractor.wait_for_actor(service_name) as sockaddr: diff --git a/tests/test_advanced_faults.py b/tests/test_advanced_faults.py index 52db139f..a4d17791 100644 --- a/tests/test_advanced_faults.py +++ b/tests/test_advanced_faults.py @@ -13,6 +13,7 @@ import trio import tractor from tractor._testing import ( examples_dir, + break_ipc, ) @@ -90,10 +91,12 @@ def test_ipc_channel_break_during_stream( # non-`trio` spawners should never hit the hang condition that # requires the user to do ctl-c to cancel the actor tree. - expect_final_exc = trio.ClosedResourceError + # expect_final_exc = trio.ClosedResourceError + expect_final_exc = tractor.TransportClosed mod: ModuleType = import_path( - examples_dir() / 'advanced_faults' / 'ipc_failure_during_stream.py', + examples_dir() / 'advanced_faults' + / 'ipc_failure_during_stream.py', root=examples_dir(), consider_namespace_packages=False, ) @@ -155,7 +158,7 @@ def test_ipc_channel_break_during_stream( if pre_aclose_msgstream: expect_final_exc = KeyboardInterrupt - # NOTE when the parent IPC side dies (even if the child's does as well + # NOTE when the parent IPC side dies (even if the child does as well # but the child fails BEFORE the parent) we always expect the # IPC layer to raise a closed-resource, NEVER do we expect # a stop msg since the parent-side ctx apis will error out @@ -167,7 +170,8 @@ def test_ipc_channel_break_during_stream( and ipc_break['break_child_ipc_after'] is False ): - expect_final_exc = trio.ClosedResourceError + # expect_final_exc = trio.ClosedResourceError + expect_final_exc = tractor.TransportClosed # BOTH but, PARENT breaks FIRST elif ( @@ -178,7 +182,8 @@ def test_ipc_channel_break_during_stream( ipc_break['break_parent_ipc_after'] ) ): - expect_final_exc = trio.ClosedResourceError + # expect_final_exc = trio.ClosedResourceError + expect_final_exc = tractor.TransportClosed with pytest.raises( expected_exception=( @@ -197,8 +202,8 @@ def test_ipc_channel_break_during_stream( **ipc_break, ) ) - except KeyboardInterrupt as kbi: - _err = kbi + except KeyboardInterrupt as _kbi: + kbi = _kbi if expect_final_exc is not KeyboardInterrupt: pytest.fail( 'Rxed unexpected KBI !?\n' @@ -207,6 +212,21 @@ def test_ipc_channel_break_during_stream( raise + except tractor.TransportClosed as _tc: + tc = _tc + if expect_final_exc is KeyboardInterrupt: + pytest.fail( + 'Unexpected transport failure !?\n' + f'{repr(tc)}' + ) + cause: Exception = tc.__cause__ + assert ( + type(cause) is trio.ClosedResourceError + and + cause.args[0] == 'another task closed this fd' + ) + raise + # get raw instance from pytest wrapper value = excinfo.value if isinstance(value, ExceptionGroup): @@ -225,9 +245,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') diff --git a/tests/test_cancellation.py b/tests/test_cancellation.py index 18ad3615..92540ed4 100644 --- a/tests/test_cancellation.py +++ b/tests/test_cancellation.py @@ -89,17 +89,30 @@ def test_remote_error(reg_addr, args_err): 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: + # 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(reg_addr): +def test_multierror( + reg_addr: tuple[str, int], +): ''' Verify we raise a ``BaseExceptionGroup`` out of a nursery where more then one actor errors. diff --git a/tests/test_caps_based_msging.py b/tests/test_caps_based_msging.py new file mode 100644 index 00000000..6064c2cf --- /dev/null +++ b/tests/test_caps_based_msging.py @@ -0,0 +1,917 @@ +''' +Low-level functional audits for our +"capability based messaging"-spec feats. + +B~) + +''' +import typing +from typing import ( + Any, + Type, + Union, +) + +from msgspec import ( + structs, + msgpack, + Struct, + ValidationError, +) +import pytest + +import tractor +from tractor import ( + _state, + MsgTypeError, + Context, +) +from tractor.msg import ( + _codec, + _ctxvar_MsgCodec, + + NamespacePath, + MsgCodec, + mk_codec, + apply_codec, + current_codec, +) +from tractor.msg.types import ( + _payload_msgs, + log, + PayloadMsg, + 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 `PayloadMsg.pld`, we still need a enc/dec_hook() pair + # to cast to/from that type on the wire. See the docs: + # https://jcristharif.com/msgspec/extending.html#mapping-to-from-native-types + + 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 mod in [ + typing, + importlib.import_module(__name__), + ]: + if type_ref := getattr( + mod, + 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 `PayloadMsg.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[PayloadMsg[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 `PayloadMsg[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: PayloadMsg = 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 `PayloadMsg` + assert chk_pld_type( + payload_spec=Any, + pld=None, + expect_roundtrip=True, + ) + + # verify that a mis-typed payload value won't decode + assert not chk_pld_type( + payload_spec=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( + payload_spec=CustomPayload, + pld='doggy', + ) + + assert chk_pld_type( + payload_spec=CustomPayload, + pld=CustomPayload(name='doggy', value='urmom') + ) + + # yah, we can `.pause_from_sync()` now! + # breakpoint() + + trio.run(main) diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index 121abaa8..8edea510 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -6,6 +6,7 @@ sync-opening a ``tractor.Context`` beforehand. ''' from itertools import count +import math import platform from pprint import pformat from typing import ( @@ -24,6 +25,7 @@ from tractor._exceptions import ( StreamOverrun, ContextCancelled, ) +from tractor._state import current_ipc_ctx from tractor._testing import ( tractor_test, @@ -143,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) @@ -203,6 +207,7 @@ def test_simple_context( block_forever=callee_blocks_forever, ) as (ctx, sent), ): + assert current_ipc_ctx() is ctx assert sent == 11 if callee_blocks_forever: @@ -795,10 +800,12 @@ async def test_callee_cancels_before_started( # raises a special cancel signal except tractor.ContextCancelled as ce: + _ce = ce # for debug on crash ce.boxed_type == trio.Cancelled # the traceback should be informative - assert '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() @@ -845,7 +852,10 @@ 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: tuple[str, int, Callable], @@ -869,29 +879,30 @@ def test_one_end_stream_not_opened( 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() @@ -1055,54 +1066,63 @@ def test_maybe_allow_overruns_stream( loglevel=loglevel, 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('Requesting `ctx.cancel()` in parent!') - await ctx.cancel() + assert isinstance(res, ContextCancelled) + assert tuple(res.canceller) == current_actor().uid - res: str|ContextCancelled = await ctx.result() - - if cancel_ctx: - assert isinstance(res, ContextCancelled) - assert tuple(res.canceller) == 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() @@ -1143,7 +1163,8 @@ def test_maybe_allow_overruns_stream( elif slow_side == 'parent': assert err.boxed_type == tractor.RemoteActorError - assert 'StreamOverrun' in err.msgdata['tb_str'] + 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 diff --git a/tests/test_debugger.py b/tests/test_debugger.py index 0de2020d..e4f28548 100644 --- a/tests/test_debugger.py +++ b/tests/test_debugger.py @@ -12,7 +12,6 @@ TODO: """ from functools import partial import itertools -from typing import Optional import platform import pathlib import time @@ -30,6 +29,7 @@ from tractor._testing import ( from tractor.devx._debug import ( _pause_msg, _crash_msg, + _repl_fail_msg, ) from .conftest import ( _ci_env, @@ -144,9 +144,10 @@ def in_prompt_msg( 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() @@ -158,6 +159,10 @@ def in_prompt_msg( 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], @@ -165,6 +170,7 @@ def assert_before( **kwargs, ) -> None: + __tracebackhide__: bool = False # as in before the prompt end before: str = str(child.before.decode()) @@ -217,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. @@ -282,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. @@ -463,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) @@ -476,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 @@ -745,8 +762,9 @@ def test_multi_daemon_subactors( # boxed error raised in root task # "Attaching to pdb in crashed actor: ('root'", _crash_msg, - "('root'", - "_exceptions.RemoteActorError: ('name_error'", + "('root'", # should attach in root + "_exceptions.RemoteActorError:", # with an embedded RAE for.. + "('name_error'", # the src subactor which raised ] ) @@ -847,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? @@ -870,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) @@ -1019,13 +1047,16 @@ 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( @@ -1040,52 +1071,289 @@ def test_pause_from_sync( ''' child = spawn('sync_bp') + + # first `sync_pause()` after nurseries open child.expect(PROMPT) assert_before( child, [ - '`greenback` portal opened!', # pre-prompt line - _pause_msg, "('root'", + _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 cd9dc022..508fdbe1 100644 --- a/tests/test_discovery.py +++ b/tests/test_discovery.py @@ -26,7 +26,7 @@ async def test_reg_then_unreg(reg_addr): portal = await n.start_actor('actor', enable_modules=[__name__]) uid = portal.channel.uid - async with tractor.get_arbiter(*reg_addr) as aportal: + async with tractor.get_registry(*reg_addr) as aportal: # this local actor should be the arbiter assert actor is aportal.actor @@ -160,7 +160,7 @@ async def spawn_and_check_registry( async with tractor.open_root_actor( registry_addrs=[reg_addr], ): - async with tractor.get_arbiter(*reg_addr) as portal: + async with tractor.get_registry(*reg_addr) as portal: # runtime needs to be up to call this actor = tractor.current_actor() @@ -298,7 +298,7 @@ async def close_chans_before_nursery( async with tractor.open_root_actor( registry_addrs=[reg_addr], ): - async with tractor.get_arbiter(*reg_addr) as aportal: + async with tractor.get_registry(*reg_addr) as aportal: try: get_reg = partial(unpack_reg, aportal) diff --git a/tests/test_docs_examples.py b/tests/test_docs_examples.py index 63ad07a2..fdf54bca 100644 --- a/tests/test_docs_examples.py +++ b/tests/test_docs_examples.py @@ -19,7 +19,7 @@ from tractor._testing import ( @pytest.fixture def run_example_in_subproc( loglevel: str, - testdir, + testdir: pytest.Testdir, reg_addr: tuple[str, int], ): diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 8d34bef4..45722a63 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -444,6 +444,7 @@ def test_basic_interloop_channel_stream(reg_addr, fan_out): infect_asyncio=True, fan_out=fan_out, ) + # should raise RAE diectly await portal.result() trio.run(main) @@ -461,12 +462,11 @@ def test_trio_error_cancels_intertask_chan(reg_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.boxed_type == Exception + # ensure boxed error type + excinfo.value.boxed_type == Exception def test_trio_closes_early_and_channel_exits(reg_addr): @@ -477,7 +477,7 @@ def test_trio_closes_early_and_channel_exits(reg_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 @@ -492,15 +492,17 @@ def test_aio_errors_and_channel_propagates_and_closes(reg_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.boxed_type == Exception + excinfo.value.boxed_type == Exception @tractor.context diff --git a/tests/test_inter_peer_cancellation.py b/tests/test_inter_peer_cancellation.py index 470287fb..7bf9a2bd 100644 --- a/tests/test_inter_peer_cancellation.py +++ b/tests/test_inter_peer_cancellation.py @@ -55,9 +55,10 @@ from tractor._testing import ( @tractor.context -async def sleep_forever( +async def open_stream_then_sleep_forever( ctx: Context, expect_ctxc: bool = False, + ) -> None: ''' Sync the context, open a stream then just sleep. @@ -67,6 +68,10 @@ async def sleep_forever( ''' 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() @@ -100,7 +105,7 @@ async def error_before_started( ''' async with tractor.wait_for_actor('sleeper') as p2: async with ( - p2.open_context(sleep_forever) as (peer_ctx, first), + 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 @@ -185,6 +190,10 @@ async def sleep_a_bit_then_cancel_peer( 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( @@ -200,9 +209,13 @@ async def stream_ints( @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 ( @@ -236,20 +249,54 @@ async def stream_from_peer( 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: - ctxerr = ctxc + # NOTE: cancellation of the (sleeper) peer should always cause + # a `ContextCancelled` raise in this streaming actor. + except ContextCancelled as _ctxc: + ctxc = _ctxc - assert peer_ctx._remote_error is ctxerr - assert peer_ctx._remote_error.msgdata == ctxerr.msgdata + # 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 ctxerr.canceller + assert "canceller" in ctxc.canceller # caller peer should not be the cancel requester assert not ctx.cancel_called @@ -273,12 +320,13 @@ async def stream_from_peer( # TODO / NOTE `.canceller` won't have been set yet # here because that machinery is inside - # `.open_context().__aexit__()` BUT, if we had + # `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! @@ -382,12 +430,13 @@ def test_peer_canceller( try: async with ( sleeper.open_context( - sleep_forever, + 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( @@ -413,10 +462,11 @@ def test_peer_canceller( # should always raise since this root task does # not request the sleeper cancellation ;) - except ContextCancelled as ctxerr: + except ContextCancelled as _ctxc: + ctxc = _ctxc print( 'CAUGHT REMOTE CONTEXT CANCEL\n\n' - f'{ctxerr}\n' + f'{ctxc}\n' ) # canceller and caller peers should not @@ -427,7 +477,7 @@ def test_peer_canceller( # we were not the actor, our peer was assert not sleeper_ctx.cancel_acked - assert ctxerr.canceller[0] == 'canceller' + assert ctxc.canceller[0] == 'canceller' # XXX NOTE XXX: since THIS `ContextCancelled` # HAS NOT YET bubbled up to the @@ -438,7 +488,7 @@ def test_peer_canceller( # CASE_1: error-during-ctxc-handling, if error_during_ctxerr_handling: - raise RuntimeError('Simulated error during teardown') + raise RuntimeError('Simulated RTE re-raise during ctxc handling') # CASE_2: standard teardown inside in `.open_context()` block raise @@ -503,6 +553,9 @@ def test_peer_canceller( # 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 @@ -511,9 +564,6 @@ def test_peer_canceller( # a `trio.Cancelled` due to a local # `._scope.cancel()` call. assert not sleeper_ctx._scope.cancelled_caught - - assert isinstance(loc_err, RuntimeError) - print(f'_loc_err: {_loc_err}\n') # assert sleeper_ctx._local_error is _loc_err # assert sleeper_ctx._local_error is _loc_err assert not ( @@ -550,9 +600,12 @@ def test_peer_canceller( else: # the other 2 ctxs assert ( - re.canceller - == - canceller.channel.uid + isinstance(re, ContextCancelled) + and ( + re.canceller + == + canceller.channel.uid + ) ) # since the sleeper errors while handling a @@ -801,8 +854,7 @@ async def serve_subactors( async with open_nursery() as an: # sanity - if debug_mode: - assert tractor._state.debug_mode() + assert tractor._state.debug_mode() == debug_mode await ctx.started(peer_name) async with ctx.open_stream() as ipc: @@ -1081,7 +1133,6 @@ def test_peer_spawns_and_cancels_service_subactor( '-> root checking `client_ctx.result()`,\n' f'-> checking that sub-spawn {peer_name} is down\n' ) - # else: try: res = await client_ctx.result(hide_tb=False) diff --git a/tests/test_local.py b/tests/test_local.py index a019d771..ecdad5fe 100644 --- a/tests/test_local.py +++ b/tests/test_local.py @@ -38,7 +38,7 @@ async def test_self_is_registered_localportal(reg_addr): "Verify waiting on the arbiter to register itself using a local portal." actor = tractor.current_actor() assert actor.is_arbiter - async with tractor.get_arbiter(*reg_addr) as portal: + async with tractor.get_registry(*reg_addr) as portal: assert isinstance(portal, tractor._portal.LocalPortal) with trio.fail_after(0.2): diff --git a/tests/test_multi_program.py b/tests/test_multi_program.py index 92f4c52d..860eeebb 100644 --- a/tests/test_multi_program.py +++ b/tests/test_multi_program.py @@ -32,7 +32,7 @@ def test_abort_on_sigint(daemon): @tractor_test async def test_cancel_remote_arbiter(daemon, reg_addr): assert not tractor.current_actor().is_arbiter - async with tractor.get_arbiter(*reg_addr) as portal: + async with tractor.get_registry(*reg_addr) as portal: await portal.cancel_actor() time.sleep(0.1) @@ -41,7 +41,7 @@ async def test_cancel_remote_arbiter(daemon, reg_addr): # no arbiter socket should exist with pytest.raises(OSError): - async with tractor.get_arbiter(*reg_addr) as portal: + async with tractor.get_registry(*reg_addr) as portal: pass diff --git a/tests/test_pldrx_limiting.py b/tests/test_pldrx_limiting.py new file mode 100644 index 00000000..bb9a3ef7 --- /dev/null +++ b/tests/test_pldrx_limiting.py @@ -0,0 +1,364 @@ +''' +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: int = 10 + elif invalid_return: + msg_type_str: str = 'Return' + bad_value: str = 'yo' + else: + # XXX but should never be used below then.. + msg_type_str: str = '' + bad_value: 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'{bad_value}', + f'has type {type(bad_value)!r}', + 'not match type-spec', + f'`{msg_type_str}.pld: PldMsg|NoneType`', + ], + # only for debug + # post_mortem=True, + ), + 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_spawning.py b/tests/test_spawning.py index 5995ed2d..99ec9abc 100644 --- a/tests/test_spawning.py +++ b/tests/test_spawning.py @@ -2,7 +2,9 @@ Spawning basics """ -from typing import Optional +from typing import ( + Any, +) import pytest import trio @@ -25,13 +27,11 @@ async def spawn( async with tractor.open_root_actor( 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: # forks here @@ -95,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 @@ -105,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, diff --git a/tractor/__init__.py b/tractor/__init__.py index 31f59598..a27a3b59 100644 --- a/tractor/__init__.py +++ b/tractor/__init__.py @@ -31,7 +31,7 @@ from ._streaming import ( stream as stream, ) from ._discovery import ( - get_arbiter as get_arbiter, + get_registry as get_registry, find_actor as find_actor, wait_for_actor as wait_for_actor, query_actor as query_actor, @@ -43,11 +43,14 @@ from ._supervise import ( from ._state import ( current_actor as current_actor, is_root_process as is_root_process, + current_ipc_ctx as current_ipc_ctx, ) from ._exceptions import ( - RemoteActorError as RemoteActorError, - ModuleNotExposed as ModuleNotExposed, ContextCancelled as ContextCancelled, + ModuleNotExposed as ModuleNotExposed, + MsgTypeError as MsgTypeError, + RemoteActorError as RemoteActorError, + TransportClosed as TransportClosed, ) from .devx import ( breakpoint as breakpoint, diff --git a/tractor/_context.py b/tractor/_context.py index 51b23302..31db2bad 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -15,17 +15,30 @@ # 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 contextvars import Token from dataclasses import ( dataclass, field, @@ -33,310 +46,75 @@ from dataclasses import ( from functools import partial import inspect from pprint import pformat +import textwrap from typing import ( Any, - Callable, AsyncGenerator, + Callable, + Type, + TypeAlias, TYPE_CHECKING, + Union, ) import warnings - +# ------ - ------ import trio - +from trio.lowlevel import Task +# ------ - ------ from ._exceptions import ( ContextCancelled, InternalError, + MsgTypeError, RemoteActorError, StreamOverrun, - pack_error, + pack_from_raise, unpack_error, - _raise_from_no_key_in_msg, ) -from .log import get_logger -from .msg import NamespacePath -from ._ipc import Channel -from ._streaming import MsgStream +from .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__) -async def _drain_to_final_msg( - ctx: Context, - - hide_tb: bool = True, - msg_limit: int = 6, - -) -> list[dict]: - ''' - Drain IPC msgs delivered to the underlying rx-mem-chan - `Context._recv_chan` from the runtime in search for a final - result or error msg. - - The motivation here is to ideally capture errors during ctxc - conditions where a canc-request/or local error is sent but the - local task also excepts and enters the - `Portal.open_context().__aexit__()` block wherein we prefer to - capture and raise any remote error or ctxc-ack as part of the - `ctx.result()` cleanup and teardown sequence. - - ''' - __tracebackhide__: bool = hide_tb - raise_overrun: bool = not ctx._allow_overruns - - # wait for a final context result by collecting (but - # basically ignoring) any bi-dir-stream msgs still in transit - # from the far end. - pre_result_drained: list[dict] = [] - while not ( - ctx.maybe_error - and not ctx._final_result_is_set() - ): - try: - # TODO: can remove? - # await trio.lowlevel.checkpoint() - - # NOTE: this REPL usage actually works here dawg! Bo - # from .devx._debug import pause - # await pause() - - # TODO: bad idea? - # -[ ] wrap final outcome channel wait in a scope so - # it can be cancelled out of band if needed? - # - # with trio.CancelScope() as res_cs: - # ctx._res_scope = res_cs - # msg: dict = await ctx._recv_chan.receive() - # if res_cs.cancelled_caught: - - # TODO: ensure there's no more hangs, debugging the - # runtime pretty preaase! - # from .devx._debug import pause - # await pause() - - # TODO: can remove this finally? - # we have no more need for the sync draining right - # since we're can kinda guarantee the async - # `.receive()` below will never block yah? - # - # if ( - # ctx._cancel_called and ( - # ctx.cancel_acked - # # or ctx.chan._cancel_called - # ) - # # or not ctx._final_result_is_set() - # # ctx.outcome is not - # # or ctx.chan._closed - # ): - # try: - # msg: dict = await ctx._recv_chan.receive_nowait()() - # except trio.WouldBlock: - # log.warning( - # 'When draining already `.cancel_called` ctx!\n' - # 'No final msg arrived..\n' - # ) - # break - # else: - # msg: dict = await ctx._recv_chan.receive() - - # TODO: don't need it right jefe? - # with trio.move_on_after(1) as cs: - # if cs.cancelled_caught: - # from .devx._debug import pause - # await pause() - - # pray to the `trio` gawds that we're corrent with this - msg: dict = await ctx._recv_chan.receive() - - # NOTE: we get here if the far end was - # `ContextCancelled` in 2 cases: - # 1. we requested the cancellation and thus - # SHOULD NOT raise that far end error, - # 2. WE DID NOT REQUEST that cancel and thus - # SHOULD RAISE HERE! - except trio.Cancelled: - - # CASE 2: mask the local cancelled-error(s) - # only when we are sure the remote error is - # the source cause of this local task's - # cancellation. - ctx.maybe_raise() - - # CASE 1: we DID request the cancel we simply - # continue to bubble up as normal. - raise - - try: - ctx._result: Any = msg['return'] - log.runtime( - 'Context delivered final draining msg:\n' - f'{pformat(msg)}' - ) - # XXX: only close the rx mem chan AFTER - # a final result is retreived. - # if ctx._recv_chan: - # await ctx._recv_chan.aclose() - # TODO: ^ we don't need it right? - break - - except KeyError: - # always capture unexpected/non-result msgs - pre_result_drained.append(msg) - - if 'yield' in msg: - # far end task is still streaming to us so discard - # and report per local context state. - 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'{pformat(msg)}\n' - ) - return pre_result_drained - - # drain up to the `msg_limit` hoping to get - # a final result or error/ctxc. - else: - log.warning( - 'Ignoring "yield" msg during `ctx.result()` drain..\n' - f'<= {ctx.chan.uid}\n' - f' |_{ctx._nsf}()\n\n' - f'=> {ctx._task}\n' - f' |_{ctx._stream}\n\n' - - f'{pformat(msg)}\n' - ) - continue - - # TODO: work out edge cases here where - # a stream is open but the task also calls - # this? - # -[ ] should be a runtime error if a stream is open - # right? - elif 'stop' in msg: - log.cancel( - 'Remote stream terminated due to "stop" msg:\n\n' - f'{pformat(msg)}\n' - ) - continue - - # It's an internal error if any other msg type without - # a`'cid'` field arrives here! - if not msg.get('cid'): - raise InternalError( - 'Unexpected cid-missing msg?\n\n' - f'{msg}\n' - ) - - # XXX fallthrough to handle expected error XXX - # TODO: replace this with `ctx.maybe_raise()` - # - # TODO: would this be handier for this case maybe? - # async with maybe_raise_on_exit() as raises: - # if raises: - # log.error('some msg about raising..') - - re: Exception|None = ctx._remote_error - if re: - log.critical( - 'Remote ctx terminated due to "error" msg:\n' - f'{re}' - ) - assert msg is ctx._cancel_msg - # NOTE: this solved a super dupe edge case XD - # this was THE super duper edge case of: - # - local task opens a remote task, - # - requests remote cancellation of far end - # ctx/tasks, - # - needs to wait for the cancel ack msg - # (ctxc) or some result in the race case - # where the other side's task returns - # before the cancel request msg is ever - # rxed and processed, - # - here this surrounding drain loop (which - # iterates all ipc msgs until the ack or - # an early result arrives) was NOT exiting - # since we are the edge case: local task - # does not re-raise any ctxc it receives - # IFF **it** was the cancellation - # requester.. - # will raise if necessary, ow break from - # loop presuming any error terminates the - # context! - ctx._maybe_raise_remote_err( - re, - # NOTE: obvi we don't care if we - # overran the far end if we're already - # waiting on a final result (msg). - # raise_overrun_from_self=False, - raise_overrun_from_self=raise_overrun, - ) - - break # OOOOOF, yeah obvi we need this.. - - # XXX we should never really get here - # right! since `._deliver_msg()` should - # always have detected an {'error': ..} - # msg and already called this right!?! - elif error := unpack_error( - msg=msg, - chan=ctx._portal.channel, - hide_tb=False, - ): - log.critical('SHOULD NEVER GET HERE!?') - assert msg is ctx._cancel_msg - assert error.msgdata == ctx._remote_error.msgdata - from .devx._debug import pause - await pause() - ctx._maybe_cancel_and_set_remote_error(error) - ctx._maybe_raise_remote_err(error) - - else: - # bubble the original src key error - raise - else: - log.cancel( - 'Skipping `MsgStream` drain since final outcome is set\n\n' - f'{ctx.outcome}\n' - ) - - return pre_result_drained - - class Unresolved: ''' Placeholder value for `Context._result` until a final return value or raised error is resolved. ''' - ... # TODO: make this a .msg.types.Struct! @@ -345,23 +123,32 @@ class Unresolved: @dataclass class Context: ''' - An inter-actor, SC transitive, `trio.Task` communication context. + An inter-actor, SC transitive, `trio.Task` (pair) + communication context. - NB: This class should **never be instatiated directly**, it is allocated - by the runtime in 2 ways: - - by entering ``Portal.open_context()`` which is the primary - public API for any "caller" task or, + (We've also considered other names and ideas: + - "communicating tasks scope": cts + - "distributed task scope": dts + - "communicating tasks context": ctc + + **Got a better idea for naming? Make an issue dawg!** + ) + + NB: This class should **never be instatiated directly**, it is + allocated by the runtime in 2 ways: + - by entering `Portal.open_context()` which is the primary + public API for any "parent" task or, - by the RPC machinery's `._rpc._invoke()` as a `ctx` arg - to a remotely scheduled "callee" function. + to a remotely scheduled "child" function. - AND is always constructed using the below ``mk_context()``. + AND is always constructed using the below `mk_context()`. Allows maintaining task or protocol specific state between 2 cancel-scope-linked, communicating and parallel executing - `trio.Task`s. Contexts are allocated on each side of any task + `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()``. + always allocated inside `._rpc._invoke()`. TODO: more detailed writeup on cancellation, error and streaming semantics.. @@ -379,9 +166,47 @@ class Context: # the "feeder" channels for delivering message values to the # local task from the runtime's msg processing loop. - _recv_chan: trio.MemoryReceiveChannel + _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 @@ -396,14 +221,24 @@ class Context: # more the the `Context` is needed? _portal: Portal | None = None + @property + def portal(self) -> Portal|None: + ''' + Return any wrapping memory-`Portal` if this is + a 'parent'-side task which called `Portal.open_context()`, + otherwise `None`. + + ''' + return self._portal + # NOTE: each side of the context has its own cancel scope # which is exactly the primitive that allows for # cross-actor-task-supervision and thus SC. _scope: trio.CancelScope|None = None - _task: trio.lowlevel.Task|None = None + _task: Task|None = None # TODO: cs around result waiting so we can cancel any - # permanently blocking `._recv_chan.receive()` call in + # permanently blocking `._rx_chan.receive()` call in # a drain loop? # _res_scope: trio.CancelScope|None = None @@ -436,7 +271,7 @@ class Context: # 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 + _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 @@ -444,23 +279,29 @@ class Context: # a call to `.cancel()` which triggers `ContextCancelled`. _cancel_msg: str|dict|None = None - # NOTE: this state var used by the runtime to determine if the + # 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_tty_for_child()` will set it to `False` if + # - `.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 # init and streaming state _started_called: 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 # task, only an ability to not lose messages when the local @@ -479,13 +320,25 @@ class Context: # boxed exception. NOW, it's used for spawning overrun queuing # tasks when `.allow_overruns == True` !!! _scope_nursery: trio.Nursery|None = None + # ^-TODO-^ change name? + # -> `._scope_tn` "scope task nursery" # streaming overrun state tracking _in_overrun: bool = False _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 __str__(self) -> str: + 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? + indent: str|None = None, + + ) -> str: ds: str = '=' # ds: str = ': ' @@ -502,12 +355,7 @@ class Context: outcome_str: str = self.repr_outcome( show_error_fields=True ) - outcome_typ_str: str = self.repr_outcome( - type_only=True - ) - - return ( - f'\n' ) + if extra_fields: + for key, val in extra_fields.items(): + fmtstr += ( + f' {key}{ds}{val!r}\n' + ) + + if indent: + fmtstr = textwrap.indent( + fmtstr, + prefix=indent, + ) + + return ( + '\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 @@ -558,7 +425,8 @@ class Context: # logging perspective over `eval()`-ability since we do NOT # target serializing non-struct instances! # def __repr__(self) -> str: - __repr__ = __str__ + __str__ = pformat + __repr__ = pformat @property def cancel_called(self) -> bool: @@ -572,10 +440,23 @@ class Context: ''' return self._cancel_called + @cancel_called.setter + def cancel_called(self, val: bool) -> None: + ''' + Set the self-cancelled request `bool` value. + + ''' + # to debug who frickin sets it.. + # if val: + # from .devx import pause_from_sync + # pause_from_sync() + + self._cancel_called = val + @property def canceller(self) -> tuple[str, str]|None: ''' - ``Actor.uid: tuple[str, str]`` of the (remote) + `Actor.uid: tuple[str, str]` of the (remote) actor-process who's task was cancelled thus causing this (side of the) context to also be cancelled. @@ -596,8 +477,12 @@ class Context: re: BaseException|None = ( remote_error - or self._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 @@ -608,10 +493,10 @@ class Context: our_canceller = self.canceller return bool( - isinstance(re, ContextCancelled) + isinstance((ctxc := re), ContextCancelled) and from_uid == self.chan.uid - and re.canceller == our_uid - and our_canceller == from_uid + and ctxc.canceller == our_uid + and our_canceller == our_uid ) @property @@ -675,7 +560,7 @@ class Context: # the local scope was never cancelled # and instead likely we received a remote side - # # cancellation that was raised inside `.result()` + # # cancellation that was raised inside `.wait_for_result()` # or ( # (se := self._local_error) # and se is re @@ -692,33 +577,63 @@ class Context: Return string indicating which task this instance is wrapping. ''' - return 'caller' if self._portal else 'callee' + 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 pause() - await self.chan.send({ - 'stop': True, - 'cid': self.cid - }) + ''' + Terminate a `MsgStream` dialog-phase by sending the IPC + equiv of a `StopIteration`. + + ''' + await self.chan.send(Stop(cid=self.cid)) def _maybe_cancel_and_set_remote_error( self, error: BaseException, + # TODO: manual toggle for cases where we wouldn't normally + # mark ourselves cancelled but want to? + # set_cancel_called: bool = False, + ) -> None: ''' (Maybe) cancel this local scope due to a received remote @@ -737,7 +652,7 @@ class Context: - `Portal.open_context()` - `Portal.result()` - `Context.open_stream()` - - `Context.result()` + - `Context.wait_for_result()` when called/closed by actor local task(s). @@ -799,52 +714,71 @@ class Context: # appropriately. log.runtime( 'Setting remote error for ctx\n\n' - f'<= remote ctx uid: {self.chan.uid}\n' - f'=>{error}' + 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 - whom: str = ( - 'us' if error.canceller == self._actor.uid - else 'peer' - ) log.cancel( - f'IPC context cancelled by {whom}!\n\n' + f'IPC context was cancelled by {whom}!\n\n' f'{error}' ) - else: + elif isinstance(error, MsgTypeError): + msgerr = True + self._canceller = error.src_uid log.error( - f'Remote context error:\n\n' - + f'IPC dialog error due to msg-type caused by {self.peer_side!r} side\n\n' f'{error}\n' f'{pformat(self)}\n' ) - # always record the cancelling actor's uid since its - # cancellation state is linked and we want to know - # which process was the cause / requester of the - # cancellation. - maybe_error_src: tuple = getattr( - error, - 'src_uid', - None, - ) - self._canceller = ( - maybe_error_src - or - # XXX: in the case we get a non-boxed error? - # -> wait but this should never happen right? - self.chan.uid - ) + 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 + # once exiting (or manually calling `.wait_for_result()`) the # `.open_context()` block. cs: trio.CancelScope = self._scope if ( @@ -857,22 +791,65 @@ class Context: # 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 ): - # TODO: it'd sure be handy to inject our own - # `trio.Cancelled` subtype here ;) - # https://github.com/goodboy/tractor/issues/368 - self._scope.cancel() + if ( + msgerr - # TODO: maybe we should also call `._res_scope.cancel()` if it - # exists to support cancelling any drain loop hangs? - # NOTE: this usage actually works here B) - # from .devx._debug import breakpoint - # await breakpoint() + # 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' + # from .devx import pause_from_sync + # pause_from_sync() + self._scope.cancel() + else: + message: str = 'NOT cancelling `Context._scope` !\n\n' + # from .devx import mk_pdb + # mk_pdb().set_trace() - # TODO: add to `Channel`? + 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 + ) + + # TODO: also add to `Channel`? @property def dst_maddr(self) -> str: chan: Channel = self.chan @@ -893,9 +870,7 @@ class Context: dmaddr = dst_maddr @property - def repr_rpc( - self, - ) -> str: + 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( @@ -904,9 +879,31 @@ class Context: ) return ( # f'{self._nsf}() -{{{codec}}}-> {repr(self.outcome)}:' - f'{self._nsf}() -> {outcome_str}:' + 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}' + ) + + return '' + + @property + def repr_api(self) -> str: + return 'Portal.open_context()' + + # TODO: use `.dev._frame_stack` scanning to find caller! + # ci: CallerInfo|None = self._caller_info + # if ci: + # return ( + # f'{ci.api_nsp}()\n' + # ) + async def cancel( self, timeout: float = 0.616, @@ -914,7 +911,7 @@ class Context: ) -> None: ''' Cancel this inter-actor IPC context by requestng the - remote side's cancel-scope-linked `trio.Task` by calling + remote side's cancel-scope-linked `Task` by calling `._scope.cancel()` and delivering an `ContextCancelled` ack msg in reponse. @@ -942,16 +939,18 @@ class Context: ''' side: str = self.side - self._cancel_called: bool = True + # XXX for debug via the `@.setter` + self.cancel_called = True header: str = ( - f'Cancelling ctx with peer from {side.upper()} side\n\n' + f'Cancelling ctx from {side.upper()}-side\n' ) reminfo: str = ( # ' =>\n' - f'Context.cancel() => {self.chan.uid}\n' + # f'Context.cancel() => {self.chan.uid}\n' + f'c)=> {self.chan.uid}\n' # f'{self.chan.uid}\n' - f' |_ @{self.dst_maddr}\n' + f' |_ @{self.dst_maddr}\n' f' >> {self.repr_rpc}\n' # f' >> {self._nsf}() -> {codec}[dict]:\n\n' # TODO: pull msg-type from spec re #320 @@ -965,13 +964,13 @@ class Context: # `._scope.cancel()` since we expect the eventual # `ContextCancelled` from the other side to trigger this # when the runtime finally receives it during teardown - # (normally in `.result()` called from + # (normally in `.wait_for_result()` called from # `Portal.open_context().__aexit__()`) - if side == 'caller': + if side == 'parent': if not self._portal: raise InternalError( 'No portal found!?\n' - 'Why is this supposed caller context missing it?' + 'Why is this supposed {self.side!r}-side ctx task missing it?!?' ) cid: str = self.cid @@ -1034,208 +1033,27 @@ class Context: assert self._scope self._scope.cancel() - # TODO? should we move this to `._streaming` much like we - # moved `Portal.open_context()`'s def to this mod? - @acm - async def open_stream( - self, - allow_overruns: bool|None = False, - msg_buffer_size: int|None = None, - - ) -> AsyncGenerator[MsgStream, None]: - ''' - Open a ``MsgStream``, a bi-directional stream connected to the - cross-actor (far end) task for this ``Context``. - - This context manager must be entered on both the caller and - callee for the stream to logically be considered "connected". - - A ``MsgStream`` is currently "one-shot" use, meaning if you - close it you can not "re-open" it for streaming and instead you - must re-establish a new surrounding ``Context`` using - ``Portal.open_context()``. In the future this may change but - currently there seems to be no obvious reason to support - "re-opening": - - pausing a stream can be done with a message. - - task errors will normally require a restart of the entire - scope of the inter-actor task context due to the nature of - ``trio``'s cancellation system. - - ''' - actor: Actor = self._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 self._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! - self.maybe_raise( - raise_ctxc_from_self_call=True, - ) - # NOTE: this is diff then calling - # `._maybe_raise_remote_err()` specifically - # because we want to raise a ctxc on any task entering this `.open_stream()` - # AFTER cancellation was already been requested, - # we DO NOT want to absorb any ctxc ACK silently! - # if self._remote_error: - # raise self._remote_error - - # XXX NOTE: if no `ContextCancelled` has been responded - # back from the other side (yet), we raise a different - # runtime error indicating that this task's usage of - # `Context.cancel()` and then `.open_stream()` is WRONG! - task: str = trio.lowlevel.current_task().name - raise RuntimeError( - 'Stream opened after `Context.cancel()` called..?\n' - f'task: {actor.uid[0]}:{task}\n' - f'{self}' - ) - - if ( - not self._portal - and not self._started_called - ): - 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: Context = actor.get_context( - chan=self.chan, - cid=self.cid, - nsf=self._nsf, - # side=self.side, - - msg_buffer_size=msg_buffer_size, - allow_overruns=allow_overruns, - ) - ctx._allow_overruns: bool = 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!\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=self, - rx_chan=ctx._recv_chan, - ) as stream: - - # NOTE: we track all existing streams per portal for - # the purposes of attempting graceful closes on runtime - # cancel requests. - if self._portal: - self._portal._streams.add(stream) - - try: - self._stream_opened: bool = True - self._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._push_result()`` the msg will be discarded and in - # the case where that msg is global debugger unlock (via - # a "stop" msg for a stream), this can result in a deadlock - # where the root is waiting on the lock to clear but the - # child has already cleared it and clobbered IPC. - # - # await maybe_wait_for_debugger() - - # XXX TODO: pretty sure this isn't needed (see - # note above this block) AND will result in - # a double `.send_stop()` call. The only reason to - # put it here would be to due with "order" in - # terms of raising any remote error (as per - # directly below) or bc the stream's - # `.__aexit__()` block might not get run - # (doubtful)? Either way if we did put this back - # in we also need a state var to avoid the double - # stop-msg send.. - # - # await stream.aclose() - - # NOTE: absorb and do not raise any - # EoC received from the other side such that - # it is not raised inside the surrounding - # context block's scope! - except trio.EndOfChannel as eoc: - if ( - eoc - and stream.closed - ): - # sanity, can remove? - assert eoc is stream._eoc - # from .devx import pause - # await pause() - log.warning( - 'Stream was terminated by EoC\n\n' - # NOTE: won't show the error but - # does show txt followed by IPC msg. - f'{str(eoc)}\n' - ) - - 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}' - ) - - # TODO: replace all the instances of this!! XD + # TODO: replace all the `._maybe_raise_remote_err()` usage + # with instances of this!! def maybe_raise( self, hide_tb: bool = True, **kwargs, ) -> Exception|None: + ''' + Check for for a remote error delivered by the runtime from + our peer (task); if set immediately raise. + + This is a convenience wrapper for + `._maybe_raise_remote_err(self._remote_error)`. + + ''' __tracebackhide__: bool = hide_tb if re := self._remote_error: return self._maybe_raise_remote_err( re, + hide_tb=hide_tb, **kwargs, ) @@ -1243,6 +1061,7 @@ class Context: self, 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, @@ -1258,12 +1077,12 @@ class Context: ''' __tracebackhide__: bool = hide_tb - our_uid: tuple = self.chan.uid + peer_uid: tuple = self.chan.uid # XXX NOTE XXX: `ContextCancelled`/`StreamOverrun` absorption - # for "graceful cancellation" case: + # for "graceful cancellation" case(s): # - # Whenever a "side" of a context (a `trio.Task` running in + # 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 @@ -1278,9 +1097,11 @@ class Context: # set to the `Actor.uid` of THIS task (i.e. the # cancellation requesting task's actor is the actor # checking whether it should absorb the ctxc). + self_ctxc: bool = self._is_self_cancelled(remote_error) if ( + self_ctxc + and not raise_ctxc_from_self_call - and self._is_self_cancelled(remote_error) # TODO: ?potentially it is useful to emit certain # warning/cancel logs for the cases where the @@ -1303,13 +1124,15 @@ class Context: # boxed `StreamOverrun`. This is mostly useful for # supressing such faults during # cancellation/error/final-result handling inside - # `_drain_to_final_msg()` such that we do not + # `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.msgdata['boxed_type_str'] == 'StreamOverrun' - and tuple(remote_error.msgdata['sender']) == our_uid + and remote_error.boxed_type is StreamOverrun + + # and tuple(remote_error.msgdata['sender']) == peer_uid + and tuple(remote_error.sender) == peer_uid ): # NOTE: we set the local scope error to any "self # cancellation" error-response thus "absorbing" @@ -1320,7 +1143,8 @@ class Context: else: log.warning( 'Local error already set for ctx?\n' - f'{self._local_error}\n' + f'{self._local_error}\n\n' + f'{self}' ) return remote_error @@ -1335,11 +1159,12 @@ class Context: # runtime frames from the tb explicitly? # https://docs.python.org/3/reference/simple_stmts.html#the-raise-statement # https://stackoverflow.com/a/24752607 - __tracebackhide__: bool = True - raise remote_error # from None + if from_src_exc is not False: + raise remote_error from from_src_exc - # TODO: change to `.wait_for_result()`? - async def result( + raise remote_error + + async def wait_for_result( self, hide_tb: bool = True, @@ -1369,49 +1194,55 @@ class Context: of the remote cancellation. ''' - __tracebackhide__ = hide_tb + __tracebackhide__: bool = hide_tb assert self._portal, ( - "Context.result() can not be called from callee side!" + '`Context.wait_for_result()` can not be called from callee side!' ) if self._final_result_is_set(): return self._result - assert self._recv_chan + assert self._rx_chan raise_overrun: bool = not self._allow_overruns - # res_placeholder: int = id(self) if ( - # self._result == res_placeholder - # and not self._remote_error self.maybe_error is None - # not self._remote_error - # and not self._local_error - and not self._recv_chan._closed # type: ignore + and + not self._rx_chan._closed # type: ignore ): - # wait for a final context result/error by "draining" # (by more or less ignoring) any bi-dir-stream "yield" # msgs still in transit from the far end. - drained_msgs: list[dict] = await _drain_to_final_msg( + # + # 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, ) - for msg in drained_msgs: - # TODO: mask this by default.. - if 'return' in msg: - # from .devx import pause - # await pause() - raise InternalError( - 'Final `return` msg should never be drained !?!?\n\n' - f'{msg}\n' - ) - - log.cancel( - 'Ctx drained pre-result msgs:\n' - f'{drained_msgs}' + 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' + ) + + log.cancel(drained_status) 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 @@ -1420,46 +1251,40 @@ class Context: # raising something we know might happen # during cancellation ;) (not self._cancel_called) - ) + ), + hide_tb=hide_tb, ) - # if ( - # (re := self._remote_error) - # # and self._result == res_placeholder - # ): - # self._maybe_raise_remote_err( - # re, - # # NOTE: obvi we don't care if we - # # overran the far end if we're already - # # waiting on a final result (msg). - # # raise_overrun_from_self=False, - # raise_overrun_from_self=( - # raise_overrun - # and - # # only when we ARE NOT the canceller - # # should we raise overruns, bc ow we're - # # raising something we know might happen - # # during cancellation ;) - # (not self._cancel_called) - # ), - # ) - # if maybe_err: - # self._result = maybe_err - + # TODO: eventually make `.outcome: Outcome` and thus return + # `self.outcome.unwrap()` here? return self.outcome - # TODO: switch this with above which should be named - # `.wait_for_outcome()` and instead do - # a `.outcome.Outcome.unwrap()` ? - # @property - # def result(self) -> Any|None: - # if self._final_result_is_set(): - # return self._result - - # raise RuntimeError('No result is available!') + # 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, + ) @property def maybe_error(self) -> BaseException|None: - le: Exception|None = self._local_error + ''' + Return the (remote) error as outcome or `None`. + + Remote errors take precedence over local ones. + + ''' + le: BaseException|None = self._local_error re: RemoteActorError|ContextCancelled|None = self._remote_error match (le, re): @@ -1487,7 +1312,7 @@ class Context: # ContextCancelled(canceller=), # ): - error: Exception|None = le or re + error: BaseException|None = le or re if error: return error @@ -1508,27 +1333,36 @@ class Context: return None def _final_result_is_set(self) -> bool: - # return not (self._result == id(self)) return self._result is not Unresolved # def get_result_nowait(self) -> Any|None: + # 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 + # TODO: make this a `outcome.Outcome`! ): ''' - The final "outcome" from an IPC context which can either be - some Value returned from the target `@context`-decorated - remote task-as-func, or an `Error` wrapping an exception - raised from an RPC task fault or cancellation. + Return the "final outcome" (state) of the far end peer task + non-blocking. If the remote task has not completed then this + field always resolves to the module defined `Unresolved` + handle. - Note that if the remote task has not terminated then this - field always resolves to the module defined `Unresolved` handle. + ------ - ------ + TODO->( this is doc-driven-dev content not yet actual ;P ) - TODO: implement this using `outcome.Outcome` types? + The final "outcome" from an IPC context which can be any of: + - some `outcome.Value` which boxes the returned output from the peer task's + `@context`-decorated remote task-as-func, or + - an `outcome.Error` wrapping an exception raised that same RPC task + after a fault or cancellation, or + - an unresolved `outcome.Outcome` when the peer task is still + executing and has not yet completed. ''' return ( @@ -1537,7 +1371,10 @@ class Context: self._result ) - # @property + @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, @@ -1559,7 +1396,8 @@ class Context: # just deliver the type name. if ( (reprol := getattr(merr, 'reprol', False)) - and show_error_fields + and + show_error_fields ): return reprol() @@ -1576,10 +1414,6 @@ class Context: repr(merr) ) - # just the type name - # else: # but wen? - # return type(merr).__name__ - # for all other errors show their regular output return ( str(merr) @@ -1593,9 +1427,88 @@ class Context: 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: ''' @@ -1616,8 +1529,52 @@ class Context: 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 + + 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, @@ -1671,7 +1628,7 @@ class Context: async def _deliver_msg( self, - msg: dict, + msg: MsgType, ) -> bool: ''' @@ -1685,6 +1642,20 @@ class Context: `._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 + `.wait_for_result()` we need the msg to be + `send_chan.send_nowait()`-ed over the `._rx_chan` so + that the error is relayed to that waiter task and thus + raised in user code! + ''' cid: str = self.cid chan: Channel = self.chan @@ -1692,6 +1663,18 @@ class Context: send_chan: trio.MemorySendChannel = self._send_chan nsf: NamespacePath = self._nsf + 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' + ) + re: Exception|None if re := unpack_error( msg, @@ -1703,64 +1686,43 @@ class Context: log_meth = log.runtime log_meth( - f'Delivering error-msg to caller\n\n' - - f'<= peer: {from_uid}\n' - f' |_ {nsf}()\n\n' - - f'=> cid: {cid}\n' - f' |_{self._task}\n\n' + 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 - # NOTE: this will not raise an error, merely set + # 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) - # XXX NEVER do this XXX..!! - # bc if the error is a ctxc and there is a task - # waiting on `.result()` we need the msg to be sent - # over the `send_chan`/`._recv_chan` so that the error - # is relayed to that waiter task.. - # return True - # - # XXX ALSO NO!! XXX - # => NEVER raise remote errors from the calling - # runtime task, they should always be raised by - # consumer side tasks operating on the - # `Portal`/`Context` APIs. - # if self._remote_error: - # self._maybe_raise_remote_err(error) - + # 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' - f'<= peer: {from_uid}\n' - f' |_ {nsf}()\n\n' + f'Queueing OVERRUN msg on caller task:\n\n' - f'=> cid: {cid}\n' - f' |_{self._task}\n\n' + f'{flow_body}' - f'{pformat(msg)}\n' + 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' - f'<= {from_uid}\n' - f' |_ {nsf}()\n\n' + f'Delivering msg from IPC ctx:\n\n' - f'=> {self._task}\n' - f' |_cid={self.cid}\n\n' + f'{flow_body}' - f'{pformat(msg)}\n' + f'{structfmt(msg)}\n' ) # NOTE: if an error is deteced we should always still @@ -1790,6 +1752,7 @@ class Context: f'cid: {self.cid}\n' 'Failed to deliver msg:\n' f'send_chan: {send_chan}\n\n' + f'{pformat(msg)}\n' ) return False @@ -1855,35 +1818,36 @@ class Context: # anything different. return False else: - txt += f'\n{msg}\n' # raise local overrun and immediately pack as IPC # msg for far end. - try: - raise StreamOverrun( + err_msg: Error = pack_from_raise( + local_err=StreamOverrun( txt, sender=from_uid, - ) - except StreamOverrun as err: - err_msg: dict[str, dict] = pack_error( - err, - cid=cid, - ) - try: - # relay condition to sender side remote task - await chan.send(err_msg) - return True + ), + cid=cid, + ) + try: + # relay condition to sender side remote task + await chan.send(err_msg) + return True - except trio.BrokenResourceError: - # XXX: local consumer has closed their side - # so cancel the far end streaming task - log.warning( - 'Channel for ctx is already closed?\n' - f'|_{chan}\n' - ) + # 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? @@ -1904,12 +1868,7 @@ async def open_context_from_portal( func: Callable, allow_overruns: bool = False, - - # TODO: if we set this the wrapping `@acm` body will - # still be shown (awkwardly) on pdb REPL entry. Ideally - # we can similarly annotate that frame to NOT show? for now - # we DO SHOW this frame since it's awkward ow.. - hide_tb: bool = False, + hide_tb: bool = True, # proxied to RPC **kwargs, @@ -1937,7 +1896,7 @@ async def open_context_from_portal( When the "callee" (side that is "called"/started by a call to *this* method) returns, the caller side (this) unblocks and any final value delivered from the other end can be - retrieved using the `Contex.result()` api. + retrieved using the `Contex.wait_for_result()` api. The yielded ``Context`` instance further allows for opening bidirectional streams, explicit cancellation and @@ -1947,12 +1906,26 @@ async def open_context_from_portal( ''' __tracebackhide__: bool = hide_tb - # conduct target func method structural checks - if not inspect.iscoroutinefunction(func) and ( - getattr(func, '_tractor_contex_function', False) + # 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} must be an async generator function!') + 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 @@ -1962,7 +1935,7 @@ async def open_context_from_portal( # XXX NOTE XXX: currenly we do NOT allow opening a contex # with "self" since the local feeder mem-chan processing # is not built for it. - if portal.channel.uid == portal.actor.uid: + if (uid := portal.channel.uid) == portal.actor.uid: raise RuntimeError( '** !! Invalid Operation !! **\n' 'Can not open an IPC ctx with the local actor!\n' @@ -1974,6 +1947,8 @@ async def open_context_from_portal( 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 @@ -1984,92 +1959,108 @@ async def open_context_from_portal( # place.. allow_overruns=allow_overruns, ) - assert ctx._remote_func_type == 'context' - msg: dict = await ctx._recv_chan.receive() - - try: - # the "first" value here is delivered by the callee's - # ``Context.started()`` call. - first: Any = msg['started'] - ctx._started_called: bool = True - - except KeyError as src_error: - _raise_from_no_key_in_msg( - ctx=ctx, - msg=msg, - src_err=src_error, - log=log, - expect_key='started', - ) - - ctx._portal: Portal = portal - uid: tuple = portal.channel.uid - cid: str = ctx.cid + assert ctx._caller_info + prior_ctx_tok: Token = _ctxvar_Context.set(ctx) # placeholder for any exception raised in the runtime # or by user tasks which cause this context's closure. scope_err: BaseException|None = None ctxc_from_callee: ContextCancelled|None = None try: - async with trio.open_nursery() as nurse: + 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 - # NOTE: used to start overrun queuing tasks - ctx._scope_nursery: trio.Nursery = nurse - ctx._scope: trio.CancelScope = nurse.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 - # deliver context instance and .started() msg value - # in enter tuple. + # 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()` + # the `else:` block handling via a `.wait_for_result()` # call below enough?? - # -[ ] pretty sure `.result()` internals do the + # + # -[ ] pretty sure `.wait_for_result()` internals do the # same as our ctxc handler below so it ended up # being same (repeated?) behaviour, but ideally we # wouldn't have that duplication either by somehow - # factoring the `.result()` handler impl in a way + # factoring the `.wait_for_result()` handler impl in a way # that we can re-use it around the `yield` ^ here # or vice versa? # - # NOTE: between the caller exiting and arriving - # here the far end may have sent a ctxc-msg or - # other error, so check for it here immediately - # and maybe raise so as to engage the ctxc - # handling block below! + # 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 ???? # - # if re := ctx._remote_error: - # maybe_ctxc: ContextCancelled|None = ctx._maybe_raise_remote_err( - # re, - # # TODO: do we want this to always raise? - # # - means that on self-ctxc, if/when the - # # block is exited before the msg arrives - # # but then the msg during __exit__ - # # calling we may not activate the - # # ctxc-handler block below? should we - # # be? - # # - if there's a remote error that arrives - # # after the child has exited, we won't - # # handle until the `finally:` block - # # where `.result()` is always called, - # # again in which case we handle it - # # differently then in the handler block - # # that would normally engage from THIS - # # block? - # raise_ctxc_from_self_call=True, - # ) - # ctxc_from_callee = maybe_ctxc + # self.maybe_raise() # when in allow_overruns mode there may be # lingering overflow sender tasks remaining? - if nurse.child_tasks: + 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(nurse.child_tasks) > 1 + or len(tn.child_tasks) > 1 ): raise InternalError( 'Context has sub-tasks but is ' @@ -2143,7 +2134,7 @@ async def open_context_from_portal( # CASE 2: context was cancelled by local task calling # `.cancel()`, we don't raise and the exit block should - # exit silently. + # finish silently. if ( ctx._cancel_called and @@ -2187,7 +2178,7 @@ async def open_context_from_portal( # AND a group-exc is only raised if there was > 1 # tasks started *here* in the "caller" / opener # block. If any one of those tasks calls - # `.result()` or `MsgStream.receive()` + # `.wait_for_result()` or `MsgStream.receive()` # `._maybe_raise_remote_err()` will be transitively # called and the remote error raised causing all # tasks to be cancelled. @@ -2207,9 +2198,17 @@ async def open_context_from_portal( # the `ContextCancelled` "self cancellation absorbed" case # handled in the block above ^^^ !! # await _debug.pause() - log.cancel( - 'Context terminated due to\n\n' - f'.outcome => {ctx.repr_outcome()}\n' + # log.cancel( + match scope_err: + case trio.Cancelled: + logmeth = log.cancel + + # XXX explicitly report on any non-graceful-taskc cases + case _: + logmeth = log.exception + + logmeth( + f'ctx {ctx.side!r}-side exited with {ctx.repr_outcome()}\n' ) if debug_mode(): @@ -2241,8 +2240,8 @@ async def open_context_from_portal( ): log.warning( 'IPC connection for context is broken?\n' - f'task:{cid}\n' - f'actor:{uid}' + f'task: {ctx.cid}\n' + f'actor: {uid}' ) raise # duh @@ -2256,7 +2255,7 @@ async def open_context_from_portal( f'|_{ctx._task}\n' ) # XXX NOTE XXX: the below call to - # `Context.result()` will ALWAYS raise + # `Context.wait_for_result()` will ALWAYS raise # a `ContextCancelled` (via an embedded call to # `Context._maybe_raise_remote_err()`) IFF # a `Context._remote_error` was set by the runtime @@ -2266,10 +2265,10 @@ async def open_context_from_portal( # ALWAYS SET any time "callee" side fails and causes "caller # side" cancellation via a `ContextCancelled` here. try: - result_or_err: Exception|Any = await ctx.result() + result_or_err: Exception|Any = await ctx.wait_for_result() except BaseException as berr: # on normal teardown, if we get some error - # raised in `Context.result()` we still want to + # raised in `Context.wait_for_result()` we still want to # save that error on the ctx's state to # determine things like `.cancelled_caught` for # cases where there was remote cancellation but @@ -2279,35 +2278,24 @@ async def open_context_from_portal( ctx._local_error: BaseException = scope_err raise - # yes! this worx Bp + # 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. - msgdata: str|None = getattr( - result_or_err, - 'msgdata', - None - ) - match (msgdata, result_or_err): - case ( - {'tb_str': tbstr}, - ContextCancelled(), - ): - log.cancel(tbstr) + match result_or_err: + case ContextCancelled() as ctxc: + log.cancel(ctxc.tb_str) - case ( - {'tb_str': tbstr}, - RemoteActorError(), - ): + case RemoteActorError() as rae: log.exception( 'Context remotely errored!\n' f'<= peer: {uid}\n' f' |_ {nsf}()\n\n' - f'{tbstr}' + f'{rae.tb_str}' ) case (None, _): log.runtime( @@ -2317,12 +2305,11 @@ async def open_context_from_portal( f'`{result_or_err}`\n' ) - finally: # XXX: (MEGA IMPORTANT) if this is a root opened process we # wait for any immediate child in debug before popping the # context from the runtime msg loop otherwise inside - # ``Actor._push_result()`` the msg will be discarded and in + # ``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 @@ -2336,7 +2323,7 @@ async def open_context_from_portal( # we tear down the runtime feeder chan last # to avoid premature stream clobbers. if ( - (rxchan := ctx._recv_chan) + (rxchan := ctx._rx_chan) # maybe TODO: yes i know the below check is # touching `trio` memchan internals..BUT, there are @@ -2377,9 +2364,9 @@ async def open_context_from_portal( # # NOTE: further, this should be the only place the # underlying feeder channel is - # once-and-only-CLOSED! + # once-forever-and-only-CLOSED! with trio.CancelScope(shield=True): - await ctx._recv_chan.aclose() + await ctx._rx_chan.aclose() # XXX: we always raise remote errors locally and # generally speaking mask runtime-machinery related @@ -2399,9 +2386,9 @@ async def open_context_from_portal( and ctx.cancel_acked ): log.cancel( - 'Context cancelled by caller task\n' - f'|_{ctx._task}\n\n' - + f'Context cancelled by local {ctx.side!r}-side task\n' + f'c)>\n' + f' |_{ctx._task}\n\n' f'{repr(scope_err)}\n' ) @@ -2417,22 +2404,27 @@ async def open_context_from_portal( # type_only=True, ) log.cancel( - f'Context terminated due to local scope error:\n\n' - f'{ctx.chan.uid} => {outcome_str}\n' + f'Context terminated due to {ctx.side!r}-side\n\n' + # TODO: do an x)> on err and c)> only for ctxc? + f'c)> {outcome_str}\n' + f' |_{ctx.repr_rpc}\n' ) # FINALLY, remove the context from runtime tracking and # exit! log.runtime( - 'Removing IPC ctx opened with peer\n' - f'{uid}\n' - f'|_{ctx}\n' + 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, cid), + (uid, ctx.cid), None, ) + # XXX revert to prior IPC-task-ctx scope + _ctxvar_Context.reset(prior_ctx_tok) + def mk_context( chan: Channel, @@ -2440,6 +2432,7 @@ def mk_context( nsf: NamespacePath, msg_buffer_size: int = 2**6, + pld_spec: Union[Type] = Any, **kwargs, @@ -2455,36 +2448,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=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, ) - # TODO: we can drop the old placeholder yah? - # 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/_discovery.py b/tractor/_discovery.py index 99a4dd68..a681c63b 100644 --- a/tractor/_discovery.py +++ b/tractor/_discovery.py @@ -26,8 +26,8 @@ from typing import ( TYPE_CHECKING, ) from contextlib import asynccontextmanager as acm -import warnings +from tractor.log import get_logger from .trionics import gather_contexts from ._ipc import _connect_chan, Channel from ._portal import ( @@ -40,11 +40,13 @@ from ._state import ( _runtime_vars, ) - if TYPE_CHECKING: from ._runtime import Actor +log = get_logger(__name__) + + @acm async def get_registry( host: str, @@ -56,14 +58,12 @@ async def get_registry( ]: ''' Return a portal instance connected to a local or remote - arbiter. + registry-service actor; if a connection already exists re-use it + (presumably to call a `.register_actor()` registry runtime RPC + ep). ''' - actor = current_actor() - - if not actor: - raise RuntimeError("No actor instance has been defined yet?") - + actor: Actor = current_actor() if actor.is_registrar: # we're already the arbiter # (likely a re-entrant call from the arbiter actor) @@ -72,6 +72,8 @@ async def get_registry( Channel((host, port)) ) else: + # TODO: try to look pre-existing connection from + # `Actor._peers` and use it instead? async with ( _connect_chan(host, port) as chan, open_portal(chan) as regstr_ptl, @@ -80,19 +82,6 @@ async def get_registry( -# TODO: deprecate and this remove _arbiter form! -@acm -async def get_arbiter(*args, **kwargs): - warnings.warn( - '`tractor.get_arbiter()` is now deprecated!\n' - 'Use `.get_registry()` instead!', - DeprecationWarning, - stacklevel=2, - ) - async with get_registry(*args, **kwargs) as to_yield: - yield to_yield - - @acm async def get_root( **kwargs, @@ -110,22 +99,53 @@ async def get_root( yield portal +def get_peer_by_name( + name: str, + # uuid: str|None = None, + +) -> list[Channel]|None: # at least 1 + ''' + Scan for an existing connection (set) to a named actor + and return any channels from `Actor._peers`. + + This is an optimization method over querying the registrar for + the same info. + + ''' + actor: Actor = current_actor() + to_scan: dict[tuple, list[Channel]] = actor._peers.copy() + pchan: Channel|None = actor._parent_chan + if pchan: + to_scan[pchan.uid].append(pchan) + + for aid, chans in to_scan.items(): + _, peer_name = aid + if name == peer_name: + if not chans: + log.warning( + 'No IPC chans for matching peer {peer_name}\n' + ) + continue + return chans + + return None + + @acm async def query_actor( name: str, - arbiter_sockaddr: tuple[str, int] | None = None, - regaddr: tuple[str, int] | None = None, + regaddr: tuple[str, int]|None = None, ) -> AsyncGenerator[ - tuple[str, int] | None, + tuple[str, int]|None, None, ]: ''' - Make a transport address lookup for an actor name to a specific - registrar. + Lookup a transport address (by actor name) via querying a registrar + listening @ `regaddr`. - Returns the (socket) address or ``None`` if no entry under that - name exists for the given registrar listening @ `regaddr`. + Returns the transport protocol (socket) address or `None` if no + entry under that name exists. ''' actor: Actor = current_actor() @@ -137,14 +157,10 @@ async def query_actor( 'The current actor IS the registry!?' ) - if arbiter_sockaddr is not None: - warnings.warn( - '`tractor.query_actor(regaddr=)` is deprecated.\n' - 'Use `registry_addrs: list[tuple]` instead!', - DeprecationWarning, - stacklevel=2, - ) - regaddr: list[tuple[str, int]] = arbiter_sockaddr + maybe_peers: list[Channel]|None = get_peer_by_name(name) + if maybe_peers: + yield maybe_peers[0].raddr + return reg_portal: Portal regaddr: tuple[str, int] = regaddr or actor.reg_addrs[0] @@ -159,10 +175,28 @@ async def query_actor( yield sockaddr +@acm +async def maybe_open_portal( + addr: tuple[str, int], + name: str, +): + async with query_actor( + name=name, + regaddr=addr, + ) as sockaddr: + pass + + if sockaddr: + async with _connect_chan(*sockaddr) as chan: + async with open_portal(chan) as portal: + yield portal + else: + yield None + + @acm async def find_actor( name: str, - arbiter_sockaddr: tuple[str, int]|None = None, registry_addrs: list[tuple[str, int]]|None = None, only_first: bool = True, @@ -179,29 +213,12 @@ async def find_actor( known to the arbiter. ''' - if arbiter_sockaddr is not None: - warnings.warn( - '`tractor.find_actor(arbiter_sockaddr=)` is deprecated.\n' - 'Use `registry_addrs: list[tuple]` instead!', - DeprecationWarning, - stacklevel=2, - ) - registry_addrs: list[tuple[str, int]] = [arbiter_sockaddr] - - @acm - async def maybe_open_portal_from_reg_addr( - addr: tuple[str, int], - ): - async with query_actor( - name=name, - regaddr=addr, - ) as sockaddr: - if sockaddr: - async with _connect_chan(*sockaddr) as chan: - async with open_portal(chan) as portal: - yield portal - else: - yield None + # optimization path, use any pre-existing peer channel + maybe_peers: list[Channel]|None = get_peer_by_name(name) + if maybe_peers and only_first: + async with open_portal(maybe_peers[0]) as peer_portal: + yield peer_portal + return if not registry_addrs: # XXX NOTE: make sure to dynamically read the value on @@ -217,10 +234,13 @@ async def find_actor( maybe_portals: list[ AsyncContextManager[tuple[str, int]] ] = list( - maybe_open_portal_from_reg_addr(addr) + maybe_open_portal( + addr=addr, + name=name, + ) for addr in registry_addrs ) - + portals: list[Portal] async with gather_contexts( mngrs=maybe_portals, ) as portals: @@ -254,31 +274,31 @@ async def find_actor( @acm async def wait_for_actor( name: str, - arbiter_sockaddr: tuple[str, int] | None = None, registry_addr: tuple[str, int] | None = None, ) -> AsyncGenerator[Portal, None]: ''' - Wait on an actor to register with the arbiter. - - A portal to the first registered actor is returned. + Wait on at least one peer actor to register `name` with the + registrar, yield a `Portal to the first registree. ''' actor: Actor = current_actor() - if arbiter_sockaddr is not None: - warnings.warn( - '`tractor.wait_for_actor(arbiter_sockaddr=)` is deprecated.\n' - 'Use `registry_addr: tuple` instead!', - DeprecationWarning, - stacklevel=2, - ) - registry_addr: tuple[str, int] = arbiter_sockaddr + # optimization path, use any pre-existing peer channel + maybe_peers: list[Channel]|None = get_peer_by_name(name) + if maybe_peers: + async with open_portal(maybe_peers[0]) as peer_portal: + yield peer_portal + return + regaddr: tuple[str, int] = ( + registry_addr + or + actor.reg_addrs[0] + ) # TODO: use `.trionics.gather_contexts()` like # above in `find_actor()` as well? reg_portal: Portal - regaddr: tuple[str, int] = registry_addr or actor.reg_addrs[0] async with get_registry(*regaddr) as reg_portal: sockaddrs = await reg_portal.run_from_ns( 'self', diff --git a/tractor/_entry.py b/tractor/_entry.py index 21c9ae48..a072706c 100644 --- a/tractor/_entry.py +++ b/tractor/_entry.py @@ -20,6 +20,8 @@ Sub-process entry points. """ from __future__ import annotations from functools import partial +import os +import textwrap from typing import ( Any, TYPE_CHECKING, @@ -32,6 +34,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, @@ -56,7 +59,7 @@ def _mp_main( ) -> None: ''' - The routine called *after fork* which invokes a fresh ``trio.run`` + The routine called *after fork* which invokes a fresh `trio.run()` ''' actor._forkserver_info = forkserver_info @@ -91,11 +94,111 @@ 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") + + +# TODO: move this func to some kinda `.devx._conc_lang.py` eventually +# as we work out our multi-domain state-flow-syntax! +def nest_from_op( + input_op: str, + # + # ?TODO? an idea for a syntax to the state of concurrent systems + # as a "3-domain" (execution, scope, storage) model and using + # a minimal ascii/utf-8 operator-set. + # + # try not to take any of this seriously yet XD + # + # > is a "play operator" indicating (CPU bound) + # exec/work/ops required at the "lowest level computing" + # + # execution primititves (tasks, threads, actors..) denote their + # lifetime with '(' and ')' since parentheses normally are used + # in many langs to denote function calls. + # + # starting = ( + # >( opening/starting; beginning of the thread-of-exec (toe?) + # (> opened/started, (finished spawning toe) + # |_ repr of toe, in py these look like + # + # >) closing/exiting/stopping, + # )> closed/exited/stopped, + # |_ + # [OR <), )< ?? ] + # + # ending = ) + # >c) cancelling to close/exit + # c)> cancelled (caused close), OR? + # |_ + # OR maybe "x) erroring to eventuall exit + # x)> errored and terminated + # |_ + # + # scopes: supers/nurseries, IPC-ctxs, sessions, perms, etc. + # >{ opening + # {> opened + # }> closed + # >} closing + # + # storage: like queues, shm-buffers, files, etc.. + # >[ opening + # [> opened + # |_ + # + # >] closing + # ]> closed + + # IPC ops: channels, transports, msging + # => req msg + # <= resp msg + # <=> 2-way streaming (of msgs) + # <- recv 1 msg + # -> send 1 msg + # + # TODO: still not sure on R/L-HS approach..? + # =>( send-req to exec start (task, actor, thread..) + # (<= recv-req to ^ + # + # (<= recv-req ^ + # <=( recv-resp opened remote exec primitive + # <=) recv-resp closed + # + # )<=c req to stop due to cancel + # c=>) req to stop due to cancel + # + # =>{ recv-req to open + # <={ send-status that it closed + + tree_str: str, + + # NOTE: so move back-from-the-left of the `input_op` by + # this amount. + back_from_op: int = 0, +) -> str: + ''' + Depth-increment the input (presumably hierarchy/supervision) + input "tree string" below the provided `input_op` execution + operator, so injecting a `"\n|_{input_op}\n"`and indenting the + `tree_str` to nest content aligned with the ops last char. + + ''' + return ( + f'{input_op}\n' + + + textwrap.indent( + tree_str, + prefix=( + len(input_op) + - + (back_from_op + 1) + ) * ' ', + ) + ) def _trio_main( - actor: Actor, *, parent_addr: tuple[str, int] | None = None, @@ -106,7 +209,8 @@ def _trio_main( Entry point for a `trio_run_in_process` subactor. ''' - __tracebackhide__: bool = True + _debug.hide_runtime_frames() + _state._current_actor = actor trio_main = partial( async_main, @@ -116,7 +220,6 @@ def _trio_main( if actor.loglevel is not None: get_console_log(actor.loglevel) - import os actor_info: str = ( f'|_{actor}\n' f' uid: {actor.uid}\n' @@ -125,27 +228,54 @@ def _trio_main( f' loglevel: {actor.loglevel}\n' ) log.info( - 'Started new trio process:\n' + 'Starting new `trio` subactor:\n' + - actor_info + nest_from_op( + input_op='>(', # see syntax ideas above + tree_str=actor_info, + back_from_op=1, + ) ) - + logmeth = log.info + exit_status: str = ( + 'Subactor exited\n' + + + nest_from_op( + input_op=')>', # like a "closed-to-play"-icon from super perspective + tree_str=actor_info, + back_from_op=1, + ) + ) try: if infect_asyncio: actor._infected_aio = True run_as_asyncio_guest(trio_main) else: trio.run(trio_main) + except KeyboardInterrupt: - log.cancel( - 'Actor received KBI\n' + logmeth = log.cancel + exit_status: str = ( + 'Actor received KBI (aka an OS-cancel)\n' + - actor_info + nest_from_op( + input_op='c)>', # closed due to cancel (see above) + tree_str=actor_info, + ) ) + except BaseException as err: + logmeth = log.error + exit_status: str = ( + 'Main actor task exited due to crash?\n' + + + nest_from_op( + input_op='x)>', # closed by error + tree_str=actor_info, + ) + ) + # NOTE since we raise a tb will already be shown on the + # console, thus we do NOT use `.exception()` above. + raise err finally: - log.info( - 'Actor terminated\n' - + - actor_info - ) + logmeth(exit_status) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 0e1d6d10..108134ca 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -22,8 +22,13 @@ 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, ) @@ -31,9 +36,29 @@ import textwrap import traceback import trio +from msgspec import ( + defstruct, + msgpack, + structs, + ValidationError, +) 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 @@ -57,21 +82,47 @@ class InternalError(RuntimeError): ''' -_body_fields: list[str] = [ - 'boxed_type', - 'src_type', - # TODO: format this better if we're going to include it. - # 'relay_path', - 'src_uid', - # only in sub-types - 'canceller', - 'sender', +# 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) ] -_msgdata_keys: list[str] = [ - 'boxed_type_str', -] + _body_fields +_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: @@ -97,7 +148,40 @@ def get_err_type(type_name: str) -> BaseException|None: return type_ref -# TODO: rename to just `RemoteError`? +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): ''' A box(ing) type which bundles a remote actor `BaseException` for @@ -110,18 +194,39 @@ class RemoteActorError(Exception): ''' reprol_fields: list[str] = [ 'src_uid', - 'relay_path', + # '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, + ipc_msg: Error|None = None, boxed_type: Type[BaseException]|None = None, - **msgdata + + # 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) + # for manual display without having to muck with `Exception.args` + self._message: str = message # TODO: maybe a better name? # - .errtype # - .retype @@ -130,12 +235,24 @@ class RemoteActorError(Exception): # - .remote_type # also pertains to our long long oustanding issue XD # https://github.com/goodboy/tractor/issues/5 - # - # TODO: always set ._boxed_type` as `None` by default - # and instead render if from `.boxed_type_str`? self._boxed_type: BaseException = boxed_type self._src_type: BaseException|None = None - self.msgdata: dict[str, Any] = msgdata + 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? @@ -154,14 +271,68 @@ class RemoteActorError(Exception): # either by customizing `ContextCancelled.__init__()` or # through a special factor func? elif boxed_type: - if not self.msgdata.get('boxed_type_str'): - self.msgdata['boxed_type_str'] = str( - type(boxed_type).__name__ - ) + 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 - assert self.boxed_type_str == self.msgdata['boxed_type_str'] + # ensure any roundtripping evals to the input value assert self.boxed_type is boxed_type + @property + 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: ''' @@ -171,17 +342,20 @@ class RemoteActorError(Exception): at the first relay/hop's receiving actor. ''' - return self.msgdata['src_type_str'] + 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.msgdata['src_type_str'] + self._ipc_msg.src_type_str ) return self._src_type @@ -192,17 +366,21 @@ class RemoteActorError(Exception): String-name of the (last hop's) boxed error type. ''' - return self.msgdata['boxed_type_str'] + bt: Type[BaseException] = self.boxed_type + if bt: + return str(bt.__name__) + + return '' @property - def boxed_type(self) -> str: + 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.msgdata['boxed_type_str'] + self._ipc_msg.boxed_type_str ) return self._boxed_type @@ -215,40 +393,44 @@ class RemoteActorError(Exception): actor's hop. NOTE: a `list` field with the same name is expected to be - passed/updated in `.msgdata`. + passed/updated in `.ipc_msg`. ''' - return self.msgdata['relay_path'] + return self._ipc_msg.relay_path @property def relay_uid(self) -> tuple[str, str]|None: return tuple( - self.msgdata['relay_path'][-1] + self._ipc_msg.relay_path[-1] ) @property def src_uid(self) -> tuple[str, str]|None: if src_uid := ( - self.msgdata.get('src_uid') + self._ipc_msg.src_uid ): return tuple(src_uid) # TODO: use path lookup instead? # return tuple( - # self.msgdata['relay_path'][0] + # self._ipc_msg.relay_path[0] # ) @property def tb_str( self, - indent: str = ' '*3, + indent: str = '', ) -> str: - if remote_tb := self.msgdata.get('tb_str'): - return textwrap.indent( - remote_tb, - prefix=indent, - ) + remote_tb: str = '' - return '' + 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, @@ -256,21 +438,32 @@ class RemoteActorError(Exception): 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 - self.msgdata.get(key) + 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}' + return _repr def reprol(self) -> str: @@ -281,55 +474,177 @@ class RemoteActorError(Exception): ''' # TODO: use this matryoshka emjoi XD # => 🪆 - reprol_str: str = f'{type(self).__name__}(' + 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 __repr__(self) -> str: + def is_inception(self) -> bool: ''' - Nicely formatted boxed error meta data + traceback. + 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. ''' - fields: str = self._mk_fields_str( - _body_fields, + # 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 ) - fields: str = textwrap.indent( - fields, - # prefix=' '*2, - prefix=' |_', - ) - indent: str = ''*1 - body: str = ( - f'{fields}' - f' |\n' - f' ------ - ------\n\n' - f'{self.tb_str}\n' - f' ------ - ------\n' - f' _|\n' - ) - if indent: - body: str = textwrap.indent( - body, - prefix=indent, + + # 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 ' + + + tail + ) + + __repr__ = pformat + + # NOTE: apparently we need this so that + # the full fields show in debugger tests? + # |_ i guess `pexepect` relies on `str`-casing + # of output? + def __str__(self) -> 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. + 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 @@ -367,14 +682,13 @@ class RemoteActorError(Exception): # # boxed_type=get_type_ref(.. # raise NotImplementedError - -class InternalActorError(RemoteActorError): - ''' - (Remote) internal `tractor` error indicating failure of some - primitive, machinery state or lowlevel task that should never - occur. - - ''' + @property + def sender(self) -> tuple[str, str]|None: + if ( + (msg := self._ipc_msg) + and (value := msg.sender) + ): + return tuple(value) class ContextCancelled(RemoteActorError): @@ -386,6 +700,10 @@ class ContextCancelled(RemoteActorError): reprol_fields: list[str] = [ 'canceller', ] + extra_body_fields: list[str] = [ + 'cid', + 'canceller', + ] @property def canceller(self) -> tuple[str, str]|None: ''' @@ -407,7 +725,7 @@ class ContextCancelled(RemoteActorError): |_`._cancel_task()` ''' - value = self.msgdata.get('canceller') + value: tuple[str, str]|None = self._ipc_msg.canceller if value: return tuple(value) @@ -421,8 +739,226 @@ class ContextCancelled(RemoteActorError): # src_actor_uid = canceller -class TransportClosed(trio.ClosedResourceError): - "Underlying channel transport was closed prior to use" +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.BrokenResourceError): + ''' + IPC transport (protocol) connection was closed or broke and + indicates that the wrapping communication `Channel` can no longer + be used to send/receive msgs from the remote peer. + + ''' + def __init__( + self, + message: str, + loglevel: str = 'transport', + cause: BaseException|None = None, + raise_on_report: bool = False, + + ) -> None: + self.message: str = message + self._loglevel = loglevel + super().__init__(message) + + if cause is not None: + self.__cause__ = cause + + # flag to toggle whether the msg loop should raise + # the exc in its `TransportClosed` handler block. + self._raise_on_report = raise_on_report + + def report_n_maybe_raise( + self, + message: str|None = None, + + ) -> None: + ''' + Using the init-specified log level emit a logging report + for this error. + + ''' + message: str = message or self.message + # when a cause is set, slap it onto the log emission. + if cause := self.__cause__: + cause_tb_str: str = ''.join( + traceback.format_tb(cause.__traceback__) + ) + message += ( + f'{cause_tb_str}\n' # tb + f' {cause}\n' # exc repr + ) + + getattr(log, self._loglevel)(message) + + # some errors we want to blow up from + # inside the RPC msg loop + if self._raise_on_report: + raise self from cause class NoResult(RuntimeError): @@ -437,23 +973,6 @@ class NoRuntime(RuntimeError): "The root actor has not been initialized yet" -class StreamOverrun( - RemoteActorError, - trio.TooSlowError, -): - reprol_fields: list[str] = [ - 'sender', - ] - ''' - This stream was overrun by sender - - ''' - @property - def sender(self) -> tuple[str, str] | None: - value = self.msgdata.get('sender') - if value: - return tuple(value) - class AsyncioCancelled(Exception): ''' @@ -464,16 +983,23 @@ class AsyncioCancelled(Exception): ''' class MessagingError(Exception): - 'Some kind of unexpected SC messaging dialog issue' + ''' + IPC related msg (typing), transaction (ordering) or dialog + handling error. + + ''' def pack_error( exc: BaseException|RemoteActorError, - tb: str|None = None, cid: str|None = None, + src_uid: tuple[str, str]|None = None, + tb: TracebackType|None = None, + tb_str: str = '', + message: str = '', -) -> dict[str, dict]: +) -> Error: ''' Create an "error message" which boxes a locally caught exception's meta-data and encodes it for wire transport via an @@ -481,10 +1007,28 @@ def pack_error( the receiver side using `unpack_error()` below. ''' - if tb: - tb_str = ''.join(traceback.format_tb(tb)) + 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' + + # 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 error_msg: dict[ # for IPC str, @@ -497,7 +1041,8 @@ def pack_error( ): error_msg.update(exc.msgdata) - # an onion/inception we need to pack + # 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) @@ -521,36 +1066,32 @@ def pack_error( error_msg['boxed_type_str'] = 'RemoteActorError' else: - error_msg['src_uid'] = our_uid + 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 alawys append us the last relay in error propagation path + # XXX always append us the last relay in error propagation path error_msg.setdefault( 'relay_path', [], ).append(our_uid) - # XXX NOTE: always ensure the traceback-str is from the - # locally raised error (**not** the prior relay's boxed - # content's `.msgdata`). + # 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 - pkt: dict = {'error': error_msg} - if cid: - pkt['cid'] = cid - - return pkt + return Error(**error_msg) def unpack_error( - msg: dict[str, Any], - - chan: Channel|None = None, + msg: Error, + chan: Channel, box_type: RemoteActorError = RemoteActorError, - hide_tb: bool = True, - ) -> None|Exception: ''' Unpack an 'error' message from the wire @@ -560,30 +1101,27 @@ def unpack_error( which is the responsibilitiy of the caller. ''' - __tracebackhide__: bool = hide_tb - - error_dict: dict[str, dict] | None - if ( - error_dict := msg.get('error') - ) is None: - # no error field, nothing to unpack. + if not isinstance(msg, Error): return None - # retrieve the remote error's msg encoded details - tb_str: str = error_dict.get('tb_str', '') - message: str = ( - f'{chan.uid}\n' - + - tb_str - ) - # 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 = error_dict['boxed_type_str'] + boxed_type_str: str = msg.boxed_type_str boxed_type: Type[BaseException] = get_err_type(boxed_type_str) - if boxed_type_str == 'ContextCancelled': - box_type = ContextCancelled + # retrieve the error's msg-encoded remotoe-env info + message: str = f'remote task raised a {msg.boxed_type_str!r}\n' + + # 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 # TODO: already included by `_this_mod` in else loop right? @@ -593,24 +1131,26 @@ def unpack_error( # original source error. elif boxed_type_str == 'RemoteActorError': assert boxed_type is RemoteActorError - assert len(error_dict['relay_path']) >= 1 + assert len(msg.relay_path) >= 1 exc = box_type( message, - **error_dict, + 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 ``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) @@ -619,14 +1159,13 @@ def is_multi_cancelled(exc: BaseException) -> bool: return False -def _raise_from_no_key_in_msg( +def _raise_from_unexpected_msg( ctx: Context, - msg: dict, - src_err: KeyError, + msg: MsgType, + src_err: Exception, log: StackLevelAdapter, # caller specific `log` obj - expect_key: str = 'yield', - stream: MsgStream | None = None, + expect_msg: Type[MsgType], # allow "deeper" tbs when debugging B^o hide_tb: bool = True, @@ -658,8 +1197,8 @@ def _raise_from_no_key_in_msg( # an internal error should never get here try: - cid: str = msg['cid'] - except KeyError as src_err: + 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' @@ -668,69 +1207,238 @@ def _raise_from_no_key_in_msg( ) 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 msg.get('error'): - raise unpack_error( + if isinstance(msg, Error): + # match msg: + # case Error(): + exc: RemoteActorError = unpack_error( msg, ctx.chan, - hide_tb=hide_tb, - - ) from None + ) + 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? - elif ( - msg.get('stop') - or ( - stream - and stream._eoc - ) - ): - log.debug( - f'Context[{cid}] stream was stopped by remote side\n' - f'cid: {cid}\n' - ) + # case Stop(): + elif stream := ctx._stream: + _type: str = 'MsgStream' - # 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! + 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) - # 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' - ) - # 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 + # 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! - # in case there already is some underlying remote error - # that arrived which is probably the source of this stream - # closure - ctx.maybe_raise() + # 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) - raise eoc from src_err + # XXX: important to set so that a new `.receive()` + # call (likely by another task using a broadcast receiver) + # doesn't accidentally pull the `return` message + # value out of the underlying feed mem chan which is + # destined for the `Context.result()` call during ctx-exit! + stream._eoc: Exception = eoc - if ( - stream - and stream._closed - ): - raise trio.ClosedResourceError('This stream was closed') + # 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. - _type: str = 'Stream' if stream else 'Context' raise MessagingError( - f"{_type} was expecting a '{expect_key}' message" - " BUT received a non-error msg:\n" - f'{pformat(msg)}' + 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 f57d3bd8..a1cb0359 100644 --- a/tractor/_ipc.py +++ b/tractor/_ipc.py @@ -23,13 +23,17 @@ from collections.abc import ( AsyncGenerator, AsyncIterator, ) -from contextlib import asynccontextmanager as acm +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, Protocol, Type, @@ -41,15 +45,38 @@ from tricycle import BufferedReceiveStream import trio from tractor.log import get_logger -from tractor._exceptions import TransportClosed +from tractor._exceptions import ( + MsgTypeError, + pack_from_raise, + TransportClosed, + _mk_send_mte, + _mk_recv_mte, +) +from tractor.msg import ( + _ctxvar_MsgCodec, + # _codec, XXX see `self._codec` sanity/debug checks + MsgCodec, + types as msgtypes, + pretty_struct, +) log = get_logger(__name__) _is_windows = platform.system() == 'Windows' -def get_stream_addrs(stream: trio.SocketStream) -> tuple: - # should both be IP sockets +def get_stream_addrs( + stream: trio.SocketStream +) -> tuple[ + tuple[str, int], # local + tuple[str, int], # remote +]: + ''' + Return the `trio` streaming transport prot's socket-addrs for + both the local and remote sides as a pair. + + ''' + # rn, should both be IP sockets lsockname = stream.socket.getsockname() rsockname = stream.socket.getpeername() return ( @@ -58,16 +85,22 @@ def get_stream_addrs(stream: trio.SocketStream) -> tuple: ) -MsgType = TypeVar("MsgType") - -# TODO: consider using a generic def and indexing with our eventual -# msg definition/types? -# - https://docs.python.org/3/library/typing.html#typing.Protocol -# - https://jcristharif.com/msgspec/usage.html#structs +# from tractor.msg.types import MsgType +# ?TODO? this should be our `Union[*msgtypes.__spec__]` alias now right..? +# => BLEH, except can't bc prots must inherit typevar or param-spec +# vars.. +MsgType = TypeVar('MsgType') +# TODO: break up this mod into a subpkg so we can start adding new +# backends and move this type stuff into a dedicated file.. Bo +# @runtime_checkable class MsgTransport(Protocol[MsgType]): +# +# ^-TODO-^ consider using a generic def and indexing with our +# eventual msg definition/types? +# - https://docs.python.org/3/library/typing.html#typing.Protocol stream: trio.SocketStream drained: list[MsgType] @@ -102,9 +135,9 @@ class MsgTransport(Protocol[MsgType]): ... -# TODO: not sure why we have to inherit here, but it seems to be an -# issue with ``get_msg_transport()`` returning a ``Type[Protocol]``; -# probably should make a `mypy` issue? +# TODO: typing oddity.. not sure why we have to inherit here, but it +# seems to be an issue with `get_msg_transport()` returning +# a `Type[Protocol]`; probably should make a `mypy` issue? class MsgpackTCPStream(MsgTransport): ''' A ``trio.SocketStream`` delivering ``msgpack`` formatted data @@ -123,6 +156,16 @@ class MsgpackTCPStream(MsgTransport): 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,30 +175,44 @@ class MsgpackTCPStream(MsgTransport): self._laddr, self._raddr = get_stream_addrs(stream) # create read loop instance - self._agen = self._iter_packets() + self._aiter_pkts = self._iter_packets() self._send_lock = trio.StrictFIFOLock() # public i guess? 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() + + # XXX for ctxvar debug only! + # self._codec: MsgCodec = ( + # codec + # or + # _codec._ctxvar_MsgCodec.get() + # ) 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`. + + This is a streaming routine implemented as an async generator + func (which was the original design, but could be changed?) + and is allocated by a `.__call__()` inside `.__init__()` where + it is assigned to the `._aiter_pkts` attr. ''' - 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, @@ -164,25 +221,122 @@ class MsgpackTCPStream(MsgTransport): # seem to be getting racy failures here on # arbiter/registry name subs.. trio.BrokenResourceError, - ): - raise TransportClosed( - f'transport {self} was already closed prior ro read' - ) + ) as trans_err: + + loglevel = 'transport' + match trans_err: + # case ( + # ConnectionResetError() + # ): + # loglevel = 'transport' + + # peer actor (graceful??) TCP EOF but `tricycle` + # seems to raise a 0-bytes-read? + case ValueError() if ( + 'unclean EOF' in trans_err.args[0] + ): + pass + + # peer actor (task) prolly shutdown quickly due + # to cancellation + case trio.BrokenResourceError() if ( + 'Connection reset by peer' in trans_err.args[0] + ): + pass + + # unless the disconnect condition falls under "a + # normal operation breakage" we usualy console warn + # about it. + case _: + loglevel: str = 'warning' + + + raise TransportClosed( + message=( + f'IPC transport already closed by peer\n' + f'x)> {type(trans_err)}\n' + f' |_{self}\n' + ), + loglevel=loglevel, + ) from trans_err + + # XXX definitely can happen if transport is closed + # manually by another `trio.lowlevel.Task` in the + # same actor; we use this in some simulated fault + # testing for ex, but generally should never happen + # under normal operation! + # + # NOTE: as such we always re-raise this error from the + # RPC msg loop! + except trio.ClosedResourceError as closure_err: + raise TransportClosed( + message=( + f'IPC transport already manually closed locally?\n' + f'x)> {type(closure_err)} \n' + f' |_{self}\n' + ), + loglevel='error', + raise_on_report=( + closure_err.args[0] == 'another task closed this fd' + or + closure_err.args[0] in ['another task closed this fd'] + ), + ) from closure_err + + # graceful TCP EOF disconnect if header == b'': raise TransportClosed( - f'transport {self} was already closed prior ro read' + message=( + f'IPC transport already gracefully closed\n' + f')>\n' + f'|_{self}\n' + ), + loglevel='transport', + # cause=??? # handy or no? ) + size: int size, = struct.unpack(" None: ''' - Send a msgpack coded blob-as-msg over TCP. + 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 + __tracebackhide__: bool = hide_tb + + # XXX see `trio._sync.AsyncContextManagerMixin` for details + # on the `.acquire()`/`.release()` sequencing.. 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() + + # XXX for ctxvar debug only! + # if self._codec.pld_spec != codec.pld_spec: + # self._codec = codec + # log.runtime( + # f'Using new codec in {self}.send()\n' + # f'codec: {self._codec}\n\n' + # f'msg: {msg}\n' + # ) + + if type(msg) not in msgtypes.__msg_types__: + if strict_types: + 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(" + # except BaseException as _err: + # err = _err + # if not isinstance(err, MsgTypeError): + # __tracebackhide__: bool = False + # raise + @property def laddr(self) -> tuple[str, int]: return self._laddr @@ -235,7 +445,7 @@ class MsgpackTCPStream(MsgTransport): return self._raddr async def recv(self) -> Any: - return await self._agen.asend(None) + return await self._aiter_pkts.asend(None) async def drain(self) -> AsyncIterator[dict]: ''' @@ -252,7 +462,7 @@ class MsgpackTCPStream(MsgTransport): yield msg def __aiter__(self): - return self._agen + return self._aiter_pkts def connected(self) -> bool: return self.stream.socket.fileno() != -1 @@ -307,7 +517,7 @@ class Channel: # set after handshake - always uid of far end self.uid: tuple[str, str]|None = None - self._agen = self._aiter_recv() + self._aiter_msgs = self._iter_msgs() self._exc: Exception|None = None # set if far end actor errors self._closed: bool = False @@ -318,7 +528,9 @@ class Channel: @property def msgstream(self) -> MsgTransport: - log.info('`Channel.msgstream` is an old name, use `._transport`') + log.info( + '`Channel.msgstream` is an old name, use `._transport`' + ) return self._transport @property @@ -349,11 +561,45 @@ class Channel: stream: trio.SocketStream, 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._transport = get_msg_transport(type_key)(stream) + 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 + @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 + + # TODO: do a .src/.dst: str for maddrs? def __repr__(self) -> str: if not self._transport: return '' @@ -397,33 +643,53 @@ class Channel: ) return transport + # TODO: something like, + # `pdbp.hideframe_on(errors=[MsgTypeError])` + # instead of the `try/except` hack we have rn.. + # seems like a pretty useful thing to have in general + # along with being able to filter certain stack frame(s / sets) + # possibly based on the current log-level? async def send( self, payload: Any, - # hide_tb: bool = False, + hide_tb: bool = False, ) -> None: ''' Send a coded msg-blob over the transport. ''' - # __tracebackhide__: bool = hide_tb - log.transport( - '=> send IPC msg:\n\n' - f'{pformat(payload)}\n' - ) # type: ignore - assert self._transport + __tracebackhide__: bool = hide_tb + try: + log.transport( + '=> send IPC msg:\n\n' + f'{pformat(payload)}\n' + ) + # assert self._transport # but why typing? + await self._transport.send( + payload, + hide_tb=hide_tb, + ) + except BaseException as _err: + err = _err # bind for introspection + if not isinstance(_err, MsgTypeError): + # assert err + __tracebackhide__: bool = False + else: + assert err.cid - await self._transport.send( - payload, - # hide_tb=hide_tb, - ) + raise async def recv(self) -> Any: 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._transport.recv() # except trio.BrokenResourceError: @@ -450,8 +716,11 @@ class Channel: await self.aclose(*args) def __aiter__(self): - return self._agen + return self._aiter_msgs + # ?TODO? run any reconnection sequence? + # -[ ] prolly should be impl-ed as deco-API? + # # async def _reconnect(self) -> None: # """Handle connection failures by polling until a reconnect can be # established. @@ -469,7 +738,6 @@ class Channel: # else: # log.transport("Stream connection re-established!") - # # TODO: run any reconnection sequence # # on_recon = self._recon_seq # # if on_recon: # # await on_recon(self) @@ -483,23 +751,42 @@ class Channel: # " for re-establishment") # await trio.sleep(1) - async def _aiter_recv( + async def _iter_msgs( self ) -> AsyncGenerator[Any, None]: ''' - Async iterate items from underlying stream. + Yield `MsgType` IPC msgs decoded and deliverd from + an underlying `MsgTransport` protocol. + + This is a streaming routine alo implemented as an async-gen + func (same a `MsgTransport._iter_pkts()`) gets allocated by + a `.__call__()` inside `.__init__()` where it is assigned to + the `._aiter_msgs` attr. ''' assert self._transport while True: try: - async for item in self._transport: - yield item - # sent = yield item - # if sent is not None: - # # optimization, passing None through all the - # # time is pointless - # await self._transport.send(sent) + 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 + except trio.BrokenResourceError: # if not self._autorecon: @@ -529,4 +816,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/_portal.py b/tractor/_portal.py index ac602dd5..f5a66836 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -31,7 +31,7 @@ from typing import ( Any, Callable, AsyncGenerator, - # Type, + TYPE_CHECKING, ) from functools import partial from dataclasses import dataclass @@ -45,9 +45,14 @@ from ._state import ( ) from ._ipc import Channel from .log import get_logger -from .msg import NamespacePath +from .msg import ( + # Error, + PayloadMsg, + NamespacePath, + Return, +) from ._exceptions import ( - unpack_error, + # unpack_error, NoResult, ) from ._context import ( @@ -58,41 +63,12 @@ from ._streaming import ( MsgStream, ) +if TYPE_CHECKING: + from ._runtime import Actor log = get_logger(__name__) -# TODO: rename to `unwrap_result()` and use -# `._raise_from_no_key_in_msg()` (after tweak to -# accept a `chan: Channel` arg) in key block! -def _unwrap_msg( - msg: dict[str, Any], - channel: Channel, - - hide_tb: bool = True, - -) -> Any: - ''' - Unwrap a final result from a `{return: }` IPC msg. - - ''' - __tracebackhide__: bool = hide_tb - - try: - return msg['return'] - except KeyError as ke: - - # internal error should never get here - assert msg.get('cid'), ( - "Received internal error at portal?" - ) - - raise unpack_error( - msg, - channel - ) from ke - - class Portal: ''' A 'portal' to a memory-domain-separated `Actor`. @@ -116,17 +92,26 @@ class Portal: # connected (peer) actors. cancel_timeout: float = 0.5 - def __init__(self, channel: Channel) -> None: - self.chan = channel + def __init__( + self, + channel: Channel, + ) -> None: + + self._chan: Channel = channel # during the portal's lifetime - self._result_msg: dict|None = 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 chan(self) -> Channel: + return self._chan @property def channel(self) -> Channel: @@ -140,6 +125,8 @@ class Portal: ) return self.chan + # TODO: factor this out into a `.highlevel` API-wrapper that uses + # a single `.open_context()` call underneath. async def _submit_for_result( self, ns: str, @@ -147,32 +134,34 @@ 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, nsf=NamespacePath(f'{ns}:{func}'), - kwargs=kwargs + kwargs=kwargs, + portal=self, ) - async def _return_once( + # TODO: we should deprecate this API right? since if we remove + # `.run_in_actor()` (and instead move it to a `.highlevel` + # wrapper api (around a single `.open_context()` call) we don't + # really have any notion of a "main" remote task any more? + # + # @api_frame + async def wait_for_result( self, - ctx: Context, - - ) -> dict[str, Any]: - - assert ctx._remote_func_type == 'asyncfunc' # single response - msg: dict = await ctx._recv_chan.receive() - return msg - - async def result(self) -> Any: + hide_tb: bool = True, + ) -> Any: ''' - Return the result(s) from the remote actor's "main" task. + Return the final result delivered by a `Return`-msg from the + remote peer actor's "main" task's `return` statement. ''' - # __tracebackhide__ = True + __tracebackhide__: bool = hide_tb # Check for non-rpc errors slapped on the # channel for which we always raise exc = self.channel._exc @@ -180,7 +169,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" @@ -188,16 +177,40 @@ 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 + + # TODO: factor this out into a `.highlevel` API-wrapper that uses + # a single `.open_context()` call underneath. + async def result( + self, + *args, + **kwargs, + ) -> Any|Exception: + typname: str = type(self).__name__ + log.warning( + f'`{typname}.result()` is DEPRECATED!\n' + f'Use `{typname}.wait_for_result()` instead!\n' + ) + return await self.wait_for_result( + *args, + **kwargs, ) async def _cancel_streams(self): @@ -240,6 +253,8 @@ class Portal: purpose. ''' + __runtimeframe__: int = 1 # noqa + chan: Channel = self.channel if not chan.connected(): log.runtime( @@ -248,14 +263,15 @@ class Portal: return False reminfo: str = ( - f'`Portal.cancel_actor()` => {self.channel.uid}\n' - f' |_{chan}\n' + f'c)=> {self.channel.uid}\n' + f' |_{chan}\n' ) log.cancel( - f'Sending runtime `.cancel()` request to peer\n\n' + f'Requesting actor-runtime cancel for peer\n\n' f'{reminfo}' ) + # XXX the one spot we set it? self.channel._cancel_called: bool = True try: # send cancel cmd - might not get response @@ -295,6 +311,8 @@ class Portal: ) return False + # TODO: do we still need this for low level `Actor`-runtime + # method calls or can we also remove it? async def run_from_ns( self, namespace_path: str, @@ -317,21 +335,23 @@ class Portal: internals! ''' + __runtimeframe__: int = 1 # noqa nsf = NamespacePath( f'{namespace_path}:{function_name}' ) - ctx = await self.actor.start_remote_task( + ctx: Context = await self.actor.start_remote_task( chan=self.channel, nsf=nsf, kwargs=kwargs, + portal=self, ) - ctx._portal = self - msg = await self._return_once(ctx) - return _unwrap_msg( - msg, - self.channel, + return await ctx._pld_rx.recv_pld( + ipc=ctx, + expect_msg=Return, ) + # TODO: factor this out into a `.highlevel` API-wrapper that uses + # a single `.open_context()` call underneath. async def run( self, func: str, @@ -347,6 +367,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" @@ -377,13 +399,15 @@ class Portal: self.channel, 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, ) + # TODO: factor this out into a `.highlevel` API-wrapper that uses + # a single `.open_context()` call underneath. @acm async def open_stream_from( self, @@ -391,6 +415,14 @@ class Portal: **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 ( @@ -404,8 +436,8 @@ class Portal: self.channel, 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' @@ -414,13 +446,13 @@ class Portal: # deliver receive only stream async with MsgStream( ctx=ctx, - rx_chan=ctx._recv_chan, - ) as rchan: - self._streams.add(rchan) - yield rchan + 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 @@ -439,7 +471,7 @@ class Portal: # XXX: should this always be done? # await recv_chan.aclose() - self._streams.remove(rchan) + self._streams.remove(stream) # NOTE: impl is found in `._context`` mod to make # reading/groking the details simpler code-org-wise. This @@ -481,7 +513,7 @@ class LocalPortal: async def open_portal( channel: Channel, - nursery: trio.Nursery|None = None, + tn: trio.Nursery|None = None, start_msg_loop: bool = True, shield: bool = False, @@ -489,15 +521,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() @@ -509,7 +545,7 @@ async def open_portal( 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, @@ -526,12 +562,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 4469f3ed..882285a5 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -18,9 +18,10 @@ Root actor runtime ignition(s). ''' -from contextlib import asynccontextmanager +from contextlib import asynccontextmanager as acm from functools import partial import importlib +import inspect import logging import os import signal @@ -60,7 +61,7 @@ _default_lo_addrs: list[tuple[str, int]] = [( logger = log.get_logger('tractor') -@asynccontextmanager +@acm async def open_root_actor( *, @@ -92,11 +93,16 @@ async def open_root_actor( # 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 @@ -110,20 +116,28 @@ async def open_root_actor( if ( debug_mode and maybe_enable_greenback - and await _debug.maybe_init_greenback( - raise_not_found=False, + and ( + maybe_mod := await _debug.maybe_init_greenback( + raise_not_found=False, + ) ) ): - os.environ['PYTHONBREAKPOINT'] = ( - 'tractor.devx._debug.pause_from_sync' + logger.info( + f'Found `greenback` installed @ {maybe_mod}\n' + 'Enabling `tractor.pause_from_sync()` support!\n' ) + os.environ['PYTHONBREAKPOINT'] = ( + 'tractor.devx._debug._sync_pause_from_builtin' + ) + _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' + '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 ' @@ -131,11 +145,13 @@ async def open_root_actor( ) sys.breakpointhook = block_bps - # os.environ['PYTHONBREAKPOINT'] = None + # 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 @@ -201,6 +217,7 @@ async def open_root_actor( ): loglevel = 'PDB' + elif debug_mode: raise RuntimeError( "Debug mode is only supported for the `trio` backend!" @@ -254,7 +271,9 @@ async def open_root_actor( except OSError: # TODO: make this a "discovery" log level? - logger.warning(f'No actor registry found @ {addr}') + logger.info( + f'No actor registry found @ {addr}\n' + ) async with trio.open_nursery() as tn: for addr in registry_addrs: @@ -268,7 +287,6 @@ async def open_root_actor( # Create a new local root-actor instance which IS NOT THE # REGISTRAR if ponged_addrs: - if ensure_registry: raise RuntimeError( f'Failed to open `{name}`@{ponged_addrs}: ' @@ -355,19 +373,25 @@ async def open_root_actor( ) try: yield actor - except ( Exception, BaseExceptionGroup, ) as err: - - entered: bool = await _debug._maybe_enter_pm(err) + # XXX NOTE XXX see equiv note inside + # `._runtime.Actor._stream_handler()` where in the + # non-root or root-that-opened-this-mahually case we + # wait for the local actor-nursery to exit before + # exiting the transport channel handler. + entered: bool = await _debug._maybe_enter_pm( + err, + api_frame=inspect.currentframe(), + ) if ( not entered and not is_multi_cancelled(err) ): - logger.exception('Root actor crashed:\n') + logger.exception('Root actor crashed\n') # ALWAYS re-raise any error bubbled up from the # runtime! @@ -392,14 +416,20 @@ async def open_root_actor( _state._last_actor_terminated = actor # restore built-in `breakpoint()` hook state - if debug_mode: + 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') + os.environ.pop('PYTHONBREAKPOINT', None) logger.runtime("Root actor terminated") diff --git a/tractor/_rpc.py b/tractor/_rpc.py index b108fdda..a77c2af7 100644 --- a/tractor/_rpc.py +++ b/tractor/_rpc.py @@ -26,6 +26,7 @@ from contextlib import ( from functools import partial import inspect from pprint import pformat +import traceback from typing import ( Any, Callable, @@ -41,22 +42,45 @@ from trio import ( TaskStatus, ) -from .msg import NamespacePath from ._ipc import Channel from ._context import ( Context, ) from ._exceptions import ( - ModuleNotExposed, - is_multi_cancelled, ContextCancelled, + RemoteActorError, + ModuleNotExposed, + MsgTypeError, + TransportClosed, + is_multi_cancelled, pack_error, unpack_error, - TransportClosed, ) -from .devx import _debug +from .devx import ( + _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 @@ -64,6 +88,16 @@ if TYPE_CHECKING: log = get_logger('tractor') +# ?TODO? move to a `tractor.lowlevel._rpc` with the below +# func-type-cases implemented "on top of" `@context` defs: +# -[ ] std async func helper decorated with `@rpc_func`? +# -[ ] `Portal.open_stream_from()` with async-gens? +# |_ possibly a duplex form of this with a +# `sent_from_peer = yield send_to_peer` form, which would require +# syncing the send/recv side with possibly `.receive_nowait()` +# on each `yield`? +# -[ ] some kinda `@rpc_acm` maybe that does a fixture style with +# user only defining a single-`yield` generator-func? async def _invoke_non_context( actor: Actor, cancel_scope: CancelScope, @@ -76,15 +110,23 @@ async def _invoke_non_context( 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 + cs: CancelScope|None = None # ref when activated - # TODO: can we unify this with the `context=True` impl below? + # ?TODO? can we unify this with the `context=True` impl below? if inspect.isasyncgen(coro): - await chan.send({'functype': 'asyncgen', 'cid': cid}) + 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 @@ -104,22 +146,30 @@ async def _invoke_non_context( # 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}) + 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': True, 'cid': cid}) + 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({'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 + await chan.send( + StartAck( + cid=cid, + functype='asyncgen', + ) + ) with cancel_scope as cs: ctx._scope = cs task_status.started(ctx) @@ -128,22 +178,23 @@ async def _invoke_non_context( 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}) + await chan.send( + Stop(cid=cid) + ) + + # simplest function/method request-response pattern + # XXX: in the most minimally used case, just a scheduled internal runtime + # call to `Actor._cancel_task()` from the ctx-peer task since we + # don't (yet) have a dedicated IPC msg. + # ------ - ------ else: - # regular async function/method - # XXX: possibly just a scheduled `Actor._cancel_task()` - # from a remote request to cancel some `Context`. - # ------ - ------ - # TODO: ideally we unify this with the above `context=True` - # block such that for any remote invocation ftype, we - # always invoke the far end RPC task scheduling the same - # way: using the linked IPC context machinery. failed_resp: bool = False try: - await chan.send({ - 'functype': 'asyncfunc', - 'cid': cid - }) + ack = StartAck( + cid=cid, + functype='asyncfunc', + ) + await chan.send(ack) except ( trio.ClosedResourceError, trio.BrokenResourceError, @@ -151,12 +202,12 @@ async def _invoke_non_context( ) as ipc_err: failed_resp = True if is_rpc: - raise + raise ipc_err else: - # TODO: should this be an `.exception()` call? - log.warning( - f'Failed to respond to non-rpc request: {func}\n' - f'{ipc_err}' + 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: @@ -177,18 +228,19 @@ async def _invoke_non_context( and chan.connected() ): try: - await chan.send( - {'return': result, - 'cid': cid} + ret_msg = return_msg_type( + cid=cid, + pld=result, ) + await chan.send(ret_msg) except ( BrokenPipeError, trio.BrokenResourceError, ): log.warning( - 'Failed to return result:\n' - f'{func}@{actor.uid}\n' - f'remote chan: {chan.uid}' + 'Failed to send RPC result?\n' + f'|_{func}@{actor.uid}() -> {ret_msg}\n\n' + f'x=> peer: {chan.uid}\n' ) @acm @@ -205,7 +257,17 @@ async def _errors_relayed_via_ipc( ] = trio.TASK_STATUS_IGNORED, ) -> None: - __tracebackhide__: bool = hide_tb # TODO: use hide_tb here? + # 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 @@ -216,24 +278,13 @@ async def _errors_relayed_via_ipc( BaseExceptionGroup, KeyboardInterrupt, ) as err: + rpc_err = err - # always hide this frame from debug REPL if the crash - # originated from an rpc task and we DID NOT fail due to - # an IPC transport error! - if ( - is_rpc - and chan.connected() - ): - __tracebackhide__: bool = hide_tb - + # 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): - - # 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 ( ( @@ -256,7 +307,6 @@ async def _errors_relayed_via_ipc( ) ) ): - # await _debug.pause() # XXX QUESTION XXX: is there any case where we'll # want to debug IPC disconnects as a default? # => I can't think of a reason that inspecting this @@ -264,20 +314,33 @@ async def _errors_relayed_via_ipc( # 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) - + 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 caller + # 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.. + # should **not** crash this actor.. await try_ship_error_to_remote( chan, err, @@ -286,53 +349,87 @@ async def _errors_relayed_via_ipc( hide_tb=hide_tb, ) - # error is probably from above coro running code *not from - # the target rpc invocation since a scope was never - # allocated around the coroutine await. + # if 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): + # always propagate KBIs at the sys-process level. + if ( + isinstance(err, KeyboardInterrupt) + + # ?TODO? except when running in asyncio mode? + # |_ wut if you want to open a `@context` FROM an + # infected_aio task? + # and not actor.is_infected_aio() + ): raise - - # RPC task bookeeping + # 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, func, is_complete = actor._rpc_tasks.pop( + 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: - # If we're cancelled before the task returns then the - # cancel scope will not have been inserted yet log.warning( - 'RPC task likely errored or cancelled before start?' - f'|_{ctx._task}\n' - f' >> {ctx.repr_rpc}\n' - ) - else: - log.cancel( - 'Failed to de-alloc internal runtime cancel task?\n' + '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") + log.runtime('All RPC tasks have completed') actor._ongoing_rpc_tasks.set() async def _invoke( - actor: Actor, cid: str, chan: Channel, @@ -341,6 +438,7 @@ async def _invoke( is_rpc: bool = True, hide_tb: bool = True, + return_msg_type: Return|CancelAck = Return, task_status: TaskStatus[ Context | BaseException @@ -357,33 +455,36 @@ async def _invoke( __tracebackhide__: bool = hide_tb treat_as_gen: bool = False - if _state.debug_mode(): + 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 = None + # tb: TracebackType = None cancel_scope = CancelScope() - # activated cancel scope ref - cs: CancelScope|None = None - ctx = actor.get_context( chan=chan, cid=cid, nsf=NamespacePath.from_ref(func), - # TODO: if we wanted to get cray and support it? - # side='callee', + # 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: bool = False + 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): @@ -409,10 +510,21 @@ async def _invoke( kwargs['stream'] = ctx - elif getattr(func, '_tractor_context_function', False): - # handle decorated ``@tractor.context`` async function - kwargs['ctx'] = ctx - context = True + # 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( @@ -424,7 +536,8 @@ async def _invoke( task_status=task_status, ): if not ( - inspect.isasyncgenfunction(func) or + inspect.isasyncgenfunction(func) + or inspect.iscoroutinefunction(func) ): raise TypeError(f'{func} must be an async function!') @@ -436,9 +549,8 @@ async def _invoke( except TypeError: raise - # TODO: implement all these cases in terms of the - # `Context` one! - if not context: + # TODO: impl all these cases in terms of the `Context` one! + if not context_ep_func: await _invoke_non_context( actor, cancel_scope, @@ -450,9 +562,10 @@ async def _invoke( kwargs, treat_as_gen, is_rpc, + return_msg_type, task_status, ) - # below is only for `@context` funcs + # XXX below fallthrough is ONLY for `@context` eps return # our most general case: a remote SC-transitive, @@ -470,52 +583,78 @@ async def _invoke( # wrapper that calls `Context.started()` and then does # the `await coro()`? - # a "context" endpoint type is the most general and - # "least sugary" type of RPC ep with support for + # ------ - ------ + # a "context" endpoint is the most general and + # "least sugary" type of RPC with support for # bi-dir streaming B) - await chan.send({ - 'functype': 'context', - 'cid': cid - }) + # + # 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 callee side by factoring the impl from + # 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 callee side instance according to current impl: + # 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: + tn: trio.Nursery + rpc_ctx_cs: CancelScope + 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 + rpc_ctx_cs = ctx._scope = tn.cancel_scope task_status.started(ctx) - # TODO: should would be nice to have our - # `TaskMngr` nursery here! - res: Any = await coro + # 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 - - # deliver final result to caller side. - await chan.send({ - 'return': res, - 'cid': cid - }) + await chan.send(return_msg) # NOTE: this happens IFF `ctx._scope.cancel()` is # called by any of, - # - *this* callee task manually calling `ctx.cancel()`. + # - *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. - # - currently a never-should-happen-fallthrough case - # inside ._context._drain_to_final_msg()`.. - # # TODO: remove this ^ right? - if ctx._scope.cancelled_caught: + if rpc_ctx_cs.cancelled_caught: our_uid: tuple = actor.uid # first check for and raise any remote error @@ -525,14 +664,9 @@ async def _invoke( if re := ctx._remote_error: ctx._maybe_raise_remote_err(re) - cs: CancelScope = ctx._scope - - if cs.cancel_called: - + if rpc_ctx_cs.cancel_called: canceller: tuple = ctx.canceller - msg: str = ( - 'actor was cancelled by ' - ) + explain: str = f'{ctx.side!r}-side task was cancelled by ' # NOTE / TODO: if we end up having # ``Actor._cancel_task()`` call @@ -542,37 +676,36 @@ async def _invoke( if ctx._cancel_called: # TODO: test for this!!!!! canceller: tuple = our_uid - msg += 'itself ' + 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: - msg += 'its caller' + explain += f'its {ctx.peer_side!r}-side peer' + + elif canceller == our_uid: + explain += 'itself' + + elif canceller: + explain += 'a remote peer' else: - msg += 'a remote peer' + explain += 'an unknown cause?' - div_chars: str = '------ - ------' - div_offset: int = ( - round(len(msg)/2)+1 + explain += ( + add_div(message=explain) + - round(len(div_chars)/2)+1 - ) - div_str: str = ( - '\n' - + - ' '*div_offset - + - f'{div_chars}\n' - ) - msg += ( - div_str + f'<= canceller: {canceller}\n' - f'=> uid: {our_uid}\n' - f' |_{ctx._task}()' + 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? @@ -584,21 +717,17 @@ async def _invoke( # f' |_{ctx}' ) - # task-contex was either cancelled by request using - # ``Portal.cancel_actor()`` or ``Context.cancel()`` - # on the far end, or it was cancelled by the local - # (callee) task, so relay this cancel signal to the + # 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( - msg, + message=explain, boxed_type=trio.Cancelled, - # boxed_type_str='Cancelled', canceller=canceller, ) - # assign local error so that the `.outcome` - # resolves to an error for both reporting and - # state checks. - ctx._local_error = ctxc raise ctxc # XXX: do we ever trigger this block any more? @@ -608,10 +737,22 @@ async def _invoke( 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 (callee) side's exception as the + # 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 @@ -634,25 +775,35 @@ async def _invoke( ctx: Context = actor._contexts.pop(( chan.uid, cid, - # ctx.side, )) + logmeth: Callable = log.runtime merr: Exception|None = ctx.maybe_error - - ( - res_type_str, - res_str, - ) = ( - ('error', f'{type(merr)}',) - if merr - else ( - 'result', - f'`{repr(ctx.outcome)}`', - ) + message: str = 'IPC context terminated ' + descr_str: str = ( + f'after having {ctx.repr_state!r}\n' ) - log.runtime( - f'IPC context terminated with a final {res_type_str}\n\n' - f'{ctx}\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 ) @@ -676,7 +827,8 @@ async def try_ship_error_to_remote( try: # NOTE: normally only used for internal runtime errors # so ship to peer actor without a cid. - msg: dict = pack_error( + # msg: dict = pack_error( + msg: Error = pack_error( err, cid=cid, @@ -692,13 +844,20 @@ async def try_ship_error_to_remote( trio.BrokenResourceError, BrokenPipeError, ): - err_msg: dict = msg['error']['tb_str'] log.critical( 'IPC transport failure -> ' f'failed to ship error to {remote_descr}!\n\n' f'X=> {channel.uid}\n\n' - f'{err_msg}\n' + + # 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( @@ -707,7 +866,10 @@ async def process_messages( shield: bool = False, task_status: TaskStatus[CancelScope] = trio.TASK_STATUS_IGNORED, -) -> bool: +) -> ( + bool, # chan diconnected + MsgType, # last msg +): ''' This is the low-level, per-IPC-channel, RPC task scheduler loop. @@ -736,18 +898,21 @@ async def process_messages( 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? - # https://github.com/python-trio/trio/issues/467 - log.runtime( - 'Entering IPC msg loop:\n' - f'peer: {chan.uid}\n' - f'|_{chan}\n' - ) + # -[ ] 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: dict | None = None + msg: MsgType|None = None try: # NOTE: this internal scope allows for keeping this # message loop running despite the current task having @@ -756,101 +921,61 @@ async def process_messages( # using ``scope = Nursery.start()`` with CancelScope(shield=shield) as loop_cs: task_status.started(loop_cs) + async for msg in chan: - - # dedicated loop terminate sentinel - if msg is None: - - tasks: dict[ - tuple[Channel, str], - tuple[Context, Callable, trio.Event] - ] = actor._rpc_tasks.copy() - log.cancel( - f'Peer IPC channel terminated via `None` setinel msg?\n' - f'=> Cancelling all {len(tasks)} local RPC tasks..\n' - f'peer: {chan.uid}\n' - f'|_{chan}\n' - ) - for (channel, cid) in tasks: - if channel is chan: - await actor._cancel_task( - cid, - channel, - requesting_uid=channel.uid, - - ipc_msg=msg, - ) - break - log.transport( # type: ignore - f'<= IPC msg from peer: {chan.uid}\n\n' + f'IPC msg from peer\n' + f'<= {chan.uid}\n\n' - # TODO: conditionally avoid fmting depending - # on log level (for perf)? - # => specifically `pformat()` sub-call..? - f'{pformat(msg)}\n' + # TODO: use of the pprinting of structs is + # FRAGILE and should prolly not be + # + # avoid fmting depending on loglevel for perf? + # -[ ] specifically `pretty_struct.pformat()` sub-call..? + # - how to only log-level-aware actually call this? + # -[ ] use `.msg.pretty_struct` here now instead! + # f'{pretty_struct.pformat(msg)}\n' + f'{msg}\n' ) - cid = msg.get('cid') - if cid: - # deliver response to local caller/waiter - # via its per-remote-context memory channel. - await actor._push_result( - chan, - cid, - msg, - ) + 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) - log.runtime( - 'Waiting on next IPC msg from\n' - f'peer: {chan.uid}:\n' - f'|_{chan}\n' + # `.cid` indicates RPC-ctx-task scoped + | Error(cid=cid) - # f'last msg: {msg}\n' - ) - continue + # 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, + ) - # process a 'cmd' request-msg upack - # TODO: impl with native `msgspec.Struct` support !! - # -[ ] implement with ``match:`` syntax? - # -[ ] discard un-authed msgs as per, - # - try: - ( - ns, - funcname, - kwargs, - actorid, - cid, - ) = msg['cmd'] + # `Actor`(-internal) runtime cancel requests + case Start( + ns='self', + func='cancel', + cid=cid, + kwargs=kwargs, + ): + kwargs |= {'req_chan': chan} - except KeyError: - # This is the non-rpc error case, that is, an - # error **not** raised inside a call to ``_invoke()`` - # (i.e. no cid was provided in the msg - see above). - # Push this error to all local channel consumers - # (normally portals) by marking the channel as errored - assert chan.uid - exc = unpack_error(msg, chan=chan) - chan._exc = exc - raise exc - - log.runtime( - 'Handling RPC cmd from\n' - f'peer: {actorid}\n' - '\n' - f'=> {ns}.{funcname}({kwargs})\n' - ) - if ns == 'self': - if funcname == 'cancel': - func: Callable = actor.cancel - kwargs |= { - 'req_chan': chan, - } - - # don't start entire actor runtime cancellation - # if this actor is currently in debug mode! - pdb_complete: trio.Event|None = _debug.Lock.local_pdb_complete + # 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() @@ -864,9 +989,10 @@ async def process_messages( actor, cid, chan, - func, + actor.cancel, kwargs, is_rpc=False, + return_msg_type=CancelAck, ) log.runtime( @@ -876,37 +1002,31 @@ async def process_messages( loop_cs.cancel() break - if funcname == '_cancel_task': - func: Callable = actor._cancel_task - - # we immediately start the runtime machinery - # shutdown - # with CancelScope(shield=True): + case Start( + ns='self', + func='_cancel_task', + cid=cid, + kwargs=kwargs, + ): target_cid: str = kwargs['cid'] kwargs |= { - # NOTE: ONLY the rpc-task-owning + '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, - 'requesting_uid': chan.uid, - 'ipc_msg': msg, } - # TODO: remove? already have emit in meth. - # log.runtime( - # f'Rx RPC task cancel request\n' - # f'<= canceller: {chan.uid}\n' - # f' |_{chan}\n\n' - # f'=> {actor}\n' - # f' |_cid: {target_cid}\n' - # ) try: await _invoke( actor, cid, chan, - func, + actor._cancel_task, kwargs, is_rpc=False, + return_msg_type=CancelAck, ) except BaseException: log.exception( @@ -916,102 +1036,157 @@ async def process_messages( f'=> {actor}\n' f' |_cid: {target_cid}\n' ) - continue - else: - # normally registry methods, eg. - # ``.register_actor()`` etc. - func: Callable = getattr(actor, funcname) - else: - # complain to client about restricted modules - try: - func = actor._get_rpc_func(ns, funcname) - except ( - ModuleNotExposed, - AttributeError, - ) as err: - err_msg: dict[str, dict] = pack_error( - err, - cid=cid, + # 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' ) - await chan.send(err_msg) - continue - # schedule a task for the requested RPC function - # in the actor's main "service nursery". - # TODO: possibly a service-tn per IPC channel for - # supervision isolation? would avoid having to - # manage RPC tasks individually in `._rpc_tasks` - # table? - log.runtime( - f'Spawning task for RPC request\n' - f'<= caller: {chan.uid}\n' - f' |_{chan}\n\n' - # TODO: maddr style repr? - # f' |_@ /ipv4/{chan.raddr}/tcp/{chan.rport}/' - # f'cid="{cid[-16:]} .."\n\n' + # runtime-internal endpoint: `Actor.` + # only registry methods exist now yah, + # like ``.register_actor()`` etc. ? + if ns == 'self': + func: Callable = getattr(actor, funcname) - f'=> {actor}\n' - f' |_cid: {cid}\n' - f' |>> {func}()\n' - ) - assert actor._service_n # wait why? do it at top? - try: - ctx: Context = await actor._service_n.start( - partial( - _invoke, - actor, - cid, - chan, - func, - kwargs, - ), - name=funcname, - ) + # 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 - except ( - RuntimeError, - BaseExceptionGroup, - ): - # avoid reporting a benign race condition - # during actor runtime teardown. - nursery_cancelled_before_task: bool = True - break + start_status += ( + f' -> func: {func}\n' + ) - # in the lone case where a ``Context`` is not - # delivered, it's likely going to be a locally - # scoped exception from ``_invoke()`` itself. - if isinstance(err := ctx, Exception): - log.warning( - 'Task for RPC failed?' - f'|_ {func}()\n\n' + # 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, + ) - f'{err}' - ) - continue + except ( + RuntimeError, + BaseExceptionGroup, + ): + # avoid reporting a benign race condition + # during actor runtime teardown. + nursery_cancelled_before_task: bool = True + break - else: - # mark that we have ongoing rpc tasks - actor._ongoing_rpc_tasks = trio.Event() + # 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 - # store cancel scope such that the rpc task can be - # cancelled gracefully if requested - actor._rpc_tasks[(chan, cid)] = ( - ctx, - func, - trio.Event(), - ) + else: + # mark our global state with ongoing rpc tasks + actor._ongoing_rpc_tasks = trio.Event() - log.runtime( + # 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, channel disconnect vis - # ``trio.EndOfChannel`` + # END-OF `async for`: + # IPC disconnected via `trio.EndOfChannel`, likely + # due to a (graceful) `Channel.aclose()`. log.runtime( - f"{chan} for {chan.uid} disconnected, cancelling tasks" + f'channel for {chan.uid} disconnected, cancelling RPC tasks\n' + f'|_{chan}\n' ) await actor.cancel_rpc_tasks( req_uid=actor.uid, @@ -1021,24 +1196,32 @@ async def process_messages( parent_chan=chan, ) - except ( - TransportClosed, - ): + except TransportClosed as tc: # channels "breaking" (for TCP streams by EOF or 104 # connection-reset) is ok since we don't have a teardown # handshake for them (yet) and instead we simply bail out of # the message loop and expect the teardown sequence to clean - # up. - # TODO: don't show this msg if it's an emphemeral - # discovery ep call? - log.runtime( - f'channel closed abruptly with\n' - f'peer: {chan.uid}\n' - f'|_{chan.raddr}\n' + # up.. + # + # TODO: maybe add a teardown handshake? and, + # -[x] don't show this msg if it's an ephemeral discovery ep call? + # |_ see the below `.report_n_maybe_raise()` impl as well as + # tc-exc input details in `MsgpackTCPStream._iter_pkts()` + # for different read-failure cases. + # -[ ] figure out how this will break with other transports? + tc.report_n_maybe_raise( + message=( + f'peer IPC channel closed abruptly?\n\n' + f'<=x {chan}\n' + f' |_{chan.raddr}\n\n' + ) + + + tc.message + ) - # transport **was** disconnected - return True + # transport **WAS** disconnected + return (True, msg) except ( Exception, @@ -1075,13 +1258,17 @@ async def process_messages( finally: # msg debugging for when he machinery is brokey - log.runtime( - 'Exiting IPC msg loop with\n' - f'peer: {chan.uid}\n' - f'|_{chan}\n\n' - 'final msg:\n' - f'{pformat(msg)}\n' - ) + 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)}' + ) - # transport **was not** disconnected - return False + log.runtime(message) + + # transport **WAS NOT** disconnected + return (False, msg) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index ed7b4503..662dd67a 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -65,7 +65,13 @@ from trio import ( TaskStatus, ) -from .msg import NamespacePath +from tractor.msg import ( + MsgType, + NamespacePath, + Stop, + pretty_struct, + types as msgtypes, +) from ._ipc import Channel from ._context import ( mk_context, @@ -73,9 +79,10 @@ from ._context import ( ) from .log import get_logger from ._exceptions import ( - unpack_error, - ModuleNotExposed, ContextCancelled, + ModuleNotExposed, + MsgTypeError, + unpack_error, TransportClosed, ) from .devx import _debug @@ -104,25 +111,26 @@ class Actor: ''' The fundamental "runtime" concurrency primitive. - An *actor* is the combination of a regular Python process executing - a ``trio`` task tree, communicating with other actors through - "memory boundary portals" - which provide a native async API around - IPC transport "channels" which themselves encapsulate various - (swappable) network protocols. + An "actor" is the combination of a regular Python process + executing a `trio.run()` task tree, communicating with other + "actors" through "memory boundary portals": `Portal`, which + provide a high-level async API around IPC "channels" (`Channel`) + which themselves encapsulate various (swappable) network + transport protocols for sending msgs between said memory domains + (processes, hosts, non-GIL threads). - - Each "actor" is ``trio.run()`` scheduled "runtime" composed of - many concurrent tasks in a single thread. The "runtime" tasks - conduct a slew of low(er) level functions to make it possible - for message passing between actors as well as the ability to - create new actors (aka new "runtimes" in new processes which - are supervised via a nursery construct). Each task which sends - messages to a task in a "peer" (not necessarily a parent-child, + Each "actor" is `trio.run()` scheduled "runtime" composed of many + concurrent tasks in a single thread. The "runtime" tasks conduct + a slew of low(er) level functions to make it possible for message + passing between actors as well as the ability to create new + actors (aka new "runtimes" in new processes which are supervised + via an "actor-nursery" construct). Each task which sends messages + to a task in a "peer" actor (not necessarily a parent-child, depth hierarchy) is able to do so via an "address", which maps IPC connections across memory boundaries, and a task request id - which allows for per-actor tasks to send and receive messages - to specific peer-actor tasks with which there is an ongoing - RPC/IPC dialog. + which allows for per-actor tasks to send and receive messages to + specific peer-actor tasks with which there is an ongoing RPC/IPC + dialog. ''' # ugh, we need to get rid of this and replace with a "registry" sys @@ -143,17 +151,15 @@ class Actor: # Information about `__main__` from parent _parent_main_data: dict[str, str] _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 - # 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], @@ -221,17 +227,20 @@ class Actor: # by the user (currently called the "arbiter") self._spawn_method: str = spawn_method - self._peers: defaultdict = defaultdict(list) + self._peers: defaultdict[ + str, # uaid + list[Channel], # IPC conns from peer + ] = defaultdict(list) self._peer_connected: dict[tuple[str, str], trio.Event] = {} self._no_more_peers = trio.Event() self._no_more_peers.set() + + # RPC state self._ongoing_rpc_tasks = trio.Event() self._ongoing_rpc_tasks.set() - - # (chan, cid) -> (cancel_scope, func) self._rpc_tasks: dict[ - tuple[Channel, str], - tuple[Context, Callable, trio.Event] + tuple[Channel, str], # (chan, cid) + tuple[Context, Callable, trio.Event] # (ctx=>, fn(), done?) ] = {} # map {actor uids -> Context} @@ -247,10 +256,13 @@ class Actor: self._listeners: list[trio.abc.Listener] = [] 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], + tuple[str, str], # sub-`Actor.uid` ActorNursery|None, - ] = {} # type: ignore # noqa + ] = {} # when provided, init the registry addresses property from # input via the validator. @@ -292,29 +304,35 @@ class Actor: 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 `uid` - using a `trio.Event` for sync. + 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") - return event, self._peers[uid][-1] + log.debug(f'{uid!r} successfully connected back to us') + return ( + event, + self._peers[uid][-1], + ) def load_modules( self, - debug_mode: bool = False, + # debug_mode: bool = False, ) -> None: ''' - Load enabled RPC py-modules locally (after process fork/spawn). + 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 (presuming it exists). + code manually (presuming it exists). ''' try: @@ -327,16 +345,21 @@ 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 @@ -374,8 +397,9 @@ 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, @@ -387,30 +411,11 @@ class Actor: ''' self._no_more_peers = trio.Event() # unset by making new chan = Channel.from_stream(stream) - their_uid: tuple[str, str]|None = chan.uid - if their_uid: - log.warning( - f'Re-connection from already known {their_uid}' - ) - else: - log.runtime(f'New connection to us @{chan.raddr}') - - con_msg: str = '' - if their_uid: - # NOTE: `.uid` is only set after first contact - con_msg = ( - 'IPC Re-connection from already known peer? ' - ) - else: - con_msg = ( - 'New IPC connection to us ' - ) - - con_msg += ( - f'<= @{chan.raddr}\n' + con_status: str = ( + 'New inbound IPC connection <=\n' f'|_{chan}\n' - # f' |_@{chan.raddr}\n\n' ) + # send/receive initial handshake response try: uid: tuple|None = await self._do_handshake(chan) @@ -422,22 +427,38 @@ class Actor: TransportClosed, ): - # XXX: This may propagate up from ``Channel._aiter_recv()`` - # and ``MsgpackStream._inter_packets()`` on a read from the + # XXX: This may propagate up from `Channel._aiter_recv()` + # and `MsgpackStream._inter_packets()` on a read from the # stream particularly when the runtime is first starting up - # inside ``open_root_actor()`` where there is a check for + # inside `open_root_actor()` where there is a check for # a bound listener on the "arbiter" addr. the reset will be # because the handshake was never meant took place. - log.warning( - con_msg + log.runtime( + con_status + ' -> But failed to handshake? Ignoring..\n' ) return - con_msg += ( - f' -> Handshake with actor `{uid[0]}[{uid[1][-6:]}]` complete\n' + familiar: str = 'new-peer' + if _pre_chan := self._peers.get(uid): + familiar: str = 'pre-existing-peer' + uid_short: str = f'{uid[0]}[{uid[1][-6:]}]' + con_status += ( + f' -> Handshake with {familiar} `{uid_short}` complete\n' ) + + if _pre_chan: + # con_status += ( + # ^TODO^ swap once we minimize conn duplication + # -[ ] last thing might be reg/unreg runtime reqs? + # log.warning( + log.debug( + f'?Wait?\n' + f'We already have IPC with peer {uid_short!r}\n' + f'|_{_pre_chan}\n' + ) + # IPC connection tracking for both peers and new children: # - if this is a new channel to a locally spawned # sub-actor there will be a spawn wait even registered @@ -448,7 +469,7 @@ class Actor: None, ) if event: - con_msg += ( + con_status += ( ' -> Waking subactor spawn waiters: ' f'{event.statistics().tasks_waiting}\n' f' -> Registered IPC chan for child actor {uid}@{chan.raddr}\n' @@ -459,7 +480,7 @@ class Actor: event.set() else: - con_msg += ( + con_status += ( f' -> Registered IPC chan for peer actor {uid}@{chan.raddr}\n' ) # type: ignore @@ -473,20 +494,26 @@ class Actor: # TODO: can we just use list-ref directly? chans.append(chan) - log.runtime(con_msg) + 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: bool = await process_messages( + ( + disconnected, + last_msg, + ) = await process_messages( self, chan, ) except trio.Cancelled: log.cancel( - 'IPC transport msg loop was cancelled for \n' - f'|_{chan}\n' + 'IPC transport msg loop was cancelled\n' + f'c)>\n' + f' |_{chan}\n' ) raise @@ -499,24 +526,51 @@ class Actor: # the peer was cancelled we try to wait for them # to tear down their side of the connection before # moving on with closing our own side. - if local_nursery: - if chan._cancel_called: - log.cancel( - 'Waiting on cancel request to peer\n' - f'`Portal.cancel_actor()` => {chan.uid}\n' - ) + 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) + + ): + log.cancel( + 'Waiting on cancel request to peer..\n' + f'c)=>\n' + f' |_{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 + # 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 # attempt to wait for the far end to close the # channel and bail after timeout (a 2-generals @@ -533,45 +587,80 @@ class Actor: # TODO: factor this into a helper? log.warning( 'Draining msg from disconnected peer\n' - f'{chan.uid}\n' - f'|_{chan}\n' - f' |_{chan.transport}\n\n' - + f'{chan_info}' f'{pformat(msg)}\n' ) - cid = msg.get('cid') + # cid: str|None = msg.get('cid') + cid: str|None = msg.cid if cid: # deliver response to local caller/waiter - await self._push_result( + await self._deliver_ctx_payload( chan, cid, msg, ) - - # NOTE: when no call to `open_root_actor()` was - # made, we implicitly make that call inside - # the first `.open_nursery()`, in this case we - # can assume that we are the root actor and do - # not have to wait for the nursery-enterer to - # exit before shutting down the actor runtime. - # - # see matching note inside `._supervise.open_nursery()` - if not local_nursery._implicit_runtime_started: - log.runtime( - 'Waiting on local actor nursery to exit..\n' - f'|_{local_nursery}\n' - ) - await local_nursery.exited.wait() - - if ( - cs.cancelled_caught - and not local_nursery._implicit_runtime_started - ): + if drain_cs.cancelled_caught: log.warning( - 'Failed to exit local actor nursery?\n' + 'Timed out waiting on IPC transport channel to drain?\n' + f'{chan_info}' + ) + + # 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' ) - # await _debug.pause() + 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 + ): + report: str = ( + 'Timed out waiting on local actor-nursery to exit?\n' + f'c)>\n' + f' |_{local_nursery}\n' + ) + if children := local_nursery._children: + # indent from above local-nurse repr + report += ( + f' |_{pformat(children)}\n' + ) + + log.warning(report) if disconnected: # if the transport died and this actor is still @@ -579,16 +668,24 @@ class Actor: # that the IPC layer may have failed # unexpectedly since it may be the cause of # other downstream errors. - entry = local_nursery._children.get(uid) + entry: tuple|None = local_nursery._children.get(uid) if entry: proc: trio.Process _, proc, _ = entry if ( (poll := getattr(proc, 'poll', None)) - and poll() is None + and + poll() is None # proc still alive ): - log.cancel( + # 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' @@ -597,39 +694,40 @@ class Actor: # ``Channel`` teardown and closure sequence # drop ref to channel so it can be gc-ed and disconnected - log.runtime( - f'Disconnected IPC channel:\n' - f'uid: {chan.uid}\n' - f'|_{pformat(chan)}\n' + 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 with {chan.uid}' + con_teardown_status += ( + f'-> No more channels with {chan.uid}' ) self._peers.pop(uid, None) peers_str: str = '' for uid, chans in self._peers.items(): peers_str += ( - f'|_ uid: {uid}\n' + f'uid: {uid}\n' ) for i, chan in enumerate(chans): peers_str += ( f' |_[{i}] {pformat(chan)}\n' ) - log.runtime( - f'Remaining IPC {len(self._peers)} peers:\n' - + peers_str + 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 @@ -643,31 +741,39 @@ class Actor: # TODO: NEEEDS TO BE TESTED! # actually, no idea if this ever even enters.. XD - pdb_user_uid: tuple = pdb_lock.global_actor_in_debug + # + # 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 ( - pdb_user_uid - and local_nursery + (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(pdb_user_uid) + 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 ' - 'a DISCONNECTED child still has the debug ' - 'lock!\n' - f'root uid: {self.uid}\n' - f'last disconnected child uid: {uid}\n' - f'locking child uid: {pdb_user_uid}\n' - ) - await _debug.maybe_wait_for_debugger( - child_in_debug=True - ) + 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 _debug.maybe_wait_for_debugger( + child_in_debug=True + ) # TODO: just bc a child's transport dropped # doesn't mean it's not still using the pdb @@ -687,9 +793,8 @@ class Actor: # if a now stale local task has the TTY lock still # we cancel it to allow servicing other requests for # the lock. - db_cs: trio.CancelScope|None = 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 ): @@ -699,43 +804,16 @@ class Actor: # TODO: figure out why this breaks tests.. db_cs.cancel() - # XXX: is this necessary (GC should do it)? - # XXX WARNING XXX - # Be AWARE OF THE INDENT LEVEL HERE - # -> ONLY ENTER THIS BLOCK WHEN ._peers IS - # EMPTY!!!! - if ( - not self._peers - and chan.connected() - ): - # if the channel is still connected it may mean the far - # end has not closed and we may have gotten here due to - # an error and so we should at least try to terminate - # the channel from this end gracefully. - log.runtime( - 'Terminating channel with `None` setinel msg\n' - f'|_{chan}\n' - ) - try: - # send msg loop terminate sentinel which - # triggers cancellation of all remotely - # started tasks. - await chan.send(None) - - # XXX: do we want this? no right? - # causes "[104] connection reset by peer" on other end - # await chan.aclose() - - except trio.BrokenResourceError: - log.runtime(f"Channel {chan.uid} was already closed") + log.runtime(con_teardown_status) + # finally block closure # TODO: rename to `._deliver_payload()` since this handles # more then just `result` msgs now obvi XD - async def _push_result( + async def _deliver_ctx_payload( self, chan: Channel, cid: str, - msg: dict[str, Any], + msg: MsgType|MsgTypeError, ) -> None|bool: ''' @@ -754,15 +832,22 @@ class Actor: # side, )] except KeyError: - log.warning( + report: str = ( 'Ignoring invalid IPC ctx msg!\n\n' - f'<= sender: {uid}\n' - f'=> cid: {cid}\n\n' - - f'{msg}\n' + f'<=? {uid}\n\n' + f' |_{pretty_struct.pformat(msg)}\n' ) + match msg: + case Stop(): + log.runtime(report) + case _: + log.warning(report) + return + # if isinstance(msg, MsgTypeError): + # return await ctx._deliver_bad_msg() + return await ctx._deliver_msg(msg) def get_context( @@ -806,7 +891,7 @@ class Actor: cid, # side, )] - log.runtime( + log.debug( f'Retreived cached IPC ctx for\n' f'peer: {chan.uid}\n' f'cid:{cid}\n' @@ -819,8 +904,8 @@ class Actor: state.max_buffer_size = msg_buffer_size except KeyError: - log.runtime( - f'Creating NEW IPC ctx for\n' + log.debug( + f'Allocate new IPC ctx for\n' f'peer: {chan.uid}\n' f'cid: {cid}\n' ) @@ -845,10 +930,14 @@ class Actor: nsf: NamespacePath, kwargs: dict, + # 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: ''' @@ -873,10 +962,12 @@ class Actor: msg_buffer_size=msg_buffer_size, allow_overruns=allow_overruns, ) + ctx._portal = portal if ( 'self' in nsf - or not load_nsf + or + not load_nsf ): ns, _, func = nsf.partition(':') else: @@ -884,36 +975,40 @@ class Actor: # -[ ] 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 cmd to\n' - f'peer: {chan.uid} => \n' - '\n' - f'=> {ns}.{func}({kwargs})\n' - ) - await chan.send( - {'cmd': ( - ns, - func, - kwargs, - self.uid, - cid, - )} - ) + '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: dict = await ctx._recv_chan.receive() - functype: str = 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 @@ -946,62 +1041,94 @@ class Actor: await self._do_handshake(chan) 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\n' + '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'{pformat(parent_data)}\n' + f'{pretty_struct.pformat(spawnspec)}\n' ) - accept_addrs: list[tuple[str, int]] = parent_data.pop('bind_addrs') - rvs = parent_data.pop('_runtime_vars') + 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']: + from .devx import ( + enable_stack_on_sig, + maybe_init_greenback, + ) try: - log.info('Enabling `stackscope` traces on SIGUSR1') - from .devx import enable_stack_on_sig + # TODO: maybe return some status msgs upward + # to that we can emit them in `con_status` + # instead? + log.devx( + 'Enabling `stackscope` traces on SIGUSR1' + ) enable_stack_on_sig() + except ImportError: log.warning( '`stackscope` not installed for use in debug mode!' ) - log.runtime(f"Runtime vars are: {rvs}") + if rvs.get('use_greenback', False): + maybe_mod: ModuleType|None = await maybe_init_greenback() + if maybe_mod: + log.devx( + 'Activated `greenback` ' + 'for `tractor.pause_from_sync()` support!' + ) + else: + rvs['use_greenback'] = False + log.warning( + '`greenback` not installed for use in debug mode!\n' + '`tractor.pause_from_sync()` not available!' + ) + rvs['_is_root'] = False _state._runtime_vars.update(rvs) - for attr, value in parent_data.items(): - if ( - attr == 'reg_addrs' - and value - ): - # XXX: ``msgspec`` doesn't support serializing tuples - # so just cash manually here since it's what our - # internals expect. - # TODO: we don't really NEED these as - # tuples so we can probably drop this - # casting since apparently in python lists - # are "more efficient"? - self.reg_addrs = [tuple(val) for val in value] + # 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 + ] - else: - setattr(self, attr, 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) - return chan, accept_addrs + return ( + chan, + accept_addrs, + ) except OSError: # failed to connect log.warning( - f'Failed to connect to parent!?\n\n' - 'Closing IPC [TCP] transport server to\n' - f'{parent_addr}\n' + f'Failed to connect to spawning parent actor!?\n' + f'x=> {parent_addr}\n' f'|_{self}\n\n' ) - await self.cancel(chan=None) # self cancel + await self.cancel(req_chan=None) # self cancel raise async def _serve_forever( @@ -1079,8 +1206,7 @@ class Actor: # 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. + # implementation does start all such tasks in a sub-nursery. req_chan: Channel|None, ) -> bool: @@ -1102,7 +1228,6 @@ class Actor: requester_type, req_chan, log_meth, - ) = ( req_chan.uid, 'peer', @@ -1120,8 +1245,9 @@ class Actor: # TODO: just use the new `Context.repr_rpc: str` (and # other) repr fields instead of doing this all manual.. msg: str = ( - f'Runtime cancel request from {requester_type}:\n\n' - f'<= .cancel(): {requesting_uid}\n' + f'Actor-runtime cancel request from {requester_type}\n\n' + f'<=c) {requesting_uid}\n' + f' |_{self}\n' ) # TODO: what happens here when we self-cancel tho? @@ -1133,13 +1259,21 @@ class Actor: # 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: + 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}\n' + '-> Cancelling active debugger request..\n' + f'|_{_debug.Lock.repr()}\n\n' + f'|_{lock_req_ctx}\n\n' ) - dbcs.cancel() + # lock_req_ctx._scope.cancel() + # TODO: wrap this in a method-API.. + debug_req.req_cs.cancel() # self-cancel **all** ongoing RPC tasks await self.cancel_rpc_tasks( @@ -1208,22 +1342,24 @@ class Actor: except KeyError: # NOTE: during msging race conditions this will often # emit, some examples: - # - callee returns a result before cancel-msg/ctxc-raised - # - callee self raises ctxc before caller send request, - # - callee errors prior to cancel req. - log.cancel( - 'Cancel request invalid, RPC task already completed?\n\n' - f'<= canceller: {requesting_uid}\n\n' + # - 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( - 'Cancel request for RPC task\n\n' - f'<= Actor._cancel_task(): {requesting_uid}\n\n' - f'=> {ctx._task}\n' - f' |_ >> {ctx.repr_rpc}\n' + 'Rxed cancel request for RPC task\n' + f'<=c) {requesting_uid}\n' + f' |_{ctx._task}\n' + f' >> {ctx.repr_rpc}\n' + # f'=> {ctx._task}\n' # f' >> Actor._cancel_task() => {ctx._task}\n' # f' |_ {ctx._task}\n\n' @@ -1273,15 +1409,15 @@ class Actor: flow_info: str = ( f'<= canceller: {requesting_uid}\n' f'=> ipc-parent: {parent_chan}\n' - f' |_{ctx}\n' + f'|_{ctx}\n' ) log.runtime( - 'Waiting on RPC task to cancel\n' + 'Waiting on RPC task to cancel\n\n' f'{flow_info}' ) await is_complete.wait() log.runtime( - f'Sucessfully cancelled RPC task\n' + f'Sucessfully cancelled RPC task\n\n' f'{flow_info}' ) return True @@ -1339,15 +1475,17 @@ class Actor: "IPC channel's " ) rent_chan_repr: str = ( - f'|_{parent_chan}' + f' |_{parent_chan}\n\n' if parent_chan else '' ) log.cancel( - f'Cancelling {descr} {len(tasks)} rpc tasks\n\n' - f'<= `Actor.cancel_rpc_tasks()`: {req_uid}\n' - f' {rent_chan_repr}\n' - # f'{self}\n' + f'Cancelling {descr} RPC tasks\n\n' + f'<=c) {req_uid} [canceller]\n' + f'{rent_chan_repr}' + f'c)=> {self.uid} [cancellee]\n' + f' |_{self} [with {len(tasks)} tasks]\n' + # f' |_tasks: {len(tasks)}\n' # f'{tasks_str}' ) for ( @@ -1377,19 +1515,27 @@ class Actor: if tasks: log.cancel( 'Waiting for remaining rpc tasks to complete\n' - f'|_{tasks}' + f'|_{tasks_str}' ) await self._ongoing_rpc_tasks.wait() - def cancel_server(self) -> None: + def cancel_server(self) -> bool: ''' 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_addrs(self) -> list[tuple[str, int]]: @@ -1408,7 +1554,7 @@ class Actor: def accept_addr(self) -> tuple[str, int]: ''' Primary address to which the IPC transport server is - bound. + bound and listening for new connections. ''' # throws OSError on failure @@ -1422,7 +1568,11 @@ class Actor: 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 IPC channels to the actor with provided `uid`. @@ -1434,7 +1584,7 @@ class Actor: self, chan: Channel - ) -> tuple[str, str]: + ) -> msgtypes.Aid: ''' Exchange `(name, UUIDs)` identifiers as the first communication step with any (peer) remote `Actor`. @@ -1443,14 +1593,27 @@ class Actor: "actor model" parlance. ''' - await chan.send(self.uid) - value: tuple = 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]) + chan.uid = uid return uid def is_infected_aio(self) -> bool: @@ -1491,8 +1654,8 @@ async def async_main( ''' # 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) is_registered: bool = False try: @@ -1510,7 +1673,8 @@ async def async_main( # because we're running in mp mode if ( set_accept_addr_says_rent - and set_accept_addr_says_rent is not None + and + set_accept_addr_says_rent is not None ): accept_addrs = set_accept_addr_says_rent @@ -1555,6 +1719,7 @@ async def async_main( assert accept_addrs try: + # TODO: why is this not with the root nursery? actor._server_n = await service_nursery.start( partial( actor._serve_forever, @@ -1570,6 +1735,9 @@ async def async_main( entered_debug: bool = await _debug._maybe_enter_pm(oserr) if not entered_debug: log.exception('Failed to init IPC channel server !?\n') + else: + log.runtime('Exited debug REPL..') + raise accept_addrs: list[tuple[str, int]] = actor.accept_addrs @@ -1587,8 +1755,8 @@ async def async_main( # Register with the arbiter if we're told its addr log.runtime( - f'Registering `{actor.name}` ->\n' - f'{pformat(accept_addrs)}' + f'Registering `{actor.name}` => {pformat(accept_addrs)}\n' + # ^-TODO-^ we should instead show the maddr here^^ ) # TODO: ideally we don't fan out to all registrars @@ -1646,57 +1814,90 @@ async def async_main( # Blocks here as expected until the root nursery is # killed (i.e. this actor is cancelled or signalled by the parent) - except Exception as err: - log.runtime("Closing all actor lifetime contexts") - actor.lifetime_stack.close() - + except Exception as internal_err: if not is_registered: + err_report: str = ( + '\n' + "Actor runtime (internally) failed BEFORE contacting the registry?\n" + f'registrars -> {actor.reg_addrs} ?!?!\n\n' + + '^^^ THIS IS PROBABLY AN INTERNAL `tractor` BUG! ^^^\n\n' + '\t>> CALMLY CANCEL YOUR CHILDREN AND CALL YOUR PARENTS <<\n\n' + + '\tIf this is a sub-actor hopefully its parent will keep running ' + 'and cancel/reap this sub-process..\n' + '(well, presuming this error was propagated upward)\n\n' + + '\t---------------------------------------------\n' + '\tPLEASE REPORT THIS TRACEBACK IN A BUG REPORT @ ' # oneline + 'https://github.com/goodboy/tractor/issues\n' + '\t---------------------------------------------\n' + ) + # TODO: I guess we could try to connect back # to the parent through a channel and engage a debugger # once we have that all working with std streams locking? - log.exception( - f"Actor errored and failed to register with arbiter " - f"@ {actor.reg_addrs[0]}?") - log.error( - "\n\n\t^^^ THIS IS PROBABLY AN INTERNAL `tractor` BUG! ^^^\n\n" - "\t>> CALMLY CALL THE AUTHORITIES AND HIDE YOUR CHILDREN <<\n\n" - "\tIf this is a sub-actor hopefully its parent will keep running " - "correctly presuming this error was safely ignored..\n\n" - "\tPLEASE REPORT THIS TRACEBACK IN A BUG REPORT: " - "https://github.com/goodboy/tractor/issues\n" - ) + log.exception(err_report) if actor._parent_chan: await try_ship_error_to_remote( actor._parent_chan, - err, + internal_err, ) # always! - match err: + match internal_err: case ContextCancelled(): log.cancel( f'Actor: {actor.uid} was task-context-cancelled with,\n' - f'str(err)' + f'str(internal_err)' ) case _: - log.exception("Actor errored:") - raise + log.exception( + 'Main actor-runtime task errored\n' + f' Closing all actor lifetime contexts..' + teardown_report: str = ( + 'Main actor-runtime task completed\n' ) - # tear down all lifetime contexts if not in guest mode - # XXX: should this just be in the entrypoint? - actor.lifetime_stack.close() - # TODO: we can't actually do this bc the debugger - # uses the _service_n to spawn the lock task, BUT, - # in theory if we had the root nursery surround this finally - # block it might be actually possible to debug THIS - # machinery in the same way as user task code? + # ?TODO? should this be in `._entry`/`._root` mods instead? + # + # teardown any actor-lifetime-bound contexts + ls: ExitStack = actor.lifetime_stack + # only report if there are any registered + cbs: list[Callable] = [ + repr(tup[1].__wrapped__) + for tup in ls._exit_callbacks + ] + if cbs: + cbs_str: str = '\n'.join(cbs) + teardown_report += ( + '-> Closing actor-lifetime-bound callbacks\n\n' + f'}}>\n' + f' |_{ls}\n' + f' |_{cbs_str}\n' + ) + # XXX NOTE XXX this will cause an error which + # prevents any `infected_aio` actor from continuing + # and any callbacks in the `ls` here WILL NOT be + # called!! + # await _debug.pause(shield=True) + + ls.close() + + # XXX TODO but hard XXX + # we can't actually do this bc the debugger uses the + # _service_n to spawn the lock task, BUT, in theory if we had + # the root nursery surround this finally block it might be + # actually possible to debug THIS machinery in the same way + # as user task code? + # # if actor.name == 'brokerd.ib': # with CancelScope(shield=True): # await _debug.breakpoint() @@ -1726,9 +1927,9 @@ async def async_main( failed = True if failed: - log.warning( - f'Failed to unregister {actor.name} from ' - f'registar @ {addr}' + teardown_report += ( + f'-> Failed to unregister {actor.name} from ' + f'registar @ {addr}\n' ) # Ensure all peers (actors connected to us as clients) are finished @@ -1736,13 +1937,23 @@ async def async_main( if any( chan.connected() for chan in chain(*actor._peers.values()) ): - log.runtime( - f"Waiting for remaining peers {actor._peers} to clear") + teardown_report += ( + f'-> Waiting for remaining peers {actor._peers} to clear..\n' + ) + log.runtime(teardown_report) with CancelScope(shield=True): await actor._no_more_peers.wait() - log.runtime("All peer channels are complete") - log.runtime("Runtime completed") + teardown_report += ( + '-> All peer channels are complete\n' + ) + + teardown_report += ( + 'Actor runtime exiting\n' + f'>)\n' + f'|_{actor}\n' + ) + log.info(teardown_report) # TODO: rename to `Registry` and move to `._discovery`! @@ -1839,13 +2050,13 @@ class Arbiter(Actor): sockaddrs: list[tuple[str, int]] = [] sockaddr: tuple[str, int] - for (aname, _), sockaddr in self._registry.items(): - log.runtime( - f'Actor mailbox info:\n' - f'aname: {aname}\n' - f'sockaddr: {sockaddr}\n' + 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 == aname: + if name == uid[0]: sockaddrs.append(sockaddr) if not sockaddrs: @@ -1857,6 +2068,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( diff --git a/tractor/_spawn.py b/tractor/_spawn.py index 48135cc9..562c7e5b 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -43,12 +43,16 @@ from tractor._state import ( 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, +) if TYPE_CHECKING: @@ -139,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: Any = await portal.result() + final: Any = await portal.wait_for_result() except ( Exception, @@ -192,7 +198,10 @@ async def cancel_on_completion( # 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: Any|Exception = await exhaust_portal(portal, actor) + result: Any|Exception = await exhaust_portal( + portal, + actor, + ) if isinstance(result, Exception): errors[actor.uid]: Exception = result log.cancel( @@ -214,8 +223,8 @@ async def cancel_on_completion( async def hard_kill( proc: trio.Process, - terminate_after: int = 1.6, + terminate_after: int = 1.6, # NOTE: for mucking with `.pause()`-ing inside the runtime # whilst also hacking on it XD # terminate_after: int = 99999, @@ -241,8 +250,9 @@ async def hard_kill( ''' log.cancel( - 'Terminating sub-proc:\n' - f'|_{proc}\n' + 'Terminating sub-proc\n' + f'>x)\n' + f' |_{proc}\n' ) # NOTE: this timeout used to do nothing since we were shielding # the ``.wait()`` inside ``new_proc()`` which will pretty much @@ -288,14 +298,13 @@ async def hard_kill( log.critical( # 'Well, the #ZOMBIE_LORD_IS_HERE# to collect\n' '#T-800 deployed to collect zombie B0\n' - f'|\n' - f'|_{proc}\n' + f'>x)\n' + f' |_{proc}\n' ) proc.kill() async def soft_kill( - proc: ProcessType, wait_func: Callable[ [ProcessType], @@ -318,13 +327,26 @@ async def soft_kill( uid: tuple[str, str] = portal.channel.uid try: log.cancel( - 'Soft killing sub-actor via `Portal.cancel_actor()`\n' - f'|_{proc}\n' + 'Soft killing sub-actor via portal request\n' + f'c)> {portal.chan.uid}\n' + f' |_{proc}\n' ) # wait on sub-proc to signal termination await wait_func(proc) 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 @@ -452,10 +474,9 @@ async def trio_proc( proc: trio.Process|None = None try: try: - # TODO: needs ``trio_typing`` patch? - proc = await trio.lowlevel.open_process(spawn_cmd) + proc: trio.Process = await trio.lowlevel.open_process(spawn_cmd) log.runtime( - 'Started new sub-proc\n' + 'Started new child\n' f'|_{proc}\n' ) @@ -493,14 +514,17 @@ async def trio_proc( portal, ) - # send additional init params - await chan.send({ - '_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, - }) + # 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: Actor = current_actor() @@ -534,8 +558,9 @@ async def trio_proc( # cancel result waiter that may have been spawned in # tandem if not done already log.cancel( - 'Cancelling existing result waiter task for ' - f'{subactor.uid}' + 'Cancelling portal result reaper task\n' + f'>c)\n' + f' |_{subactor.uid}\n' ) nursery.cancel_scope.cancel() @@ -544,9 +569,13 @@ async def trio_proc( # allowed! Do this **after** cancellation/teardown to avoid # killing the process too early. if proc: - log.cancel(f'Hard reap sequence starting for {subactor.uid}') - with trio.CancelScope(shield=True): + log.cancel( + f'Hard reap sequence starting for subactor\n' + f'>x)\n' + f' |_{subactor}@{subactor.uid}\n' + ) + with trio.CancelScope(shield=True): # don't clobber an ongoing pdb if cancelled_during_spawn: # Try again to avoid TTY clobbering. diff --git a/tractor/_state.py b/tractor/_state.py index b76e8ac9..9f896005 100644 --- a/tractor/_state.py +++ b/tractor/_state.py @@ -19,13 +19,19 @@ Per process state """ from __future__ import annotations +from contextvars import ( + ContextVar, +) from typing import ( Any, TYPE_CHECKING, ) +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 @@ -38,7 +44,7 @@ _runtime_vars: dict[str, Any] = { '_root_mailbox': (None, None), '_registry_addrs': [], - # for `breakpoint()` support + # for `tractor.pause_from_sync()` & `breakpoint()` support 'use_greenback': False, } @@ -66,7 +72,7 @@ def current_actor( err_on_no_runtime and _current_actor is None ): - msg: str = 'No local actor has been initialized yet' + msg: str = 'No local actor has been initialized yet?\n' from ._exceptions import NoRuntime if last := last_actor(): @@ -79,8 +85,8 @@ def current_actor( # this process. else: msg += ( - 'No last actor found?\n' - 'Did you forget to open one of:\n\n' + # 'No last actor found?\n' + '\nDid you forget to call one of,\n' '- `tractor.open_root_actor()`\n' '- `tractor.open_nursery()`\n' ) @@ -110,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 e0015fe4..bc87164e 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -26,6 +26,7 @@ import inspect from pprint import pformat from typing import ( Any, + AsyncGenerator, Callable, AsyncIterator, TYPE_CHECKING, @@ -35,17 +36,25 @@ import warnings import trio from ._exceptions import ( - _raise_from_no_key_in_msg, ContextCancelled, + RemoteActorError, ) 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__) @@ -59,10 +68,10 @@ log = get_logger(__name__) 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: @@ -78,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 @@ -89,35 +98,44 @@ class MsgStream(trio.abc.Channel): self._eoc: bool|trio.EndOfChannel = False self._closed: bool|trio.ClosedResourceError = False + @property + def ctx(self) -> Context: + ''' + A read-only ref to this stream's inter-actor-task `Context`. + + ''' + 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, - allow_msg_keys: list[str] = ['yield'], + expect_msg: MsgType = Yield, ): - msg: dict = self._rx_chan.receive_nowait() - for ( - i, - key, - ) in enumerate(allow_msg_keys): - try: - return msg[key] - except KeyError as kerr: - if i < (len(allow_msg_keys) - 1): - continue - - _raise_from_no_key_in_msg( - ctx=self._ctx, - msg=msg, - src_err=kerr, - log=log, - expect_key=key, - stream=self, - ) + ctx: Context = self._ctx + return ctx._pld_rx.recv_pld_nowait( + ipc=self, + expect_msg=expect_msg, + ) async def receive( self, - hide_tb: bool = True, + hide_tb: bool = False, ): ''' Receive a single msg from the IPC transport, the next in @@ -127,9 +145,8 @@ class MsgStream(trio.abc.Channel): ''' __tracebackhide__: bool = hide_tb - # NOTE: `trio.ReceiveChannel` implements - # EOC handling as follows (aka uses it - # to gracefully exit async for loops): + # NOTE FYI: `trio.ReceiveChannel` implements EOC handling as + # follows (aka uses it to gracefully exit async for loops): # # async def __anext__(self) -> ReceiveType: # try: @@ -147,62 +164,29 @@ class MsgStream(trio.abc.Channel): src_err: Exception|None = None # orig tb try: - try: - msg = await self._rx_chan.receive() - return msg['yield'] - - except KeyError as kerr: - src_err = kerr - - # NOTE: may raise any of the below error types - # includg EoC when a 'stop' msg is found. - _raise_from_no_key_in_msg( - ctx=self._ctx, - msg=msg, - src_err=kerr, - log=log, - expect_key='yield', - stream=self, - ) + 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._push_result()`, but now the 'stop' message handling - # has been put just above inside `_raise_from_no_key_in_msg()`. - except ( - trio.EndOfChannel, - ) as eoc: - src_err = eoc + # - `self._rx_chan.receive()` raising after manual closure + # by the rpc-runtime, + # OR + # - via a `Stop`-msg received from remote peer task. + # NOTE + # |_ previously this was triggered by calling + # ``._rx_chan.aclose()`` on the send side of the channel + # inside `Actor._deliver_ctx_payload()`, but now the 'stop' + # message handling gets delegated to `PldRFx.recv_pld()` + # internals. + except trio.EndOfChannel as eoc: + # a graceful stream finished signal self._eoc = eoc + src_err = eoc - # TODO: Locally, we want to close this stream gracefully, by - # terminating any local consumers tasks deterministically. - # Once we have broadcast support, we **don't** want to be - # closing this stream and not flushing a final value to - # remaining (clone) consumers who may not have been - # scheduled to receive it yet. - # try: - # maybe_err_msg_or_res: dict = self._rx_chan.receive_nowait() - # if maybe_err_msg_or_res: - # log.warning( - # 'Discarding un-processed msg:\n' - # f'{maybe_err_msg_or_res}' - # ) - # except trio.WouldBlock: - # # no queued msgs that might be another remote - # # error, so just raise the original EoC - # pass - - # raise eoc - - # a ``ClosedResourceError`` indicates that the internal - # feeder memory receive channel was closed likely by the - # runtime after the associated transport-channel - # disconnected or broke. + # a `ClosedResourceError` indicates that the internal feeder + # memory receive channel was closed likely by the runtime + # after the associated transport-channel disconnected or + # broke. except trio.ClosedResourceError as cre: # by self._rx_chan.receive() src_err = cre log.warning( @@ -214,47 +198,57 @@ class MsgStream(trio.abc.Channel): # terminated and signal this local iterator to stop drained: list[Exception|dict] = await self.aclose() if drained: + # ?TODO? pass these to the `._ctx._drained_msgs: deque` + # and then iterate them as part of any `.wait_for_result()` call? + # # from .devx import pause # await pause() log.warning( - 'Drained context msgs during closure:\n' + 'Drained context msgs during closure\n\n' f'{drained}' ) - # TODO: pass these to the `._ctx._drained_msgs: deque` - # and then iterate them as part of any `.result()` call? # NOTE XXX: if the context was cancelled or remote-errored # but we received the stream close msg first, we # 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: 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. + # propagate any error but hide low-level frame details from + # the caller by default for console/debug-REPL noise + # reduction. if ( hide_tb + and ( - # XXX NOTE XXX don't reraise on certain - # stream-specific internal error types like, - # - # - `trio.EoC` since we want to use the exact instance - # to ensure that it is the error that bubbles upward - # for silent absorption by `Context.open_stream()`. - and not self._eoc + # XXX NOTE special conditions: don't reraise on + # certain stream-specific internal error types like, + # + # - `trio.EoC` since we want to use the exact instance + # to ensure that it is the error that bubbles upward + # for silent absorption by `Context.open_stream()`. + not self._eoc - # - `RemoteActorError` (or `ContextCancelled`) if it gets - # raised from `_raise_from_no_key_in_msg()` since we - # want the same (as the above bullet) for any - # `.open_context()` block bubbled error raised by - # any nearby ctx API remote-failures. - # and not isinstance(src_err, RemoteActorError) + # - `RemoteActorError` (or subtypes like ctxc) + # since we want to present the error as though it is + # "sourced" directly from this `.receive()` call and + # generally NOT include the stack frames raised from + # inside the `PldRx` and/or the transport stack + # layers. + or isinstance(src_err, RemoteActorError) + ) ): raise type(src_err)(*src_err.args) from src_err else: + # for any non-graceful-EOC we want to NOT hide this frame + if not self._eoc: + __tracebackhide__: bool = False + raise src_err async def aclose(self) -> list[Exception|dict]: @@ -292,7 +286,8 @@ class MsgStream(trio.abc.Channel): while not drained: try: maybe_final_msg = self.receive_nowait( - allow_msg_keys=['yield', 'return'], + # allow_msgs=[Yield, Return], + expect_msg=Yield, ) if maybe_final_msg: log.debug( @@ -377,14 +372,15 @@ class MsgStream(trio.abc.Channel): # await rx_chan.aclose() if not self._eoc: - log.cancel( - 'Stream closed before it received an EoC?\n' - 'Setting eoc manually..\n..' - ) - self._eoc: bool = trio.EndOfChannel( - f'Context stream closed by {self._ctx.side}\n' + message: str = ( + f'Stream self-closed by {self._ctx.side!r}-side before EoC\n' + # } bc a stream is a "scope"/msging-phase inside an IPC + f'x}}>\n' f'|_{self}\n' ) + log.cancel(message) + self._eoc = trio.EndOfChannel(message) + # ?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 @@ -469,6 +465,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, ) @@ -515,11 +514,10 @@ class MsgStream(trio.abc.Channel): try: await self._ctx.chan.send( - payload={ - 'yield': data, - 'cid': self._ctx.cid, - }, - # hide_tb=hide_tb, + payload=Yield( + cid=self._ctx.cid, + pld=data, + ), ) except ( trio.ClosedResourceError, @@ -533,6 +531,224 @@ class MsgStream(trio.abc.Channel): 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}' + ) + + def stream(func: Callable) -> Callable: ''' @@ -541,7 +757,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 be81e4e6..4a538e9f 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -80,15 +80,19 @@ class ActorNursery: ''' def __init__( self, + # TODO: maybe def these as fields of a struct looking type? actor: Actor, ria_nursery: trio.Nursery, da_nursery: trio.Nursery, errors: dict[tuple[str, str], BaseException], + ) -> None: # self.supervisor = supervisor # TODO self._actor: Actor = actor - self._ria_nursery = ria_nursery + + # TODO: rename to `._tn` for our conventional "task-nursery" self._da_nursery = da_nursery + self._children: dict[ tuple[str, str], tuple[ @@ -97,13 +101,12 @@ class ActorNursery: Portal | None, ] ] = {} - # portals spawned with ``run_in_actor()`` are - # cancelled when their "main" result arrives - self._cancel_after_result_on_exit: set = set() + self.cancelled: bool = False self._join_procs = trio.Event() self._at_least_one_child_in_debug: bool = False self.errors = errors + self._scope_error: BaseException|None = None self.exited = trio.Event() # NOTE: when no explicit call is made to @@ -114,24 +117,43 @@ class ActorNursery: # and syncing purposes to any actor opened nurseries. self._implicit_runtime_started: bool = False + # TODO: remove the `.run_in_actor()` API and thus this 2ndary + # nursery when that API get's moved outside this primitive! + self._ria_nursery = ria_nursery + # portals spawned with ``run_in_actor()`` are + # cancelled when their "main" result arrives + self._cancel_after_result_on_exit: set = set() + async def start_actor( self, name: str, + *, + 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, + + # TODO: ideally we can rm this once we no longer have + # a `._ria_nursery` since the dependent APIs have been + # removed! + nursery: trio.Nursery|None = None, + ) -> Portal: ''' Start a (daemon) actor: an process that has no designated "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() @@ -184,6 +206,14 @@ class ActorNursery: ) ) + # 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. + # |_ mention how it's similar to `trio-parallel` API? + # -[ ] use @api_frame on the wrapper async def run_in_actor( self, @@ -209,13 +239,14 @@ class ActorNursery: the actor is terminated. ''' + __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 [] @@ -244,19 +275,24 @@ class ActorNursery: ) return portal + # @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. + Cancel this actor-nursery by instructing each subactor's + runtime to cancel and wait for all underlying sub-processes + to terminate. - If ``hard_killl`` is set to ``True`` then kill the processes - directly without any far end graceful ``trio`` cancellation. + If `hard_kill` is set then kill the processes directly using + the spawning-backend's API/OS-machinery without any attempt + at (graceful) `trio`-style cancellation using our + `Actor.cancel()`. ''' + __runtimeframe__: int = 1 # noqa self.cancelled = True # TODO: impl a repr for spawn more compact @@ -337,11 +373,15 @@ class ActorNursery: @acm async def _open_and_supervise_one_cancels_all_nursery( actor: Actor, + tb_hide: bool = False, ) -> typing.AsyncGenerator[ActorNursery, None]: - # TODO: yay or nay? - __tracebackhide__ = True + # normally don't need to show user by default + __tracebackhide__: bool = tb_hide + + 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] = {} @@ -351,7 +391,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: @@ -386,7 +426,8 @@ async def _open_and_supervise_one_cancels_all_nursery( ) 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 @@ -464,8 +505,10 @@ 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. @@ -480,7 +523,7 @@ async def _open_and_supervise_one_cancels_all_nursery( if an._children: log.cancel( 'Actor-nursery cancelling due error type:\n' - f'{err}\n' + f'{outer_err}\n' ) with trio.CancelScope(shield=True): await an.cancel() @@ -507,11 +550,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, @@ -531,6 +582,7 @@ async def open_nursery( which cancellation scopes correspond to each spawned subactor set. ''' + __tracebackhide__: bool = True implicit_runtime: bool = False actor: Actor = current_actor(err_on_no_runtime=False) an: ActorNursery|None = None @@ -581,13 +633,25 @@ async def open_nursery( an.exited.set() finally: + # 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: + # shutdown runtime if it was started and report noisly + # that we're did so. msg += '=> Shutting down actor runtime <=\n' + log.info(msg) - log.info(msg) + else: + # keep noise low during std operation. + log.runtime(msg) diff --git a/tractor/_testing/__init__.py b/tractor/_testing/__init__.py index 876c87e8..fd79fe20 100644 --- a/tractor/_testing/__init__.py +++ b/tractor/_testing/__init__.py @@ -26,6 +26,9 @@ import tractor from .pytest import ( tractor_test as tractor_test ) +from .fault_simulation import ( + break_ipc as break_ipc, +) def repodir() -> pathlib.Path: 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/devx/__init__.py b/tractor/devx/__init__.py index 75aec953..cfcff931 100644 --- a/tractor/devx/__init__.py +++ b/tractor/devx/__init__.py @@ -27,21 +27,23 @@ from ._debug import ( pause as pause, pause_from_sync as pause_from_sync, shield_sigint_handler as shield_sigint_handler, - MultiActorPdb as MultiActorPdb, open_crash_handler as open_crash_handler, maybe_open_crash_handler as maybe_open_crash_handler, + maybe_init_greenback as maybe_init_greenback, post_mortem as post_mortem, + mk_pdb as mk_pdb, ) 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, -# ) +from .pformat import ( + add_div as add_div, + pformat_caller_frame as pformat_caller_frame, + pformat_boxed_tb as pformat_boxed_tb, +) +# TODO, move this to a new `.devx._pdbp` mod? def _enable_readline_feats() -> str: ''' Handle `readline` when compiled with `libedit` to avoid breaking @@ -73,5 +75,4 @@ def _enable_readline_feats() -> str: return 'readline' -# TODO, move this to a new `.devx._pdbp` mod? _enable_readline_feats() diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 255b1dbd..1135932c 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -25,14 +25,18 @@ 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 ( @@ -40,104 +44,735 @@ from typing import ( Callable, AsyncIterator, AsyncGenerator, + TypeAlias, TYPE_CHECKING, ) from types import ( + FunctionType, FrameType, ModuleType, + TracebackType, + CodeType, ) +from msgspec import Struct import pdbp import sniffio -import tractor import trio -from trio.lowlevel import current_task +from trio import CancelScope +from trio.lowlevel import ( + current_task, +) from trio import ( TaskStatus, - # Task, ) - +import tractor from tractor.log import get_logger +from tractor._context import Context +from tractor import _state from tractor._state import ( current_actor, is_root_process, debug_mode, + current_ipc_ctx, ) -from tractor._exceptions import ( - is_multi_cancelled, - ContextCancelled, -) -from tractor._ipc import Channel +# 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, ) log = get_logger(__name__) - -__all__ = [ - 'breakpoint', - 'post_mortem', -] +# 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 global debug lock state. + Actor-tree-global debug lock state, exists only in a root process. - Mostly to avoid a lot of ``global`` declarations for now XD. + 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: Callable | None = None + @staticmethod + def get_locking_task_cs() -> CancelScope|None: + if not is_root_process(): + raise RuntimeError( + '`Lock.locking_task_cs` is invalid in subactors!' + ) - _trio_handler: Callable[ - [int, FrameType | None], Any - ] | int | None = None + if ctx := Lock.ctx_in_debug: + return ctx._scope - # actor-wide variable pointing to current task name using debugger - local_task_in_debug: str | None = None + return 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: trio.CancelScope|None = None + # TODO: once we convert to singleton-per-actor-style + # @property + # def stats(cls) -> trio.LockStatistics: + # return cls._debug_lock.statistics() - # 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: trio.CancelScope|None = None + # @property + # def owner(cls) -> Task: + # return cls._debug_lock.statistics().owner - # actor tree-wide actor uid that supposedly has the tty lock - global_actor_in_debug: tuple[str, str] = None + # 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 - local_pdb_complete: trio.Event | None = None - no_remote_has_tty: trio.Event | None = None - - # lock in root actor preventing multi-access to local tty + _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() - _orig_sigint_handler: Callable | None = None - _blocked: set[tuple[str, str]] = 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 - `trio.Task` cancellation) in subactors when the `pdb` REPL + `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 a signal is received from - an external request; explicit runtime cancel requests are - allowed until the use exits the REPL session using - 'continue' or 'quit', at which point the orig SIGINT - handler is restored. + 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 ''' # @@ -161,6 +796,12 @@ class Lock: @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... @@ -171,6 +812,11 @@ class Lock: 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, @@ -209,42 +855,57 @@ class Lock: # ) @classmethod - def release(cls): - try: - if not cls.is_main_trio_thread(): - trio.from_thread.run_sync( - cls._debug_lock.release - ) - else: - cls._debug_lock.release() + def cancel(cls) -> bool: + if (req_cs := cls.req_cs): + req_cs.cancel() + return True - 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 + return False + @classmethod + @pdbp.hideframe + def release( + cls, + cancel_req_task: bool = False, + ): + repl_release: trio.Event = cls.repl_release 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() + # 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: - # restore original sigint handler - cls.unshield_sigint() - cls.repl = None + # 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.global_actor_in_debug = None - cls.local_task_in_debug = None + cls.repl_task = None + cls.repl = None + # restore original sigint handler + cls.unshield_sigint() class TractorConfig(pdbp.DefaultConfig): ''' - Custom ``pdbp`` goodness :surfer: + Custom `pdbp` config which tries to use the best tradeoff + between pretty and minimal. ''' use_pygments: bool = True @@ -255,32 +916,91 @@ class TractorConfig(pdbp.DefaultConfig): # 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 MultiActorPdb(pdbp.Pdb): + +class PdbREPL(pdbp.Pdb): ''' - Add teardown hooks to the regular ``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: figure out how to disallow recursive .set_trace() entry - # since that'll cause deadlock for us. + # 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: - Lock.release() + # 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: - Lock.release() + 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 @@ -312,255 +1032,279 @@ class MultiActorPdb(pdbp.Pdb): 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: str = current_task().name - we_acquired: bool = False - - log.runtime( - f"Attempting to acquire TTY lock, remote task: {task_name}:{uid}" - ) - try: - log.runtime( - f"entering lock checkpoint, remote task: {task_name}:{uid}" - ) - # 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() - we_acquired = True - - 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: str = 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: bool = 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( +async def request_root_stdio_lock( actor_uid: tuple[str, str], - task_status: TaskStatus[trio.CancelScope] = trio.TASK_STATUS_IGNORED + task_uid: tuple[str, int], + + shield: bool = False, + task_status: TaskStatus[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*. + 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. - 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 ``pause()``). 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()``). + 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. ''' - from .._discovery import get_root + 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, - with trio.CancelScope(shield=True) as cs: - Lock._debugger_request_cs = cs + # 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__, - try: - async with get_root() as portal: + ) as (req_ctx, status): - # this syncs to child's ``Context.started()`` call. - async with portal.open_context( - lock_tty_for_child, - subactor_uid=actor_uid, + DebugStatus.req_ctx = req_ctx + log.devx( + 'Subactor locked TTY with msg\n\n' + f'{status}\n' + ) - ) as (ctx, val): + # try: + assert status.subactor_uid == actor_uid + assert status.cid + # except AttributeError: + # log.exception('failed pldspec asserts!') + # mk_pdb().set_trace() + # raise - log.debug('locked context') - assert val == 'Locked' + # set last rxed lock dialog status. + DebugStatus.lock_status = status - async with ctx.open_stream() as stream: - try: - # unblock local caller - assert Lock.local_pdb_complete - task_status.started(cs) + async with req_ctx.open_stream() as stream: + task_status.started(req_ctx) - # wait for local task to exit and - # release the REPL - await Lock.local_pdb_complete.wait() + # 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, + ) + ) - finally: - # TODO: shielding currently can cause hangs... - # with trio.CancelScope(shield=True): - await stream.send('pdb_unlock') + # sync with child-side root locker task + # completion + status: LockStatus = await req_ctx.result() + assert not status.locked + DebugStatus.lock_status = status - # sync with callee termination - assert await ctx.result() == "pdb_unlock_complete" + 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' + ) - log.debug('exitting child side locking task context') + 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 ContextCancelled: - log.warning('Root actor cancelled debug lock') - raise + except ( + BaseException, + ) as ctx_err: + message: str = ( + 'Failed during debug request dialog with root actor?\n\n' + ) - finally: - Lock.local_task_in_debug = None - log.debug('Exiting debugger from child') + 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 -def mk_mpdb() -> MultiActorPdb: + 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 `MultiActorPdb`: a multi-process safe `pdbp` - REPL using the magic of SC! + 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' TTY & stdstreams - via an IPC managed `Lock` singleton per process tree. + - 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 either explicit requests in the runtime or + - 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 = MultiActorPdb() - - # Always shield out SIGINTs for subactors when REPL is active. - # - # 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 - Lock.shield_sigint() + 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 - # pdb_obj: MultiActorPdb | None = None, *args, ) -> None: @@ -574,8 +1318,6 @@ def shield_sigint_handler( ''' __tracebackhide__: bool = True - uid_in_debug: tuple[str, str]|None = Lock.global_actor_in_debug - actor: Actor = current_actor() def do_cancel(): @@ -586,107 +1328,191 @@ def shield_sigint_handler( 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 + # 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 - any_connected: bool = False + # only set in the actor actually running the REPL + repl: PdbREPL|None = DebugStatus.repl - if uid_in_debug is not None: + # 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. - chans: list[tractor.Channel] = 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() + any_connected: bool = any_connected_locker_child() + # if not any_connected: + # return do_cancel() - # only set in the actor actually running the REPL - pdb_obj: MultiActorPdb|None = Lock.repl + problem = ( + f'root {actor.uid} handling SIGINT\n' + f'any_connected: {any_connected}\n\n' - # 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" - ) + f'{Lock.repr()}\n' + ) if ( - Lock._root_local_task_cs_in_debug - and not Lock._root_local_task_cs_in_debug.cancel_called + (ctx := Lock.ctx_in_debug) + and + (uid_in_debug := ctx.chan.uid) # "someone" is (ostensibly) using debug `Lock` ): - Lock._root_local_task_cs_in_debug.cancel() + 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' - # revert back to ``trio`` handler asap! - Lock.unshield_sigint() + # 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' + ) - chan: Channel = actor._parent_chan - if not chan or not chan.connected(): + rent_chan: Channel = actor._parent_chan + if ( + rent_chan is None + or + not rent_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' + 'This sub-actor thinks it is debugging ' + 'but it has no connection to its parent ??\n' + f'{actor.uid}\n' 'Allowing SIGINT propagation..' ) - return do_cancel() + DebugStatus.unshield_sigint() + # do_cancel() - task: str | None = Lock.local_task_in_debug + repl_task: str|None = DebugStatus.repl_task + req_task: str|None = DebugStatus.req_task if ( - task - and pdb_obj + repl_task + and + repl ): log.pdb( - f"Ignoring SIGINT while task in debug mode: `{task}`" + 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(): - else: # XXX: shouldn't ever get here? - raise RuntimeError("WTFWTFWTF") - # raise KeyboardInterrupt("WTFWTFWTF") - # 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. @@ -695,99 +1521,49 @@ def shield_sigint_handler( # 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 + repl # only when current 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) + if repl.shname == 'xonsh': + repl.stdout.write(repl.prompt) - pdb_obj.stdout.flush() + 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? - # pdb_obj.sticky = True - # pdb_obj._print_if_sticky() + # 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' -def _set_trace( - actor: tractor.Actor|None = None, - pdb: MultiActorPdb|None = None, - shield: bool = False, +class DebugRequestError(RuntimeError): + ''' + Failed to request stdio lock from root actor! - extra_frames_up_when_async: int = 1, - hide_tb: bool = True, -): - __tracebackhide__: bool = hide_tb + ''' - actor: tractor.Actor = ( - actor - or - current_actor() - ) - # always start 1 level up from THIS in user code. - frame: FrameType|None - if frame := sys._getframe(): - frame: FrameType = frame.f_back # type: ignore - - if ( - frame - and ( - pdb - and actor is not None - ) - # or shield - ): - msg: str = _pause_msg - if shield: - # log.warning( - msg = ( - '\n\n' - ' ------ - ------\n' - 'Debugger invoked with `shield=True` so an extra\n' - '`trio.CancelScope.__exit__()` frame is shown..\n' - '\n' - 'Try going up one frame to see your pause point!\n' - '\n' - ' SORRY we need to fix this!\n' - ' ------ - ------\n\n' - ) + msg - - # pdbp.set_trace() - # TODO: maybe print the actor supervion tree up to the - # root here? Bo - log.pdb( - f'{msg}\n' - '|\n' - f'|_ {actor.uid}\n' - ) - # no f!#$&* idea, but when we're in async land - # we need 2x frames up? - for i in range(extra_frames_up_when_async): - frame: FrameType = frame.f_back - log.debug( - f'Going up frame {i} -> {frame}\n' - ) - - # engage ze REPL - # B~() - pdb.set_trace(frame=frame) +_repl_fail_msg: str = ( + 'Failed to REPl via `_pause()` ' +) async def _pause( - debug_func: Callable = _set_trace, + debug_func: Callable|partial|None, # NOTE: must be passed in the `.pause_from_sync()` case! - pdb: MultiActorPdb|None = None, + repl: PdbREPL|None = None, # TODO: allow caller to pause despite task cancellation, # exactly the same as wrapping with: @@ -799,9 +1575,15 @@ async def _pause( # shield: bool = False, hide_tb: bool = True, - task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED + 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, -) -> None: +) -> 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 @@ -813,178 +1595,483 @@ async def _pause( __tracebackhide__: bool = hide_tb actor: Actor = current_actor() try: - task_name: str = trio.lowlevel.current_task().name + task: Task = current_task() except RuntimeError as rte: - if actor.is_infected_aio(): - raise RuntimeError( - '`tractor.pause[_from_sync]()` not yet supported ' - 'for infected `asyncio` mode!' - ) from rte + __tracebackhide__: bool = False + log.exception( + 'Failed to get current `trio`-task?' + ) + # if actor.is_infected_aio(): + # mk_pdb().set_trace() + # raise RuntimeError( + # '`tractor.pause[_from_sync]()` not yet supported ' + # 'directly (infected) `asyncio` tasks!' + # ) from rte - if ( - not Lock.local_pdb_complete - or Lock.local_pdb_complete.is_set() - ): - Lock.local_pdb_complete = trio.Event() + raise if debug_func is not None: - debug_func = partial( - debug_func, + 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' ) - if pdb is None: - pdb: MultiActorPdb = mk_mpdb() - - # 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? + # 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: - with trio.CancelScope(shield=True): - await actor._service_n.start( - wait_for_parent_stdin_hijack, - actor.uid, + 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, ) - Lock.repl = pdb - except RuntimeError: - Lock.release() + # 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 - 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 + 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 - elif is_root_process(): + log.devx( + 'Entering `._pause()` for requesting task\n' + f'|_{task}\n' + ) - # 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. + # 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 - # 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() + log.exception( + _repl_fail_msg + + + f'on behalf of {repl_task} ??\n' + ) - Lock.global_actor_in_debug = actor.uid - Lock.local_task_in_debug = task_name - Lock.repl = pdb + DebugStatus.release(cancel_req_task=True) - try: - # TODO: do we want to support using this **just** for the - # locking / common code (prolly to help address #320)? - # - if debug_func is None: - task_status.started(Lock) + # sanity checks for ^ on request/status teardown + assert DebugStatus.repl is None + assert DebugStatus.repl_task is None - else: - # block here one (at the appropriate frame *up*) where - # ``breakpoint()`` was awaited and begin handling stdio. - log.debug('Entering sync world of the `pdb` REPL..') - try: - debug_func( - actor, - pdb, - extra_frames_up_when_async=2, - shield=shield, - ) - except BaseException: - log.exception( - 'Failed to invoke internal `debug_func = ' - f'{debug_func.func.__name__}`\n' - ) - raise + # 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 - except bdb.BdbQuit: - Lock.release() raise - except BaseException: - log.exception( - 'Failed to engage debugger via `_pause()` ??\n' - ) - 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 -# 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) + # 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, - debug_func: Callable|None = _set_trace, - - # 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?.. - # + # 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, - task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED, - **_pause_kwargs, ) -> None: @@ -1003,44 +2090,38 @@ async def pause( https://en.wikipedia.org/wiki/Breakpoint ''' - __tracebackhide__: bool = True + __tracebackhide__: bool = hide_tb - if shield: - # NOTE XXX: even hard coding this inside the `class CancelScope:` - # doesn't seem to work for me!? - # ^ XXX ^ + # 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() - # def _exit(self, *args, **kwargs): - # __tracebackhide__: bool = True - # super().__exit__(*args, **kwargs) - - trio.CancelScope.__enter__.__tracebackhide__ = True - trio.CancelScope.__exit__.__tracebackhide__ = True - - # import types - # with trio.CancelScope(shield=shield) as cs: - # cs.__exit__ = types.MethodType(_exit, cs) - # cs.__exit__.__tracebackhide__ = True - - with trio.CancelScope(shield=shield) as cs: - # setattr(cs.__exit__.__func__, '__tracebackhide__', True) - # setattr(cs.__enter__.__func__, '__tracebackhide__', True) - - # NOTE: so the caller can always cancel even if shielded - task_status.started(cs) - return await _pause( - debug_func=debug_func, - shield=True, - task_status=task_status, - **_pause_kwargs - ) - else: - return await _pause( - debug_func=debug_func, - shield=False, - task_status=task_status, - **_pause_kwargs - ) + # 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 @@ -1084,58 +2165,184 @@ def maybe_import_greenback( return False -async def maybe_init_greenback( - **kwargs, -) -> None|ModuleType: - - if mod := maybe_import_greenback(**kwargs): - await mod.ensure_portal() - log.info( - '`greenback` portal opened!\n' - 'Sync debug support activated!\n' - ) - return mod +async def maybe_init_greenback(**kwargs) -> None|ModuleType: + try: + if mod := maybe_import_greenback(**kwargs): + await mod.ensure_portal() + log.devx( + '`greenback` portal opened!\n' + 'Sync debug support activated!\n' + ) + return mod + except BaseException: + log.exception('Failed to init `greenback`..') + raise return None -# TODO: allow pausing from sync code. -# normally by remapping python's builtin breakpoint() hook to this -# runtime aware version which takes care of all . +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 = False, + 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 - actor: tractor.Actor = current_actor( - err_on_no_runtime=False, - ) - log.debug( - f'{actor.uid}: JUST ENTERED `tractor.pause_from_sync()`' - f'|_{actor}\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' + try: + actor: tractor.Actor = current_actor( + err_on_no_runtime=False, ) - - # NOTE: 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!' + 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' + ) - # raises on not-found by default - greenback: ModuleType = maybe_import_greenback() - mdb: MultiActorPdb = mk_mpdb() + # TODO: once supported, remove this AND the one + # inside `._pause()`! + # outstanding impl fixes: + # -[ ] need to make `.shield_sigint()` below work here! + # -[ ] how to handle `asyncio`'s new SIGINT-handler + # injection? + # -[ ] should `breakpoint()` work and what does it normally + # do in `asyncio` ctxs? + if actor.is_infected_aio(): + raise RuntimeError( + '`tractor.pause[_from_sync]()` not yet supported ' + 'for infected `asyncio` mode!' + ) - # run async task which will lock out the root proc's TTY. - if not Lock.is_main_trio_thread(): + 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 @@ -1144,58 +2351,165 @@ def pause_from_sync( # `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() + + # TODO: how to ensure this is either dynamically (if + # needed) called here (in some bg tn??) or that the + # subactor always already called it? + # greenback: ModuleType = await maybe_init_greenback() + + message += f'-> imported {greenback}\n' + repl_owner: Task = current_task() + message += '-> calling `greenback.await_(_pause(debug_func=None))` from sync caller..\n' + try: + out = greenback.await_( + _pause( + debug_func=None, + repl=repl, + hide_tb=hide_tb, + called_from_sync=True, + **_pause_kwargs, + ) + ) + except RuntimeError as rte: + if not _state._runtime_vars.get( + 'use_greenback', + False, + ): + raise RuntimeError( + '`greenback` was never initialized in this actor!?\n\n' + f'{_state._runtime_vars}\n' + ) from rte + + raise + + if out: + bg_task, repl = out + assert repl is repl + 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.. # - trio.from_thread.run( - partial( - pause, - debug_func=None, - pdb=mdb, - hide_tb=hide_tb, - ) - ) - # TODO: maybe the `trio.current_task()` id/name if avail? - Lock.local_task_in_debug: str = str(threading.current_thread().name) + # 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 - else: # we are presumably the `trio.run()` + main thread - greenback.await_( - pause( - debug_func=None, - pdb=mdb, - hide_tb=hide_tb, - ) - ) - Lock.local_task_in_debug: str = current_task().name - # TODO: ensure we aggressively make the user aware about - # entering the global ``breakpoint()`` built-in from sync - # code? - _set_trace( - actor=actor, - pdb=mdb, - hide_tb=hide_tb, - extra_frames_up_when_async=1, +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. - # TODO? will we ever need it? - # -> the gb._await() won't be affected by cancellation? - # shield=shield, + Note: this assigned to `os.environ['PYTHONBREAKPOINT']` inside `._root` + + ''' + pause_from_sync( + *args, + called_from_builtin=True, + api_frame=inspect.currentframe(), + **kwargs, ) - # 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 # 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(**kwargs): +async def breakpoint( + hide_tb: bool = True, + **kwargs, +): log.warning( '`tractor.breakpoint()` is deprecated!\n' 'Please use `tractor.pause()` instead!\n' ) - await pause(**kwargs) + __tracebackhide__: bool = hide_tb + await pause( + api_frame=inspect.currentframe(), + **kwargs, + ) _crash_msg: str = ( @@ -1204,12 +2518,15 @@ _crash_msg: str = ( def _post_mortem( - actor: tractor.Actor, - pdb: MultiActorPdb, - shield: bool = False, + # provided and passed by `_pause()` + repl: PdbREPL, - # only for compat with `._set_trace()`.. - extra_frames_up_when_async=0, + # XXX all `partial`-ed in by `post_mortem()` below! + tb: TracebackType, + api_frame: FrameType, + + shield: bool = False, + hide_tb: bool = False, ) -> None: ''' @@ -1217,32 +2534,92 @@ def _post_mortem( 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'|_ {actor.uid}\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' ) - # TODO: only replacing this to add the + # NOTE only replacing this from `pdbp.xpm()` to add the # `end=''` to the print XD - # pdbp.xpm(Pdb=lambda: pdb) - info = sys.exc_info() print(traceback.format_exc(), end='') - pdbp.post_mortem( - t=info[2], - Pdb=lambda: pdb, + + 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 ) -post_mortem = partial( - pause, - debug_func=_post_mortem, -) - - -async def _maybe_enter_pm(err): +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() @@ -1261,12 +2638,13 @@ async def _maybe_enter_pm(err): # 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 + 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 @@ -1275,9 +2653,12 @@ async def _maybe_enter_pm(err): @acm async def acquire_debug_lock( subactor_uid: tuple[str, str], -) -> AsyncGenerator[None, tuple]: +) -> AsyncGenerator[ + trio.CancelScope|None, + tuple, +]: ''' - Grab root's debug lock on entry, release on exit. + Request to acquire the TTY `Lock` in the root actor, release on exit. This helper is for actor's who don't actually need to acquired the debugger but want to wait until the lock is free in the @@ -1290,12 +2671,12 @@ async def acquire_debug_lock( return async with trio.open_nursery() as n: - cs = await n.start( - wait_for_parent_stdin_hijack, + ctx: Context = await n.start( + request_root_stdio_lock, subactor_uid, ) - yield None - cs.cancel() + yield ctx + ctx.cancel() async def maybe_wait_for_debugger( @@ -1304,6 +2685,7 @@ async def maybe_wait_for_debugger( child_in_debug: bool = False, header_msg: str = '', + _ll: str = 'devx', ) -> bool: # was locked and we polled? @@ -1313,6 +2695,7 @@ async def maybe_wait_for_debugger( ): return False + logmeth: Callable = getattr(log, _ll) msg: str = header_msg if ( @@ -1324,9 +2707,12 @@ async def maybe_wait_for_debugger( # will make the pdb repl unusable. # Instead try to wait for pdb to be released before # tearing down. - in_debug: tuple[str, str]|None = Lock.global_actor_in_debug - debug_complete: trio.Event|None = Lock.no_remote_has_tty - + 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 @@ -1337,7 +2723,7 @@ async def maybe_wait_for_debugger( elif in_debug: msg += ( - f'Debug `Lock` in use by subactor: {in_debug}\n' + 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 @@ -1346,7 +2732,7 @@ async def maybe_wait_for_debugger( # XXX => but it doesn't seem to work.. # await trio.testing.wait_all_tasks_blocked(cushion=0) else: - log.debug( + logmeth( msg + 'Root immediately acquired debug TTY LOCK' @@ -1355,31 +2741,39 @@ async def maybe_wait_for_debugger( for istep in range(poll_steps): if ( - debug_complete - and not debug_complete.is_set() + Lock.req_handler_finished is not None + and not Lock.req_handler_finished.is_set() and in_debug is not None ): - log.pdb( + # caller_frame_info: str = pformat_caller_frame() + logmeth( msg + - 'Root is waiting on tty lock to release..\n' + '\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 debug_complete.wait() - log.pdb( - f'Child subactor released debug lock\n' + 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 ( - debug_complete is None - or debug_complete.is_set() + Lock.req_handler_finished is None + or Lock.req_handler_finished.is_set() ) ): - log.pdb( + logmeth( msg + 'Root acquired tty lock!' @@ -1387,25 +2781,16 @@ async def maybe_wait_for_debugger( break else: - # TODO: don't need this right? - # await trio.lowlevel.checkpoint() - - log.debug( + logmeth( 'Root polling for debug:\n' f'poll step: {istep}\n' - f'poll delya: {poll_delay}' + f'poll delya: {poll_delay}\n\n' + f'{Lock.repr()}\n' ) - with trio.CancelScope(shield=True): + with CancelScope(shield=True): await trio.sleep(poll_delay) continue - # fallthrough on failure to acquire.. - # else: - # raise RuntimeError( - # msg - # + - # 'Root actor failed to acquire debug lock?' - # ) return True # else: @@ -1443,10 +2828,10 @@ def open_crash_handler( `trio.run()`. ''' + err: BaseException try: yield except tuple(catch) as err: - if type(err) not in ignore: pdbp.xpm() @@ -1454,7 +2839,9 @@ def open_crash_handler( @cm -def maybe_open_crash_handler(pdb: bool = False): +def maybe_open_crash_handler( + pdb: bool = False, +): ''' Same as `open_crash_handler()` but with bool input flag to allow conditional handling. diff --git a/tractor/devx/_code.py b/tractor/devx/_frame_stack.py similarity index 52% rename from tractor/devx/_code.py rename to tractor/devx/_frame_stack.py index 01d64cd1..89a9e849 100644 --- a/tractor/devx/_code.py +++ b/tractor/devx/_frame_stack.py @@ -20,9 +20,8 @@ as it pertains to improving the grok-ability of our runtime! ''' from __future__ import annotations +from functools import partial import inspect -# import msgspec -# from pprint import pformat from types import ( FrameType, FunctionType, @@ -30,9 +29,8 @@ from types import ( # CodeType, ) from typing import ( - # Any, + Any, Callable, - # TYPE_CHECKING, Type, ) @@ -40,6 +38,7 @@ from tractor.msg import ( pretty_struct, NamespacePath, ) +import wrapt # TODO: yeah, i don't love this and we should prolly just @@ -81,6 +80,31 @@ def get_class_from_frame(fr: FrameType) -> ( 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: @@ -96,34 +120,63 @@ def func_ref_from_frame( ) -# TODO: move all this into new `.devx._code`! -# -[ ] prolly create a `@runtime_api` dec? -# -[ ] ^- make it capture and/or accept buncha optional -# meta-data like a fancier version of `@pdbp.hideframe`. -# class CallerInfo(pretty_struct.Struct): - rt_fi: inspect.FrameInfo - call_frame: FrameType + # 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_func_ref(self) -> Callable|None: - return func_ref_from_frame(self.rt_fi.frame) + 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_ref + func: FunctionType = self.api_func if func: return NamespacePath.from_ref(func) return '' @property - def caller_func_ref(self) -> Callable|None: - return func_ref_from_frame(self.call_frame) + 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.caller_func_ref + func: FunctionType = self.api_func if func: return NamespacePath.from_ref(func) @@ -170,8 +223,66 @@ def find_caller_info( call_frame = call_frame.f_back return CallerInfo( - rt_fi=fi, - call_frame=call_frame, + _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/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/log.py b/tractor/log.py index edb058e3..47f1f259 100644 --- a/tractor/log.py +++ b/tractor/log.py @@ -54,11 +54,12 @@ LOG_FORMAT = ( DATE_FORMAT = '%b %d %H:%M:%S' # FYI, ERROR is 40 +# TODO: use a `bidict` to avoid the :155 check? CUSTOM_LEVELS: dict[str, int] = { 'TRANSPORT': 5, 'RUNTIME': 15, 'DEVX': 17, - 'CANCEL': 18, + 'CANCEL': 22, 'PDB': 500, } STD_PALETTE = { @@ -147,6 +148,8 @@ class StackLevelAdapter(LoggerAdapter): Delegate a log call to the underlying logger, after adding contextual information from this adapter instance. + NOTE: all custom level methods (above) delegate to this! + ''' if self.isEnabledFor(level): stacklevel: int = 3 diff --git a/tractor/msg/__init__.py b/tractor/msg/__init__.py index 906627cf..44586f2d 100644 --- a/tractor/msg/__init__.py +++ b/tractor/msg/__init__.py @@ -18,9 +18,56 @@ Built-in messaging patterns, types, APIs and helpers. ''' +from typing import ( + TypeAlias, +) from .ptr import ( NamespacePath as NamespacePath, ) -from .types import ( +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..2faadb9f --- /dev/null +++ b/tractor/msg/_ops.py @@ -0,0 +1,842 @@ +# 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( + 'Invalid 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 check on IPC 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 + + # when the applied spec is unwound/removed, the same IPC-ctx + # should still be in scope. + curr_ctx: Context = current_ipc_ctx() + assert ctx is curr_ctx + + +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 context's + rx-mem-chan (i.e. from `Context._rx_chan`) in search for a final + `Return` or `Error` msg. + + Deliver the `Return` + preceding drained msgs (`list[MsgType]`) + as a pair unless an `Error` is found, in which unpack and raise + it. + + The motivation here is to always capture any remote error relayed + by the remote peer task during a ctxc condition. + + For eg. a ctxc-request may be sent to the peer as part of the + local task's (request for) cancellation but then that same task + **also errors** before executing the teardown in the + `Portal.open_context().__aexit__()` block. In such error-on-exit + cases we want to always capture and raise any delivered remote + error (like an expected ctxc-ACK) as part of the final + `ctx.wait_for_result()` teardown sequence such that the + `Context.outcome` related state always reflect what transpired + even after ctx closure and the `.open_context()` block exit. + + ''' + __tracebackhide__: bool = hide_tb + 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: + taskc: trio.Cancelled = _taskc + + # report when the cancellation wasn't (ostensibly) due to + # RPC operation, some surrounding parent cancel-scope. + if not ctx._scope.cancel_called: + task: trio.lowlevel.Task = trio.lowlevel.current_task() + rent_n: trio.Nursery = task.parent_nursery + if ( + (local_cs := rent_n.cancel_scope).cancel_called + ): + log.cancel( + 'RPC-ctx cancelled by local-parent scope during drain!\n\n' + f'c}}>\n' + f' |_{rent_n}\n' + f' |_.cancel_scope = {local_cs}\n' + f' |_>c}}\n' + f' |_{ctx.pformat(indent=" "*9)}' + # ^TODO, some (other) simpler repr here? + ) + __tracebackhide__: bool = False + + # CASE 2: mask the local cancelled-error(s) + # only when we are sure the remote error is + # the source cause of this local task's + # cancellation. + ctx.maybe_raise( + hide_tb=hide_tb, + from_src_exc=taskc, + # ?TODO? when *should* we use this? + ) + + # 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.runtime( # normal/expected shutdown transaction + '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! + report: str = ( + f'Invalid or unknown msg type {type(msg)!r}!?\n' + ) + if not msg.cid: + report += ( + '\nWhich also has no `.cid` field?\n' + ) + + raise MessagingError( + report + + + f'\n{msg}\n' + ) + + 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..15e469e2 --- /dev/null +++ b/tractor/msg/pretty_struct.py @@ -0,0 +1,292 @@ +# 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, +) + +from tractor.log import get_logger + +log = get_logger() +# TODO: auto-gen type sig for input func both for +# type-msgs and logging of RPC tasks? +# taken and modified from: +# 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 + try: + val_str: str = saferepr(v) + except Exception: + log.exception( + 'Failed to `saferepr({type(struct)})` !?\n' + ) + return _Struct.__repr__(struct) + + # TODO: LOLOL use `textwrap.indent()` instead dawwwwwg! + obj_str += (field_ws + f'{k}: {typ_name} = {val_str},\n') + + 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 + # __repr__ = pformat + # __str__ = __repr__ = pformat + # TODO: use a pprint.PrettyPrinter instance around ONLY rendering + # inside a known tty? + # def __repr__(self) -> str: + # ... + def __repr__(self) -> str: + try: + return pformat(self) + except Exception: + log.exception( + f'Failed to `pformat({type(self)})` !?\n' + ) + return _Struct.__repr__(self) + + 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 index 4d089c3e..abe5406e 100644 --- a/tractor/msg/ptr.py +++ b/tractor/msg/ptr.py @@ -76,9 +76,11 @@ class NamespacePath(str): return self._ref @staticmethod - def _mk_fqnp(ref: type | object) -> tuple[str, str]: + def _mk_fqnp( + ref: type|object, + ) -> tuple[str, str]: ''' - Generate a minial ``str`` pair which describes a python + Generate a minial `str` pair which describes a python object's namespace path and object/type name. In more precise terms something like: @@ -87,10 +89,9 @@ class NamespacePath(str): of THIS type XD ''' - if ( - isfunction(ref) - ): + if isfunction(ref): name: str = getattr(ref, '__name__') + mod_name: str = ref.__module__ elif ismethod(ref): # build out the path manually i guess..? @@ -99,15 +100,19 @@ class NamespacePath(str): 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] = ( - ref.__module__, + mod_name, name, ) return fqnp @@ -115,7 +120,7 @@ class NamespacePath(str): @classmethod def from_ref( cls, - ref: type | object, + ref: type|object, ) -> NamespacePath: diff --git a/tractor/msg/types.py b/tractor/msg/types.py index 3ceff845..0904411f 100644 --- a/tractor/msg/types.py +++ b/tractor/msg/types.py @@ -15,256 +15,716 @@ # along with this program. If not, see . ''' -Extensions to built-in or (heavily used but 3rd party) friend-lib -types. +Define our strictly typed IPC message spec for the SCIPP: + +that is, + +the "Structurred-Concurrency-Inter-Process-(dialog)-(un)Protocol". ''' from __future__ import annotations -from collections import UserList -from pprint import ( - saferepr, -) +import types from typing import ( Any, - Iterator, + Generic, + Literal, + Type, + TypeVar, + TypeAlias, + Union, ) from msgspec import ( - msgpack, - Struct as _Struct, - structs, + defstruct, + # field, + Raw, + Struct, + # UNSET, + # UnsetType, ) -# 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) +from tractor.msg import ( + pretty_struct, +) +from tractor.log import get_logger -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. +log = get_logger('tractor.msgspec') - ''' - 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 +# type variable for the boxed payload field `.pld` +PayloadT = TypeVar('PayloadT') -class Struct( - _Struct, +class PayloadMsg( + Struct, + Generic[PayloadT], # https://jcristharif.com/msgspec/structs.html#tagged-unions - # tag='pikerstruct', - # tag=True, + 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, ): ''' - A "human friendlier" (aka repl buddy) struct subtype. + 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. ''' - def _sin_props(self) -> Iterator[ - tuple[ - structs.FieldIinfo, - str, - Any, - ] - ]: - ''' - Iterate over all non-@property fields of this struct. + 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 - ''' - fi: structs.FieldInfo - for fi in structs.fields(self): - key: str = fi.name - val: Any = getattr(self, key) - yield fi, key, val + # The msg's "payload" (spelled without vowels): + # https://en.wikipedia.org/wiki/Payload_(computing) + pld: Raw - def to_dict( - self, - include_non_members: bool = True, + # ^-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. - ) -> 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. +# TODO: complete rename +Msg = PayloadMsg - ''' - asdict: dict = structs.asdict(self) - if include_non_members: - return asdict - # only return a dict of the struct members - # which were provided as input, NOT anything - # added as type-defined `@property` methods! - sin_props: dict = {} - fi: structs.FieldInfo - for fi, k, v in self._sin_props(): - sin_props[k] = asdict[k] +class Aid( + Struct, + tag=True, + tag_field='msg_type', +): + ''' + Actor-identity msg. - return sin_props + Initial contact exchange enabling an actor "mailbox handshake" + delivering the peer identity (and maybe eventually contact) + info. - def pformat( - self, - field_indent: int = 2, - indent: int = 0, + Used by discovery protocol to register actors as well as + conduct the initial comms (capability) filtering. - ) -> str: - ''' - Recursion-safe `pprint.pformat()` style formatting of - a `msgspec.Struct` for sane reading by a human using a REPL. + ''' + 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 - ''' - # global whitespace indent - ws: str = ' '*indent - # field whitespace indent - field_ws: str = ' '*(field_indent + indent) +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. - # qtn: str = ws + self.__class__.__qualname__ - qtn: str = self.__class__.__qualname__ + ''' + # 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] - obj_str: str = '' # accumulator - fi: structs.FieldInfo - k: str - v: Any - for fi, k, v in self._sin_props(): + # module import capability + enable_modules: dict[str, str] - # 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)) + # TODO: not just sockaddr pairs? + # -[ ] abstract into a `TransportAddr` type? + reg_addrs: list[tuple[str, int]] + bind_addrs: list[tuple[str, int]] - # 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: 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] +# ) - # TODO: LOLOL use `textwrap.indent()` instead dawwwwwg! - obj_str += (field_ws + f'{k}: {typ_name} = {val_str},\n') +# class Params( +# Struct, +# Generic[PayloadT], +# ): +# spec: PayloadT|ParamSpec +# inputs: InputsT|dict[str, Any] - return ( - f'{qtn}(\n' - f'{obj_str}' - f'{ws})' + # 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] ) - # TODO: use a pprint.PrettyPrinter instance around ONLY rendering - # inside a known tty? - # def __repr__(self) -> str: - # ... + # with `msgspec.structs.defstruct` + # XXX ALSO DOESN'T WORK + defstruct_msgtype = defstruct( + name=msgtype.__name__, + fields=[ + ('cid', str), - # __str__ = __repr__ = pformat - __repr__ = pformat + # XXX doesn't seem to work.. + # ('pld', PayloadT), - 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) + ('pld', payload_type_union), + ], + bases=defstruct_bases, ) + defs_msg_types.append(defstruct_msgtype) - def typecast( - self, + # assert index_paramed_msg_type == manual_paramed_msg_subtype - # TODO: allow only casting a named subset? - # fields: set[str] | None = None, + # paramed_msg_type = manual_paramed_msg_subtype - ) -> None: - ''' - Cast all fields using their declared type annotations - (kinda like what `pydantic` does by default). + # ipc_payload_msgs_type_union |= index_paramed_msg_type - NOTE: this of course won't work on frozen types, use - ``.copy()`` above in such cases. + 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] - ''' - # 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)), - ) + 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, + } - def __sub__( - self, - other: Struct, + # 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() - ) -> 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 + 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 585b0b00..d1451b4c 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -577,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/_broadcast.py b/tractor/trionics/_broadcast.py index a5d31871..977b6828 100644 --- a/tractor/trionics/_broadcast.py +++ b/tractor/trionics/_broadcast.py @@ -156,11 +156,12 @@ class BroadcastState(Struct): class BroadcastReceiver(ReceiveChannel): ''' - A memory receive channel broadcaster which is non-lossy for the - fastest consumer. + A memory receive channel broadcaster which is non-lossy for + the fastest consumer. - Additional consumer tasks can receive all produced values by registering - with ``.subscribe()`` and receiving from the new instance it delivers. + Additional consumer tasks can receive all produced values by + registering with ``.subscribe()`` and receiving from the new + instance it delivers. ''' def __init__( diff --git a/tractor/trionics/_mngrs.py b/tractor/trionics/_mngrs.py index f57be0a7..fd224d65 100644 --- a/tractor/trionics/_mngrs.py +++ b/tractor/trionics/_mngrs.py @@ -18,8 +18,12 @@ Async context manager primitives with hard ``trio``-aware semantics ''' -from contextlib import asynccontextmanager as acm +from __future__ import annotations +from contextlib import ( + asynccontextmanager as acm, +) import inspect +from types import ModuleType from typing import ( Any, AsyncContextManager, @@ -30,13 +34,16 @@ from typing import ( Optional, Sequence, TypeVar, + TYPE_CHECKING, ) import trio - from tractor._state import current_actor from tractor.log import get_logger +if TYPE_CHECKING: + from tractor import ActorNursery + log = get_logger(__name__) @@ -46,8 +53,10 @@ T = TypeVar("T") @acm async def maybe_open_nursery( - nursery: trio.Nursery | None = None, + nursery: trio.Nursery|ActorNursery|None = None, shield: bool = False, + lib: ModuleType = trio, + ) -> AsyncGenerator[trio.Nursery, Any]: ''' Create a new nursery if None provided. @@ -58,13 +67,12 @@ async def maybe_open_nursery( if nursery is not None: yield nursery else: - async with trio.open_nursery() as nursery: + async with lib.open_nursery() as nursery: nursery.cancel_scope.shield = shield yield nursery async def _enter_and_wait( - mngr: AsyncContextManager[T], unwrapped: dict[int, T], all_entered: trio.Event, @@ -91,7 +99,6 @@ async def _enter_and_wait( @acm async def gather_contexts( - mngrs: Sequence[AsyncContextManager[T]], ) -> AsyncGenerator[ @@ -102,15 +109,17 @@ async def gather_contexts( None, ]: ''' - Concurrently enter a sequence of async context managers, each in - a separate ``trio`` task and deliver the unwrapped values in the - same order once all managers have entered. On exit all contexts are - subsequently and concurrently exited. + Concurrently enter a sequence of async context managers (acms), + each from a separate `trio` task and deliver the unwrapped + `yield`-ed values in the same order once all managers have entered. - This function is somewhat similar to common usage of - ``contextlib.AsyncExitStack.enter_async_context()`` (in a loop) in - combo with ``asyncio.gather()`` except the managers are concurrently - entered and exited, and cancellation just works. + On exit, all acms are subsequently and concurrently exited. + + This function is somewhat similar to a batch of non-blocking + calls to `contextlib.AsyncExitStack.enter_async_context()` + (inside a loop) *in combo with* a `asyncio.gather()` to get the + `.__aenter__()`-ed values, except the managers are both + concurrently entered and exited and *cancellation just works*(R). ''' seed: int = id(mngrs) @@ -210,9 +219,10 @@ async def maybe_open_context( ) -> AsyncIterator[tuple[bool, T]]: ''' - Maybe open a context manager if there is not already a _Cached - version for the provided ``key`` for *this* actor. Return the - _Cached instance on a _Cache hit. + Maybe open an async-context-manager (acm) if there is not already + a `_Cached` version for the provided (input) `key` for *this* actor. + + Return the `_Cached` instance on a _Cache hit. ''' fid = id(acm_func) @@ -271,8 +281,16 @@ async def maybe_open_context( yield False, yielded else: - log.info(f'Reusing _Cached resource for {ctx_key}') _Cache.users += 1 + log.runtime( + f'Re-using cached resource for user {_Cache.users}\n\n' + f'{ctx_key!r} -> {type(yielded)}\n' + + # TODO: make this work with values but without + # `msgspec.Struct` causing frickin crashes on field-type + # lookups.. + # f'{ctx_key!r} -> {yielded!r}\n' + ) lock.release() yield True, yielded