From c04d77a3c9b1347b9f90eb61f827bf0b1c30383d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 20 Mar 2024 19:13:13 -0400 Subject: [PATCH 1/3] First draft workin minus non-main-thread usage! --- examples/debugging/sync_bp.py | 69 +++++++++++++ tractor/devx/_debug.py | 177 ++++++++++++++++++++-------------- 2 files changed, 172 insertions(+), 74 deletions(-) create mode 100644 examples/debugging/sync_bp.py diff --git a/examples/debugging/sync_bp.py b/examples/debugging/sync_bp.py new file mode 100644 index 0000000..49f4d9a --- /dev/null +++ b/examples/debugging/sync_bp.py @@ -0,0 +1,69 @@ +import trio +import tractor + + +def sync_pause(): + tractor.pause_from_sync() + + +@tractor.context +async def start_n_sync_pause( + ctx: tractor.Context, +): + # sync to requesting peer + await ctx.started() + + actor: tractor.Actor = tractor.current_actor() + print(f'entering SYNC PAUSE in {actor.uid}') + sync_pause() + print(f'back from SYNC PAUSE in {actor.uid}') + + +async def main() -> None: + + from tractor._rpc import maybe_import_gb + + async with tractor.open_nursery( + debug_mode=True, + ) as an: + + # TODO: where to put this? + # => just inside `open_root_actor()` yah? + await maybe_import_gb() + + p: tractor.Portal = await an.start_actor( + 'subactor', + enable_modules=[__name__], + # infect_asyncio=True, + debug_mode=True, + loglevel='cancel', + ) + + # TODO: 3 sub-actor usage cases: + # -[ ] via a `.run_in_actor()` call + # -[ ] via a `.run()` + # -[ ] via a `.open_context()` + # + async with p.open_context( + start_n_sync_pause, + ) as (ctx, first): + assert first is None + + await tractor.pause() + sync_pause() + + # TODO: make this work!! + await trio.to_thread.run_sync( + sync_pause, + abandon_on_cancel=False, + ) + + await ctx.cancel() + + # TODO: case where we cancel from trio-side while asyncio task + # has debugger lock? + await p.cancel_actor() + + +if __name__ == '__main__': + trio.run(main) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 3203af1..105d2ca 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -46,7 +46,7 @@ import pdbp import tractor import trio from trio.lowlevel import current_task -from trio_typing import ( +from trio import ( TaskStatus, # Task, ) @@ -400,7 +400,6 @@ async def wait_for_parent_stdin_hijack( # this syncs to child's ``Context.started()`` call. async with portal.open_context( - lock_tty_for_child, subactor_uid=actor_uid, @@ -682,7 +681,10 @@ def _set_trace( async def _pause( debug_func: Callable = _set_trace, - release_lock_signal: trio.Event | None = None, + + # NOTE: must be passed in the `.pause_from_sync()` case! + pdb: MultiActorPdb|None = None, + undo_sigint: Callable|None = None, # TODO: allow caller to pause despite task cancellation, # exactly the same as wrapping with: @@ -691,8 +693,7 @@ async def _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 @@ -707,7 +708,6 @@ async def _pause( ''' __tracebackhide__: bool = True actor = current_actor() - pdb, undo_sigint = mk_mpdb() task_name: str = trio.lowlevel.current_task().name if ( @@ -716,9 +716,14 @@ async def _pause( ): Lock.local_pdb_complete = trio.Event() - debug_func = partial( - debug_func, - ) + if debug_func is not None: + debug_func = partial( + debug_func, + ) + + if pdb is None: + assert undo_sigint is None, 'You must pass both!?!' + pdb, undo_sigint = mk_mpdb() # TODO: need a more robust check for the "root" actor if ( @@ -761,12 +766,14 @@ async def _pause( # ``` # but not entirely sure if that's a sane way to implement it? try: + print("ACQUIRING TTY LOCK from CHILD") with trio.CancelScope(shield=True): await actor._service_n.start( wait_for_parent_stdin_hijack, actor.uid, ) Lock.repl = pdb + except RuntimeError: Lock.release() @@ -779,11 +786,13 @@ async def _pause( raise elif is_root_process(): + print("ROOT TTY LOCK BRANCH") # 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: + print("ROOT ALREADY HAS TTY?") # re-entrant root process already has it: noop. return @@ -797,11 +806,14 @@ async def _pause( # must shield here to avoid hitting a ``Cancelled`` and # a child getting stuck bc we clobbered the tty + print("ACQUIRING TTY LOCK from ROOT") with trio.CancelScope(shield=True): await Lock._debug_lock.acquire() else: # may be cancelled + print("ROOT TRYING LOCK ACQUIRE") await Lock._debug_lock.acquire() + print("ROOT LOCKED TTY") Lock.global_actor_in_debug = actor.uid Lock.local_task_in_debug = task_name @@ -811,32 +823,27 @@ async def _pause( # 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") - # with trio.CancelScope(shield=True): - # await release_lock_signal.wait() + if debug_func is None: + task_status.started(Lock) + print("ROOT .started(Lock) now!") - # else: + else: # block here one (at the appropriate frame *up*) where # ``breakpoint()`` was awaited and begin handling stdio. - log.debug('Entering sync world of the `pdb` REPL..') - try: - debug_func( - actor, - pdb, - extra_frames_up_when_async=2, - shield=shield, - ) - except BaseException: - log.exception( - 'Failed to invoke internal `debug_func = ' - f'{debug_func.func.__name__}`\n' - ) - raise + log.debug('Entering sync world of the `pdb` REPL..') + try: + debug_func( + actor, + pdb, + extra_frames_up_when_async=2, + shield=shield, + ) + except BaseException: + log.exception( + 'Failed to invoke internal `debug_func = ' + f'{debug_func.func.__name__}`\n' + ) + raise except bdb.BdbQuit: Lock.release() @@ -862,8 +869,7 @@ async def _pause( async def pause( - debug_func: Callable = _set_trace, - release_lock_signal: trio.Event | None = None, + debug_func: Callable|None = _set_trace, # TODO: allow caller to pause despite task cancellation, # exactly the same as wrapping with: @@ -872,10 +878,11 @@ async def 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 + task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED, + + **_pause_kwargs, ) -> None: ''' @@ -920,16 +927,16 @@ async def pause( task_status.started(cs) return await _pause( debug_func=debug_func, - release_lock_signal=release_lock_signal, shield=True, task_status=task_status, + **_pause_kwargs ) else: return await _pause( debug_func=debug_func, - release_lock_signal=release_lock_signal, shield=False, task_status=task_status, + **_pause_kwargs ) @@ -938,46 +945,64 @@ async def pause( # 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") +def pause_from_sync( + hide_tb: bool = True +) -> None: + + __tracebackhide__: bool = hide_tb actor: tractor.Actor = current_actor( err_on_no_runtime=False, ) - if actor: - try: - import greenback - # __tracebackhide__ = True + print( + f'{actor.uid}: JUST ENTERED `tractor.pause_from_sync()`' + f'|_{actor}\n' + ) + if not actor: + raise RuntimeError( + 'Not inside the `tractor`-runtime?\n' + '`tractor.pause_from_sync()` is not functional without a wrapping\n' + '- `async with tractor.open_nursery()` or,\n' + '- `async with tractor.open_root_actor()`\n' + ) + try: + import greenback + except ModuleNotFoundError: + raise RuntimeError( + 'The `greenback` lib is required to use `tractor.pause_from_sync()`!\n' + 'https://github.com/oremanj/greenback\n' + ) - # 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') + # out = 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 ;) db, undo_sigint = mk_mpdb() - Lock.local_task_in_debug = 'sync' - # db.config.enable_hidden_frames = True + greenback.await_( + pause( + debug_func=None, + pdb=db, + undo_sigint=undo_sigint, + ) + ) - # we entered the global ``breakpoint()`` built-in from sync + Lock.local_task_in_debug = 'sync' + + # TODO: ensure we aggressively make the user aware about + # entering 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 + + # db.config.enable_hidden_frames = True + # assert not db._is_hidden(frame) # print(f'FRAME: {str(frame)}') # if not db._is_hidden(frame): # pdbp.set_trace() @@ -985,17 +1010,21 @@ def pause_from_sync() -> None: # (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 + + # XXX NOTE XXX no other LOC can be here without it + # showing up in the REPL's last stack frame !?! + # -[ ] tried to use `@pdbp.hideframe` decoration but + # still doesn't work + # + # FROM BEFORE: 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? + # + # NOTE: not needed any more anyway since it's all in + # `Lock.release()` now! # undo_sigint() - # Lock.global_actor_in_debug = actor.uid - # Lock.release() - # task_can_release_tty_lock.set() - # using the "pause" semantics instead since # that better covers actually somewhat "pausing the runtime" -- 2.34.1 From 4f863a698953c19ad5b1e66c78407cf1193ebe22 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 22 Mar 2024 16:41:49 -0400 Subject: [PATCH 2/3] Refine and test `tractor.pause_from_sync()` Now supports use from any `trio` task, any sync thread started with `trio.to_thread.run_sync()` AND also via `breakpoint()` builtin API! The only bit missing now is support for `asyncio` tasks when in infected mode.. Bo `greenback` setup/API adjustments: - move `._rpc.maybe_import_gb()` to -> `devx._debug` and factor out the cached import checking into a sync func whilst placing the async `.ensure_portal()` bootstrapping into a new async `maybe_init_greenback()`. - use the new init-er func inside `open_root_actor()` with the output predicating whether we override the `breakpoint()` hook. core `devx._debug` implementation deatz: - make `mk_mpdb()` only return the `pdp.Pdb` subtype instance since the sigint unshielding func is now accessible from the `Lock` singleton from anywhere. - add non-main thread support (at least for `trio.to_thread` use cases) to our `Lock` with a new `.is_trio_thread()` predicate that delegates directly to `trio`'s internal version. - do `Lock.is_trio_thread()` checks inside any methods which require special provisions when invoked from a non-main `trio` thread: - `.[un]shield_sigint()` methods since `signal.signal` usage is only allowed from cpython's main thread. - `.release()` since `trio.StrictFIFOLock` can only be called from a `trio` task. - rework `.pause_from_sync()` itself to directly call `._set_trace()` and don't bother with `greenback._await()` when we're already calling it from a `.to_thread.run_sync()` thread, oh and try to use the thread/task name when setting `Lock.local_task_in_debug`. - make it an RTE for now if you try to use `.pause_from_sync()` from any infected-`asyncio` task, but support is (hopefully) coming soon! For testing we add a new `test_debugger.py::test_pause_from_sync()` which includes a ctrl-c parametrization around the `examples/debugging/sync_bp.py` script which includes all currently supported/working usages: - `tractor.pause_from_sync()`. - via `breakpoint()` overload. - from a `trio.to_thread.run_sync()` spawn. --- examples/debugging/sync_bp.py | 20 ++- tests/test_debugger.py | 64 +++++++ tractor/_root.py | 18 +- tractor/_rpc.py | 26 +-- tractor/devx/_debug.py | 330 +++++++++++++++++++++++----------- 5 files changed, 322 insertions(+), 136 deletions(-) diff --git a/examples/debugging/sync_bp.py b/examples/debugging/sync_bp.py index 49f4d9a..23469d6 100644 --- a/examples/debugging/sync_bp.py +++ b/examples/debugging/sync_bp.py @@ -2,8 +2,18 @@ import trio import tractor -def sync_pause(): - tractor.pause_from_sync() +def sync_pause( + use_builtin: bool = True, + error: bool = False, +): + if use_builtin: + breakpoint() + + else: + tractor.pause_from_sync() + + if error: + raise RuntimeError('yoyo sync code error') @tractor.context @@ -21,16 +31,10 @@ async def start_n_sync_pause( async def main() -> None: - from tractor._rpc import maybe_import_gb - async with tractor.open_nursery( debug_mode=True, ) as an: - # TODO: where to put this? - # => just inside `open_root_actor()` yah? - await maybe_import_gb() - p: tractor.Portal = await an.start_actor( 'subactor', enable_modules=[__name__], diff --git a/tests/test_debugger.py b/tests/test_debugger.py index 20e67ab..38a3bc2 100644 --- a/tests/test_debugger.py +++ b/tests/test_debugger.py @@ -1027,3 +1027,67 @@ def test_different_debug_mode_per_actor( # instead crashed completely assert "tractor._exceptions.RemoteActorError: ('crash_boi'" in before assert "RuntimeError" in before + + + +def test_pause_from_sync( + spawn, + ctlc: bool +): + ''' + Verify we can use the `pdbp` REPL from sync functions AND from + any thread spawned with `trio.to_thread.run_sync()`. + + `examples/debugging/sync_bp.py` + + ''' + child = spawn('sync_bp') + child.expect(PROMPT) + assert_before( + child, + [ + '`greenback` portal opened!', + # pre-prompt line + _pause_msg, "('root'", + ] + ) + if ctlc: + do_ctlc(child) + child.sendline('c') + child.expect(PROMPT) + + # XXX shouldn't see gb loaded again + before = str(child.before.decode()) + assert not in_prompt_msg( + before, + ['`greenback` portal opened!'], + ) + assert_before( + child, + [_pause_msg, "('root'",], + ) + + if ctlc: + do_ctlc(child) + child.sendline('c') + child.expect(PROMPT) + assert_before( + child, + [_pause_msg, "('subactor'",], + ) + + if ctlc: + do_ctlc(child) + child.sendline('c') + child.expect(PROMPT) + # non-main thread case + # TODO: should we agument the pre-prompt msg in this case? + assert_before( + child, + [_pause_msg, "('root'",], + ) + + if ctlc: + do_ctlc(child) + child.sendline('c') + child.expect(pexpect.EOF) diff --git a/tractor/_root.py b/tractor/_root.py index 5445191..91d7c83 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -94,12 +94,24 @@ async def open_root_actor( Runtime init entry point for ``tractor``. ''' + # TODO: stick this in a `@cm` defined in `devx._debug`? + # # Override the global debugger hook to make it play nice with # ``trio``, see much discussion in: # https://github.com/python-trio/trio/issues/1155#issuecomment-742964018 - builtin_bp_handler = sys.breakpointhook - orig_bp_path: str | None = os.environ.get('PYTHONBREAKPOINT', None) - os.environ['PYTHONBREAKPOINT'] = 'tractor.devx._debug.pause_from_sync' + if ( + await _debug.maybe_init_greenback( + raise_not_found=False, + ) + ): + builtin_bp_handler = sys.breakpointhook + orig_bp_path: str|None = os.environ.get( + 'PYTHONBREAKPOINT', + None, + ) + os.environ['PYTHONBREAKPOINT'] = ( + 'tractor.devx._debug.pause_from_sync' + ) # attempt to retreive ``trio``'s sigint handler and stash it # on our debugger lock state. diff --git a/tractor/_rpc.py b/tractor/_rpc.py index d369b41..91482a0 100644 --- a/tractor/_rpc.py +++ b/tractor/_rpc.py @@ -26,7 +26,6 @@ from contextlib import ( from functools import partial import inspect from pprint import pformat -from types import ModuleType from typing import ( Any, Callable, @@ -337,27 +336,6 @@ async def _errors_relayed_via_ipc( actor._ongoing_rpc_tasks.set() -_gb_mod: ModuleType|None|False = None - - -async def maybe_import_gb(): - global _gb_mod - if _gb_mod is False: - return - - try: - import greenback - _gb_mod = greenback - await greenback.ensure_portal() - - except ModuleNotFoundError: - log.debug( - '`greenback` is not installed.\n' - 'No sync debug support!\n' - ) - _gb_mod = False - - async def _invoke( actor: Actor, @@ -385,7 +363,9 @@ async def _invoke( treat_as_gen: bool = False if _state.debug_mode(): - await maybe_import_gb() + # XXX for .pause_from_sync()` usage we need to make sure + # `greenback` is boostrapped in the subactor! + await _debug.maybe_init_greenback() # TODO: possibly a specially formatted traceback # (not sure what typing is for this..)? diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 105d2ca..770995a 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -33,14 +33,19 @@ from functools import ( import os import signal import sys +import threading import traceback from typing import ( Any, Callable, AsyncIterator, AsyncGenerator, + TYPE_CHECKING, +) +from types import ( + FrameType, + ModuleType, ) -from types import FrameType import pdbp import tractor @@ -51,17 +56,22 @@ from trio import ( # Task, ) -from ..log import get_logger -from .._state import ( +from tractor.log import get_logger +from tractor._state import ( current_actor, is_root_process, debug_mode, ) -from .._exceptions import ( +from tractor._exceptions import ( is_multi_cancelled, ContextCancelled, ) -from .._ipc import Channel +from tractor._ipc import Channel + +if TYPE_CHECKING: + from tractor._runtime import ( + Actor, + ) log = get_logger(__name__) @@ -116,10 +126,36 @@ class Lock: @classmethod def shield_sigint(cls): - cls._orig_sigint_handler = signal.signal( - signal.SIGINT, - shield_sigint_handler, - ) + ''' + Shield out SIGINT handling (which by default triggers + `trio.Task` cancellation) in subactors when the `pdb` REPL + is active. + + Avoids cancellation of the current actor (task) when the + user mistakenly sends ctl-c or a signal is received from + an external request; explicit runtime cancel requests are + allowed until the use exits the REPL session using + 'continue' or 'quit', at which point the orig SIGINT + handler is restored. + + ''' + # + # XXX detect whether we're running from a non-main thread + # in which case schedule the SIGINT shielding override + # to in the main thread. + # https://docs.python.org/3/library/signal.html#signals-and-threads + if not cls.is_trio_thread(): + cls._orig_sigint_handler: Callable = trio.from_thread.run_sync( + signal.signal, + signal.SIGINT, + shield_sigint_handler, + ) + + else: + cls._orig_sigint_handler = signal.signal( + signal.SIGINT, + shield_sigint_handler, + ) @classmethod @pdbp.hideframe # XXX NOTE XXX see below in `.pause_from_sync()` @@ -127,13 +163,46 @@ class Lock: # always restore ``trio``'s sigint handler. see notes below in # the pdb factory about the nightmare that is that code swapping # out the handler when the repl activates... - signal.signal(signal.SIGINT, cls._trio_handler) + if not cls.is_trio_thread(): + trio.from_thread.run_sync( + signal.signal, + signal.SIGINT, + cls._trio_handler, + ) + else: + signal.signal( + signal.SIGINT, + cls._trio_handler, + ) + cls._orig_sigint_handler = None + @classmethod + def is_trio_thread(self) -> bool: + ''' + Check if we're the "main" thread (as in the first one + started by cpython) and presume that it is the thread that + called `trio.run()` and not some thread spawned with + `trio.to_thread.run_sync()`. + + ''' + return trio._util.is_main_thread() + # XXX apparently unreliable..see ^ + # ( + # threading.current_thread() + # is not threading.main_thread() + # ) + @classmethod def release(cls): try: - cls._debug_lock.release() + if not cls.is_trio_thread(): + trio.from_thread.run_sync( + cls._debug_lock.release + ) + else: + cls._debug_lock.release() + except RuntimeError: # uhhh makes no sense but been seeing the non-owner # release error even though this is definitely the task @@ -437,11 +506,31 @@ async def wait_for_parent_stdin_hijack( log.debug('Exiting debugger from child') -def mk_mpdb() -> tuple[MultiActorPdb, Callable]: +def mk_mpdb() -> MultiActorPdb: + ''' + Deliver a new `MultiActorPdb`: a multi-process safe `pdbp` + REPL using the magic of SC! + Our `pdb.Pdb` subtype accomplishes multi-process safe debugging + by: + + - mutexing access to the root process' TTY & stdstreams + via an IPC managed `Lock` singleton per process tree. + + - temporarily overriding any subactor's SIGINT handler to shield during + live REPL sessions in sub-actors such that cancellation is + never (mistakenly) triggered by a ctrl-c and instead only + by either explicit requests in the runtime or + + ''' pdb = MultiActorPdb() - # signal.signal = pdbp.hideframe(signal.signal) + # Always shield out SIGINTs for subactors when REPL is active. + # + # XXX detect whether we're running from a non-main thread + # in which case schedule the SIGINT shielding override + # to in the main thread. + # https://docs.python.org/3/library/signal.html#signals-and-threads Lock.shield_sigint() # XXX: These are the important flags mentioned in @@ -450,7 +539,7 @@ def mk_mpdb() -> tuple[MultiActorPdb, Callable]: pdb.allow_kbdint = True pdb.nosigint = True - return pdb, Lock.unshield_sigint + return pdb def shield_sigint_handler( @@ -463,17 +552,16 @@ def shield_sigint_handler( ''' Specialized, debugger-aware SIGINT handler. - In childred we always ignore to avoid deadlocks since cancellation - should always be managed by the parent supervising actor. The root - is always cancelled on ctrl-c. + In childred we always ignore/shield for SIGINT to avoid + deadlocks since cancellation should always be managed by the + supervising parent actor. The root actor-proces is always + cancelled on ctrl-c. ''' - __tracebackhide__ = True + __tracebackhide__: bool = True + uid_in_debug: tuple[str, str]|None = Lock.global_actor_in_debug - uid_in_debug: tuple[str, str] | None = Lock.global_actor_in_debug - - actor = current_actor() - # print(f'{actor.uid} in HANDLER with ') + actor: Actor = current_actor() def do_cancel(): # If we haven't tried to cancel the runtime then do that instead @@ -508,7 +596,7 @@ def shield_sigint_handler( return do_cancel() # only set in the actor actually running the REPL - pdb_obj: MultiActorPdb | None = Lock.repl + pdb_obj: MultiActorPdb|None = Lock.repl # root actor branch that reports whether or not a child # has locked debugger. @@ -615,14 +703,20 @@ _pause_msg: str = 'Attaching to pdb REPL in actor' def _set_trace( - actor: tractor.Actor | None = None, - pdb: MultiActorPdb | None = None, + actor: tractor.Actor|None = None, + pdb: MultiActorPdb|None = None, shield: bool = False, extra_frames_up_when_async: int = 1, + hide_tb: bool = True, ): - __tracebackhide__: bool = True - actor: tractor.Actor = actor or current_actor() + __tracebackhide__: bool = hide_tb + + actor: tractor.Actor = ( + actor + or + current_actor() + ) # always start 1 level up from THIS in user code. frame: FrameType|None @@ -668,13 +762,8 @@ def _set_trace( f'Going up frame {i} -> {frame}\n' ) - else: - pdb, undo_sigint = mk_mpdb() - - # we entered the global ``breakpoint()`` built-in from sync - # code? - Lock.local_task_in_debug = 'sync' - + # engage ze REPL + # B~() pdb.set_trace(frame=frame) @@ -684,7 +773,6 @@ async def _pause( # NOTE: must be passed in the `.pause_from_sync()` case! pdb: MultiActorPdb|None = None, - undo_sigint: Callable|None = None, # TODO: allow caller to pause despite task cancellation, # exactly the same as wrapping with: @@ -695,6 +783,7 @@ async def _pause( # be no way to override it?.. # shield: bool = False, + hide_tb: bool = True, task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED ) -> None: @@ -706,9 +795,16 @@ async def _pause( Hopefully we won't need this in the long run. ''' - __tracebackhide__: bool = True - actor = current_actor() - task_name: str = trio.lowlevel.current_task().name + __tracebackhide__: bool = hide_tb + actor: Actor = current_actor() + try: + task_name: str = trio.lowlevel.current_task().name + except RuntimeError as rte: + if actor.is_infected_aio(): + raise RuntimeError( + '`tractor.pause[_from_sync]()` not yet supported ' + 'for infected `asyncio` mode!' + ) from rte if ( not Lock.local_pdb_complete @@ -722,8 +818,7 @@ async def _pause( ) if pdb is None: - assert undo_sigint is None, 'You must pass both!?!' - pdb, undo_sigint = mk_mpdb() + pdb: MultiActorPdb = mk_mpdb() # TODO: need a more robust check for the "root" actor if ( @@ -766,7 +861,6 @@ async def _pause( # ``` # but not entirely sure if that's a sane way to implement it? try: - print("ACQUIRING TTY LOCK from CHILD") with trio.CancelScope(shield=True): await actor._service_n.start( wait_for_parent_stdin_hijack, @@ -786,13 +880,11 @@ async def _pause( raise elif is_root_process(): - print("ROOT TTY LOCK BRANCH") # 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: - print("ROOT ALREADY HAS TTY?") # re-entrant root process already has it: noop. return @@ -806,14 +898,11 @@ async def _pause( # must shield here to avoid hitting a ``Cancelled`` and # a child getting stuck bc we clobbered the tty - print("ACQUIRING TTY LOCK from ROOT") with trio.CancelScope(shield=True): await Lock._debug_lock.acquire() else: # may be cancelled - print("ROOT TRYING LOCK ACQUIRE") await Lock._debug_lock.acquire() - print("ROOT LOCKED TTY") Lock.global_actor_in_debug = actor.uid Lock.local_task_in_debug = task_name @@ -825,7 +914,6 @@ async def _pause( # if debug_func is None: task_status.started(Lock) - print("ROOT .started(Lock) now!") else: # block here one (at the appropriate frame *up*) where @@ -940,20 +1028,74 @@ async def pause( ) +_gb_mod: None|ModuleType|False = None + + +def maybe_import_greenback( + raise_not_found: bool = True, + force_reload: bool = False, + +) -> ModuleType|False: + # be cached-fast on module-already-inited + global _gb_mod + + if _gb_mod is False: + return False + + elif ( + _gb_mod is not None + and not force_reload + ): + return _gb_mod + + try: + import greenback + _gb_mod = greenback + return greenback + + except ModuleNotFoundError as mnf: + log.debug( + '`greenback` is not installed.\n' + 'No sync debug support!\n' + ) + _gb_mod = False + + if raise_not_found: + raise RuntimeError( + 'The `greenback` lib is required to use `tractor.pause_from_sync()`!\n' + 'https://github.com/oremanj/greenback\n' + ) from mnf + + return False + + +async def maybe_init_greenback( + **kwargs, +) -> None|ModuleType: + + if mod := maybe_import_greenback(**kwargs): + await mod.ensure_portal() + log.info( + '`greenback` portal opened!\n' + 'Sync debug support activated!\n' + ) + return mod + + return None # TODO: allow pausing from sync code. # normally by remapping python's builtin breakpoint() hook to this # runtime aware version which takes care of all . def pause_from_sync( - hide_tb: bool = True + hide_tb: bool = False, ) -> None: __tracebackhide__: bool = hide_tb actor: tractor.Actor = current_actor( err_on_no_runtime=False, ) - print( + log.debug( f'{actor.uid}: JUST ENTERED `tractor.pause_from_sync()`' f'|_{actor}\n' ) @@ -965,73 +1107,57 @@ def pause_from_sync( '- `async with tractor.open_root_actor()`\n' ) - try: - import greenback - except ModuleNotFoundError: - raise RuntimeError( - 'The `greenback` lib is required to use `tractor.pause_from_sync()`!\n' - 'https://github.com/oremanj/greenback\n' + # raises on not-found by default + greenback: ModuleType = maybe_import_greenback() + mdb: MultiActorPdb = mk_mpdb() + + # run async task which will lock out the root proc's TTY. + if not Lock.is_trio_thread(): + trio.from_thread.run( + partial( + pause, + debug_func=None, + pdb=mdb, + hide_tb=hide_tb, + ) ) + # TODO: maybe the `trio.current_task()` id/name if avail? + Lock.local_task_in_debug: str = str(threading.current_thread().name) - # out = 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 ;) - db, undo_sigint = mk_mpdb() - greenback.await_( - pause( - debug_func=None, - pdb=db, - undo_sigint=undo_sigint, + else: # we are presumably the `trio.run()` + main thread + greenback.await_( + pause( + debug_func=None, + pdb=mdb, + hide_tb=hide_tb, + ) ) - ) - - Lock.local_task_in_debug = 'sync' + Lock.local_task_in_debug: str = current_task().name # TODO: ensure we aggressively make the user aware about # entering the global ``breakpoint()`` built-in from sync # code? - frame: FrameType | None = sys._getframe() - frame: FrameType = frame.f_back # type: ignore - - # db.config.enable_hidden_frames = True - # assert not db._is_hidden(frame) - # 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) + _set_trace( + actor=actor, + pdb=mdb, + hide_tb=hide_tb, + extra_frames_up_when_async=1, + # TODO? will we ever need it? + # -> the gb._await() won't be affected by cancellation? + # shield=shield, + ) + # LEGACY NOTE on next LOC's frame showing weirdness.. + # # XXX NOTE XXX no other LOC can be here without it # showing up in the REPL's last stack frame !?! # -[ ] tried to use `@pdbp.hideframe` decoration but # still doesn't work - # - # FROM BEFORE: 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? - # - # NOTE: not needed any more anyway since it's all in - # `Lock.release()` now! - # 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) -# pp = pause # short-hand for "pause point" +# NOTE prefer a new "pause" semantic since it better describes +# "pausing the actor's runtime" for this particular +# paralell task to do debugging in a REPL. async def breakpoint(**kwargs): log.warning( '`tractor.breakpoint()` is deprecated!\n' -- 2.34.1 From 0055c1d954b814848530fda81489d578c8545058 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 24 Mar 2024 16:39:59 -0400 Subject: [PATCH 3/3] Tweak main thread predicate to ensure `trio.run()` Change the name to `Lock.is_main_trio_thread()` indicating that when `True` the thread is both the main one **and** the one that called `trio.run()`. Add a todo for just copying the `trio._util.is_main_thread()` impl (since it's private / may change) and some brief notes about potential usage of `trio.from_thread.check_cancelled()` to detect non-`.to_thread` thread spawns. --- tractor/devx/_debug.py | 30 +++++++++++++++++++++++------- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 770995a..bb5740b 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -48,6 +48,7 @@ from types import ( ) import pdbp +import sniffio import tractor import trio from trio.lowlevel import current_task @@ -144,7 +145,7 @@ class Lock: # in which case schedule the SIGINT shielding override # to in the main thread. # https://docs.python.org/3/library/signal.html#signals-and-threads - if not cls.is_trio_thread(): + if not cls.is_main_trio_thread(): cls._orig_sigint_handler: Callable = trio.from_thread.run_sync( signal.signal, signal.SIGINT, @@ -163,7 +164,7 @@ class Lock: # always restore ``trio``'s sigint handler. see notes below in # the pdb factory about the nightmare that is that code swapping # out the handler when the repl activates... - if not cls.is_trio_thread(): + if not cls.is_main_trio_thread(): trio.from_thread.run_sync( signal.signal, signal.SIGINT, @@ -178,15 +179,21 @@ class Lock: cls._orig_sigint_handler = None @classmethod - def is_trio_thread(self) -> bool: + def is_main_trio_thread(cls) -> bool: ''' Check if we're the "main" thread (as in the first one - started by cpython) and presume that it is the thread that + started by cpython) AND that it is ALSO the thread that called `trio.run()` and not some thread spawned with `trio.to_thread.run_sync()`. ''' - return trio._util.is_main_thread() + return ( + # TODO: since this is private, @oremanj says + # we should just copy the impl for now.. + trio._util.is_main_thread() + and + sniffio.current_async_library() == 'trio' + ) # XXX apparently unreliable..see ^ # ( # threading.current_thread() @@ -196,7 +203,7 @@ class Lock: @classmethod def release(cls): try: - if not cls.is_trio_thread(): + if not cls.is_main_trio_thread(): trio.from_thread.run_sync( cls._debug_lock.release ) @@ -1112,7 +1119,16 @@ def pause_from_sync( mdb: MultiActorPdb = mk_mpdb() # run async task which will lock out the root proc's TTY. - if not Lock.is_trio_thread(): + if not Lock.is_main_trio_thread(): + + # TODO: we could also check for a non-`.to_thread` context + # using `trio.from_thread.check_cancelled()` (says + # oremanj) wherein we get the following outputs: + # + # `RuntimeError`: non-`.to_thread` spawned thread + # noop: non-cancelled `.to_thread` + # `trio.Cancelled`: cancelled `.to_thread` + # trio.from_thread.run( partial( pause, -- 2.34.1