diff --git a/tractor/devx/__init__.py b/tractor/devx/__init__.py index c4676e3..7ea2b25 100644 --- a/tractor/devx/__init__.py +++ b/tractor/devx/__init__.py @@ -27,7 +27,6 @@ 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, post_mortem as post_mortem, diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 26155b2..51e7437 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -25,6 +25,7 @@ from contextlib import ( asynccontextmanager as acm, contextmanager as cm, nullcontext, + _GeneratorContextManager, ) from functools import ( partial, @@ -33,6 +34,7 @@ from functools import ( import os import signal import sys +import textwrap import threading import traceback from typing import ( @@ -40,6 +42,7 @@ from typing import ( Callable, AsyncIterator, AsyncGenerator, + TypeAlias, TYPE_CHECKING, ) from types import ( @@ -47,17 +50,23 @@ from types import ( ModuleType, ) +from msgspec import Struct import pdbp import sniffio import tractor import trio -from trio.lowlevel import current_task +from trio.lowlevel import ( + current_task, + Task, +) from trio import ( TaskStatus, - # Task, ) from tractor.log import get_logger +from tractor.msg import ( + _codec, +) from tractor._state import ( current_actor, is_root_process, @@ -76,6 +85,36 @@ if TYPE_CHECKING: log = get_logger(__name__) +# 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? +# +pdbp.hideframe(trio._core._run.NurseryManager.__aexit__) +pdbp.hideframe(trio._core._run.CancelScope.__exit__) +pdbp.hideframe(_GeneratorContextManager.__exit__) __all__ = [ 'breakpoint', @@ -83,6 +122,28 @@ __all__ = [ ] +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 + + +__msg_spec__: TypeAlias = LockStatus|LockRelease + + class Lock: ''' Actor global debug lock state. @@ -90,41 +151,111 @@ class Lock: Mostly to avoid a lot of ``global`` declarations for now XD. ''' - repl: MultiActorPdb | None = None + # XXX local ref to the `Pbp` instance, ONLY set in the + # actor-process that currently has activated a REPL + # i.e. it will be `None` (unset) in any other actor-process + # that does not have this lock acquired in the root proc. + repl: PdbREPL|None = None + # placeholder for function to set a ``trio.Event`` on debugger exit # pdb_release_hook: Callable | None = None - _trio_handler: Callable[ - [int, FrameType | None], Any - ] | int | None = None + _trio_handler: ( + Callable[[int, FrameType|None], Any] + |int + | None + ) = None - # actor-wide variable pointing to current task name using debugger - local_task_in_debug: str | None = None + remote_task_in_debug: str|None = None - # NOTE: set by the current task waiting on the root tty lock from - # the CALLER side of the `lock_tty_for_child()` context entry-call - # and must be cancelled if this actor is cancelled via IPC - # request-message otherwise deadlocks with the parent actor may - # ensure + @staticmethod + def get_locking_task_cs() -> trio.CancelScope|None: + if is_root_process(): + return Lock._locking_task_cs + + raise RuntimeError( + '`Lock.locking_task_cs` is invalid in subactors!' + ) + + @staticmethod + def set_locking_task_cs( + cs: trio.CancelScope, + ) -> None: + if not is_root_process(): + raise RuntimeError( + '`Lock.locking_task_cs` is invalid in subactors!' + ) + + Lock._locking_task_cs = cs + + # SUBACTOR ONLY + # ------ - ------- + local_task_in_debug: Task|None = None _debugger_request_cs: trio.CancelScope|None = None + local_pdb_complete: trio.Event|None = None - # NOTE: set only in the root actor for the **local** root spawned task - # which has acquired the lock (i.e. this is on the callee side of - # the `lock_tty_for_child()` context entry). - _root_local_task_cs_in_debug: trio.CancelScope|None = 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 + global_actor_in_debug: tuple[str, str]|None = None + no_remote_has_tty: trio.Event|None = None + _locking_task_cs: trio.CancelScope|None = None - # actor tree-wide actor uid that supposedly has the tty lock - global_actor_in_debug: tuple[str, str] = 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 _debug_lock: trio.StrictFIFOLock = trio.StrictFIFOLock() + _blocked: set[tuple[str, str]] = set() # `Actor.uid` block list + # TODO: should go on `PbpREPL`? _orig_sigint_handler: Callable | None = None - _blocked: set[tuple[str, str]] = set() + @classmethod + def repr(cls) -> str: + + # both root and subs + fields: str = ( + f'repl: {cls.repl}\n' + f'local_repl_task: {cls.local_task_in_debug}\n' + ) + + if is_root_process(): + lock_stats: trio.LockStatistics = cls._debug_lock.statistics() + fields += ( + f'global_actor_in_debug: {cls.global_actor_in_debug}\n' + f'no_remote_has_tty: {cls.no_remote_has_tty}\n' + f'remote_task_in_debug: {cls.remote_task_in_debug}\n' + f'_locking_task_cs: {cls.get_locking_task_cs()}\n' + f'_blocked: {cls._blocked}\n\n' + + f'_debug_lock: {cls._debug_lock}\n' + f'lock_stats: {lock_stats}\n' + ) + else: + fields += ( + f'local_task_in_debug: {cls.local_task_in_debug}\n' + f'local_pdb_complete: {cls.local_pdb_complete}\n' + f'_debugger_request_cs: {cls._debugger_request_cs}\n' + ) + + body: str = textwrap.indent( + fields, + prefix=' |_', + ) + return ( + f'<{cls.__name__}(\n' + f'{body}' + ')>' + ) + + # TODO: move to PdbREPL! @classmethod def shield_sigint(cls): ''' @@ -218,19 +349,35 @@ class Lock: else: cls._debug_lock.release() - except RuntimeError: + except RuntimeError as rte: # 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 + # and + # cls.remote_task_in_debug is None + # ): + # raise RuntimeError( + # 'Stale `Lock` detected, no remote task active!?\n' + # f'|_{owner}\n' + # # f'{Lock}' + # ) from rte + if owner: - raise + raise rte + + # OW suppress, can't member why tho .. XD + # something somethin corrupts a cancel-scope + # somewhere.. try: # sometimes the ``trio`` might already be terminated in # which case this call will raise. if cls.local_pdb_complete is not None: cls.local_pdb_complete.set() + finally: # restore original sigint handler cls.unshield_sigint() @@ -241,10 +388,33 @@ class Lock: cls.local_task_in_debug = None +# TODO: actually use this instead throughout for subs! +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. + + ''' + repl: PdbREPL|None = None + lock_status: LockStatus|None = None + + repl_task: Task|None = None + # local_task_in_debug: Task|None = None + + req_cs: trio.CancelScope|None = None + # _debugger_request_cs: trio.CancelScope|None = None + + repl_release: trio.Event|None = None + # local_pdb_complete: trio.Event|None = None + 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,21 +425,41 @@ 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 + # 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() @@ -282,6 +472,17 @@ class MultiActorPdb(pdbp.Pdb): finally: Lock.release() + # TODO: special handling where we just want the next LOC and + # not to resume to the next pause/crash point? + # def set_next( + # self, + # frame: FrameType + # ) -> None: + # try: + # super().set_next(frame) + # finally: + # Lock.release() + # XXX NOTE: we only override this because apparently the stdlib pdb # bois likes to touch the SIGINT handler as much as i like to touch # my d$%&. @@ -314,7 +515,8 @@ class MultiActorPdb(pdbp.Pdb): @acm async def _acquire_debug_lock_from_root_task( - uid: tuple[str, str] + subactor_uid: tuple[str, str], + remote_task_uid: str, ) -> AsyncIterator[trio.StrictFIFOLock]: ''' @@ -326,16 +528,31 @@ async def _acquire_debug_lock_from_root_task( to the ``pdb`` repl. ''' - task_name: str = current_task().name + # task_name: str = current_task().name we_acquired: bool = False log.runtime( - f"Attempting to acquire TTY lock, remote task: {task_name}:{uid}" + f'Attempting to acquire TTY lock for,\n' + f'subactor_uid: {subactor_uid}\n' + f'remote task: {remote_task_uid}\n' ) try: - log.runtime( - f"entering lock checkpoint, remote task: {task_name}:{uid}" + pre_msg: str = ( + f'Entering lock checkpoint for sub-actor\n' + f'subactor_uid: {subactor_uid}\n' + f'remote task: {remote_task_uid}\n' ) + stats = Lock._debug_lock.statistics() + if owner := stats.owner: + # and Lock.no_remote_has_tty is not None + pre_msg += ( + f'\n' + f'`Lock` already held by local task\n' + f'{owner}\n\n' + f'On behalf of remote task: {Lock.remote_task_in_debug!r}\n' + ) + log.runtime(pre_msg) + # 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 @@ -349,9 +566,14 @@ async def _acquire_debug_lock_from_root_task( # can try to avoid clobbering any connection from a child # that's currently relying on it. Lock.no_remote_has_tty = trio.Event() + Lock.remote_task_in_debug = remote_task_uid - Lock.global_actor_in_debug = uid - log.runtime(f"TTY lock acquired, remote task: {task_name}:{uid}") + Lock.global_actor_in_debug = subactor_uid + log.runtime( + f'TTY lock acquired for,\n' + f'subactor_uid: {subactor_uid}\n' + f'remote task: {remote_task_uid}\n' + ) # NOTE: critical section: this yield is unshielded! @@ -368,7 +590,8 @@ async def _acquire_debug_lock_from_root_task( finally: if ( we_acquired - and Lock._debug_lock.locked() + and + Lock._debug_lock.locked() ): Lock._debug_lock.release() @@ -380,16 +603,37 @@ async def _acquire_debug_lock_from_root_task( stats = Lock._debug_lock.statistics() if ( not stats.owner + # and Lock.no_remote_has_tty is not None ): - log.runtime(f"No more tasks waiting on tty lock! says {uid}") + # log.runtime( + log.info( + f'No more child ctx tasks hold the TTY lock!\n' + f'last subactor: {subactor_uid}\n' + f'remote task: {remote_task_uid}\n' + ) if Lock.no_remote_has_tty is not None: + # set and release Lock.no_remote_has_tty.set() Lock.no_remote_has_tty = None + Lock.remote_task_in_debug = None + else: + log.warning( + 'Not signalling `Lock.no_remote_has_tty` since it has value:\n' + f'{Lock.no_remote_has_tty}\n' + ) + else: + log.info( + f'A child ctx tasks still holds the TTY lock ??\n' + f'last subactor: {subactor_uid}\n' + f'remote task: {remote_task_uid}\n' + f'current local owner task: {stats.owner}\n' + ) Lock.global_actor_in_debug = None - log.runtime( - f"TTY lock released, remote task: {task_name}:{uid}" + 'TTY lock released by child\n' + f'last subactor: {subactor_uid}\n' + f'remote task: {remote_task_uid}\n' ) @@ -397,9 +641,14 @@ async def _acquire_debug_lock_from_root_task( async def lock_tty_for_child( ctx: tractor.Context, - subactor_uid: tuple[str, str] -) -> str: + # TODO: when we finally get a `Start.params: ParamSpec` + # working it'd sure be nice to have `msgspec` auto-decode this + # to an actual tuple XD + subactor_uid: tuple[str, str], + 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 @@ -411,53 +660,141 @@ async def lock_tty_for_child( highly reliable at releasing the mutex complete! ''' - task_name: str = current_task().name + + req_task_uid: tuple = tuple(subactor_task_uid) + if req_task_uid in Lock._blocked: + raise RuntimeError( + f'The same remote task already has an active request for TTY lock ??\n\n' + f'task uid: {req_task_uid}\n' + f'subactor uid: {subactor_uid}\n' + ) + + Lock._blocked.add(req_task_uid) + + root_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}" + 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 await ctx.cancel(f'Debug lock blocked for {subactor_uid}') - return 'pdb_lock_blocked' + return LockStatus( + subactor_uid=subactor_uid, + cid=ctx.cid, + locked=False, + ) # 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}" + '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' ) - - log.debug(f"Actor {subactor_uid} is WAITING on stdin hijack lock") Lock.shield_sigint() - try: with ( + # NOTE: though a cs is created for every subactor lock + # REQUEST in this ctx-child task, only the root-task + # holding the `Lock` (on behalf of the ctx parent task + # in a subactor) will set + # `Lock._locking_task_cs` such that if the + # lock holdingn task ever needs to be cancelled (since + # it's shielded by default) that global ref can be + # used to do so! trio.CancelScope(shield=True) as debug_lock_cs, + + _codec.limit_msg_spec( + payload_spec=__msg_spec__, + ) as codec, ): - Lock._root_local_task_cs_in_debug = debug_lock_cs - async with _acquire_debug_lock_from_root_task(subactor_uid): + # sanity? + # TODO: don't need the ref right? + assert codec is _codec.current_codec() + + async with _acquire_debug_lock_from_root_task( + subactor_uid, + subactor_task_uid, + ): + # XXX SUPER IMPORTANT BELOW IS ON THIS LINE XXX + # without that the root cs might be, + # - set and then removed in the finally block by + # a task that never acquired the lock, leaving + # - the task that DID acquire the lock STUCK since + # it's original cs was GC-ed bc the first task + # already set the global ref to `None` + Lock.set_locking_task_cs(debug_lock_cs) # indicate to child that we've locked stdio - await ctx.started('Locked') - log.debug( - f"Actor {subactor_uid} acquired stdin hijack lock" + await ctx.started( + LockStatus( + subactor_uid=subactor_uid, + cid=ctx.cid, + locked=True, + ) ) + log.debug( f'Actor {subactor_uid} acquired TTY lock') + # wait for unlock pdb by child async with ctx.open_stream() as stream: - assert await stream.receive() == 'pdb_unlock' + release_msg: LockRelease = await stream.receive() - return "pdb_unlock_complete" + # TODO: security around only releasing if + # these match? + log.pdb( + 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.debug(f'Actor {subactor_uid} released TTY lock') + + return LockStatus( + subactor_uid=subactor_uid, + cid=ctx.cid, + locked=False, + ) finally: - Lock._root_local_task_cs_in_debug = None + debug_lock_cs.cancel() + Lock.set_locking_task_cs(None) Lock.unshield_sigint() +@cm +def apply_debug_codec() -> _codec.MsgCodec: + ''' + Apply the subactor TTY `Lock`-ing protocol's msgspec temporarily + (only in the current task). + + ''' + with ( + _codec.limit_msg_spec( + payload_spec=__msg_spec__, + ) as debug_codec, + ): + assert debug_codec is _codec.current_codec() + log.pdb( + 'Applied `.devx._debug` msg-spec via codec\n' + f'{debug_codec}\n' + ) + yield debug_codec + + log.pdb( + 'REMOVED `.devx._debug` msg-spec via codec\n' + f'{debug_codec}\n' + ) + + async def wait_for_parent_stdin_hijack( actor_uid: tuple[str, str], + task_uid: tuple[str, int], task_status: TaskStatus[trio.CancelScope] = trio.TASK_STATUS_IGNORED ): ''' @@ -476,25 +813,30 @@ async def wait_for_parent_stdin_hijack( ''' from .._discovery import get_root - with trio.CancelScope(shield=True) as cs: + with ( + trio.CancelScope(shield=True) as cs, + apply_debug_codec(), + ): Lock._debugger_request_cs = cs - try: + # TODO: merge into sync async with ? async with get_root() as portal: - # this syncs to child's ``Context.started()`` call. async with portal.open_context( lock_tty_for_child, subactor_uid=actor_uid, + subactor_task_uid=task_uid, - ) as (ctx, val): - - log.debug('locked context') - assert val == 'Locked' + ) as (ctx, resp): + log.pdb( + 'Subactor locked TTY per msg\n' + f'{resp}\n' + ) + assert resp.subactor_uid == actor_uid + assert resp.cid async with ctx.open_stream() as stream: - try: - # unblock local caller + try: # to unblock local caller assert Lock.local_pdb_complete task_status.started(cs) @@ -503,14 +845,22 @@ async def wait_for_parent_stdin_hijack( await Lock.local_pdb_complete.wait() finally: - # TODO: shielding currently can cause hangs... - # with trio.CancelScope(shield=True): - await stream.send('pdb_unlock') + await stream.send( + LockRelease( + subactor_uid=actor_uid, + cid=resp.cid, + ) + ) # sync with callee termination - assert await ctx.result() == "pdb_unlock_complete" + status: LockStatus = await ctx.result() + assert not status.locked - log.debug('exitting child side locking task context') + log.pdb( + 'TTY lock was released for subactor with msg\n\n' + f'{status}\n\n' + 'Exitting {ctx.side!r} side locking of locking ctx' + ) except ContextCancelled: log.warning('Root actor cancelled debug lock') @@ -518,12 +868,17 @@ async def wait_for_parent_stdin_hijack( finally: Lock.local_task_in_debug = None - log.debug('Exiting debugger from child') + log.debug('Exiting debugger TTY lock request func from child') -def mk_mpdb() -> MultiActorPdb: + log.cancel('Reverting SIGINT handler!') + Lock.unshield_sigint() + + + +def mk_mpdb() -> PdbREPL: ''' - Deliver a new `MultiActorPdb`: a multi-process safe `pdbp` + Deliver a new `PdbREPL`: a multi-process safe `pdbp` REPL using the magic of SC! Our `pdb.Pdb` subtype accomplishes multi-process safe debugging @@ -538,7 +893,7 @@ def mk_mpdb() -> MultiActorPdb: by either explicit requests in the runtime or ''' - pdb = MultiActorPdb() + pdb = PdbREPL() # Always shield out SIGINTs for subactors when REPL is active. # @@ -560,7 +915,6 @@ def mk_mpdb() -> MultiActorPdb: def shield_sigint_handler( signum: int, frame: 'frame', # type: ignore # noqa - # pdb_obj: MultiActorPdb | None = None, *args, ) -> None: @@ -577,6 +931,7 @@ def shield_sigint_handler( uid_in_debug: tuple[str, str]|None = Lock.global_actor_in_debug actor: Actor = current_actor() + case_handled: bool = False def do_cancel(): # If we haven't tried to cancel the runtime then do that instead @@ -586,107 +941,202 @@ 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 + # try to see if the supposed (sub)actor in debug still + # has an active connection to *this* actor, and if not + # it's likely they aren't using the TTY lock / debugger + # and we should propagate SIGINT normally. any_connected: bool = False - if uid_in_debug is not None: - # try to see if the supposed (sub)actor in debug still - # has an active connection to *this* actor, and if not - # it's likely they aren't using the TTY lock / debugger - # and we should propagate SIGINT normally. - chans: list[tractor.Channel] = actor._peers.get(tuple(uid_in_debug)) + 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' + 'but no connection exists for this child!?\n' + f'subactor_uid: {uid_in_debug}\n\n' 'Allowing SIGINT propagation..' ) return do_cancel() # only set in the actor actually running the REPL - pdb_obj: MultiActorPdb|None = Lock.repl + repl: PdbREPL|None = Lock.repl + # TODO: maybe we should flatten out all these cases using + # a match/case? + # # root actor branch that reports whether or not a child # has locked debugger. - if ( - is_root_process() - and uid_in_debug is not None + if is_root_process(): + lock_cs: trio.CancelScope = Lock.get_locking_task_cs() - # 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: + log.warning( + f'root {actor.uid} handling SIGINT\n' + f'any_connected: {any_connected}\n\n' - if pdb_obj: - name = uid_in_debug[0] - if name != 'root': - log.pdb( - f"Ignoring SIGINT, child in debug mode: `{uid_in_debug}`" - ) + f'{Lock.repr()}\n' + ) + + maybe_stale_lock_cs: bool = ( + lock_cs is not None + # and not lock_cs.cancel_called + and uid_in_debug is None + ) + if maybe_stale_lock_cs: + log.warning( + 'Stale `Lock._locking_task_cs: CancelScope` DETECTED?\n' + f'|_{lock_cs}\n\n' + ) + lock_cs.cancel() + + if uid_in_debug: # "someone" is (ostensibly) using debug `Lock` + name_in_debug: str = uid_in_debug[0] + 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' + + # 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. + if any_connected: # there are subactors we can contact + # NOTE: don't emit this with `.pdb()` level in + # root without a higher level. + log.debug( + f'Ignoring SIGINT while debug REPL in use by child\n' + f'subactor: {uid_in_debug}\n' + ) + # returns here minus tail logic + case_handled = True + + else: + message: str = ( + f'Ignoring SIGINT while debug REPL SUPPOSEDLY in use by child\n' + f'subactor: {uid_in_debug}\n\n' + f'BUT, no child actors are contactable!?!?\n\n' + + # f'Reverting to def `trio` SIGINT handler..\n' + ) + + if maybe_stale_lock_cs: + lock_cs.cancel() + message += ( + 'Maybe `Lock._locking_task_cs: CancelScope` is stale?\n' + f'|_{lock_cs}\n\n' + ) + + log.warning(message) + Lock.unshield_sigint() + case_handled = True else: + assert name_in_debug == 'root' # we are the registered locker + assert repl # we have a pdb REPL engaged log.pdb( - "Ignoring SIGINT while in debug mode" + f'Ignoring SIGINT while debug REPL in use\n' + f'root actor: {uid_in_debug}\n' ) - elif ( - is_root_process() - ): - if pdb_obj: - log.pdb( - "Ignoring SIGINT since debug mode is enabled" + # returns here minus tail logic + case_handled = True + + # 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! + else: + # XXX revert back to ``trio`` handler since this handler shouldn't + # be enabled withtout an actor using a debug REPL! + log.warning( + 'Ignoring SIGINT in root actor but no actor using a `pdb` REPL?\n' + 'Reverting SIGINT handler to `trio` default!\n' ) - if ( - Lock._root_local_task_cs_in_debug - and not Lock._root_local_task_cs_in_debug.cancel_called - ): - Lock._root_local_task_cs_in_debug.cancel() + if maybe_stale_lock_cs: + lock_cs.cancel() - # revert back to ``trio`` handler asap! Lock.unshield_sigint() + case_handled = True # child actor that has locked the debugger elif not is_root_process(): + log.warning( + 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' + 'A global sub-actor reported to be in debug ' + 'but it has no connection to its parent ??\n' f'{uid_in_debug}\n' 'Allowing SIGINT propagation..' ) - return do_cancel() + Lock.unshield_sigint() + # do_cancel() + case_handled = True - task: str | None = Lock.local_task_in_debug + task: str|None = Lock.local_task_in_debug if ( task - and pdb_obj + and + repl ): + # if repl: log.pdb( - f"Ignoring SIGINT while task in debug mode: `{task}`" + f'Ignoring SIGINT while local task using debug REPL\n' + f'|_{task}\n' + f' |_{repl}\n' ) + case_handled = True + else: + msg: str = ( + 'SIGINT shield handler still active BUT, \n\n' + ) + if task is None: + msg += ( + f'- No local task claims to be in debug?\n' + f' |_{task}\n\n' + ) + + if repl is None: + msg += ( + f'- No local REPL is currently active?\n' + f' |_{repl}\n\n' + ) + + log.warning( + msg + + + 'Reverting handler to `trio` default!\n' + ) + Lock.unshield_sigint() + case_handled = True + + # XXX ensure that the reverted-to-handler actually is + # able to rx what should have been **this** KBI ;) + do_cancel() + # raise KeyboardInterrupt # 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,31 +1145,37 @@ 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 this actor has a REPL engaged ): # XXX: yah, mega hack, but how else do we catch this madness XD - if pdb_obj.shname == 'xonsh': - pdb_obj.stdout.write(pdb_obj.prompt) + 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 + if not case_handled: + log.critical( + f'{actor.uid} UNHANDLED SIGINT !?!?\n' + # TODO: pprint for `Lock`? + ) + _pause_msg: str = 'Attaching to pdb REPL in actor' def _set_trace( actor: tractor.Actor|None = None, - pdb: MultiActorPdb|None = None, + pdb: PdbREPL|None = None, shield: bool = False, extra_frames_up_when_async: int = 1, @@ -767,14 +1223,16 @@ def _set_trace( log.pdb( f'{msg}\n' '|\n' - f'|_ {actor.uid}\n' + # TODO: make an `Actor.__repr()__` + # f'|_ {current_task()} @ {actor.name}\n' + f'|_ {current_task()}\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' + f'Going up frame_{i}:\n|_{frame}\n' ) # engage ze REPL @@ -787,7 +1245,7 @@ async def _pause( debug_func: Callable = _set_trace, # NOTE: must be passed in the `.pause_from_sync()` case! - pdb: MultiActorPdb|None = None, + pdb: PdbREPL|None = None, # TODO: allow caller to pause despite task cancellation, # exactly the same as wrapping with: @@ -799,6 +1257,8 @@ async def _pause( # shield: bool = False, hide_tb: bool = True, + extra_frames_up_when_async: int = 4, + task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED ) -> None: @@ -813,7 +1273,9 @@ async def _pause( __tracebackhide__: bool = hide_tb actor: Actor = current_actor() try: - task_name: str = trio.lowlevel.current_task().name + # TODO: use the `Task` instance instead for `is` checks + # below! + task: Task = trio.lowlevel.current_task() except RuntimeError as rte: if actor.is_infected_aio(): raise RuntimeError( @@ -821,48 +1283,142 @@ async def _pause( 'for infected `asyncio` mode!' ) from rte + # task_name: str = task.name + if ( not Lock.local_pdb_complete - or Lock.local_pdb_complete.is_set() + or + Lock.local_pdb_complete.is_set() ): Lock.local_pdb_complete = trio.Event() if debug_func is not None: - debug_func = partial( - debug_func, - ) + debug_func = partial(debug_func) if pdb is None: - pdb: MultiActorPdb = mk_mpdb() + pdb: PdbREPL = mk_mpdb() + + def _enter_repl_sync( + debug_func: Callable, + ) -> None: + __tracebackhide__: bool = hide_tb + 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) + 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: + # log.critical( + # f'stack len: {len(pdb.stack)}\n' + # ) + debug_func( + actor, + pdb, + extra_frames_up_when_async=extra_frames_up_when_async, + shield=shield, + ) + except BaseException: + log.exception( + 'Failed to invoke internal `debug_func = ' + f'{debug_func.func.__name__}`\n' + ) + raise + + except bdb.BdbQuit: + Lock.release() + raise + + except BaseException: + log.exception( + 'Failed to engage debugger via `_pause()` ??\n' + ) + raise + + 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 acquiring it though? + if Lock.global_actor_in_debug == actor.uid: + # re-entrant root process already has it: noop. + log.warning( + f'{task.name}@{actor.uid} already has TTY lock\n' + f'ignoring..' + ) + await trio.lowlevel.checkpoint() + 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( + 'attempting to shield-acquire active TTY lock' + f' owned by {Lock.global_actor_in_debug}' + ) + + # must shield here to avoid hitting a ``Cancelled`` and + # a child getting stuck bc we clobbered the tty + with trio.CancelScope(shield=True): + await Lock._debug_lock.acquire() + else: + # may be cancelled + await Lock._debug_lock.acquire() + + Lock.global_actor_in_debug = actor.uid + Lock.local_task_in_debug = task + Lock.repl = pdb + + # enter REPL from root, no TTY locking IPC ctx necessary + _enter_repl_sync(debug_func) + return # next branch is mutex and for subactors # TODO: need a more robust check for the "root" actor - if ( + elif ( 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. + # + # NOTE: 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 that would otherwise not bubble until the + # next checkpoint was hit. + if ( + (repl_task := Lock.local_task_in_debug) + and + repl_task is task + ): + log.warning( + f'{task.name}@{actor.uid} already has TTY lock\n' + f'ignoring..' + ) 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...") - + # 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.name}@{actor.uid} already has TTY lock\n' + f'waiting for release..' + ) 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 + Lock.local_task_in_debug = task # this **must** be awaited by the caller and is done using the # root nursery so that the debugger can continue to run without @@ -875,91 +1431,54 @@ async def _pause( # actor._service_n.cancel_scope.shield = shield # ``` # but not entirely sure if that's a sane way to implement it? - try: - with trio.CancelScope(shield=True): - await actor._service_n.start( + + # NOTE: MUST it here bc multiple tasks are spawned by any + # one 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 here. SO ensure + # the codec is set before either are run! + # + with ( + # _codec.limit_msg_spec( + # payload_spec=__msg_spec__, + # ) as debug_codec, + trio.CancelScope(shield=shield), + ): + # async with trio.open_nursery() as tn: + # tn.cancel_scope.shield = True + try: + # cs: trio.CancelScope = await tn.start( + cs: trio.CancelScope = await actor._service_n.start( wait_for_parent_stdin_hijack, actor.uid, + (task.name, id(task)), ) + # our locker task should be the one in ctx + # with the root actor + assert Lock._debugger_request_cs is cs + + # XXX used by the SIGINT handler to check if + # THIS actor is in REPL interaction Lock.repl = pdb - except RuntimeError: - Lock.release() + except RuntimeError: + Lock.release() - if actor._cancel_called: - # service nursery won't be usable and we - # don't want to lock up the root either way since - # we're in (the midst of) cancellation. - return + if actor._cancel_called: + # service nursery won't be usable and we + # don't want to lock up the root either way since + # we're in (the midst of) cancellation. + return - raise - - elif is_root_process(): - - # we also wait in the root-parent for any child that - # may have the tty locked prior - # TODO: wait, what about multiple root tasks acquiring it though? - if Lock.global_actor_in_debug == actor.uid: - # re-entrant root process already has it: noop. - return - - # XXX: since we need to enter pdb synchronously below, - # we have to release the lock manually from pdb completion - # callbacks. Can't think of a nicer way then this atm. - if Lock._debug_lock.locked(): - log.warning( - 'Root actor attempting to shield-acquire active tty lock' - f' owned by {Lock.global_actor_in_debug}') - - # must shield here to avoid hitting a ``Cancelled`` and - # a child getting stuck bc we clobbered the tty - with trio.CancelScope(shield=True): - await Lock._debug_lock.acquire() - else: - # may be cancelled - await Lock._debug_lock.acquire() - - Lock.global_actor_in_debug = actor.uid - Lock.local_task_in_debug = task_name - Lock.repl = pdb - - try: - # 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) - - 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: - # log.critical( - # f'stack len: {len(pdb.stack)}\n' - # ) - 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 - except bdb.BdbQuit: - Lock.release() - raise + # enter REPL + + try: + _enter_repl_sync(debug_func) + finally: + Lock.unshield_sigint() - except BaseException: - log.exception( - 'Failed to engage debugger via `_pause()` ??\n' - ) - raise # XXX: apparently we can't do this without showing this frame # in the backtrace on first entry to the REPL? Seems like an odd @@ -1017,15 +1536,21 @@ async def pause( # __tracebackhide__: bool = True # super().__exit__(*args, **kwargs) - trio.CancelScope.__enter__.__tracebackhide__ = True - trio.CancelScope.__exit__.__tracebackhide__ = True + # 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: + # TODO: LOL, solved this with the `pdb.hideframe` stuff + # at top-of-mod.. so I guess we can just only use this + # block right? + with trio.CancelScope( + shield=shield, + ) as cs: + print(f'debug cs is {cs}\n') # setattr(cs.__exit__.__func__, '__tracebackhide__', True) # setattr(cs.__enter__.__func__, '__tracebackhide__', True) @@ -1135,7 +1660,7 @@ def pause_from_sync( # raises on not-found by default greenback: ModuleType = maybe_import_greenback() - mdb: MultiActorPdb = mk_mpdb() + mdb: PdbREPL = mk_mpdb() # run async task which will lock out the root proc's TTY. if not Lock.is_main_trio_thread(): @@ -1157,7 +1682,7 @@ def pause_from_sync( ) ) # TODO: maybe the `trio.current_task()` id/name if avail? - Lock.local_task_in_debug: str = str(threading.current_thread().name) + Lock.local_task_in_debug: str = str(threading.current_thread()) else: # we are presumably the `trio.run()` + main thread greenback.await_( @@ -1167,7 +1692,7 @@ def pause_from_sync( hide_tb=hide_tb, ) ) - Lock.local_task_in_debug: str = current_task().name + Lock.local_task_in_debug: str = current_task() # TODO: ensure we aggressively make the user aware about # entering the global ``breakpoint()`` built-in from sync @@ -1198,7 +1723,11 @@ async def breakpoint(**kwargs): '`tractor.breakpoint()` is deprecated!\n' 'Please use `tractor.pause()` instead!\n' ) - await pause(**kwargs) + __tracebackhide__: bool = True + await pause( + # extra_frames_up_when_async=6, + **kwargs + ) _crash_msg: str = ( @@ -1208,11 +1737,11 @@ _crash_msg: str = ( def _post_mortem( actor: tractor.Actor, - pdb: MultiActorPdb, + pdb: PdbREPL, shield: bool = False, # only for compat with `._set_trace()`.. - extra_frames_up_when_async=0, + extra_frames_up_when_async=1, ) -> None: ''' @@ -1225,7 +1754,11 @@ def _post_mortem( log.pdb( f'{_crash_msg}\n' '|\n' - f'|_ {actor.uid}\n' + f'|_ {current_task()}\n' + + # f'|_ @{actor.uid}\n' + # TODO: make an `Actor.__repr()__` + # f'|_ {current_task()} @ {actor.name}\n' ) # TODO: only replacing this to add the @@ -1278,9 +1811,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 @@ -1297,7 +1833,7 @@ async def acquire_debug_lock( wait_for_parent_stdin_hijack, subactor_uid, ) - yield None + yield cs cs.cancel() @@ -1328,7 +1864,6 @@ async def maybe_wait_for_debugger( # 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 if in_debug == current_actor().uid: log.debug( @@ -1340,7 +1875,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 @@ -1358,17 +1893,17 @@ async def maybe_wait_for_debugger( for istep in range(poll_steps): if ( - debug_complete - and not debug_complete.is_set() + Lock.no_remote_has_tty is not None + and not Lock.no_remote_has_tty.is_set() and in_debug is not None ): log.pdb( msg + - 'Root is waiting on tty lock to release..\n' + '\nRoot is waiting on tty lock to release..\n' ) with trio.CancelScope(shield=True): - await debug_complete.wait() + await Lock.no_remote_has_tty.wait() log.pdb( f'Child subactor released debug lock\n' f'|_{in_debug}\n' @@ -1378,8 +1913,8 @@ async def maybe_wait_for_debugger( if ( in_debug is None and ( - debug_complete is None - or debug_complete.is_set() + Lock.no_remote_has_tty is None + or Lock.no_remote_has_tty.is_set() ) ): log.pdb(