From 42cf9e11a43584f2e367be0ac092cef4464f82b8 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 11 Apr 2025 14:30:21 -0400 Subject: [PATCH] Mv `Actor._stream_handler()` to `.ipc._server` func Call it `handle_stream_from_peer()` and bind in the `actor: Actor` via a `handler=partial()` to `trio.serve_listeners()`. With this (minus the `Actor._peers/._peer_connected/._no_more_peers` attrs ofc) we get nearly full separation of IPC-connection-processing (concerns) from `Actor` state. Thus it's a first look at modularizing the low-level runtime into isolated subsystems which will hopefully improve the entire code base's grok-ability and ease any new feature design discussions especially pertaining to introducing and/or composing-together any new transport protocols. --- tractor/_portal.py | 4 +- tractor/_runtime.py | 439 +------------------------------------- tractor/ipc/_server.py | 468 ++++++++++++++++++++++++++++++++++++++++- 3 files changed, 471 insertions(+), 440 deletions(-) diff --git a/tractor/_portal.py b/tractor/_portal.py index a4ae431a..d5dd8369 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -578,10 +578,10 @@ async def open_portal( msg_loop_cs: trio.CancelScope|None = None if start_msg_loop: - from ._runtime import process_messages + from . import _rpc msg_loop_cs = await tn.start( partial( - process_messages, + _rpc.process_messages, actor, channel, # if the local task is cancelled we want to keep diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 4daa4742..b21ed0ee 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -96,18 +96,13 @@ from ._exceptions import ( ModuleNotExposed, MsgTypeError, unpack_error, - TransportClosed, ) from .devx import _debug from ._discovery import get_registry from ._portal import Portal from . import _state from . import _mp_fixup_main -from ._rpc import ( - process_messages, - try_ship_error_to_remote, -) - +from . import _rpc if TYPE_CHECKING: from ._supervise import ActorNursery @@ -493,434 +488,6 @@ class Actor: raise mne - # TODO: maybe change to mod-func and rename for implied - # multi-transport semantics? - async def _stream_handler( - self, - stream: trio.SocketStream, - - ) -> None: - ''' - Entry point for new inbound IPC connections on a specific - transport server. - - ''' - self._no_more_peers = trio.Event() # unset by making new - # with _debug.maybe_open_crash_handler( - # pdb=True, - # ) as boxerr: - chan = Channel.from_stream(stream) - con_status: str = ( - 'New inbound IPC connection <=\n' - f'|_{chan}\n' - ) - - # send/receive initial handshake response - try: - peer_aid: msgtypes.Aid = await chan._do_handshake( - aid=self.aid, - ) - except ( - TransportClosed, - # ^XXX NOTE, the above wraps `trio` exc types raised - # during various `SocketStream.send/receive_xx()` calls - # under different fault conditions such as, - # - # trio.BrokenResourceError, - # trio.ClosedResourceError, - # - # Inside our `.ipc._transport` layer we absorb and - # re-raise our own `TransportClosed` exc such that this - # higher level runtime code can only worry one - # "kinda-error" that we expect to tolerate during - # discovery-sys related pings, queires, DoS etc. - ): - # 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 - # a bound listener on the "arbiter" addr. the reset will be - # because the handshake was never meant took place. - log.runtime( - con_status - + - ' -> But failed to handshake? Ignoring..\n' - ) - return - - uid: tuple[str, str] = ( - peer_aid.name, - peer_aid.uuid, - ) - # TODO, can we make this downstream peer tracking use the - # `peer_aid` instead? - 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 - # by a call to `.wait_for_peer()`. - # - if a peer is connecting no such event will exit. - event: trio.Event|None = self._peer_connected.pop( - uid, - None, - ) - if event: - con_status += ( - ' -> Waking subactor spawn waiters: ' - f'{event.statistics().tasks_waiting}\n' - f' -> Registered IPC chan for child actor {uid}@{chan.raddr}\n' - # f' {event}\n' - # f' |{event.statistics()}\n' - ) - # wake tasks waiting on this IPC-transport "connect-back" - event.set() - - else: - con_status += ( - f' -> Registered IPC chan for peer actor {uid}@{chan.raddr}\n' - ) # type: ignore - - chans: list[Channel] = self._peers[uid] - # if chans: - # # TODO: re-use channels for new connections instead - # # of always new ones? - # # => will require changing all the discovery funcs.. - - # append new channel - # TODO: can we just use list-ref directly? - chans.append(chan) - - 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, - last_msg, - ) = await process_messages( - self, - chan, - ) - except trio.Cancelled: - log.cancel( - 'IPC transport msg loop was cancelled\n' - f'c)>\n' - f' |_{chan}\n' - ) - raise - - finally: - local_nursery: ( - ActorNursery|None - ) = self._actoruid2nursery.get(uid) - - # This is set in ``Portal.cancel_actor()``. So if - # the peer was cancelled we try to wait for them - # to tear down their side of the connection before - # moving on with closing our own side. - if ( - local_nursery - 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 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 - # problem on closure). - assert chan.transport - async for msg in chan.transport.drain(): - - # try to deliver any lingering msgs - # before we destroy the channel. - # This accomplishes deterministic - # ``Portal.cancel_actor()`` cancellation by - # making sure any RPC response to that call is - # delivered the local calling task. - # TODO: factor this into a helper? - log.warning( - 'Draining msg from disconnected peer\n' - f'{chan_info}' - f'{pformat(msg)}\n' - ) - # cid: str|None = msg.get('cid') - cid: str|None = msg.cid - if cid: - # deliver response to local caller/waiter - await self._deliver_ctx_payload( - chan, - cid, - msg, - ) - if drain_cs.cancelled_caught: - log.warning( - 'Timed out waiting on IPC transport channel to drain?\n' - f'{chan_info}' - ) - - # XXX NOTE XXX when no explicit call to - # `open_root_actor()` was made by the application - # (normally we implicitly make that call inside - # the first `.open_nursery()` in root-actor - # user/app code), we can assume that either we - # are NOT the root actor or are root but the - # runtime was started manually. and thus DO have - # to wait for the nursery-enterer to exit before - # shutting down the local runtime to avoid - # clobbering any ongoing subactor - # teardown/debugging/graceful-cancel. - # - # see matching note inside `._supervise.open_nursery()` - # - # TODO: should we have a separate cs + timeout - # block here? - if ( - # XXX SO either, - # - not root OR, - # - is root but `open_root_actor()` was - # entered manually (in which case we do - # the equiv wait there using the - # `devx._debug` sub-sys APIs). - not local_nursery._implicit_runtime_started - ): - log.runtime( - 'Waiting on local actor nursery to exit..\n' - f'|_{local_nursery}\n' - ) - with trio.move_on_after(0.5) as an_exit_cs: - an_exit_cs.shield = True - await local_nursery.exited.wait() - - # TODO: currently this is always triggering for every - # sub-daemon spawned from the `piker.services._mngr`? - # -[ ] how do we ensure that the IPC is supposed to - # be long lived and isn't just a register? - # |_ in the register case how can we signal that the - # ephemeral msg loop was intentional? - if ( - # not local_nursery._implicit_runtime_started - # and - an_exit_cs.cancelled_caught - ): - 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 - # registered within a local nursery, we report - # that the IPC layer may have failed - # unexpectedly since it may be the cause of - # other downstream errors. - entry: tuple|None = local_nursery._children.get(uid) - if entry: - proc: trio.Process - _, proc, _ = entry - - if ( - (poll := getattr(proc, 'poll', None)) - and - poll() is None # proc still alive - ): - # TODO: change log level based on - # detecting whether chan was created for - # ephemeral `.register_actor()` request! - # -[ ] also, that should be avoidable by - # re-using any existing chan from the - # `._discovery.get_registry()` call as - # well.. - log.runtime( - f'Peer IPC broke but subproc is alive?\n\n' - - f'<=x {chan.uid}@{chan.raddr}\n' - f' |_{proc}\n' - ) - - # ``Channel`` teardown and closure sequence - # drop ref to channel so it can be gc-ed and disconnected - 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: - 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' - ) - for i, chan in enumerate(chans): - peers_str += ( - f' |_[{i}] {pformat(chan)}\n' - ) - - con_teardown_status += ( - f'-> Remaining IPC {len(self._peers)} peers: {peers_str}\n' - ) - - # No more channels to other actors (at all) registered - # as connected. - if not self._peers: - con_teardown_status += ( - 'Signalling no more peer channel connections' - ) - self._no_more_peers.set() - - # NOTE: block this actor from acquiring the - # debugger-TTY-lock since we have no way to know if we - # cancelled it and further there is no way to ensure the - # lock will be released if acquired due to having no - # more active IPC channels. - if _state.is_root_process(): - pdb_lock = _debug.Lock - pdb_lock._blocked.add(uid) - - # TODO: NEEEDS TO BE TESTED! - # actually, no idea if this ever even enters.. XD - # - # XXX => YES IT DOES, when i was testing ctl-c - # from broken debug TTY locking due to - # msg-spec races on application using RunVar... - if ( - (ctx_in_debug := pdb_lock.ctx_in_debug) - and - (pdb_user_uid := ctx_in_debug.chan.uid) - and - local_nursery - ): - entry: tuple|None = local_nursery._children.get( - tuple(pdb_user_uid) - ) - if entry: - proc: trio.Process - _, proc, _ = entry - - if ( - (poll := getattr(proc, 'poll', None)) - and poll() is None - ): - log.cancel( - 'Root actor reports no-more-peers, BUT\n' - 'a DISCONNECTED child still has the debug ' - 'lock!\n\n' - # f'root uid: {self.uid}\n' - f'last disconnected child uid: {uid}\n' - f'locking child uid: {pdb_user_uid}\n' - ) - await _debug.maybe_wait_for_debugger( - child_in_debug=True - ) - - # TODO: just bc a child's transport dropped - # doesn't mean it's not still using the pdb - # REPL! so, - # -[ ] ideally we can check out child proc - # tree to ensure that its alive (and - # actually using the REPL) before we cancel - # it's lock acquire by doing the below! - # -[ ] create a way to read the tree of each actor's - # grandchildren such that when an - # intermediary parent is cancelled but their - # child has locked the tty, the grandparent - # will not allow the parent to cancel or - # zombie reap the child! see open issue: - # - https://github.com/goodboy/tractor/issues/320 - # ------ - ------ - # if a now stale local task has the TTY lock still - # we cancel it to allow servicing other requests for - # the lock. - if ( - (db_cs := pdb_lock.get_locking_task_cs()) - and not db_cs.cancel_called - and uid == pdb_user_uid - ): - log.critical( - f'STALE DEBUG LOCK DETECTED FOR {uid}' - ) - # TODO: figure out why this breaks tests.. - db_cs.cancel() - - 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 _deliver_ctx_payload( @@ -1916,7 +1483,7 @@ async def async_main( if actor._parent_chan: await root_nursery.start( partial( - process_messages, + _rpc.process_messages, actor, actor._parent_chan, shield=True, @@ -1959,7 +1526,7 @@ async def async_main( log.exception(err_report) if actor._parent_chan: - await try_ship_error_to_remote( + await _rpc.try_ship_error_to_remote( actor._parent_chan, internal_err, ) diff --git a/tractor/ipc/_server.py b/tractor/ipc/_server.py index f23cf697..7d2ab217 100644 --- a/tractor/ipc/_server.py +++ b/tractor/ipc/_server.py @@ -24,6 +24,7 @@ from contextlib import ( ) from functools import partial import inspect +from pprint import pformat from types import ( ModuleType, ) @@ -40,24 +41,484 @@ from trio import ( SocketListener, ) -from ..msg import Struct +from ..devx import _debug +from .._exceptions import ( + TransportClosed, +) +from .. import _rpc +from ..msg import ( + MsgType, + Struct, + types as msgtypes, +) from ..trionics import maybe_open_nursery from .. import ( _state, log, ) from .._addr import Address +from ._chan import Channel from ._transport import MsgTransport from ._uds import UDSAddress from ._tcp import TCPAddress if TYPE_CHECKING: from .._runtime import Actor + from .._supervise import ActorNursery log = log.get_logger(__name__) +# TODO multi-tpt support with per-proto peer tracking? +# +# -[x] maybe change to mod-func and rename for implied +# multi-transport semantics? +# +# -[ ] register each stream/tpt/chan with the owning `IPCEndpoint` +# so that we can query per tpt all peer contact infos? +# |_[ ] possibly provide a global viewing via a +# `collections.ChainMap`? +# +async def handle_stream_from_peer( + stream: trio.SocketStream, + actor: Actor, + +) -> None: + ''' + Top-level `trio.abc.Stream` (i.e. normally `trio.SocketStream`) + handler-callback as spawn-invoked by `trio.serve_listeners()`. + + Note that each call to this handler is as a spawned task inside + any `IPCServer.listen_on()` passed `stream_handler_tn: Nursery` + such that it is invoked as, + + IPCEndpoint.stream_handler_tn.start_soon( + handle_stream, + stream, + ) + + ''' + actor._no_more_peers = trio.Event() # unset by making new + + # TODO, debug_mode tooling for when hackin this lower layer? + # with _debug.maybe_open_crash_handler( + # pdb=True, + # ) as boxerr: + + chan = Channel.from_stream(stream) + con_status: str = ( + 'New inbound IPC connection <=\n' + f'|_{chan}\n' + ) + + # initial handshake with peer phase + try: + peer_aid: msgtypes.Aid = await chan._do_handshake( + aid=actor.aid, + ) + except ( + TransportClosed, + # ^XXX NOTE, the above wraps `trio` exc types raised + # during various `SocketStream.send/receive_xx()` calls + # under different fault conditions such as, + # + # trio.BrokenResourceError, + # trio.ClosedResourceError, + # + # Inside our `.ipc._transport` layer we absorb and + # re-raise our own `TransportClosed` exc such that this + # higher level runtime code can only worry one + # "kinda-error" that we expect to tolerate during + # discovery-sys related pings, queires, DoS etc. + ): + # 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 + # a bound listener on the "arbiter" addr. the reset will be + # because the handshake was never meant took place. + log.runtime( + con_status + + + ' -> But failed to handshake? Ignoring..\n' + ) + return + + uid: tuple[str, str] = ( + peer_aid.name, + peer_aid.uuid, + ) + # TODO, can we make this downstream peer tracking use the + # `peer_aid` instead? + familiar: str = 'new-peer' + if _pre_chan := actor._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 + # by a call to `.wait_for_peer()`. + # - if a peer is connecting no such event will exit. + event: trio.Event|None = actor._peer_connected.pop( + uid, + None, + ) + if event: + con_status += ( + ' -> Waking subactor spawn waiters: ' + f'{event.statistics().tasks_waiting}\n' + f' -> Registered IPC chan for child actor {uid}@{chan.raddr}\n' + # f' {event}\n' + # f' |{event.statistics()}\n' + ) + # wake tasks waiting on this IPC-transport "connect-back" + event.set() + + else: + con_status += ( + f' -> Registered IPC chan for peer actor {uid}@{chan.raddr}\n' + ) # type: ignore + + chans: list[Channel] = actor._peers[uid] + # if chans: + # # TODO: re-use channels for new connections instead + # # of always new ones? + # # => will require changing all the discovery funcs.. + + # append new channel + # TODO: can we just use list-ref directly? + chans.append(chan) + + 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, + last_msg, + ) = await _rpc.process_messages( + actor, + chan, + ) + except trio.Cancelled: + log.cancel( + 'IPC transport msg loop was cancelled\n' + f'c)>\n' + f' |_{chan}\n' + ) + raise + + finally: + local_nursery: ( + ActorNursery|None + ) = actor._actoruid2nursery.get(uid) + + # This is set in ``Portal.cancel_actor()``. So if + # the peer was cancelled we try to wait for them + # to tear down their side of the connection before + # moving on with closing our own side. + if ( + local_nursery + and ( + actor._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 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 + # problem on closure). + assert chan.transport + async for msg in chan.transport.drain(): + + # try to deliver any lingering msgs + # before we destroy the channel. + # This accomplishes deterministic + # ``Portal.cancel_actor()`` cancellation by + # making sure any RPC response to that call is + # delivered the local calling task. + # TODO: factor this into a helper? + log.warning( + 'Draining msg from disconnected peer\n' + f'{chan_info}' + f'{pformat(msg)}\n' + ) + # cid: str|None = msg.get('cid') + cid: str|None = msg.cid + if cid: + # deliver response to local caller/waiter + await actor._deliver_ctx_payload( + chan, + cid, + msg, + ) + if drain_cs.cancelled_caught: + log.warning( + 'Timed out waiting on IPC transport channel to drain?\n' + f'{chan_info}' + ) + + # XXX NOTE XXX when no explicit call to + # `open_root_actor()` was made by the application + # (normally we implicitly make that call inside + # the first `.open_nursery()` in root-actor + # user/app code), we can assume that either we + # are NOT the root actor or are root but the + # runtime was started manually. and thus DO have + # to wait for the nursery-enterer to exit before + # shutting down the local runtime to avoid + # clobbering any ongoing subactor + # teardown/debugging/graceful-cancel. + # + # see matching note inside `._supervise.open_nursery()` + # + # TODO: should we have a separate cs + timeout + # block here? + if ( + # XXX SO either, + # - not root OR, + # - is root but `open_root_actor()` was + # entered manually (in which case we do + # the equiv wait there using the + # `devx._debug` sub-sys APIs). + not local_nursery._implicit_runtime_started + ): + log.runtime( + 'Waiting on local actor nursery to exit..\n' + f'|_{local_nursery}\n' + ) + with trio.move_on_after(0.5) as an_exit_cs: + an_exit_cs.shield = True + await local_nursery.exited.wait() + + # TODO: currently this is always triggering for every + # sub-daemon spawned from the `piker.services._mngr`? + # -[ ] how do we ensure that the IPC is supposed to + # be long lived and isn't just a register? + # |_ in the register case how can we signal that the + # ephemeral msg loop was intentional? + if ( + # not local_nursery._implicit_runtime_started + # and + an_exit_cs.cancelled_caught + ): + 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 + # registered within a local nursery, we report + # that the IPC layer may have failed + # unexpectedly since it may be the cause of + # other downstream errors. + entry: tuple|None = local_nursery._children.get(uid) + if entry: + proc: trio.Process + _, proc, _ = entry + + if ( + (poll := getattr(proc, 'poll', None)) + and + poll() is None # proc still alive + ): + # TODO: change log level based on + # detecting whether chan was created for + # ephemeral `.register_actor()` request! + # -[ ] also, that should be avoidable by + # re-using any existing chan from the + # `._discovery.get_registry()` call as + # well.. + log.runtime( + f'Peer IPC broke but subproc is alive?\n\n' + + f'<=x {chan.uid}@{chan.raddr}\n' + f' |_{proc}\n' + ) + + # ``Channel`` teardown and closure sequence + # drop ref to channel so it can be gc-ed and disconnected + 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: + con_teardown_status += ( + f'-> No more channels with {chan.uid}' + ) + actor._peers.pop(uid, None) + + peers_str: str = '' + for uid, chans in actor._peers.items(): + peers_str += ( + f'uid: {uid}\n' + ) + for i, chan in enumerate(chans): + peers_str += ( + f' |_[{i}] {pformat(chan)}\n' + ) + + con_teardown_status += ( + f'-> Remaining IPC {len(actor._peers)} peers: {peers_str}\n' + ) + + # No more channels to other actors (at all) registered + # as connected. + if not actor._peers: + con_teardown_status += ( + 'Signalling no more peer channel connections' + ) + actor._no_more_peers.set() + + # NOTE: block this actor from acquiring the + # debugger-TTY-lock since we have no way to know if we + # cancelled it and further there is no way to ensure the + # lock will be released if acquired due to having no + # more active IPC channels. + if _state.is_root_process(): + pdb_lock = _debug.Lock + pdb_lock._blocked.add(uid) + + # TODO: NEEEDS TO BE TESTED! + # actually, no idea if this ever even enters.. XD + # + # XXX => YES IT DOES, when i was testing ctl-c + # from broken debug TTY locking due to + # msg-spec races on application using RunVar... + if ( + (ctx_in_debug := pdb_lock.ctx_in_debug) + and + (pdb_user_uid := ctx_in_debug.chan.uid) + and + local_nursery + ): + entry: tuple|None = local_nursery._children.get( + tuple(pdb_user_uid) + ) + if entry: + proc: trio.Process + _, proc, _ = entry + + if ( + (poll := getattr(proc, 'poll', None)) + and poll() is None + ): + log.cancel( + 'Root actor reports no-more-peers, BUT\n' + 'a DISCONNECTED child still has the debug ' + 'lock!\n\n' + # f'root uid: {actor.uid}\n' + f'last disconnected child uid: {uid}\n' + f'locking child uid: {pdb_user_uid}\n' + ) + await _debug.maybe_wait_for_debugger( + child_in_debug=True + ) + + # TODO: just bc a child's transport dropped + # doesn't mean it's not still using the pdb + # REPL! so, + # -[ ] ideally we can check out child proc + # tree to ensure that its alive (and + # actually using the REPL) before we cancel + # it's lock acquire by doing the below! + # -[ ] create a way to read the tree of each actor's + # grandchildren such that when an + # intermediary parent is cancelled but their + # child has locked the tty, the grandparent + # will not allow the parent to cancel or + # zombie reap the child! see open issue: + # - https://github.com/goodboy/tractor/issues/320 + # ------ - ------ + # if a now stale local task has the TTY lock still + # we cancel it to allow servicing other requests for + # the lock. + if ( + (db_cs := pdb_lock.get_locking_task_cs()) + and not db_cs.cancel_called + and uid == pdb_user_uid + ): + log.critical( + f'STALE DEBUG LOCK DETECTED FOR {uid}' + ) + # TODO: figure out why this breaks tests.. + db_cs.cancel() + + log.runtime(con_teardown_status) + # finally block closure + + class IPCEndpoint(Struct): ''' An instance of an IPC "bound" address where the lifetime of the @@ -379,7 +840,10 @@ async def _serve_ipc_eps( _listeners: list[SocketListener] = await listen_tn.start( partial( trio.serve_listeners, - handler=actor._stream_handler, + handler=partial( + handle_stream_from_peer, + actor=actor, + ), listeners=listeners, # NOTE: configured such that new