diff --git a/examples/debugging/sync_bp.py b/examples/debugging/sync_bp.py new file mode 100644 index 0000000..23469d6 --- /dev/null +++ b/examples/debugging/sync_bp.py @@ -0,0 +1,73 @@ +import trio +import tractor + + +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 +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: + + async with tractor.open_nursery( + debug_mode=True, + ) as an: + + 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/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 3203af1..bb5740b 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -33,35 +33,46 @@ 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 sniffio import tractor import trio from trio.lowlevel import current_task -from trio_typing import ( +from trio import ( TaskStatus, # 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 +127,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_main_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 +164,52 @@ 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_main_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_main_trio_thread(cls) -> bool: + ''' + Check if we're the "main" thread (as in the first one + 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 ( + # 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() + # is not threading.main_thread() + # ) + @classmethod def release(cls): try: - cls._debug_lock.release() + if not cls.is_main_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 @@ -400,7 +476,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, @@ -438,11 +513,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 @@ -451,7 +546,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( @@ -464,17 +559,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 @@ -509,7 +603,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. @@ -616,14 +710,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 @@ -669,20 +769,17 @@ 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) 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, # TODO: allow caller to pause despite task cancellation, # exactly the same as wrapping with: @@ -691,9 +788,9 @@ 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, + hide_tb: bool = True, task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED ) -> None: @@ -705,10 +802,16 @@ async def _pause( Hopefully we won't need this in the long run. ''' - __tracebackhide__: bool = True - actor = current_actor() - pdb, undo_sigint = mk_mpdb() - 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 @@ -716,9 +819,13 @@ 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: + pdb: MultiActorPdb = mk_mpdb() # TODO: need a more robust check for the "root" actor if ( @@ -767,6 +874,7 @@ async def _pause( actor.uid, ) Lock.repl = pdb + except RuntimeError: Lock.release() @@ -811,32 +919,26 @@ 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) - # 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 +964,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 +973,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,89 +1022,158 @@ 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 ) +_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() -> None: - print("ENTER SYNC PAUSE") +def pause_from_sync( + hide_tb: bool = False, +) -> None: + + __tracebackhide__: bool = hide_tb actor: tractor.Actor = current_actor( err_on_no_runtime=False, ) - if actor: - try: - import greenback - # __tracebackhide__ = True + log.debug( + f'{actor.uid}: JUST ENTERED `tractor.pause_from_sync()`' + f'|_{actor}\n' + ) + if not actor: + raise RuntimeError( + 'Not inside the `tractor`-runtime?\n' + '`tractor.pause_from_sync()` is not functional without a wrapping\n' + '- `async with tractor.open_nursery()` or,\n' + '- `async with tractor.open_root_actor()`\n' + ) + # raises on not-found by default + greenback: ModuleType = maybe_import_greenback() + mdb: MultiActorPdb = mk_mpdb() - # task_can_release_tty_lock = trio.Event() + # run async task which will lock out the root proc's TTY. + if not Lock.is_main_trio_thread(): - # 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, - )) + # 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, + 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) - except ModuleNotFoundError: - log.warning('NO GREENBACK FOUND') - else: - log.warning('Not inside actor-runtime') + else: # we are presumably the `trio.run()` + main thread + greenback.await_( + pause( + debug_func=None, + pdb=mdb, + hide_tb=hide_tb, + ) + ) + Lock.local_task_in_debug: str = current_task().name - db, undo_sigint = mk_mpdb() - Lock.local_task_in_debug = 'sync' - # db.config.enable_hidden_frames = True - - # we entered the global ``breakpoint()`` built-in from 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) + _set_trace( + actor=actor, + pdb=mdb, + hide_tb=hide_tb, + extra_frames_up_when_async=1, - 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() - - # 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" -# for this particular paralell task to do debugging B) -# pp = pause # short-hand for "pause point" + # 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 +# 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'