From d65e4bbad763163c340e11b1cadfe7c65b170639 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 7 Mar 2023 17:37:06 -0500 Subject: [PATCH 01/30] Add (first-draft) infected-`asyncio` actor task uses debugger example --- examples/debugging/asyncio_bp.py | 79 ++++++++++++++++++++++++++++++++ 1 file changed, 79 insertions(+) create mode 100644 examples/debugging/asyncio_bp.py diff --git a/examples/debugging/asyncio_bp.py b/examples/debugging/asyncio_bp.py new file mode 100644 index 00000000..882ebbed --- /dev/null +++ b/examples/debugging/asyncio_bp.py @@ -0,0 +1,79 @@ +import asyncio + +import trio +import tractor + + +async def bp_then_error( + to_trio: trio.MemorySendChannel, + from_trio: asyncio.Queue, + +) -> None: + + # sync with ``trio``-side (caller) task + to_trio.send_nowait('start') + + # NOTE: what happens here inside the hook needs some refinement.. + # => seems like it's still `._debug._set_trace()` but + # we set `Lock.local_task_in_debug = 'sync'`, we probably want + # some further, at least, meta-data about the task/actoq in debug + # in terms of making it clear it's asyncio mucking about. + + breakpoint() + + await asyncio.sleep(0.5) + raise ValueError('blah') + + +async def aio_sleep_forever(): + await asyncio.sleep(float('inf')) + + +@tractor.context +async def trio_ctx( + ctx: tractor.Context, +): + + # this will block until the ``asyncio`` task sends a "first" + # message, see first line in above func. + async with ( + tractor.to_asyncio.open_channel_from(bp_then_error) as (first, chan), + trio.open_nursery() as n, + ): + + assert first == 'start' + await ctx.started(first) + + n.start_soon( + tractor.to_asyncio.run_task, + aio_sleep_forever, + ) + await trio.sleep_forever() + + +async def main(): + + async with tractor.open_nursery() as n: + + p = await n.start_actor( + 'aio_daemon', + enable_modules=[__name__], + infect_asyncio=True, + debug_mode=True, + loglevel='cancel', + ) + + async with p.open_context(trio_ctx) as (ctx, first): + + assert first == 'start' + await trio.sleep_forever() + + assert 0 + + # TODO: case where we cancel from trio-side while asyncio task + # has debugger lock? + # await p.cancel_actor() + + +if __name__ == '__main__': + trio.run(main) From 62a0fff2fd88e11c224828f08f80185482fe5b70 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 27 Mar 2023 19:05:00 -0400 Subject: [PATCH 02/30] Add a debug-mode-breakpoint-causes-hang case! Only found this by luck more or less (while working on something in a client project) and it turns out we can actually get to (yet another) hang state where SIGINT will be ignored by the root actor on teardown.. I've added all the necessary logic flags to reproduce. We obviously need a follow up bug issue and a test suite to replicate! It appears as though the following are required based on very light tinkering: - infected asyncio mode active - debug mode active - the `trio` context must breakpoint *before* `.started()`-ing - the `asyncio` must **not** error --- examples/debugging/asyncio_bp.py | 56 +++++++++++++++++++++++++++----- 1 file changed, 47 insertions(+), 9 deletions(-) diff --git a/examples/debugging/asyncio_bp.py b/examples/debugging/asyncio_bp.py index 882ebbed..b32ad1d8 100644 --- a/examples/debugging/asyncio_bp.py +++ b/examples/debugging/asyncio_bp.py @@ -2,12 +2,19 @@ import asyncio import trio import tractor +from tractor import to_asyncio + + +async def aio_sleep_forever(): + await asyncio.sleep(float('inf')) async def bp_then_error( to_trio: trio.MemorySendChannel, from_trio: asyncio.Queue, + raise_after_bp: bool = True, + ) -> None: # sync with ``trio``-side (caller) task @@ -18,40 +25,57 @@ async def bp_then_error( # we set `Lock.local_task_in_debug = 'sync'`, we probably want # some further, at least, meta-data about the task/actoq in debug # in terms of making it clear it's asyncio mucking about. - breakpoint() + # short checkpoint / delay await asyncio.sleep(0.5) - raise ValueError('blah') + if raise_after_bp: + raise ValueError('blah') -async def aio_sleep_forever(): - await asyncio.sleep(float('inf')) + # TODO: test case with this so that it gets cancelled? + else: + # XXX NOTE: this is required in order to get the SIGINT-ignored + # hang case documented in the module script section! + await aio_sleep_forever() @tractor.context async def trio_ctx( ctx: tractor.Context, + bp_before_started: bool = False, ): # this will block until the ``asyncio`` task sends a "first" # message, see first line in above func. async with ( - tractor.to_asyncio.open_channel_from(bp_then_error) as (first, chan), + + to_asyncio.open_channel_from( + bp_then_error, + raise_after_bp=not bp_before_started, + ) as (first, chan), + trio.open_nursery() as n, ): assert first == 'start' + + if bp_before_started: + await tractor.breakpoint() + await ctx.started(first) n.start_soon( - tractor.to_asyncio.run_task, + to_asyncio.run_task, aio_sleep_forever, ) await trio.sleep_forever() -async def main(): +async def main( + bps_all_over: bool = False, + +) -> None: async with tractor.open_nursery() as n: @@ -63,11 +87,18 @@ async def main(): loglevel='cancel', ) - async with p.open_context(trio_ctx) as (ctx, first): + async with p.open_context( + trio_ctx, + bp_before_started=bps_all_over, + ) as (ctx, first): assert first == 'start' - await trio.sleep_forever() + if bps_all_over: + await tractor.breakpoint() + + # await trio.sleep_forever() + await ctx.cancel() assert 0 # TODO: case where we cancel from trio-side while asyncio task @@ -76,4 +107,11 @@ async def main(): if __name__ == '__main__': + + # works fine B) trio.run(main) + + # will hang and ignores SIGINT !! + # NOTE: you'll need to send a SIGQUIT (via ctl-\) to kill it + # manually.. + # trio.run(main, True) From f0417d802ba085a33bcbeaf081d10564aa7ef25d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 21 Jun 2023 16:08:18 -0400 Subject: [PATCH 03/30] First proto: use `greenback` for sync func breakpointing This works now for supporting a new `tractor.pause_from_sync()` `tractor`-aware-replacement for `Pdb.set_trace()` from sync functions which are also scheduled from our runtime. Uses `greenback` to do all the magic of scheduling the bg `tractor._debug._pause()` task and engaging the normal TTY locking machinery triggered by `await tractor.breakpoint()` Further this starts some public API renaming, making a switch to `tractor.pause()` from `.breakpoint()` which IMO much better expresses the semantics of the runtime intervention required to suffice multi-process "breakpointing"; it also is an alternate name for the same in computer science more generally: https://en.wikipedia.org/wiki/Breakpoint It also avoids using the same name as the `breakpoint()` built-in which is important since there **is alot more going on** when you call our equivalent API. Deats of that: - add deprecation warning for `tractor.breakpoint()` - add `tractor.pause()` and a shorthand, easier-to-type, alias `.pp()` for "pause-point" B) - add `pause_from_sync()` as the new `breakpoint()`-from-sync-function hack which does all the `greenback` stuff for the user. Still TODO: - figure out where in the runtime and when to call `greenback.ensure_portal()`. - fix the frame selection issue where `trio._core._ki._ki_protection_decorator:wrapper` seems to be always shown on REPL start as the selected frame.. --- tractor/__init__.py | 8 ++- tractor/_debug.py | 116 ++++++++++++++++++++++++++++++++++++-------- tractor/_runtime.py | 4 +- 3 files changed, 106 insertions(+), 22 deletions(-) diff --git a/tractor/__init__.py b/tractor/__init__.py index 7af40c6e..ad71db1e 100644 --- a/tractor/__init__.py +++ b/tractor/__init__.py @@ -46,6 +46,9 @@ from ._exceptions import ( ) from ._debug import ( breakpoint, + pause, + pp, + pause_from_sync, post_mortem, ) from . import msg @@ -59,12 +62,12 @@ from ._runtime import Actor __all__ = [ 'Actor', + 'BaseExceptionGroup', 'Channel', 'Context', 'ContextCancelled', 'ModuleNotExposed', 'MsgStream', - 'BaseExceptionGroup', 'Portal', 'RemoteActorError', 'breakpoint', @@ -77,7 +80,10 @@ __all__ = [ 'open_actor_cluster', 'open_nursery', 'open_root_actor', + 'pause', 'post_mortem', + 'pp', + 'pause_from_sync' 'query_actor', 'run_daemon', 'stream', diff --git a/tractor/_debug.py b/tractor/_debug.py index b0482f18..eec6fc50 100644 --- a/tractor/_debug.py +++ b/tractor/_debug.py @@ -374,7 +374,7 @@ async def wait_for_parent_stdin_hijack( This function is used by any sub-actor to acquire mutex access to the ``pdb`` REPL and thus the root's TTY for interactive debugging - (see below inside ``_breakpoint()``). It can be used to ensure that + (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()``). @@ -440,17 +440,29 @@ def mk_mpdb() -> tuple[MultiActorPdb, Callable]: return pdb, Lock.unshield_sigint -async def _breakpoint( +async def _pause( - debug_func, + debug_func: Callable | None = None, + release_lock_signal: trio.Event | None = None, # TODO: # shield: bool = False + task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED ) -> None: ''' - Breakpoint entry for engaging debugger instance sync-interaction, - from async code, executing in actor runtime (task). + A pause point (more commonly known as a "breakpoint") interrupt + instruction for engaging a blocking debugger instance to + conduct manual console-based-REPL-interaction from within + `tractor`'s async runtime, normally from some single-threaded + and currently executing actor-hosted-`trio`-task in some + (remote) process. + + NOTE: we use the semantics "pause" since it better encompasses + the entirety of the necessary global-runtime-state-mutation any + actor-task must access and lock in order to get full isolated + control over the process tree's root TTY: + https://en.wikipedia.org/wiki/Breakpoint ''' __tracebackhide__ = True @@ -559,10 +571,21 @@ async def _breakpoint( Lock.repl = pdb try: - # block here one (at the appropriate frame *up*) where - # ``breakpoint()`` was awaited and begin handling stdio. - log.debug("Entering the synchronous world of pdb") - debug_func(actor, pdb) + # breakpoint() + if debug_func is None: + assert release_lock_signal, ( + 'Must pass `release_lock_signal: trio.Event` if no ' + 'trace func provided!' + ) + print(f"{actor.uid} ENTERING WAIT") + task_status.started() + await release_lock_signal.wait() + + else: + # block here one (at the appropriate frame *up*) where + # ``breakpoint()`` was awaited and begin handling stdio. + log.debug("Entering the synchronous world of pdb") + debug_func(actor, pdb) except bdb.BdbQuit: Lock.release() @@ -708,8 +731,8 @@ def shield_sigint_handler( # elif debug_mode(): else: # XXX: shouldn't ever get here? - print("WTFWTFWTF") - raise KeyboardInterrupt + 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) @@ -737,21 +760,18 @@ def shield_sigint_handler( # https://github.com/goodboy/tractor/issues/130#issuecomment-663752040 # https://github.com/prompt-toolkit/python-prompt-toolkit/blob/c2c6af8a0308f9e5d7c0e28cb8a02963fe0ce07a/prompt_toolkit/patch_stdout.py - # XXX LEGACY: lol, see ``pdbpp`` issue: - # https://github.com/pdbpp/pdbpp/issues/496 - def _set_trace( actor: tractor.Actor | None = None, pdb: MultiActorPdb | None = None, ): __tracebackhide__ = True - actor = actor or tractor.current_actor() + actor: tractor.Actor = actor or tractor.current_actor() # start 2 levels up in user code - frame: Optional[FrameType] = sys._getframe() + frame: FrameType | None = sys._getframe() if frame: - frame = frame.f_back # type: ignore + frame: FrameType = frame.f_back # type: ignore if ( frame @@ -773,10 +793,66 @@ def _set_trace( pdb.set_trace(frame=frame) -breakpoint = partial( - _breakpoint, +# TODO: allow pausing from sync code, normally by remapping +# python's builtin breakpoint() hook to this runtime aware version. +def pause_from_sync() -> None: + import greenback + + actor: tractor.Actor = tractor.current_actor() + task_can_release_tty_lock = trio.Event() + + # spawn bg task which will lock out the TTY, we poll + # just below until the release event is reporting that task as + # waiting.. not the most ideal but works for now ;) + greenback.await_( + actor._service_n.start(partial( + _pause, + debug_func=None, + release_lock_signal=task_can_release_tty_lock, + )) + ) + print("ENTER SYNC PAUSE") + pdb, undo_sigint = mk_mpdb() + try: + print("ENTER SYNC PAUSE") + # _set_trace(actor=actor) + + # we entered the global ``breakpoint()`` built-in from sync + # code? + Lock.local_task_in_debug = 'sync' + frame: FrameType | None = sys._getframe() + print(f'FRAME: {str(frame)}') + + frame: FrameType = frame.f_back # type: ignore + print(f'FRAME: {str(frame)}') + + frame: FrameType = frame.f_back # type: ignore + print(f'FRAME: {str(frame)}') + + pdb.set_trace(frame=frame) + # pdb.do_frame( + # pdb.curindex + + finally: + task_can_release_tty_lock.set() + undo_sigint() + +# using the "pause" semantics instead since +# that better covers actually somewhat "pausing the runtime" +# for this particular paralell task to do debugging B) +pause = partial( + _pause, _set_trace, ) +pp = pause # short-hand for "pause point" + + +async def breakpoint(**kwargs): + log.warning( + '`tractor.breakpoint()` is deprecated!\n' + 'Please use `tractor.pause()` instead!\n' + ) + await pause(**kwargs) def _post_mortem( @@ -801,7 +877,7 @@ def _post_mortem( post_mortem = partial( - _breakpoint, + _pause, _post_mortem, ) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index cc8eaf5f..6c2e028f 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -1722,4 +1722,6 @@ class Arbiter(Actor): ) -> None: uid = (str(uid[0]), str(uid[1])) - self._registry.pop(uid) + entry: tuple = self._registry.pop(uid, None) + if entry is None: + log.warning(f'Request to de-register {uid} failed?') From 36d2aa185286ff9ed2864815832f3179baccc2f8 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 7 Jul 2023 14:47:42 -0400 Subject: [PATCH 04/30] Add longer "required reading" list B) --- docs/README.rst | 25 ++++++++++++++++++------- 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/docs/README.rst b/docs/README.rst index 9dfe2f60..9dd7faf4 100644 --- a/docs/README.rst +++ b/docs/README.rst @@ -3,8 +3,8 @@ |gh_actions| |docs| -``tractor`` is a `structured concurrent`_, multi-processing_ runtime -built on trio_. +``tractor`` is a `structured concurrent`_, (optionally +distributed_) multi-processing_ runtime built on trio_. Fundamentally, ``tractor`` gives you parallelism via ``trio``-"*actors*": independent Python processes (aka @@ -17,11 +17,20 @@ protocol" constructed on top of multiple Pythons each running a ``trio`` scheduled runtime - a call to ``trio.run()``. We believe the system adheres to the `3 axioms`_ of an "`actor model`_" -but likely *does not* look like what *you* probably think an "actor -model" looks like, and that's *intentional*. +but likely **does not** look like what **you** probably *think* an "actor +model" looks like, and that's **intentional**. -The first step to grok ``tractor`` is to get the basics of ``trio`` down. -A great place to start is the `trio docs`_ and this `blog post`_. + +Where do i start!? +------------------ +The first step to grok ``tractor`` is to get an intermediate +knowledge of ``trio`` and **structured concurrency** B) + +Some great places to start are, +- the seminal `blog post`_ +- obviously the `trio docs`_ +- wikipedia's nascent SC_ page +- the fancy diagrams @ libdill-docs_ Features @@ -593,6 +602,7 @@ matrix seems too hip, we're also mostly all in the the `trio gitter channel`_! .. _structured concurrent: https://trio.discourse.group/t/concise-definition-of-structured-concurrency/228 +.. _distributed: https://en.wikipedia.org/wiki/Distributed_computing .. _multi-processing: https://en.wikipedia.org/wiki/Multiprocessing .. _trio: https://github.com/python-trio/trio .. _nurseries: https://vorpus.org/blog/notes-on-structured-concurrency-or-go-statement-considered-harmful/#nurseries-a-structured-replacement-for-go-statements @@ -611,8 +621,9 @@ channel`_! .. _trio docs: https://trio.readthedocs.io/en/latest/ .. _blog post: https://vorpus.org/blog/notes-on-structured-concurrency-or-go-statement-considered-harmful/ .. _structured concurrency: https://en.wikipedia.org/wiki/Structured_concurrency +.. _SC: https://en.wikipedia.org/wiki/Structured_concurrency +.. _libdill-docs: https://sustrik.github.io/libdill/structured-concurrency.html .. _structured chadcurrency: https://en.wikipedia.org/wiki/Structured_concurrency -.. _structured concurrency: https://en.wikipedia.org/wiki/Structured_concurrency .. _unrequirements: https://en.wikipedia.org/wiki/Actor_model#Direct_communication_and_asynchrony .. _async generators: https://www.python.org/dev/peps/pep-0525/ .. _trio-parallel: https://github.com/richardsheridan/trio-parallel From 1deed8dbee168c352ad2641712b837ee2283eb04 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 7 Jul 2023 14:49:23 -0400 Subject: [PATCH 05/30] ._runtime: log level tweaks, use crit for stale debug lock detection --- tractor/_runtime.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 6c2e028f..dcc4c97d 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -632,7 +632,7 @@ class Actor: and not db_cs.cancel_called and uid == pdb_user_uid ): - log.warning( + log.critical( f'STALE DEBUG LOCK DETECTED FOR {uid}' ) # TODO: figure out why this breaks tests.. From ab8bd9b787931a23fb1826e645c7ca581dcd473a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 7 Jul 2023 14:51:44 -0400 Subject: [PATCH 06/30] Fix frame-selection display on first REPL entry For whatever reason pdb(p), and in general, will show the frame of the *next* python instruction/LOC on initial entry (at least using `.set_trace()`), as such remove the `try/finally` block in the sync code entrypoint `.pause_from_sync()`, and also since doesn't seem like we really need it anyway. Further, and to this end: - enable hidden frames support in our default config. - fix/drop/mask all the frame ref-ing/mangling we had prior since it's no longer needed as well as manual `Lock` releasing which seems to work already by having the `greenback` spawned task do it's normal thing? - move to no `Union` type annots. - hide all frames that can add "this is the runtime confusion" to traces. --- tractor/_debug.py | 132 +++++++++++++++++++++++++--------------------- 1 file changed, 72 insertions(+), 60 deletions(-) diff --git a/tractor/_debug.py b/tractor/_debug.py index eec6fc50..d5f5f4f1 100644 --- a/tractor/_debug.py +++ b/tractor/_debug.py @@ -30,7 +30,6 @@ from functools import ( from contextlib import asynccontextmanager as acm from typing import ( Any, - Optional, Callable, AsyncIterator, AsyncGenerator, @@ -40,7 +39,10 @@ from types import FrameType import pdbp import tractor import trio -from trio_typing import TaskStatus +from trio_typing import ( + TaskStatus, + # Task, +) from .log import get_logger from ._discovery import get_root @@ -69,10 +71,10 @@ class Lock: ''' repl: MultiActorPdb | None = None # placeholder for function to set a ``trio.Event`` on debugger exit - # pdb_release_hook: Optional[Callable] = None + # pdb_release_hook: Callable | None = None _trio_handler: Callable[ - [int, Optional[FrameType]], Any + [int, FrameType | None], Any ] | int | None = None # actor-wide variable pointing to current task name using debugger @@ -83,23 +85,23 @@ class Lock: # and must be cancelled if this actor is cancelled via IPC # request-message otherwise deadlocks with the parent actor may # ensure - _debugger_request_cs: Optional[trio.CancelScope] = None + _debugger_request_cs: trio.CancelScope | 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: Optional[trio.CancelScope] = None + _root_local_task_cs_in_debug: trio.CancelScope | None = None # actor tree-wide actor uid that supposedly has the tty lock - global_actor_in_debug: Optional[tuple[str, str]] = None + global_actor_in_debug: tuple[str, str] = None - local_pdb_complete: Optional[trio.Event] = None - no_remote_has_tty: Optional[trio.Event] = 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() - _orig_sigint_handler: Optional[Callable] = None + _orig_sigint_handler: Callable | None = None _blocked: set[tuple[str, str]] = set() @classmethod @@ -110,6 +112,7 @@ class Lock: ) @classmethod + @pdbp.hideframe # XXX NOTE XXX see below in `.pause_from_sync()` def unshield_sigint(cls): # always restore ``trio``'s sigint handler. see notes below in # the pdb factory about the nightmare that is that code swapping @@ -129,10 +132,6 @@ class Lock: if owner: raise - # actor-local state, irrelevant for non-root. - cls.global_actor_in_debug = None - cls.local_task_in_debug = None - try: # sometimes the ``trio`` might already be terminated in # which case this call will raise. @@ -143,6 +142,11 @@ class Lock: cls.unshield_sigint() cls.repl = None + # actor-local state, irrelevant for non-root. + cls.global_actor_in_debug = None + cls.local_task_in_debug = None + + class TractorConfig(pdbp.DefaultConfig): ''' @@ -151,7 +155,7 @@ class TractorConfig(pdbp.DefaultConfig): ''' use_pygments: bool = True sticky_by_default: bool = False - enable_hidden_frames: bool = False + enable_hidden_frames: bool = True # much thanks @mdmintz for the hot tip! # fixes line spacing issue when resizing terminal B) @@ -228,26 +232,23 @@ async def _acquire_debug_lock_from_root_task( to the ``pdb`` repl. ''' - task_name = trio.lowlevel.current_task().name + task_name: str = trio.lowlevel.current_task().name + we_acquired: bool = False log.runtime( f"Attempting to acquire TTY lock, remote task: {task_name}:{uid}" ) - - we_acquired = False - try: log.runtime( f"entering lock checkpoint, remote task: {task_name}:{uid}" ) - we_acquired = True - # NOTE: if the surrounding cancel scope from the # `lock_tty_for_child()` caller is cancelled, this line should # unblock and NOT leave us in some kind of # a "child-locked-TTY-but-child-is-uncontactable-over-IPC" # condition. await Lock._debug_lock.acquire() + we_acquired = True if Lock.no_remote_has_tty is None: # mark the tty lock as being in use so that the runtime @@ -573,13 +574,15 @@ async def _pause( try: # breakpoint() if debug_func is None: - assert release_lock_signal, ( - 'Must pass `release_lock_signal: trio.Event` if no ' - 'trace func provided!' - ) + # assert release_lock_signal, ( + # 'Must pass `release_lock_signal: trio.Event` if no ' + # 'trace func provided!' + # ) print(f"{actor.uid} ENTERING WAIT") task_status.started() - await release_lock_signal.wait() + + # with trio.CancelScope(shield=True): + # await release_lock_signal.wait() else: # block here one (at the appropriate frame *up*) where @@ -606,7 +609,7 @@ async def _pause( def shield_sigint_handler( signum: int, frame: 'frame', # type: ignore # noqa - # pdb_obj: Optional[MultiActorPdb] = None, + # pdb_obj: MultiActorPdb | None = None, *args, ) -> None: @@ -620,7 +623,7 @@ def shield_sigint_handler( ''' __tracebackhide__ = True - uid_in_debug = Lock.global_actor_in_debug + uid_in_debug: tuple[str, str] | None = Lock.global_actor_in_debug actor = tractor.current_actor() # print(f'{actor.uid} in HANDLER with ') @@ -638,14 +641,14 @@ def shield_sigint_handler( else: raise KeyboardInterrupt - any_connected = False + 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 = 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: @@ -658,7 +661,7 @@ def shield_sigint_handler( return do_cancel() # only set in the actor actually running the REPL - pdb_obj = Lock.repl + pdb_obj: MultiActorPdb | None = Lock.repl # root actor branch that reports whether or not a child # has locked debugger. @@ -716,7 +719,7 @@ def shield_sigint_handler( ) return do_cancel() - task = Lock.local_task_in_debug + task: str | None = Lock.local_task_in_debug if ( task and pdb_obj @@ -791,15 +794,18 @@ def _set_trace( Lock.local_task_in_debug = 'sync' pdb.set_trace(frame=frame) + # undo_ # TODO: allow pausing from sync code, normally by remapping # python's builtin breakpoint() hook to this runtime aware version. def pause_from_sync() -> None: + print("ENTER SYNC PAUSE") import greenback + __tracebackhide__ = True actor: tractor.Actor = tractor.current_actor() - task_can_release_tty_lock = trio.Event() + # task_can_release_tty_lock = trio.Event() # spawn bg task which will lock out the TTY, we poll # just below until the release event is reporting that task as @@ -808,34 +814,39 @@ def pause_from_sync() -> None: actor._service_n.start(partial( _pause, debug_func=None, - release_lock_signal=task_can_release_tty_lock, + # release_lock_signal=task_can_release_tty_lock, )) ) - print("ENTER SYNC PAUSE") - pdb, undo_sigint = mk_mpdb() - try: - print("ENTER SYNC PAUSE") - # _set_trace(actor=actor) - # we entered the global ``breakpoint()`` built-in from sync - # code? - Lock.local_task_in_debug = 'sync' - frame: FrameType | None = sys._getframe() - print(f'FRAME: {str(frame)}') + db, undo_sigint = mk_mpdb() + Lock.local_task_in_debug = 'sync' + # db.config.enable_hidden_frames = True - frame: FrameType = frame.f_back # type: ignore - print(f'FRAME: {str(frame)}') + # we entered the global ``breakpoint()`` built-in from sync + # code? + frame: FrameType | None = sys._getframe() + # print(f'FRAME: {str(frame)}') + # assert not db._is_hidden(frame) - frame: FrameType = frame.f_back # type: ignore - print(f'FRAME: {str(frame)}') + frame: FrameType = frame.f_back # type: ignore + # print(f'FRAME: {str(frame)}') + # if not db._is_hidden(frame): + # pdbp.set_trace() + # db._hidden_frames.append( + # (frame, frame.f_lineno) + # ) + db.set_trace(frame=frame) + # NOTE XXX: see the `@pdbp.hideframe` decoration + # on `Lock.unshield_sigint()`.. I have NO CLUE why + # the next instruction's def frame is being shown + # in the tb but it seems to be something wonky with + # the way `pdb` core works? + # undo_sigint() - pdb.set_trace(frame=frame) - # pdb.do_frame( - # pdb.curindex + # Lock.global_actor_in_debug = actor.uid + # Lock.release() + # task_can_release_tty_lock.set() - finally: - task_can_release_tty_lock.set() - undo_sigint() # using the "pause" semantics instead since # that better covers actually somewhat "pausing the runtime" @@ -959,8 +970,7 @@ async def maybe_wait_for_debugger( # will make the pdb repl unusable. # Instead try to wait for pdb to be released before # tearing down. - - sub_in_debug = None + sub_in_debug: tuple[str, str] | None = None for _ in range(poll_steps): @@ -980,13 +990,15 @@ async def maybe_wait_for_debugger( debug_complete = Lock.no_remote_has_tty if ( - (debug_complete and - not debug_complete.is_set()) + debug_complete + and sub_in_debug is not None + and not debug_complete.is_set() ): - log.debug( + log.pdb( 'Root has errored but pdb is in use by ' f'child {sub_in_debug}\n' - 'Waiting on tty lock to release..') + 'Waiting on tty lock to release..' + ) await debug_complete.wait() From 6de4a5a9f3e956d6f95a5fedecdae55e279819e5 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 7 Jul 2023 15:35:52 -0400 Subject: [PATCH 07/30] Map `breakpoint()` built-in to new `.pause_from_sync()` ep --- tractor/_root.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tractor/_root.py b/tractor/_root.py index 881dc90f..8c15f706 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -90,7 +90,7 @@ async def open_root_actor( # https://github.com/python-trio/trio/issues/1155#issuecomment-742964018 builtin_bp_handler = sys.breakpointhook orig_bp_path: str | None = os.environ.get('PYTHONBREAKPOINT', None) - os.environ['PYTHONBREAKPOINT'] = 'tractor._debug._set_trace' + os.environ['PYTHONBREAKPOINT'] = 'tractor._debug.pause_from_sync' # attempt to retreive ``trio``'s sigint handler and stash it # on our debugger lock state. @@ -237,10 +237,10 @@ async def open_root_actor( ) as err: entered = await _debug._maybe_enter_pm(err) - if ( not entered - and not is_multi_cancelled(err) + and + not is_multi_cancelled(err) ): logger.exception('Root actor crashed:\n') From fa5f458de00263ac0d40d3fccbebe39220995b17 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 21 Jul 2023 15:08:46 -0400 Subject: [PATCH 08/30] Move `.to_asyncio` to modern optional value type annots --- tractor/to_asyncio.py | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index 174a99d3..7c88edd2 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -28,7 +28,6 @@ from typing import ( Callable, AsyncIterator, Awaitable, - Optional, ) import trio @@ -65,9 +64,9 @@ class LinkedTaskChannel(trio.abc.Channel): _trio_exited: bool = False # set after ``asyncio.create_task()`` - _aio_task: Optional[asyncio.Task] = None - _aio_err: Optional[BaseException] = None - _broadcaster: Optional[BroadcastReceiver] = None + _aio_task: asyncio.Task | None = None + _aio_err: BaseException | None = None + _broadcaster: BroadcastReceiver | None = None async def aclose(self) -> None: await self._from_aio.aclose() @@ -188,7 +187,7 @@ def _run_asyncio_task( cancel_scope = trio.CancelScope() aio_task_complete = trio.Event() - aio_err: Optional[BaseException] = None + aio_err: BaseException | None = None chan = LinkedTaskChannel( aio_q, # asyncio.Queue @@ -270,7 +269,7 @@ def _run_asyncio_task( ''' nonlocal chan aio_err = chan._aio_err - task_err: Optional[BaseException] = None + task_err: BaseException | None = None # only to avoid ``asyncio`` complaining about uncaptured # task exceptions @@ -350,11 +349,11 @@ async def translate_aio_errors( ''' trio_task = trio.lowlevel.current_task() - aio_err: Optional[BaseException] = None + aio_err: BaseException | None = None # TODO: make thisi a channel method? def maybe_raise_aio_err( - err: Optional[Exception] = None + err: Exception | None = None ) -> None: aio_err = chan._aio_err if ( From 93f489e263238e0982fc9dcc5f53cf286a0d3758 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 18 Aug 2023 10:18:25 -0400 Subject: [PATCH 09/30] Expose `Channel` @ pkg level, drop `_debug.pp()` alias --- tractor/__init__.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tractor/__init__.py b/tractor/__init__.py index ad71db1e..b6adba07 100644 --- a/tractor/__init__.py +++ b/tractor/__init__.py @@ -19,7 +19,6 @@ tractor: structured concurrent ``trio``-"actors". """ from ._clustering import open_actor_cluster -from ._ipc import Channel from ._context import ( Context, # the type context, # a func-decorator @@ -47,7 +46,6 @@ from ._exceptions import ( from ._debug import ( breakpoint, pause, - pp, pause_from_sync, post_mortem, ) @@ -56,6 +54,7 @@ from ._root import ( run_daemon, open_root_actor, ) +from ._ipc import Channel from ._portal import Portal from ._runtime import Actor @@ -74,6 +73,7 @@ __all__ = [ 'context', 'current_actor', 'find_actor', + 'query_actor', 'get_arbiter', 'is_root_process', 'msg', @@ -82,8 +82,7 @@ __all__ = [ 'open_root_actor', 'pause', 'post_mortem', - 'pp', - 'pause_from_sync' + 'pause_from_sync', 'query_actor', 'run_daemon', 'stream', From b00ba158f1927414052d0c79c1fa66be43b8814c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 28 Sep 2023 14:14:50 -0400 Subject: [PATCH 10/30] Kick off `.devx` subpkg for our dev tools B) Where `.devx` is "developer experience", a hopefully broad enough subpkg name for all the slick stuff planned to augment working on the actor runtime :boom: Move the `._debug` module into the new subpkg and adjust rest of core code base to reflect import path change. Also add a new `.devx._debug.open_crash_handler()` manager for wrapping any sync code outside a `trio.run()` which is handy for eventual CLI addons for popular frameworks like `click`/`typer`. --- tractor/__init__.py | 2 +- tractor/_context.py | 3 +++ tractor/_root.py | 6 ++--- tractor/_runtime.py | 4 +-- tractor/_spawn.py | 2 +- tractor/_supervise.py | 2 +- tractor/devx/__init__.py | 45 +++++++++++++++++++++++++++++++++ tractor/{ => devx}/_debug.py | 49 ++++++++++++++++++++++++++++++------ 8 files changed, 97 insertions(+), 16 deletions(-) create mode 100644 tractor/devx/__init__.py rename tractor/{ => devx}/_debug.py (96%) diff --git a/tractor/__init__.py b/tractor/__init__.py index b6adba07..5c16bc4e 100644 --- a/tractor/__init__.py +++ b/tractor/__init__.py @@ -43,7 +43,7 @@ from ._exceptions import ( ModuleNotExposed, ContextCancelled, ) -from ._debug import ( +from .devx import ( breakpoint, pause, pause_from_sync, diff --git a/tractor/_context.py b/tractor/_context.py index 50f7bfa5..0be54860 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -868,6 +868,9 @@ class Context: # 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() # TODO: add to `Channel`? @property diff --git a/tractor/_root.py b/tractor/_root.py index 8c15f706..d81cae78 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -38,7 +38,7 @@ from ._runtime import ( # Arbiter as Registry, async_main, ) -from . import _debug +from .devx import _debug from . import _spawn from . import _state from . import log @@ -90,7 +90,7 @@ async def open_root_actor( # https://github.com/python-trio/trio/issues/1155#issuecomment-742964018 builtin_bp_handler = sys.breakpointhook orig_bp_path: str | None = os.environ.get('PYTHONBREAKPOINT', None) - os.environ['PYTHONBREAKPOINT'] = 'tractor._debug.pause_from_sync' + os.environ['PYTHONBREAKPOINT'] = 'tractor.devx._debug.pause_from_sync' # attempt to retreive ``trio``'s sigint handler and stash it # on our debugger lock state. @@ -138,7 +138,7 @@ async def open_root_actor( # expose internal debug module to every actor allowing # for use of ``await tractor.breakpoint()`` - enable_modules.append('tractor._debug') + enable_modules.append('tractor.devx._debug') # if debug mode get's enabled *at least* use that level of # logging for some informative console prompts. diff --git a/tractor/_runtime.py b/tractor/_runtime.py index dcc4c97d..6b3d9461 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -78,7 +78,7 @@ from ._exceptions import ( TransportClosed, ) from ._discovery import get_arbiter -from . import _debug +from .devx import _debug from ._portal import Portal from . import _state from . import _mp_fixup_main @@ -187,7 +187,7 @@ class Actor: self._parent_main_data = _mp_fixup_main._mp_figure_out_main() # always include debugging tools module - enable_modules.append('tractor._debug') + enable_modules.append('tractor.devx._debug') mods = {} for name in enable_modules: diff --git a/tractor/_spawn.py b/tractor/_spawn.py index e91638bc..9f2c5713 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -34,7 +34,7 @@ from typing import ( import trio from trio import TaskStatus -from ._debug import ( +from .devx._debug import ( maybe_wait_for_debugger, acquire_debug_lock, ) diff --git a/tractor/_supervise.py b/tractor/_supervise.py index 1e5ea387..c8c2336d 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -31,7 +31,7 @@ import warnings import trio -from ._debug import maybe_wait_for_debugger +from .devx._debug import maybe_wait_for_debugger from ._state import current_actor, is_main_process from .log import get_logger, get_loglevel from ._runtime import Actor diff --git a/tractor/devx/__init__.py b/tractor/devx/__init__.py new file mode 100644 index 00000000..e24405a0 --- /dev/null +++ b/tractor/devx/__init__.py @@ -0,0 +1,45 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +""" +Runtime "developer experience" utils and addons to aid our +(advanced) users and core devs in building distributed applications +and working with/on the actor runtime. + +""" +from ._debug import ( + maybe_wait_for_debugger, + acquire_debug_lock, + breakpoint, + pause, + pause_from_sync, + shield_sigint_handler, + MultiActorPdb, + open_crash_handler, + post_mortem, +) + +__all__ = [ + 'maybe_wait_for_debugger', + 'acquire_debug_lock', + 'breakpoint', + 'pause', + 'pause_from_sync', + 'shield_sigint_handler', + 'MultiActorPdb', + 'open_crash_handler', + 'post_mortem', +] diff --git a/tractor/_debug.py b/tractor/devx/_debug.py similarity index 96% rename from tractor/_debug.py rename to tractor/devx/_debug.py index d5f5f4f1..6575c223 100644 --- a/tractor/_debug.py +++ b/tractor/devx/_debug.py @@ -28,6 +28,7 @@ from functools import ( cached_property, ) from contextlib import asynccontextmanager as acm +from contextlib import contextmanager as cm from typing import ( Any, Callable, @@ -44,22 +45,25 @@ from trio_typing import ( # Task, ) -from .log import get_logger -from ._discovery import get_root -from ._state import ( +from ..log import get_logger +from .._discovery import get_root +from .._state import ( is_root_process, debug_mode, ) -from ._exceptions import ( +from .._exceptions import ( is_multi_cancelled, ContextCancelled, ) -from ._ipc import Channel +from .._ipc import Channel log = get_logger(__name__) -__all__ = ['breakpoint', 'post_mortem'] +__all__ = [ + 'breakpoint', + 'post_mortem', +] class Lock: @@ -390,7 +394,7 @@ async def wait_for_parent_stdin_hijack( # this syncs to child's ``Context.started()`` call. async with portal.open_context( - tractor._debug.lock_tty_for_child, + lock_tty_for_child, subactor_uid=actor_uid, ) as (ctx, val): @@ -855,7 +859,7 @@ pause = partial( _pause, _set_trace, ) -pp = pause # short-hand for "pause point" +# pp = pause # short-hand for "pause point" async def breakpoint(**kwargs): @@ -1008,3 +1012,32 @@ async def maybe_wait_for_debugger( log.debug( 'Root acquired TTY LOCK' ) + + +# TODO: better naming and what additionals? +# - optional runtime plugging? +# - detection for sync vs. async code? +# - specialized REPL entry when in distributed mode? +@cm +def open_crash_handler( + catch: set[BaseException] = { + Exception, + BaseException, + } +): + ''' + Generic "post mortem" crash handler using `pdbp` REPL debugger. + + We expose this as a CLI framework addon to both `click` and + `typer` users so they can quickly wrap cmd endpoints which get + automatically wrapped to use the runtime's `debug_mode: bool` + AND `pdbp.pm()` around any code that is PRE-runtime entry + - any sync code which runs BEFORE the main call to + `trio.run()`. + + ''' + try: + yield + except tuple(catch): + pdbp.xpm() + raise From ab0c0fb71d91dfac410601bc6cf3d2ca14b17b4b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 28 Sep 2023 15:36:24 -0400 Subject: [PATCH 11/30] Start `.devx.cli` extensions for pop CLI frameworks Starting of with just a `typer` (and thus transitively `click`) `typer.Typer.callback` hook which allows passthrough of the `--ll ` and `--pdb ` flags for use when building CLIs that use the runtime Bo Still needs lotsa refinement and obviously better docs but, the doc string for `load_runtime_vars()` shows how to use the underlying `.devx._debug.open_crash_handler()` via a wrapper that can be passed the `--pdb` flag and then enable debug mode throughout the entire actor system. --- tractor/devx/_debug.py | 6 +- tractor/devx/cli.py | 149 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 153 insertions(+), 2 deletions(-) create mode 100644 tractor/devx/cli.py diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 6575c223..eef5c843 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -27,8 +27,10 @@ from functools import ( partial, cached_property, ) -from contextlib import asynccontextmanager as acm -from contextlib import contextmanager as cm +from contextlib import ( + asynccontextmanager as acm, + contextmanager as cm, +) from typing import ( Any, Callable, diff --git a/tractor/devx/cli.py b/tractor/devx/cli.py new file mode 100644 index 00000000..353389da --- /dev/null +++ b/tractor/devx/cli.py @@ -0,0 +1,149 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +""" +CLI framework extensions for hacking on the actor runtime. + +Currently popular frameworks supported are: + + - `typer` via the `@callback` API + +""" +from __future__ import annotations +from contextlib import ( + # asynccontextmanager as acm, + nullcontext, + contextmanager as cm, +) +from typing import ( + Any, + Callable, +) +from typing_extensions import Annotated + +import typer + + +from ._debug import open_crash_handler + + +_runtime_vars: dict[str, Any] = {} + + +def load_runtime_vars( + ctx: typer.Context, + callback: Callable, + pdb: bool = False, # --pdb + ll: Annotated[ + str, + typer.Option( + '--loglevel', + '-l', + help='BigD logging level', + ), + ] = 'cancel', # -l info +): + ''' + Maybe engage crash handling with `pdbp` when code inside + a `typer` CLI endpoint cmd raises. + + To use this callback simply take your `app = typer.Typer()` instance + and decorate this function with it like so: + + .. code:: python + + from tractor.devx import cli + + app = typer.Typer() + + # manual decoration to hook into `click`'s context system! + cli.load_runtime_vars = app.callback( + invoke_without_command=True, + ) + + And then you can use the now augmented `click` CLI context as so, + + .. code:: python + + @app.command( + context_settings={ + "allow_extra_args": True, + "ignore_unknown_options": True, + } + ) + def my_cli_cmd( + ctx: typer.Context, + ): + rtvars: dict = ctx.runtime_vars + pdb: bool = rtvars['pdb'] + + with tractor.devx.cli.maybe_open_crash_handler(pdb=pdb): + trio.run( + partial( + my_tractor_main_task_func, + debug_mode=pdb, + loglevel=rtvars['ll'], + ) + ) + + which will enable log level and debug mode globally for the entire + `tractor` + `trio` runtime thereafter! + + Bo + + ''' + global _runtime_vars + _runtime_vars |= { + 'pdb': pdb, + 'll': ll, + } + + ctx.runtime_vars: dict[str, Any] = _runtime_vars + print( + f'`typer` sub-cmd: {ctx.invoked_subcommand}\n' + f'`tractor` runtime vars: {_runtime_vars}' + ) + + # XXX NOTE XXX: hackzone.. if no sub-cmd is specified (the + # default if the user just invokes `bigd`) then we simply + # invoke the sole `_bigd()` cmd passing in the "parent" + # typer.Context directly to that call since we're treating it + # as a "non sub-command" or wtv.. + # TODO: ideally typer would have some kinda built-in way to get + # this behaviour without having to construct and manually + # invoke our own cmd.. + if ( + ctx.invoked_subcommand is None + or ctx.invoked_subcommand == callback.__name__ + ): + cmd: typer.core.TyperCommand = typer.core.TyperCommand( + name='bigd', + callback=callback, + ) + ctx.params = {'ctx': ctx} + cmd.invoke(ctx) + + +@cm +def maybe_open_crash_handler(pdb: bool = False): + # if the --pdb flag is passed always engage + # the pdb REPL on any crashes B) + rtctx = nullcontext + if pdb: + rtctx = open_crash_handler + + with rtctx(): + yield From cca4f952ed5fd6ea6011bce53086b6f86e2bc84f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 2 Oct 2023 18:10:34 -0400 Subject: [PATCH 12/30] Move `maybe_open_crash_handler()` CLI `--pdb`-driven wrapper to debug mod --- tractor/devx/__init__.py | 2 ++ tractor/devx/_debug.py | 18 ++++++++++++++++++ tractor/devx/cli.py | 13 ------------- 3 files changed, 20 insertions(+), 13 deletions(-) diff --git a/tractor/devx/__init__.py b/tractor/devx/__init__.py index e24405a0..89b9a336 100644 --- a/tractor/devx/__init__.py +++ b/tractor/devx/__init__.py @@ -29,6 +29,7 @@ from ._debug import ( shield_sigint_handler, MultiActorPdb, open_crash_handler, + maybe_open_crash_handler, post_mortem, ) @@ -41,5 +42,6 @@ __all__ = [ 'shield_sigint_handler', 'MultiActorPdb', 'open_crash_handler', + 'maybe_open_crash_handler', 'post_mortem', ] diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index eef5c843..24baba06 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -30,6 +30,7 @@ from functools import ( from contextlib import ( asynccontextmanager as acm, contextmanager as cm, + nullcontext, ) from typing import ( Any, @@ -1043,3 +1044,20 @@ def open_crash_handler( except tuple(catch): pdbp.xpm() raise + + +@cm +def maybe_open_crash_handler(pdb: bool = False): + ''' + Same as `open_crash_handler()` but with bool input flag + to allow conditional handling. + + Normally this is used with CLI endpoints such that if the --pdb + flag is passed the pdb REPL is engaed on any crashes B) + ''' + rtctx = nullcontext + if pdb: + rtctx = open_crash_handler + + with rtctx(): + yield diff --git a/tractor/devx/cli.py b/tractor/devx/cli.py index 353389da..76890669 100644 --- a/tractor/devx/cli.py +++ b/tractor/devx/cli.py @@ -25,7 +25,6 @@ Currently popular frameworks supported are: from __future__ import annotations from contextlib import ( # asynccontextmanager as acm, - nullcontext, contextmanager as cm, ) from typing import ( @@ -135,15 +134,3 @@ def load_runtime_vars( ) ctx.params = {'ctx': ctx} cmd.invoke(ctx) - - -@cm -def maybe_open_crash_handler(pdb: bool = False): - # if the --pdb flag is passed always engage - # the pdb REPL on any crashes B) - rtctx = nullcontext - if pdb: - rtctx = open_crash_handler - - with rtctx(): - yield From 5912fecdc9c8bbbe6a05e4588cb0a156498a0948 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 6 Oct 2023 15:49:23 -0400 Subject: [PATCH 13/30] Add shielding support to `.pause()` Implement it like you'd expect using simply a wrapping `trio.CancelScope` which is itself shielded by the input `shield: bool` B) There's seemingly still some issues with the frame selection when the REPL engages and not sure how to resolve it yet but at least this does indeed work for practical purposes. Still needs a test obviously! --- tractor/devx/_debug.py | 361 +++++++++++++++++++++-------------------- 1 file changed, 186 insertions(+), 175 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 24baba06..d3ad1bd4 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -382,7 +382,7 @@ async def wait_for_parent_stdin_hijack( 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 + (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()``). @@ -448,171 +448,6 @@ def mk_mpdb() -> tuple[MultiActorPdb, Callable]: return pdb, Lock.unshield_sigint -async def _pause( - - debug_func: Callable | None = None, - release_lock_signal: trio.Event | None = None, - - # TODO: - # shield: bool = False - task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED - -) -> None: - ''' - A pause point (more commonly known as a "breakpoint") interrupt - instruction for engaging a blocking debugger instance to - conduct manual console-based-REPL-interaction from within - `tractor`'s async runtime, normally from some single-threaded - and currently executing actor-hosted-`trio`-task in some - (remote) process. - - NOTE: we use the semantics "pause" since it better encompasses - the entirety of the necessary global-runtime-state-mutation any - actor-task must access and lock in order to get full isolated - control over the process tree's root TTY: - https://en.wikipedia.org/wiki/Breakpoint - - ''' - __tracebackhide__ = True - actor = tractor.current_actor() - pdb, undo_sigint = mk_mpdb() - task_name = trio.lowlevel.current_task().name - - # TODO: is it possible to debug a trio.Cancelled except block? - # right now it seems like we can kinda do with by shielding - # around ``tractor.breakpoint()`` but not if we move the shielded - # scope here??? - # with trio.CancelScope(shield=shield): - # await trio.lowlevel.checkpoint() - - if ( - not Lock.local_pdb_complete - or Lock.local_pdb_complete.is_set() - ): - Lock.local_pdb_complete = trio.Event() - - # TODO: need a more robust check for the "root" actor - if ( - not is_root_process() - and actor._parent_chan # a connected child - ): - - if Lock.local_task_in_debug: - - # Recurrence entry case: this task already has the lock and - # is likely recurrently entering a breakpoint - if Lock.local_task_in_debug == task_name: - # noop on recurrent entry case but we want to trigger - # a checkpoint to allow other actors error-propagate and - # potetially avoid infinite re-entries in some subactor. - await trio.lowlevel.checkpoint() - return - - # if **this** actor is already in debug mode block here - # waiting for the control to be released - this allows - # support for recursive entries to `tractor.breakpoint()` - log.warning(f"{actor.uid} already has a debug lock, waiting...") - - await Lock.local_pdb_complete.wait() - await trio.sleep(0.1) - - # mark local actor as "in debug mode" to avoid recurrent - # entries/requests to the root process - Lock.local_task_in_debug = task_name - - # this **must** be awaited by the caller and is done using the - # root nursery so that the debugger can continue to run without - # being restricted by the scope of a new task nursery. - - # TODO: if we want to debug a trio.Cancelled triggered exception - # we have to figure out how to avoid having the service nursery - # cancel on this task start? I *think* this works below: - # ```python - # actor._service_n.cancel_scope.shield = shield - # ``` - # but not entirely sure if that's a sane way to implement it? - try: - with trio.CancelScope(shield=True): - await actor._service_n.start( - wait_for_parent_stdin_hijack, - actor.uid, - ) - Lock.repl = pdb - except RuntimeError: - Lock.release() - - if actor._cancel_called: - # service nursery won't be usable and we - # don't want to lock up the root either way since - # we're in (the midst of) cancellation. - return - - raise - - elif is_root_process(): - - # we also wait in the root-parent for any child that - # may have the tty locked prior - # TODO: wait, what about multiple root tasks acquiring it though? - if Lock.global_actor_in_debug == actor.uid: - # re-entrant root process already has it: noop. - return - - # XXX: since we need to enter pdb synchronously below, - # we have to release the lock manually from pdb completion - # callbacks. Can't think of a nicer way then this atm. - if Lock._debug_lock.locked(): - log.warning( - 'Root actor attempting to shield-acquire active tty lock' - f' owned by {Lock.global_actor_in_debug}') - - # must shield here to avoid hitting a ``Cancelled`` and - # a child getting stuck bc we clobbered the tty - with trio.CancelScope(shield=True): - await Lock._debug_lock.acquire() - else: - # may be cancelled - await Lock._debug_lock.acquire() - - Lock.global_actor_in_debug = actor.uid - Lock.local_task_in_debug = task_name - Lock.repl = pdb - - try: - # breakpoint() - if debug_func is None: - # assert release_lock_signal, ( - # 'Must pass `release_lock_signal: trio.Event` if no ' - # 'trace func provided!' - # ) - print(f"{actor.uid} ENTERING WAIT") - task_status.started() - - # with trio.CancelScope(shield=True): - # await release_lock_signal.wait() - - else: - # block here one (at the appropriate frame *up*) where - # ``breakpoint()`` was awaited and begin handling stdio. - log.debug("Entering the synchronous world of pdb") - debug_func(actor, pdb) - - except bdb.BdbQuit: - Lock.release() - raise - - # XXX: apparently we can't do this without showing this frame - # in the backtrace on first entry to the REPL? Seems like an odd - # behaviour that should have been fixed by now. This is also why - # we scrapped all the @cm approaches that were tried previously. - # finally: - # __tracebackhide__ = True - # # frame = sys._getframe() - # # last_f = frame.f_back - # # last_f.f_globals['__tracebackhide__'] = True - # # signal.signal = pdbp.hideframe(signal.signal) - - def shield_sigint_handler( signum: int, frame: 'frame', # type: ignore # noqa @@ -774,6 +609,7 @@ def shield_sigint_handler( def _set_trace( actor: tractor.Actor | None = None, pdb: MultiActorPdb | None = None, + shield: bool = False, ): __tracebackhide__ = True actor: tractor.Actor = actor or tractor.current_actor() @@ -785,14 +621,20 @@ def _set_trace( if ( frame - and pdb - and actor is not None + and ( + pdb + and actor is not None + ) or shield ): + # pdbp.set_trace() log.pdb(f"\nAttaching pdb to actor: {actor.uid}\n") # no f!#$&* idea, but when we're in async land # we need 2x frames up? frame = frame.f_back + # if shield: + # frame = frame.f_back + else: pdb, undo_sigint = mk_mpdb() @@ -804,8 +646,181 @@ def _set_trace( # undo_ -# TODO: allow pausing from sync code, normally by remapping -# python's builtin breakpoint() hook to this runtime aware version. + +async def pause( + + debug_func: Callable = _set_trace, + release_lock_signal: trio.Event | None = None, + + # allow caller to pause despite task cancellation, + # exactly the same as wrapping with: + # with CancelScope(shield=True): + # await pause() + shield: bool = False, + + # TODO: + # shield: bool = False + task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED + +) -> None: + ''' + A pause point (more commonly known as a "breakpoint") interrupt + instruction for engaging a blocking debugger instance to + conduct manual console-based-REPL-interaction from within + `tractor`'s async runtime, normally from some single-threaded + and currently executing actor-hosted-`trio`-task in some + (remote) process. + + NOTE: we use the semantics "pause" since it better encompasses + the entirety of the necessary global-runtime-state-mutation any + actor-task must access and lock in order to get full isolated + control over the process tree's root TTY: + https://en.wikipedia.org/wiki/Breakpoint + + ''' + __tracebackhide__ = True + actor = tractor.current_actor() + pdb, undo_sigint = mk_mpdb() + task_name = trio.lowlevel.current_task().name + + if ( + not Lock.local_pdb_complete + or Lock.local_pdb_complete.is_set() + ): + Lock.local_pdb_complete = trio.Event() + + if shield: + debug_func = partial( + debug_func, + shield=shield, + ) + + with trio.CancelScope(shield=shield): + + # TODO: need a more robust check for the "root" actor + if ( + not is_root_process() + and actor._parent_chan # a connected child + ): + + if Lock.local_task_in_debug: + + # Recurrence entry case: this task already has the lock and + # is likely recurrently entering a breakpoint + if Lock.local_task_in_debug == task_name: + # noop on recurrent entry case but we want to trigger + # a checkpoint to allow other actors error-propagate and + # potetially avoid infinite re-entries in some subactor. + await trio.lowlevel.checkpoint() + return + + # if **this** actor is already in debug mode block here + # waiting for the control to be released - this allows + # support for recursive entries to `tractor.breakpoint()` + log.warning(f"{actor.uid} already has a debug lock, waiting...") + + await Lock.local_pdb_complete.wait() + await trio.sleep(0.1) + + # mark local actor as "in debug mode" to avoid recurrent + # entries/requests to the root process + Lock.local_task_in_debug = task_name + + # this **must** be awaited by the caller and is done using the + # root nursery so that the debugger can continue to run without + # being restricted by the scope of a new task nursery. + + # TODO: if we want to debug a trio.Cancelled triggered exception + # we have to figure out how to avoid having the service nursery + # cancel on this task start? I *think* this works below: + # ```python + # actor._service_n.cancel_scope.shield = shield + # ``` + # but not entirely sure if that's a sane way to implement it? + try: + with trio.CancelScope(shield=True): + await actor._service_n.start( + wait_for_parent_stdin_hijack, + actor.uid, + ) + Lock.repl = pdb + except RuntimeError: + Lock.release() + + if actor._cancel_called: + # service nursery won't be usable and we + # don't want to lock up the root either way since + # we're in (the midst of) cancellation. + return + + raise + + elif is_root_process(): + + # we also wait in the root-parent for any child that + # may have the tty locked prior + # TODO: wait, what about multiple root tasks acquiring it though? + if Lock.global_actor_in_debug == actor.uid: + # re-entrant root process already has it: noop. + return + + # XXX: since we need to enter pdb synchronously below, + # we have to release the lock manually from pdb completion + # callbacks. Can't think of a nicer way then this atm. + if Lock._debug_lock.locked(): + log.warning( + 'Root actor attempting to shield-acquire active tty lock' + f' owned by {Lock.global_actor_in_debug}') + + # must shield here to avoid hitting a ``Cancelled`` and + # a child getting stuck bc we clobbered the tty + with trio.CancelScope(shield=True): + await Lock._debug_lock.acquire() + else: + # may be cancelled + await Lock._debug_lock.acquire() + + Lock.global_actor_in_debug = actor.uid + Lock.local_task_in_debug = task_name + Lock.repl = pdb + + try: + if debug_func is None: + # assert release_lock_signal, ( + # 'Must pass `release_lock_signal: trio.Event` if no ' + # 'trace func provided!' + # ) + print(f"{actor.uid} ENTERING WAIT") + task_status.started() + + # with trio.CancelScope(shield=True): + # await release_lock_signal.wait() + + else: + # block here one (at the appropriate frame *up*) where + # ``breakpoint()`` was awaited and begin handling stdio. + log.debug("Entering the synchronous world of pdb") + debug_func(actor, pdb) + + except bdb.BdbQuit: + Lock.release() + raise + + # XXX: apparently we can't do this without showing this frame + # in the backtrace on first entry to the REPL? Seems like an odd + # behaviour that should have been fixed by now. This is also why + # we scrapped all the @cm approaches that were tried previously. + # finally: + # __tracebackhide__ = True + # # frame = sys._getframe() + # # last_f = frame.f_back + # # last_f.f_globals['__tracebackhide__'] = True + # # signal.signal = pdbp.hideframe(signal.signal) + + +# TODO: allow pausing from sync code. +# normally by remapping python's builtin breakpoint() hook to this +# runtime aware version which takes care of all . def pause_from_sync() -> None: print("ENTER SYNC PAUSE") import greenback @@ -858,10 +873,6 @@ def pause_from_sync() -> None: # using the "pause" semantics instead since # that better covers actually somewhat "pausing the runtime" # for this particular paralell task to do debugging B) -pause = partial( - _pause, - _set_trace, -) # pp = pause # short-hand for "pause point" @@ -895,7 +906,7 @@ def _post_mortem( post_mortem = partial( - _pause, + pause, _post_mortem, ) From 5042f1fdb898a1e8f86d50196fa8de193ac427e0 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 10 Oct 2023 09:55:11 -0400 Subject: [PATCH 14/30] Comment all `.pause(shield=True)` attempts again, need to solve cancel scope `.__exit__()` frame hiding issue.. --- tractor/devx/_debug.py | 244 +++++++++++++++++++++-------------------- 1 file changed, 128 insertions(+), 116 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index d3ad1bd4..561c387c 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -631,6 +631,7 @@ def _set_trace( # no f!#$&* idea, but when we're in async land # we need 2x frames up? frame = frame.f_back + # frame = frame.f_back # if shield: # frame = frame.f_back @@ -646,17 +647,19 @@ def _set_trace( # undo_ - async def pause( debug_func: Callable = _set_trace, release_lock_signal: trio.Event | None = None, - # allow caller to pause despite task cancellation, + # TODO: allow caller to pause despite task cancellation, # exactly the same as wrapping with: # with CancelScope(shield=True): # await pause() - shield: bool = False, + # => the REMAINING ISSUE is that the scope's .__exit__() frame + # is always show in the debugger on entry.. and there seems to + # be no way to override it?.. + # shield: bool = False, # TODO: # shield: bool = False @@ -689,133 +692,142 @@ async def pause( ): Lock.local_pdb_complete = trio.Event() - if shield: - debug_func = partial( - debug_func, - shield=shield, - ) + # if shield: + debug_func = partial( + debug_func, + # shield=shield, + ) - with trio.CancelScope(shield=shield): + # def _exit(self, *args, **kwargs): + # __tracebackhide__: bool = True + # super().__exit__(*args, **kwargs) + + # trio.CancelScope.__exit__.__tracebackhide__ = True + + # import types + # with trio.CancelScope(shield=shield) as cs: + # cs.__exit__ = types.MethodType(_exit, cs) + # cs.__exit__.__tracebackhide__ = True # TODO: need a more robust check for the "root" actor - if ( - not is_root_process() - and actor._parent_chan # a connected child - ): + if ( + not is_root_process() + and actor._parent_chan # a connected child + ): - if Lock.local_task_in_debug: + if Lock.local_task_in_debug: - # Recurrence entry case: this task already has the lock and - # is likely recurrently entering a breakpoint - if Lock.local_task_in_debug == task_name: - # noop on recurrent entry case but we want to trigger - # a checkpoint to allow other actors error-propagate and - # potetially avoid infinite re-entries in some subactor. - await trio.lowlevel.checkpoint() - return - - # if **this** actor is already in debug mode block here - # waiting for the control to be released - this allows - # support for recursive entries to `tractor.breakpoint()` - log.warning(f"{actor.uid} already has a debug lock, waiting...") - - await Lock.local_pdb_complete.wait() - await trio.sleep(0.1) - - # mark local actor as "in debug mode" to avoid recurrent - # entries/requests to the root process - Lock.local_task_in_debug = task_name - - # this **must** be awaited by the caller and is done using the - # root nursery so that the debugger can continue to run without - # being restricted by the scope of a new task nursery. - - # TODO: if we want to debug a trio.Cancelled triggered exception - # we have to figure out how to avoid having the service nursery - # cancel on this task start? I *think* this works below: - # ```python - # actor._service_n.cancel_scope.shield = shield - # ``` - # but not entirely sure if that's a sane way to implement it? - try: - with trio.CancelScope(shield=True): - await actor._service_n.start( - wait_for_parent_stdin_hijack, - actor.uid, - ) - Lock.repl = pdb - except RuntimeError: - Lock.release() - - if actor._cancel_called: - # service nursery won't be usable and we - # don't want to lock up the root either way since - # we're in (the midst of) cancellation. - return - - raise - - elif is_root_process(): - - # we also wait in the root-parent for any child that - # may have the tty locked prior - # TODO: wait, what about multiple root tasks acquiring it though? - if Lock.global_actor_in_debug == actor.uid: - # re-entrant root process already has it: noop. + # 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 - # 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}') + # 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...") - # 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() + await Lock.local_pdb_complete.wait() + await trio.sleep(0.1) - Lock.global_actor_in_debug = actor.uid - Lock.local_task_in_debug = task_name - Lock.repl = pdb + # mark local actor as "in debug mode" to avoid recurrent + # entries/requests to the root process + Lock.local_task_in_debug = task_name + # this **must** be awaited by the caller and is done using the + # root nursery so that the debugger can continue to run without + # being restricted by the scope of a new task nursery. + + # TODO: if we want to debug a trio.Cancelled triggered exception + # we have to figure out how to avoid having the service nursery + # cancel on this task start? I *think* this works below: + # ```python + # actor._service_n.cancel_scope.shield = shield + # ``` + # but not entirely sure if that's a sane way to implement it? try: - if debug_func is None: - # assert release_lock_signal, ( - # 'Must pass `release_lock_signal: trio.Event` if no ' - # 'trace func provided!' - # ) - print(f"{actor.uid} ENTERING WAIT") - task_status.started() - - # with trio.CancelScope(shield=True): - # await release_lock_signal.wait() - - else: - # block here one (at the appropriate frame *up*) where - # ``breakpoint()`` was awaited and begin handling stdio. - log.debug("Entering the synchronous world of pdb") - debug_func(actor, pdb) - - except bdb.BdbQuit: + with trio.CancelScope(shield=True): + await actor._service_n.start( + wait_for_parent_stdin_hijack, + actor.uid, + ) + Lock.repl = pdb + except RuntimeError: Lock.release() + + if actor._cancel_called: + # service nursery won't be usable and we + # don't want to lock up the root either way since + # we're in (the midst of) cancellation. + return + raise - # 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) + 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: + if debug_func is None: + # assert release_lock_signal, ( + # 'Must pass `release_lock_signal: trio.Event` if no ' + # 'trace func provided!' + # ) + print(f"{actor.uid} ENTERING WAIT") + task_status.started() + + # with trio.CancelScope(shield=True): + # await release_lock_signal.wait() + + else: + # block here one (at the appropriate frame *up*) where + # ``breakpoint()`` was awaited and begin handling stdio. + log.debug("Entering the synchronous world of pdb") + debug_func(actor, pdb) + + except bdb.BdbQuit: + Lock.release() + raise + + # XXX: apparently we can't do this without showing this frame + # in the backtrace on first entry to the REPL? Seems like an odd + # behaviour that should have been fixed by now. This is also why + # we scrapped all the @cm approaches that were tried previously. + # finally: + # __tracebackhide__ = True + # # frame = sys._getframe() + # # last_f = frame.f_back + # # last_f.f_globals['__tracebackhide__'] = True + # # signal.signal = pdbp.hideframe(signal.signal) # TODO: allow pausing from sync code. From cebc2cb515e26d8bf061d853d6644bc082bab7f4 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 16 Oct 2023 15:45:34 -0400 Subject: [PATCH 15/30] Ignore kbis in `open_crash_handler()` by default --- tractor/devx/_debug.py | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 561c387c..06e6071b 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -1041,15 +1041,19 @@ async def maybe_wait_for_debugger( # TODO: better naming and what additionals? -# - optional runtime plugging? -# - detection for sync vs. async code? -# - specialized REPL entry when in distributed mode? +# - [ ] optional runtime plugging? +# - [ ] detection for sync vs. async code? +# - [ ] specialized REPL entry when in distributed mode? +# - [x] allow ignoring kbi Bo @cm def open_crash_handler( catch: set[BaseException] = { Exception, BaseException, - } + }, + ignore: set[BaseException] = { + KeyboardInterrupt, + }, ): ''' Generic "post mortem" crash handler using `pdbp` REPL debugger. @@ -1064,8 +1068,11 @@ def open_crash_handler( ''' try: yield - except tuple(catch): - pdbp.xpm() + except tuple(catch) as err: + + if type(err) not in ignore: + pdbp.xpm() + raise From f5c35dca558e93578e0b0e4c48ff22fc459a7ead Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 17 Oct 2023 16:52:31 -0400 Subject: [PATCH 16/30] Runtime import `.get_root()` in stdin hijacker to avoid import cycle --- tractor/devx/_debug.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 06e6071b..e636e49e 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -49,7 +49,6 @@ from trio_typing import ( ) from ..log import get_logger -from .._discovery import get_root from .._state import ( is_root_process, debug_mode, @@ -331,7 +330,7 @@ async def lock_tty_for_child( f'Actor {subactor_uid} is blocked from acquiring debug lock\n' f"remote task: {task_name}:{subactor_uid}" ) - ctx._enter_debugger_on_cancel = False + ctx._enter_debugger_on_cancel: bool = False await ctx.cancel(f'Debug lock blocked for {subactor_uid}') return 'pdb_lock_blocked' @@ -388,6 +387,8 @@ async def wait_for_parent_stdin_hijack( ``maybe_wait_for_debugger()``). ''' + from .._discovery import get_root + with trio.CancelScope(shield=True) as cs: Lock._debugger_request_cs = cs @@ -611,7 +612,7 @@ def _set_trace( pdb: MultiActorPdb | None = None, shield: bool = False, ): - __tracebackhide__ = True + __tracebackhide__: bool = True actor: tractor.Actor = actor or tractor.current_actor() # start 2 levels up in user code From ba9448d52f1ab2085966fec9a3e9bb63bc3c574e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 18 Oct 2023 15:01:04 -0400 Subject: [PATCH 17/30] Change old `._debug._pause()` name, cherry to #362 re `greenback` --- tractor/devx/_debug.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index e636e49e..1b225052 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -847,7 +847,7 @@ def pause_from_sync() -> None: # waiting.. not the most ideal but works for now ;) greenback.await_( actor._service_n.start(partial( - _pause, + pause, debug_func=None, # release_lock_signal=task_can_release_tty_lock, )) From 8647421ef95c80e331569f26598e2b2c1904f444 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 19 Oct 2023 12:41:15 -0400 Subject: [PATCH 18/30] Ignore `greenback` import error if not installed --- tractor/devx/_debug.py | 31 +++++++++++++++++-------------- 1 file changed, 17 insertions(+), 14 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 1b225052..3bef7bd6 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -836,22 +836,25 @@ async def pause( # runtime aware version which takes care of all . def pause_from_sync() -> None: print("ENTER SYNC PAUSE") - import greenback - __tracebackhide__ = True + try: + import greenback + __tracebackhide__ = True - actor: tractor.Actor = tractor.current_actor() - # task_can_release_tty_lock = trio.Event() + actor: tractor.Actor = tractor.current_actor() + # task_can_release_tty_lock = trio.Event() - # spawn bg task which will lock out the TTY, we poll - # just below until the release event is reporting that task as - # waiting.. not the most ideal but works for now ;) - greenback.await_( - actor._service_n.start(partial( - pause, - debug_func=None, - # release_lock_signal=task_can_release_tty_lock, - )) - ) + # spawn bg task which will lock out the TTY, we poll + # just below until the release event is reporting that task as + # waiting.. not the most ideal but works for now ;) + greenback.await_( + actor._service_n.start(partial( + pause, + debug_func=None, + # release_lock_signal=task_can_release_tty_lock, + )) + ) + except ModuleNotFoundError: + log.warning('NO GREENBACK FOUND') db, undo_sigint = mk_mpdb() Lock.local_task_in_debug = 'sync' From 5b3bcbaa7d1532708cd0645f85a67fe36569f41f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 2 Jan 2024 11:28:02 -0500 Subject: [PATCH 19/30] Only use `greenback` if actor-runtime is up.. --- tractor/devx/_debug.py | 43 ++++++++++++++++++++++++------------------ 1 file changed, 25 insertions(+), 18 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 3bef7bd6..f3550ba6 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -682,7 +682,7 @@ async def pause( https://en.wikipedia.org/wiki/Breakpoint ''' - __tracebackhide__ = True + # __tracebackhide__ = True actor = tractor.current_actor() pdb, undo_sigint = mk_mpdb() task_name = trio.lowlevel.current_task().name @@ -836,25 +836,32 @@ async def pause( # runtime aware version which takes care of all . def pause_from_sync() -> None: print("ENTER SYNC PAUSE") - try: - import greenback - __tracebackhide__ = True + actor: tractor.Actor = tractor.current_actor( + err_on_no_runtime=False, + ) + if actor: + try: + import greenback + # __tracebackhide__ = True - actor: tractor.Actor = tractor.current_actor() - # task_can_release_tty_lock = trio.Event() - # spawn bg task which will lock out the TTY, we poll - # just below until the release event is reporting that task as - # waiting.. not the most ideal but works for now ;) - greenback.await_( - actor._service_n.start(partial( - pause, - debug_func=None, - # release_lock_signal=task_can_release_tty_lock, - )) - ) - except ModuleNotFoundError: - log.warning('NO GREENBACK FOUND') + # task_can_release_tty_lock = trio.Event() + + # spawn bg task which will lock out the TTY, we poll + # just below until the release event is reporting that task as + # waiting.. not the most ideal but works for now ;) + greenback.await_( + actor._service_n.start(partial( + pause, + debug_func=None, + # release_lock_signal=task_can_release_tty_lock, + )) + ) + + except ModuleNotFoundError: + log.warning('NO GREENBACK FOUND') + else: + log.warning('Not inside actor-runtime') db, undo_sigint = mk_mpdb() Lock.local_task_in_debug = 'sync' From 199ca48cc4a605b82172016bc63507565f5eafc1 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 20 Feb 2024 08:59:21 -0500 Subject: [PATCH 20/30] Add `stackscope` tree pprinter triggered by SIGUSR1 Can be optionally enabled via a new `enable_stack_on_sig()` which will swap in the SIGUSR1 handler. Much thanks to @oremanj for writing this amazing project, it's thus far helped me fix some very subtle hangs inside our new IPC-context cancellation machinery that would have otherwise taken much more manual pdb-ing and hair pulling XD Full credit for `dump_task_tree()` goes to the original project author with some minor tweaks as was handed to me via the trio-general matrix room B) Slight changes from orig version: - use a `log.pdb()` emission to pprint to console - toss in an ex sh CLI cmd to trigger the dump from another terminal using `kill` + `pgrep`. --- tractor/devx/__init__.py | 3 ++ tractor/devx/_stackscope.py | 84 +++++++++++++++++++++++++++++++++++++ 2 files changed, 87 insertions(+) create mode 100644 tractor/devx/_stackscope.py diff --git a/tractor/devx/__init__.py b/tractor/devx/__init__.py index 89b9a336..5f832615 100644 --- a/tractor/devx/__init__.py +++ b/tractor/devx/__init__.py @@ -32,6 +32,9 @@ from ._debug import ( maybe_open_crash_handler, post_mortem, ) +from ._stackscope import ( + enable_stack_on_sig as enable_stack_on_sig, +) __all__ = [ 'maybe_wait_for_debugger', diff --git a/tractor/devx/_stackscope.py b/tractor/devx/_stackscope.py new file mode 100644 index 00000000..706b85d3 --- /dev/null +++ b/tractor/devx/_stackscope.py @@ -0,0 +1,84 @@ +# tractor: structured concurrent "actors". +# Copyright eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +''' +The fundamental cross process SC abstraction: an inter-actor, +cancel-scope linked task "context". + +A ``Context`` is very similar to the ``trio.Nursery.cancel_scope`` built +into each ``trio.Nursery`` except it links the lifetimes of memory space +disjoint, parallel executing tasks in separate actors. + +''' +from signal import ( + signal, + SIGUSR1, +) + +import trio + +@trio.lowlevel.disable_ki_protection +def dump_task_tree() -> None: + import stackscope + from tractor.log import get_console_log + + tree_str: str = str( + stackscope.extract( + trio.lowlevel.current_root_task(), + recurse_child_tasks=True + ) + ) + log = get_console_log('cancel') + log.pdb( + f'Dumping `stackscope` tree:\n\n' + f'{tree_str}\n' + ) + # import logging + # try: + # with open("/dev/tty", "w") as tty: + # tty.write(tree_str) + # except BaseException: + # logging.getLogger( + # "task_tree" + # ).exception("Error printing task tree") + + +def signal_handler(sig: int, frame: object) -> None: + import traceback + try: + trio.lowlevel.current_trio_token( + ).run_sync_soon(dump_task_tree) + except RuntimeError: + # not in async context -- print a normal traceback + traceback.print_stack() + + + +def enable_stack_on_sig( + sig: int = SIGUSR1 +) -> None: + ''' + Enable `stackscope` tracing on reception of a signal; by + default this is SIGUSR1. + + ''' + signal( + sig, + signal_handler, + ) + # NOTE: not the above can be triggered from + # a (xonsh) shell using: + # kill -SIGUSR1 @$(pgrep -f '') From 1fb4d7318b8bba79f05e2211a053202f489f9c44 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 20 Feb 2024 15:39:45 -0500 Subject: [PATCH 21/30] Fix `.devx.maybe_wait_for_debugger()` polling deats When entered by the root actor avoid excessive polling cycles by, - blocking on the `Lock.no_remote_has_tty: trio.Event` and breaking *immediately* when set (though we should really also lock it from the root right?) to avoid extra loops.. - shielding the `await trio.sleep(poll_delay)` call to avoid any local cancellation causing the (presumably root-actor task) caller to move on (possibly to cancel its children) and instead to continue poll-blocking until the lock is actually released by its user. - `break` the poll loop immediately if no remote locker is detected. - use `.pdb()` level for reporting lock state changes. Also add a #TODO to handle calls by non-root actors as it pertains to --- tractor/devx/_debug.py | 139 +++++++++++++++++++++++++---------------- 1 file changed, 86 insertions(+), 53 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index f3550ba6..43fd9018 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -1,18 +1,19 @@ # 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 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. +# 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 . +# You should have received a copy of the GNU Affero General Public +# License along with this program. If not, see +# . """ Multi-core debugging for da peeps! @@ -43,6 +44,7 @@ from types import FrameType import pdbp import tractor import trio +from trio.lowlevel import current_task from trio_typing import ( TaskStatus, # Task, @@ -50,6 +52,7 @@ from trio_typing import ( from ..log import get_logger from .._state import ( + current_actor, is_root_process, debug_mode, ) @@ -238,7 +241,7 @@ async def _acquire_debug_lock_from_root_task( to the ``pdb`` repl. ''' - task_name: str = trio.lowlevel.current_task().name + task_name: str = current_task().name we_acquired: bool = False log.runtime( @@ -323,8 +326,7 @@ async def lock_tty_for_child( highly reliable at releasing the mutex complete! ''' - task_name = trio.lowlevel.current_task().name - + 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' @@ -407,11 +409,13 @@ async def wait_for_parent_stdin_hijack( assert val == 'Locked' async with ctx.open_stream() as stream: - # unblock local caller - try: + # unblock local caller assert Lock.local_pdb_complete task_status.started(cs) + + # wait for local task to exit and + # release the REPL await Lock.local_pdb_complete.wait() finally: @@ -468,7 +472,7 @@ def shield_sigint_handler( uid_in_debug: tuple[str, str] | None = Lock.global_actor_in_debug - actor = tractor.current_actor() + actor = current_actor() # print(f'{actor.uid} in HANDLER with ') def do_cancel(): @@ -613,7 +617,7 @@ def _set_trace( shield: bool = False, ): __tracebackhide__: bool = True - actor: tractor.Actor = actor or tractor.current_actor() + actor: tractor.Actor = actor or current_actor() # start 2 levels up in user code frame: FrameType | None = sys._getframe() @@ -683,9 +687,9 @@ async def pause( ''' # __tracebackhide__ = True - actor = tractor.current_actor() + actor = current_actor() pdb, undo_sigint = mk_mpdb() - task_name = trio.lowlevel.current_task().name + task_name: str = trio.lowlevel.current_task().name if ( not Lock.local_pdb_complete @@ -836,7 +840,7 @@ async def pause( # runtime aware version which takes care of all . def pause_from_sync() -> None: print("ENTER SYNC PAUSE") - actor: tractor.Actor = tractor.current_actor( + actor: tractor.Actor = current_actor( err_on_no_runtime=False, ) if actor: @@ -971,9 +975,10 @@ async def acquire_debug_lock( ''' Grab root's debug lock on entry, release on exit. - This helper is for actor's who don't actually need - to acquired the debugger but want to wait until the - lock is free in the process-tree root. + This helper is for actor's who don't actually need to acquired + the debugger but want to wait until the lock is free in the + process-tree root such that they don't clobber an ongoing pdb + REPL session in some peer or child! ''' if not debug_mode(): @@ -1013,43 +1018,71 @@ async def maybe_wait_for_debugger( # tearing down. sub_in_debug: tuple[str, str] | None = None - for _ in range(poll_steps): + for istep in range(poll_steps): - if Lock.global_actor_in_debug: - sub_in_debug = tuple(Lock.global_actor_in_debug) - - log.debug('Root polling for debug') - - with trio.CancelScope(shield=True): - await trio.sleep(poll_delay) - - # TODO: could this make things more deterministic? wait - # to see if a sub-actor task will be scheduled and grab - # the tty lock on the next tick? - # XXX: doesn't seem to work + if sub_in_debug := Lock.global_actor_in_debug: + log.pdb( + f'Lock in use by {sub_in_debug}' + ) + # TODO: could this make things more deterministic? + # wait to see if a sub-actor task will be + # scheduled and grab the tty lock on the next + # tick? + # XXX => but it doesn't seem to work.. # await trio.testing.wait_all_tasks_blocked(cushion=0) - debug_complete = Lock.no_remote_has_tty - if ( - debug_complete - and sub_in_debug is not None - and not debug_complete.is_set() - ): - log.pdb( - 'Root has errored but pdb is in use by ' - f'child {sub_in_debug}\n' - 'Waiting on tty lock to release..' - ) + debug_complete: trio.Event|None = Lock.no_remote_has_tty + if ( + debug_complete + and not debug_complete.is_set() + and sub_in_debug is not None + ): + log.pdb( + 'Root has errored but pdb is in use by child\n' + 'Waiting on tty lock to release..\n' + f'uid: {sub_in_debug}\n' + ) + await debug_complete.wait() + log.pdb( + f'Child subactor released debug lock!\n' + f'uid: {sub_in_debug}\n' + ) + if debug_complete.is_set(): + break - await debug_complete.wait() + # is no subactor locking debugger currently? + elif ( + debug_complete is None + or sub_in_debug is None + ): + log.pdb( + 'Root acquired debug TTY LOCK from child\n' + f'uid: {sub_in_debug}' + ) + break - await trio.sleep(poll_delay) - continue + else: + # TODO: don't need this right? + # await trio.lowlevel.checkpoint() + + log.debug( + 'Root polling for debug:\n' + f'poll step: {istep}\n' + f'poll delya: {poll_delay}' + ) + with trio.CancelScope(shield=True): + await trio.sleep(poll_delay) + continue else: - log.debug( - 'Root acquired TTY LOCK' - ) + log.pdb('Root acquired debug TTY LOCK') + # else: + # # TODO: non-root call for #320? + # this_uid: tuple[str, str] = current_actor().uid + # async with acquire_debug_lock( + # subactor_uid=this_uid, + # ): + # pass # TODO: better naming and what additionals? # - [ ] optional runtime plugging? From 526add2cae4a2749344b09a60364ab297f6ceef6 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 22 Feb 2024 15:08:10 -0500 Subject: [PATCH 22/30] Support `maybe_wait_for_debugger(header_msg: str)` Allow callers to stick in a header to the `.pdb()` level emitted msg(s) such that any "waiting status" content is only shown if the caller actually get's blocked waiting for the debug lock; use it inside the `._spawn` sub-process reaper call. Also, return early if `Lock.global_actor_in_debug == None` and thus only enter the poll loop when actually needed, consequently raise if we fall through the loop without acquisition. --- tractor/_spawn.py | 7 +++-- tractor/devx/_debug.py | 71 +++++++++++++++++++++++++++--------------- 2 files changed, 50 insertions(+), 28 deletions(-) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index 9f2c5713..78c38c84 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -551,13 +551,14 @@ async def trio_proc( with trio.move_on_after(0.5): await proc.wait() - log.pdb( - 'Delaying subproc reaper while debugger locked..' - ) await maybe_wait_for_debugger( child_in_debug=_runtime_vars.get( '_debug_mode', False ), + header_msg=( + 'Delaying subproc reaper while debugger locked..\n' + ), + # TODO: need a diff value then default? # poll_steps=9999999, ) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 43fd9018..d3bf4fe0 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -999,6 +999,8 @@ async def maybe_wait_for_debugger( poll_delay: float = 0.1, child_in_debug: bool = False, + header_msg: str = '', + ) -> None: if ( @@ -1007,6 +1009,8 @@ async def maybe_wait_for_debugger( ): return + + msg: str = header_msg if ( is_root_process() ): @@ -1016,48 +1020,59 @@ async def maybe_wait_for_debugger( # will make the pdb repl unusable. # Instead try to wait for pdb to be released before # tearing down. - sub_in_debug: tuple[str, str] | None = None + sub_in_debug: tuple[str, str]|None = Lock.global_actor_in_debug + debug_complete: trio.Event|None = Lock.no_remote_has_tty + + if sub_in_debug := Lock.global_actor_in_debug: + msg += ( + 'Debug `Lock` in use by subactor\n' + f'|_{sub_in_debug}\n' + ) + # TODO: could this make things more deterministic? + # wait to see if a sub-actor task will be + # scheduled and grab the tty lock on the next + # tick? + # XXX => but it doesn't seem to work.. + # await trio.testing.wait_all_tasks_blocked(cushion=0) + else: + log.pdb( + msg + + + 'Root immediately acquired debug TTY LOCK' + ) + return for istep in range(poll_steps): - if sub_in_debug := Lock.global_actor_in_debug: - log.pdb( - f'Lock in use by {sub_in_debug}' - ) - # TODO: could this make things more deterministic? - # wait to see if a sub-actor task will be - # scheduled and grab the tty lock on the next - # tick? - # XXX => but it doesn't seem to work.. - # await trio.testing.wait_all_tasks_blocked(cushion=0) - debug_complete: trio.Event|None = Lock.no_remote_has_tty if ( debug_complete and not debug_complete.is_set() and sub_in_debug is not None ): log.pdb( - 'Root has errored but pdb is in use by child\n' - 'Waiting on tty lock to release..\n' - f'uid: {sub_in_debug}\n' + msg + + + 'Root is waiting on tty lock to release..\n' ) await debug_complete.wait() log.pdb( - f'Child subactor released debug lock!\n' - f'uid: {sub_in_debug}\n' + f'Child subactor released debug lock:' + f'|_{sub_in_debug}\n' ) - if debug_complete.is_set(): - break # is no subactor locking debugger currently? - elif ( - debug_complete is None - or sub_in_debug is None + if ( + sub_in_debug is None + and ( + debug_complete is None + or debug_complete.is_set() + ) ): log.pdb( - 'Root acquired debug TTY LOCK from child\n' - f'uid: {sub_in_debug}' + msg + + + 'Root acquired tty lock!' ) break @@ -1073,8 +1088,14 @@ async def maybe_wait_for_debugger( with trio.CancelScope(shield=True): await trio.sleep(poll_delay) continue + + # fallthrough on failure to acquire.. else: - log.pdb('Root acquired debug TTY LOCK') + raise RuntimeError( + msg + + + 'Root actor failed to acquire debug lock?' + ) # else: # # TODO: non-root call for #320? From e3bb9c914cb0560f1b35d6de6d01a037f4312e1e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 5 Mar 2024 11:43:23 -0500 Subject: [PATCH 23/30] Mk debugger tests work for arbitrary pre-REPL format Since this was changed as part of overall project wide logging format updates, and i ended up changing the both the crash and pause `.pdb()` msgs to include some multi-line-ascii-"stuff", might as well make the pre-prompt checks in the test suite more flexible to match. As such, this exposes 2 new constants inside the `.devx._debug` mod: - `._pause_msg: str` for the pre `tractor.pause()` header emitted via `log.pdb()` and, - `._crash_msg: str` for the pre `._post_mortem()` equiv when handling errors in debug mode. Adjust the test suite to use these values and thus make us more capable to absorb changes in the future as well: - add a new `in_prompt_msg()` predicate, very similar to `assert_before()` but minus `assert`s which takes in a `parts: list[str]` to match in the pre-prompt stdout. - delegate to `in_prompt_msg()` in `assert_before()` since it was mostly duplicate minus `assert`. - adjust all previous ` in before` asserts to instead use `in_prompt_msg()` with separated pre-prompt-header vs. actor-name `parts`. - use new `._pause/crash_msg` values in all such calls including any `assert_before()` cases. --- tests/test_debugger.py | 182 +++++++++++++++++++++++++++++---------- tractor/devx/__init__.py | 33 +++---- tractor/devx/_debug.py | 80 +++++++++++------ tractor/devx/cli.py | 7 -- 4 files changed, 200 insertions(+), 102 deletions(-) diff --git a/tests/test_debugger.py b/tests/test_debugger.py index 889e7c74..a10ecad9 100644 --- a/tests/test_debugger.py +++ b/tests/test_debugger.py @@ -10,6 +10,7 @@ TODO: - wonder if any of it'll work on OS X? """ +from functools import partial import itertools from typing import Optional import platform @@ -26,6 +27,10 @@ from pexpect.exceptions import ( from tractor._testing import ( examples_dir, ) +from tractor.devx._debug import ( + _pause_msg, + _crash_msg, +) from .conftest import ( _ci_env, ) @@ -123,20 +128,52 @@ def expect( raise +def in_prompt_msg( + prompt: str, + parts: list[str], + + pause_on_false: bool = False, + print_prompt_on_false: bool = True, + +) -> bool: + ''' + Predicate check if (the prompt's) std-streams output has all + `str`-parts in it. + + Can be used in test asserts for bulk matching expected + log/REPL output for a given `pdb` interact point. + + ''' + for part in parts: + if part not in prompt: + + if pause_on_false: + import pdbp + pdbp.set_trace() + + if print_prompt_on_false: + print(prompt) + + return False + + return True + def assert_before( child, patts: list[str], + **kwargs, + ) -> None: - before = str(child.before.decode()) + # as in before the prompt end + before: str = str(child.before.decode()) + assert in_prompt_msg( + prompt=before, + parts=patts, - for patt in patts: - try: - assert patt in before - except AssertionError: - print(before) - raise + **kwargs + ) @pytest.fixture( @@ -195,7 +232,10 @@ def test_root_actor_error(spawn, user_in_out): before = str(child.before.decode()) # make sure expected logging and error arrives - assert "Attaching to pdb in crashed actor: ('root'" in before + assert in_prompt_msg( + before, + [_crash_msg, "('root'"] + ) assert 'AssertionError' in before # send user command @@ -332,7 +372,10 @@ def test_subactor_error( child.expect(PROMPT) before = str(child.before.decode()) - assert "Attaching to pdb in crashed actor: ('name_error'" in before + assert in_prompt_msg( + before, + [_crash_msg, "('name_error'"] + ) if do_next: child.sendline('n') @@ -353,9 +396,15 @@ def test_subactor_error( before = str(child.before.decode()) # root actor gets debugger engaged - assert "Attaching to pdb in crashed actor: ('root'" in before + assert in_prompt_msg( + before, + [_crash_msg, "('root'"] + ) # error is a remote error propagated from the subactor - assert "RemoteActorError: ('name_error'" in before + assert in_prompt_msg( + before, + [_crash_msg, "('name_error'"] + ) # another round if ctlc: @@ -380,7 +429,10 @@ def test_subactor_breakpoint( child.expect(PROMPT) before = str(child.before.decode()) - assert "Attaching pdb to actor: ('breakpoint_forever'" in before + assert in_prompt_msg( + before, + [_pause_msg, "('breakpoint_forever'"] + ) # do some "next" commands to demonstrate recurrent breakpoint # entries @@ -396,7 +448,10 @@ def test_subactor_breakpoint( child.sendline('continue') child.expect(PROMPT) before = str(child.before.decode()) - assert "Attaching pdb to actor: ('breakpoint_forever'" in before + assert in_prompt_msg( + before, + [_pause_msg, "('breakpoint_forever'"] + ) if ctlc: do_ctlc(child) @@ -441,7 +496,10 @@ def test_multi_subactors( child.expect(PROMPT) before = str(child.before.decode()) - assert "Attaching pdb to actor: ('breakpoint_forever'" in before + assert in_prompt_msg( + before, + [_pause_msg, "('breakpoint_forever'"] + ) if ctlc: do_ctlc(child) @@ -461,7 +519,10 @@ def test_multi_subactors( # first name_error failure child.expect(PROMPT) before = str(child.before.decode()) - assert "Attaching to pdb in crashed actor: ('name_error'" in before + assert in_prompt_msg( + before, + [_crash_msg, "('name_error'"] + ) assert "NameError" in before if ctlc: @@ -487,7 +548,10 @@ def test_multi_subactors( child.sendline('c') child.expect(PROMPT) before = str(child.before.decode()) - assert "Attaching pdb to actor: ('breakpoint_forever'" in before + assert in_prompt_msg( + before, + [_pause_msg, "('breakpoint_forever'"] + ) if ctlc: do_ctlc(child) @@ -527,17 +591,21 @@ def test_multi_subactors( child.expect(PROMPT) before = str(child.before.decode()) - assert_before(child, [ - # debugger attaches to root - "Attaching to pdb in crashed actor: ('root'", + assert_before( + child, [ + # debugger attaches to root + # "Attaching to pdb in crashed actor: ('root'", + _crash_msg, + "('root'", - # expect a multierror with exceptions for each sub-actor - "RemoteActorError: ('breakpoint_forever'", - "RemoteActorError: ('name_error'", - "RemoteActorError: ('spawn_error'", - "RemoteActorError: ('name_error_1'", - 'bdb.BdbQuit', - ]) + # expect a multierror with exceptions for each sub-actor + "RemoteActorError: ('breakpoint_forever'", + "RemoteActorError: ('name_error'", + "RemoteActorError: ('spawn_error'", + "RemoteActorError: ('name_error_1'", + 'bdb.BdbQuit', + ] + ) if ctlc: do_ctlc(child) @@ -574,15 +642,22 @@ def test_multi_daemon_subactors( # the root's tty lock first so anticipate either crash # message on the first entry. - bp_forever_msg = "Attaching pdb to actor: ('bp_forever'" + bp_forev_parts = [_pause_msg, "('bp_forever'"] + bp_forev_in_msg = partial( + in_prompt_msg, + parts=bp_forev_parts, + ) + name_error_msg = "NameError: name 'doggypants' is not defined" + name_error_parts = [name_error_msg] before = str(child.before.decode()) - if bp_forever_msg in before: - next_msg = name_error_msg + + if bp_forev_in_msg(prompt=before): + next_parts = name_error_parts elif name_error_msg in before: - next_msg = bp_forever_msg + next_parts = bp_forev_parts else: raise ValueError("Neither log msg was found !?") @@ -599,7 +674,10 @@ def test_multi_daemon_subactors( child.sendline('c') child.expect(PROMPT) - assert_before(child, [next_msg]) + assert_before( + child, + next_parts, + ) # XXX: hooray the root clobbering the child here was fixed! # IMO, this demonstrates the true power of SC system design. @@ -623,9 +701,15 @@ def test_multi_daemon_subactors( child.expect(PROMPT) try: - assert_before(child, [bp_forever_msg]) + assert_before( + child, + bp_forev_parts, + ) except AssertionError: - assert_before(child, [name_error_msg]) + assert_before( + child, + name_error_parts, + ) else: if ctlc: @@ -637,7 +721,10 @@ def test_multi_daemon_subactors( child.sendline('c') child.expect(PROMPT) - assert_before(child, [name_error_msg]) + assert_before( + child, + name_error_parts, + ) # wait for final error in root # where it crashs with boxed error @@ -647,7 +734,7 @@ def test_multi_daemon_subactors( child.expect(PROMPT) assert_before( child, - [bp_forever_msg] + bp_forev_parts ) except AssertionError: break @@ -656,7 +743,9 @@ def test_multi_daemon_subactors( child, [ # boxed error raised in root task - "Attaching to pdb in crashed actor: ('root'", + # "Attaching to pdb in crashed actor: ('root'", + _crash_msg, + "('root'", "_exceptions.RemoteActorError: ('name_error'", ] ) @@ -770,7 +859,7 @@ def test_multi_nested_subactors_error_through_nurseries( child = spawn('multi_nested_subactors_error_up_through_nurseries') - timed_out_early: bool = False + # timed_out_early: bool = False for send_char in itertools.cycle(['c', 'q']): try: @@ -871,11 +960,14 @@ def test_root_nursery_cancels_before_child_releases_tty_lock( if not timed_out_early: before = str(child.before.decode()) - assert_before(child, [ - "tractor._exceptions.RemoteActorError: ('spawner0'", - "tractor._exceptions.RemoteActorError: ('name_error'", - "NameError: name 'doggypants' is not defined", - ]) + assert_before( + child, + [ + "tractor._exceptions.RemoteActorError: ('spawner0'", + "tractor._exceptions.RemoteActorError: ('name_error'", + "NameError: name 'doggypants' is not defined", + ], + ) def test_root_cancels_child_context_during_startup( @@ -909,8 +1001,10 @@ def test_different_debug_mode_per_actor( # only one actor should enter the debugger before = str(child.before.decode()) - assert "Attaching to pdb in crashed actor: ('debugged_boi'" in before - assert "RuntimeError" in before + assert in_prompt_msg( + before, + [_crash_msg, "('debugged_boi'", "RuntimeError"], + ) if ctlc: do_ctlc(child) diff --git a/tractor/devx/__init__.py b/tractor/devx/__init__.py index 5f832615..c4676e3f 100644 --- a/tractor/devx/__init__.py +++ b/tractor/devx/__init__.py @@ -21,30 +21,17 @@ and working with/on the actor runtime. """ from ._debug import ( - maybe_wait_for_debugger, - acquire_debug_lock, - breakpoint, - pause, - pause_from_sync, - shield_sigint_handler, - MultiActorPdb, - open_crash_handler, - maybe_open_crash_handler, - post_mortem, + maybe_wait_for_debugger as maybe_wait_for_debugger, + acquire_debug_lock as acquire_debug_lock, + breakpoint as breakpoint, + pause as pause, + pause_from_sync as pause_from_sync, + shield_sigint_handler as shield_sigint_handler, + MultiActorPdb as MultiActorPdb, + open_crash_handler as open_crash_handler, + maybe_open_crash_handler as maybe_open_crash_handler, + post_mortem as post_mortem, ) from ._stackscope import ( enable_stack_on_sig as enable_stack_on_sig, ) - -__all__ = [ - 'maybe_wait_for_debugger', - 'acquire_debug_lock', - 'breakpoint', - 'pause', - 'pause_from_sync', - 'shield_sigint_handler', - 'MultiActorPdb', - 'open_crash_handler', - 'maybe_open_crash_handler', - 'post_mortem', -] diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index d3bf4fe0..e174b848 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -21,18 +21,19 @@ Multi-core debugging for da peeps! """ from __future__ import annotations import bdb -import os -import sys -import signal -from functools import ( - partial, - cached_property, -) from contextlib import ( asynccontextmanager as acm, contextmanager as cm, nullcontext, ) +from functools import ( + partial, + cached_property, +) +import os +import signal +import sys +import traceback from typing import ( Any, Callable, @@ -611,6 +612,9 @@ def shield_sigint_handler( # https://github.com/prompt-toolkit/python-prompt-toolkit/blob/c2c6af8a0308f9e5d7c0e28cb8a02963fe0ce07a/prompt_toolkit/patch_stdout.py +_pause_msg: str = 'Attaching to pdb REPL in actor' + + def _set_trace( actor: tractor.Actor | None = None, pdb: MultiActorPdb | None = None, @@ -632,7 +636,13 @@ def _set_trace( ) or shield ): # pdbp.set_trace() - log.pdb(f"\nAttaching pdb to actor: {actor.uid}\n") + # TODO: maybe print the actor supervion tree up to the + # root here? Bo + log.pdb( + f'{_pause_msg}\n' + '|\n' + f'|_ {actor.uid}\n' + ) # no f!#$&* idea, but when we're in async land # we need 2x frames up? frame = frame.f_back @@ -911,6 +921,11 @@ async def breakpoint(**kwargs): await pause(**kwargs) +_crash_msg: str = ( + 'Attaching to pdb REPL in crashed actor' +) + + def _post_mortem( actor: tractor.Actor, pdb: MultiActorPdb, @@ -921,15 +936,23 @@ def _post_mortem( debugger instance. ''' - log.pdb(f"\nAttaching to pdb in crashed actor: {actor.uid}\n") + # TODO: print the actor supervion tree up to the root + # here! Bo + log.pdb( + f'{_crash_msg}\n' + '|\n' + f'|_ {actor.uid}\n' + ) - # TODO: you need ``pdbpp`` master (at least this commit - # https://github.com/pdbpp/pdbpp/commit/b757794857f98d53e3ebbe70879663d7d843a6c2) - # to fix this and avoid the hang it causes. See issue: - # https://github.com/pdbpp/pdbpp/issues/480 - # TODO: help with a 3.10+ major release if/when it arrives. - - pdbp.xpm(Pdb=lambda: pdb) + # TODO: only replacing this 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, + ) post_mortem = partial( @@ -1001,13 +1024,13 @@ async def maybe_wait_for_debugger( header_msg: str = '', -) -> None: +) -> bool: # was locked and we polled? if ( not debug_mode() and not child_in_debug ): - return + return False msg: str = header_msg @@ -1025,8 +1048,7 @@ async def maybe_wait_for_debugger( if sub_in_debug := Lock.global_actor_in_debug: msg += ( - 'Debug `Lock` in use by subactor\n' - f'|_{sub_in_debug}\n' + f'Debug `Lock` in use by subactor: {sub_in_debug}\n' ) # TODO: could this make things more deterministic? # wait to see if a sub-actor task will be @@ -1035,12 +1057,12 @@ 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.pdb( + log.debug( msg + 'Root immediately acquired debug TTY LOCK' ) - return + return False for istep in range(poll_steps): @@ -1090,12 +1112,13 @@ async def maybe_wait_for_debugger( continue # fallthrough on failure to acquire.. - else: - raise RuntimeError( - msg - + - 'Root actor failed to acquire debug lock?' - ) + # else: + # raise RuntimeError( + # msg + # + + # 'Root actor failed to acquire debug lock?' + # ) + return True # else: # # TODO: non-root call for #320? @@ -1104,6 +1127,7 @@ async def maybe_wait_for_debugger( # subactor_uid=this_uid, # ): # pass + return False # TODO: better naming and what additionals? # - [ ] optional runtime plugging? diff --git a/tractor/devx/cli.py b/tractor/devx/cli.py index 76890669..c44f9686 100644 --- a/tractor/devx/cli.py +++ b/tractor/devx/cli.py @@ -23,10 +23,6 @@ Currently popular frameworks supported are: """ from __future__ import annotations -from contextlib import ( - # asynccontextmanager as acm, - contextmanager as cm, -) from typing import ( Any, Callable, @@ -36,9 +32,6 @@ from typing_extensions import Annotated import typer -from ._debug import open_crash_handler - - _runtime_vars: dict[str, Any] = {} From e834297503217fcb31a3afd55e9037ba09b3afed Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 6 Mar 2024 14:37:54 -0500 Subject: [PATCH 24/30] Add `shield: bool` support to `.pause()` It's been on the todo for a while and I've given up trying to properly hide the `trio.CancelScope.__exit__()` frame for now instead opting to just `log.pdb()` a big apology XD Users can obvi still just not use the flag and wrap `tractor.pause()` in their own cs block if they want to avoid having to hit `'up'` in the pdb REPL if needed in a cancelled task-scope. Impl deatz: - factor orig `.pause()` impl into new `._pause()` so that we can more tersely wrap the original content depending on `shield: bool` input; only open the cancel-scope when shield is set to avoid aforemented extra strack frame annoyance. - pass through `shield` to underlying `_pause` and `debug_func()` so we can actually know when so log our apology. - add a buncha notes to new `.pause()` wrapper regarding the inability to hide the cancel-scope `.__exit__()`, inluding that overriding the code in `trio._core._run.CancelScope` doesn't seem to solve the issue either.. Unrelated `maybe_wait_for_debugger()` tweaks: - don't read `Lock.global_actor_in_debug` more then needed, rename local read var to `in_debug` (since it can also hold the root actor uid, not just sub-actors). - shield the `await debug_complete.wait()` since ideally we avoid the root cancellation child-actors in debug even when the root calls this func in a cancelled scope. --- tractor/devx/_debug.py | 206 +++++++++++++++++++++++++++++------------ 1 file changed, 145 insertions(+), 61 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index e174b848..2839e597 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -619,13 +619,15 @@ def _set_trace( actor: tractor.Actor | None = None, pdb: MultiActorPdb | None = None, shield: bool = False, + + extra_frames_up_when_async: int = 1, ): __tracebackhide__: bool = True actor: tractor.Actor = actor or current_actor() - # start 2 levels up in user code - frame: FrameType | None = sys._getframe() - if frame: + # 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 ( @@ -633,23 +635,39 @@ def _set_trace( and ( pdb and actor is not None - ) or shield + ) + # 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'{_pause_msg}\n' + f'{msg}\n' '|\n' f'|_ {actor.uid}\n' ) # no f!#$&* idea, but when we're in async land # we need 2x frames up? - frame = frame.f_back - # frame = frame.f_back - - # if shield: - # frame = frame.f_back + for i in range(extra_frames_up_when_async): + frame: FrameType = frame.f_back + log.debug( + f'Going up frame {i} -> {frame}\n' + ) else: pdb, undo_sigint = mk_mpdb() @@ -659,10 +677,9 @@ def _set_trace( Lock.local_task_in_debug = 'sync' pdb.set_trace(frame=frame) - # undo_ -async def pause( +async def _pause( debug_func: Callable = _set_trace, release_lock_signal: trio.Event | None = None, @@ -676,27 +693,19 @@ async def pause( # be no way to override it?.. # shield: bool = False, - # TODO: - # shield: bool = False + shield: bool = False, task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED ) -> None: ''' - A pause point (more commonly known as a "breakpoint") interrupt - instruction for engaging a blocking debugger instance to - conduct manual console-based-REPL-interaction from within - `tractor`'s async runtime, normally from some single-threaded - and currently executing actor-hosted-`trio`-task in some - (remote) process. + Inner impl for `pause()` to avoid the `trio.CancelScope.__exit__()` + stack frame when not shielded (since apparently i can't figure out + how to hide it using the normal mechanisms..) - NOTE: we use the semantics "pause" since it better encompasses - the entirety of the necessary global-runtime-state-mutation any - actor-task must access and lock in order to get full isolated - control over the process tree's root TTY: - https://en.wikipedia.org/wiki/Breakpoint + Hopefully we won't need this in the long run. ''' - # __tracebackhide__ = True + __tracebackhide__: bool = True actor = current_actor() pdb, undo_sigint = mk_mpdb() task_name: str = trio.lowlevel.current_task().name @@ -707,24 +716,11 @@ async def pause( ): Lock.local_pdb_complete = trio.Event() - # if shield: debug_func = partial( debug_func, - # shield=shield, ) - # def _exit(self, *args, **kwargs): - # __tracebackhide__: bool = True - # super().__exit__(*args, **kwargs) - - # trio.CancelScope.__exit__.__tracebackhide__ = True - - # import types - # with trio.CancelScope(shield=shield) as cs: - # cs.__exit__ = types.MethodType(_exit, cs) - # cs.__exit__.__tracebackhide__ = True - - # TODO: need a more robust check for the "root" actor + # TODO: need a more robust check for the "root" actor if ( not is_root_process() and actor._parent_chan # a connected child @@ -818,7 +814,7 @@ async def pause( # 'trace func provided!' # ) print(f"{actor.uid} ENTERING WAIT") - task_status.started() + task_status.started(cs) # with trio.CancelScope(shield=True): # await release_lock_signal.wait() @@ -827,22 +823,103 @@ async def pause( # block here one (at the appropriate frame *up*) where # ``breakpoint()`` was awaited and begin handling stdio. log.debug("Entering the synchronous world of pdb") - debug_func(actor, pdb) + debug_func( + actor, + pdb, + extra_frames_up_when_async=2, + shield=shield, + ) + assert cs except bdb.BdbQuit: Lock.release() raise - # XXX: apparently we can't do this without showing this frame - # in the backtrace on first entry to the REPL? Seems like an odd - # behaviour that should have been fixed by now. This is also why - # we scrapped all the @cm approaches that were tried previously. - # finally: - # __tracebackhide__ = True - # # frame = sys._getframe() - # # last_f = frame.f_back - # # last_f.f_globals['__tracebackhide__'] = True - # # signal.signal = pdbp.hideframe(signal.signal) +# 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) + + +async def pause( + + debug_func: Callable = _set_trace, + release_lock_signal: trio.Event | None = None, + + # TODO: allow caller to pause despite task cancellation, + # exactly the same as wrapping with: + # with CancelScope(shield=True): + # await pause() + # => the REMAINING ISSUE is that the scope's .__exit__() frame + # is always show in the debugger on entry.. and there seems to + # be no way to override it?.. + # shield: bool = False, + + shield: bool = False, + task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED + +) -> None: + ''' + A pause point (more commonly known as a "breakpoint") interrupt + instruction for engaging a blocking debugger instance to + conduct manual console-based-REPL-interaction from within + `tractor`'s async runtime, normally from some single-threaded + and currently executing actor-hosted-`trio`-task in some + (remote) process. + + NOTE: we use the semantics "pause" since it better encompasses + the entirety of the necessary global-runtime-state-mutation any + actor-task must access and lock in order to get full isolated + control over the process tree's root TTY: + https://en.wikipedia.org/wiki/Breakpoint + + ''' + __tracebackhide__: bool = True + + if shield: + # NOTE XXX: even hard coding this inside the `class CancelScope:` + # doesn't seem to work for me!? + # ^ XXX ^ + + # 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) + await _pause( + debug_func=debug_func, + release_lock_signal=release_lock_signal, + shield=True, + task_status=task_status, + ) + else: + await _pause( + debug_func=debug_func, + release_lock_signal=release_lock_signal, + shield=False, + task_status=task_status, + ) + + # TODO: allow pausing from sync code. @@ -1043,12 +1120,20 @@ async def maybe_wait_for_debugger( # will make the pdb repl unusable. # Instead try to wait for pdb to be released before # tearing down. - sub_in_debug: tuple[str, str]|None = Lock.global_actor_in_debug + in_debug: tuple[str, str]|None = Lock.global_actor_in_debug debug_complete: trio.Event|None = Lock.no_remote_has_tty - if sub_in_debug := Lock.global_actor_in_debug: + if in_debug == current_actor().uid: + log.debug( + msg + + + 'Root already owns the TTY LOCK' + ) + return True + + elif in_debug: msg += ( - f'Debug `Lock` in use by subactor: {sub_in_debug}\n' + f'Debug `Lock` in use by subactor: {in_debug}\n' ) # TODO: could this make things more deterministic? # wait to see if a sub-actor task will be @@ -1065,27 +1150,26 @@ async def maybe_wait_for_debugger( return False for istep in range(poll_steps): - - if ( debug_complete and not debug_complete.is_set() - and sub_in_debug is not None + and in_debug is not None ): log.pdb( msg + 'Root is waiting on tty lock to release..\n' ) - await debug_complete.wait() + with trio.CancelScope(shield=True): + await debug_complete.wait() log.pdb( f'Child subactor released debug lock:' - f'|_{sub_in_debug}\n' + f'|_{in_debug}\n' ) # is no subactor locking debugger currently? if ( - sub_in_debug is None + in_debug is None and ( debug_complete is None or debug_complete.is_set() From 65e49696e7e98fdf8f3b416fffeb4fce127a7705 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 7 Mar 2024 21:14:40 -0500 Subject: [PATCH 25/30] Woops, fix `_post_mortem()` type sig.. We're passing a `extra_frames_up_when_async=2` now (from prior attempt to hide `CancelScope.__exit__()` when `shield=True`) and thus both `debug_func`s must accept it :facepalm: On the brighter side found out that the `TypeError` from the call-sig mismatch was actually being swallowed entirely so add some `.exception()` msgs for such cases to at least alert the dev they broke stuff XD --- tractor/devx/_debug.py | 43 +++++++++++++++++++++++++++++------------- 1 file changed, 30 insertions(+), 13 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 2839e597..3203af1b 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -95,12 +95,12 @@ class Lock: # 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 + _debugger_request_cs: trio.CancelScope|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_local_task_cs_in_debug: trio.CancelScope|None = None # actor tree-wide actor uid that supposedly has the tty lock global_actor_in_debug: tuple[str, str] = None @@ -808,33 +808,46 @@ async def _pause( Lock.repl = pdb try: - if debug_func is None: + # TODO: do we want to support using this **just** for the + # locking / common code (prolly to help address #320)? + # + # if debug_func is None: # assert release_lock_signal, ( # 'Must pass `release_lock_signal: trio.Event` if no ' # 'trace func provided!' # ) - print(f"{actor.uid} ENTERING WAIT") - task_status.started(cs) - + # print(f"{actor.uid} ENTERING WAIT") # with trio.CancelScope(shield=True): # await release_lock_signal.wait() - else: + # else: # block here one (at the appropriate frame *up*) where # ``breakpoint()`` was awaited and begin handling stdio. - log.debug("Entering the synchronous world of pdb") + log.debug('Entering sync world of the `pdb` REPL..') + try: debug_func( actor, pdb, extra_frames_up_when_async=2, shield=shield, ) - assert cs + 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 + # 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 @@ -905,14 +918,14 @@ async def pause( # NOTE: so the caller can always cancel even if shielded task_status.started(cs) - await _pause( + return await _pause( debug_func=debug_func, release_lock_signal=release_lock_signal, shield=True, task_status=task_status, ) else: - await _pause( + return await _pause( debug_func=debug_func, release_lock_signal=release_lock_signal, shield=False, @@ -1006,6 +1019,10 @@ _crash_msg: str = ( def _post_mortem( actor: tractor.Actor, pdb: MultiActorPdb, + shield: bool = False, + + # only for compat with `._set_trace()`.. + extra_frames_up_when_async=0, ) -> None: ''' @@ -1034,7 +1051,7 @@ def _post_mortem( post_mortem = partial( pause, - _post_mortem, + debug_func=_post_mortem, ) @@ -1163,7 +1180,7 @@ async def maybe_wait_for_debugger( with trio.CancelScope(shield=True): await debug_complete.wait() log.pdb( - f'Child subactor released debug lock:' + f'Child subactor released debug lock\n' f'|_{in_debug}\n' ) From 3a317c15819f69af66f0c3ac65026ab71c14110b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 20 Feb 2024 13:23:16 -0500 Subject: [PATCH 26/30] Enable `stackscope` render via root in debug mode If `stackscope` is importable and debug_mode is enabled then we by default call and report `.devx.enable_stack_on_sig()` is set B) This makes debugging unexpected (SIGINT ignoring) hangs a cinch! --- tractor/_root.py | 29 ++++++++++++++++++++++++----- 1 file changed, 24 insertions(+), 5 deletions(-) diff --git a/tractor/_root.py b/tractor/_root.py index d81cae78..a1a11d3b 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -131,13 +131,19 @@ async def open_root_actor( ) ) - loglevel = (loglevel or log._default_loglevel).upper() + loglevel = ( + loglevel + or log._default_loglevel + ).upper() - if debug_mode and _spawn._spawn_method == 'trio': + if ( + debug_mode + and _spawn._spawn_method == 'trio' + ): _state._runtime_vars['_debug_mode'] = True - # expose internal debug module to every actor allowing - # for use of ``await tractor.breakpoint()`` + # expose internal debug module to every actor allowing for + # use of ``await tractor.pause()`` enable_modules.append('tractor.devx._debug') # if debug mode get's enabled *at least* use that level of @@ -156,7 +162,20 @@ async def open_root_actor( "Debug mode is only supported for the `trio` backend!" ) - log.get_console_log(loglevel) + assert loglevel + _log = log.get_console_log(loglevel) + assert _log + + # TODO: factor this into `.devx._stackscope`!! + if debug_mode: + try: + logger.info('Enabling `stackscope` traces on SIGUSR1') + from .devx import enable_stack_on_sig + enable_stack_on_sig() + except ImportError: + logger.warning( + '`stackscope` not installed for use in debug mode!' + ) try: # make a temporary connection to see if an arbiter exists, From f028181e19b322c2bd330a9bba3b29f357f87d0b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 20 Feb 2024 15:29:31 -0500 Subject: [PATCH 27/30] Add `stackscope` as dep, drop legacy `pdb` issue cruft --- setup.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/setup.py b/setup.py index a4e5e1ed..d7801d8e 100755 --- a/setup.py +++ b/setup.py @@ -26,7 +26,7 @@ with open('docs/README.rst', encoding='utf-8') as f: setup( name="tractor", version='0.1.0a6dev0', # alpha zone - description='structured concurrrent `trio`-"actors"', + description='structured concurrent `trio`-"actors"', long_description=readme, license='AGPLv3', author='Tyler Goodlet', @@ -53,6 +53,7 @@ setup( # 'exceptiongroup', # in stdlib as of 3.11! # tooling + 'stackscope', 'tricycle', 'trio_typing', 'colorlog', @@ -64,16 +65,15 @@ setup( # debug mode REPL 'pdbp', + # TODO: distributed transport using + # linux kernel networking + # 'pyroute2', + # pip ref docs on these specs: # https://pip.pypa.io/en/stable/reference/requirement-specifiers/#examples # and pep: # https://peps.python.org/pep-0440/#version-specifiers - # windows deps workaround for ``pdbpp`` - # https://github.com/pdbpp/pdbpp/issues/498 - # https://github.com/pdbpp/fancycompleter/issues/37 - 'pyreadline3 ; platform_system == "Windows"', - ], tests_require=['pytest'], python_requires=">=3.11", From a88564549a7bdb839c975c465543fe6e07096d77 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 6 Mar 2024 09:55:05 -0500 Subject: [PATCH 28/30] Yahh, add `.devx` package to installed subpkgs.. --- setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/setup.py b/setup.py index d7801d8e..66b2622d 100755 --- a/setup.py +++ b/setup.py @@ -40,6 +40,7 @@ setup( 'tractor.trionics', # trio extensions 'tractor.msg', # lowlevel data types 'tractor._testing', # internal cross-subsys suite utils + 'tractor.devx', # "dev-experience" ], install_requires=[ From d5eec6eb6c703b0b064c9ba71ebabce8bb1f9858 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 19 Mar 2025 15:59:56 -0400 Subject: [PATCH 29/30] Re-revert back to `.devx` subpkg after rebase.. --- tractor/_context.py | 6 +++--- tractor/_rpc.py | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index 0be54860..a2860f3d 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -313,7 +313,7 @@ async def _drain_to_final_msg( log.critical('SHOULD NEVER GET HERE!?') assert msg is ctx._cancel_msg assert error.msgdata == ctx._remote_error.msgdata - from ._debug import pause + from .devx._debug import pause await pause() ctx._maybe_cancel_and_set_remote_error(error) ctx._maybe_raise_remote_err(error) @@ -2202,7 +2202,7 @@ async def open_context_from_portal( # pass # TODO: factor ^ into below for non-root cases? # - from ._debug import maybe_wait_for_debugger + from .devx._debug import maybe_wait_for_debugger was_acquired: bool = await maybe_wait_for_debugger( # header_msg=( # 'Delaying `ctx.cancel()` until debug lock ' @@ -2313,7 +2313,7 @@ async def open_context_from_portal( # where the root is waiting on the lock to clear but the # child has already cleared it and clobbered IPC. if debug_mode(): - from ._debug import maybe_wait_for_debugger + from .devx._debug import maybe_wait_for_debugger await maybe_wait_for_debugger() # though it should be impossible for any tasks diff --git a/tractor/_rpc.py b/tractor/_rpc.py index 47548106..89c97381 100644 --- a/tractor/_rpc.py +++ b/tractor/_rpc.py @@ -55,7 +55,7 @@ from ._exceptions import ( unpack_error, TransportClosed, ) -from . import _debug +from .devx import _debug from . import _state from .log import get_logger From 8d318a8ac540307dc3502196e0208a0ba6906f0c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 14 Apr 2024 19:39:57 -0400 Subject: [PATCH 30/30] Flip a last `MultiError` to a beg, add todo on `@stream` func --- tests/test_context_stream_semantics.py | 4 ++-- tests/test_legacy_one_way_streaming.py | 5 ++++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index 42b1f7d0..40247fd7 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -245,10 +245,10 @@ def test_simple_context( trio.run(main) except error_parent: pass - except trio.MultiError as me: + except BaseExceptionGroup as beg: # XXX: on windows it seems we may have to expect the group error from tractor._exceptions import is_multi_cancelled - assert is_multi_cancelled(me) + assert is_multi_cancelled(beg) else: trio.run(main) diff --git a/tests/test_legacy_one_way_streaming.py b/tests/test_legacy_one_way_streaming.py index 5d7787fa..61fff75c 100644 --- a/tests/test_legacy_one_way_streaming.py +++ b/tests/test_legacy_one_way_streaming.py @@ -38,10 +38,13 @@ async def async_gen_stream(sequence): assert cs.cancelled_caught +# TODO: deprecated either remove entirely +# or re-impl in terms of `MsgStream` one-sides +# wrapper, but at least remove `Portal.open_stream_from()` @tractor.stream async def context_stream( ctx: tractor.Context, - sequence + sequence: list[int], ): for i in sequence: await ctx.send_yield(i)