From 969bce3aa4c01b46a0ad41c90b3ab0970baaeb7c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 10 May 2021 07:25:55 -0400 Subject: [PATCH 01/22] Use context for remote debugger locking A context is the natural fit (vs. a receive stream) for locking the root proc's tty usage via it's `.started()` sync point. Simplify the `_breakpoin()` routine to be a simple async func instead of all this "returning a coroutine" stuff from before we decided that `tractor.breakpoint()` must be async. Use `runtime` level for locking logging making it easier to trace. --- tractor/_debug.py | 209 ++++++++++++++++++++++++++++------------------ 1 file changed, 127 insertions(+), 82 deletions(-) diff --git a/tractor/_debug.py b/tractor/_debug.py index c76422a..843b9e1 100644 --- a/tractor/_debug.py +++ b/tractor/_debug.py @@ -7,7 +7,6 @@ from functools import partial from contextlib import asynccontextmanager from typing import Awaitable, Tuple, Optional, Callable, AsyncIterator -from async_generator import aclosing import tractor import trio @@ -38,7 +37,9 @@ _pdb_release_hook: Optional[Callable] = None _in_debug = False # lock in root actor preventing multi-access to local tty -_debug_lock = trio.StrictFIFOLock() +_debug_lock: trio.StrictFIFOLock = trio.StrictFIFOLock() +_debug_lock._uid = None +_pdb_complete: trio.Event = None # XXX: set by the current task waiting on the root tty lock # and must be cancelled if this actor is cancelled via message @@ -119,18 +120,21 @@ async def _acquire_debug_lock(uid: Tuple[str, str]) -> AsyncIterator[None]: """Acquire a actor local FIFO lock meant to mutex entry to a local debugger entry point to avoid tty clobbering by multiple processes. """ - task_name = trio.lowlevel.current_task().name - try: - log.debug( - f"Attempting to acquire TTY lock, remote task: {task_name}:{uid}") - await _debug_lock.acquire() + global _debug_lock - log.debug(f"TTY lock acquired, remote task: {task_name}:{uid}") + task_name = trio.lowlevel.current_task().name + + log.runtime( + f"Attempting to acquire TTY lock, remote task: {task_name}:{uid}") + + async with _debug_lock: + + _debug_lock._uid = uid + log.runtime(f"TTY lock acquired, remote task: {task_name}:{uid}") yield - finally: - _debug_lock.release() - log.debug(f"TTY lock released, remote task: {task_name}:{uid}") + _debug_lock._uid = None + log.runtime(f"TTY lock released, remote task: {task_name}:{uid}") # @contextmanager @@ -144,118 +148,159 @@ async def _acquire_debug_lock(uid: Tuple[str, str]) -> AsyncIterator[None]: # signal.signal(signal.SIGINT, prior_handler) +@tractor.context async def _hijack_stdin_relay_to_child( + + ctx: tractor.context, subactor_uid: Tuple[str, str] + ) -> AsyncIterator[str]: + + global _pdb_complete + + task_name = trio.lowlevel.current_task().name + # TODO: when we get to true remote debugging - # this will deliver stdin data - log.warning(f"Actor {subactor_uid} is WAITING on stdin hijack lock") + # this will deliver stdin data? + + log.debug( + "Attempting to acquire TTY lock, " + f"remote task: {task_name}:{subactor_uid}" + ) + + log.runtime(f"Actor {subactor_uid} is WAITING on stdin hijack lock") + async with _acquire_debug_lock(subactor_uid): - log.warning(f"Actor {subactor_uid} ACQUIRED stdin hijack lock") - # with _disable_sigint(): + with trio.CancelScope(shield=True): - # indicate to child that we've locked stdio - yield 'Locked' + # indicate to child that we've locked stdio + await ctx.started('Locked') + log.runtime(f"Actor {subactor_uid} ACQUIRED stdin hijack lock") - # wait for cancellation of stream by child - # indicating debugger is dis-engaged - await trio.sleep_forever() + # wait for unlock pdb by child + async with ctx.open_stream() as stream: + assert await stream.receive() == 'Unlock' + + log.runtime( + f"TTY lock released, remote task: {task_name}:{subactor_uid}") log.debug(f"Actor {subactor_uid} RELEASED stdin hijack lock") # XXX: We only make this sync in case someone wants to # overload the ``breakpoint()`` built-in. -def _breakpoint(debug_func) -> Awaitable[None]: +async def _breakpoint(debug_func) -> Awaitable[None]: """``tractor`` breakpoint entry for engaging pdb machinery in subactors. """ actor = tractor.current_actor() - do_unlock = trio.Event() + task_name = trio.lowlevel.current_task().name + + global _pdb_complete + global _pdb_release_hook + global _in_debug async def wait_for_parent_stdin_hijack( task_status=trio.TASK_STATUS_IGNORED ): global _debugger_request_cs + with trio.CancelScope() as cs: _debugger_request_cs = cs + try: async with get_root() as portal: - async with portal.open_stream_from( - tractor._debug._hijack_stdin_relay_to_child, - subactor_uid=actor.uid, - ) as stream: - # block until first yield above - async for val in stream: + # this syncs to child's ``Context.started()`` call. + async with portal.open_context( - assert val == 'Locked' - task_status.started() + tractor._debug._hijack_stdin_relay_to_child, + subactor_uid=actor.uid, - # with trio.CancelScope(shield=True): - await do_unlock.wait() + ) as (ctx, val): + + assert val == 'Locked' + + async with ctx.open_stream() as stream: + + # unblock local caller + task_status.started() + + await _pdb_complete.wait() + await stream.send('Unlock') - # trigger cancellation of remote stream - break finally: log.debug(f"Exiting debugger for actor {actor}") global _in_debug _in_debug = False log.debug(f"Child {actor} released parent stdio lock") - async def _bp(): - """Async breakpoint which schedules a parent stdio lock, and once complete - enters the ``pdbpp`` debugging console. - """ - task_name = trio.lowlevel.current_task().name + if not _pdb_complete or _pdb_complete.is_set(): + _pdb_complete = trio.Event() - global _in_debug - - # TODO: need a more robust check for the "root" actor - if actor._parent_chan and not is_root_process(): - if _in_debug: - if _in_debug == task_name: - # this task already has the lock and is - # likely recurrently entering a breakpoint - 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 {actor.uid} already has a debug lock, waiting...") - await do_unlock.wait() - await trio.sleep(0.1) - - # assign unlock callback for debugger teardown hooks - global _pdb_release_hook - _pdb_release_hook = do_unlock.set - - # mark local actor as "in debug mode" to avoid recurrent - # entries/requests to the root process - _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. - await actor._service_n.start(wait_for_parent_stdin_hijack) - - elif is_root_process(): - # we also wait in the root-parent for any child that - # may have the tty locked prior - if _debug_lock.locked(): # root process already has it; ignore + # TODO: need a more robust check for the "root" actor + if actor._parent_chan and not is_root_process(): + if _in_debug: + if _in_debug == task_name: + # this task already has the lock and is + # likely recurrently entering a breakpoint return - await _debug_lock.acquire() - _pdb_release_hook = _debug_lock.release - # 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) + # 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 {actor.uid} already has a debug lock, waiting...") + await _pdb_complete.wait() + await trio.sleep(0.1) - # user code **must** await this! - return _bp() + # mark local actor as "in debug mode" to avoid recurrent + # entries/requests to the root process + _in_debug = task_name + + # assign unlock callback for debugger teardown hooks + _pdb_release_hook = _pdb_complete.set + + # 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. + await actor._service_n.start(wait_for_parent_stdin_hijack) + + elif is_root_process(): + + # we also wait in the root-parent for any child that + # may have the tty locked prior + global _debug_lock + + # TODO: wait, what about multiple root tasks acquiring + # it though.. shrug? + # root process (us) already has it; ignore + if _debug_lock._uid == actor.uid: + 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. + await _debug_lock.acquire() + + _debug_lock._uid = actor.uid + + # the lock must be released on pdb completion + def teardown(): + global _pdb_complete + global _debug_lock + + _debug_lock.release() + _debug_lock._uid = None + _pdb_complete.set() + + _pdb_release_hook = teardown + + # 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) def _mk_pdb(): From b369b9135705d630ee5285b53a631bf869ea8861 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 12 May 2021 12:01:43 -0400 Subject: [PATCH 02/22] Fix up var naming and typing --- tractor/_debug.py | 86 ++++++++++++++++++++++++++--------------------- 1 file changed, 47 insertions(+), 39 deletions(-) diff --git a/tractor/_debug.py b/tractor/_debug.py index 843b9e1..da05480 100644 --- a/tractor/_debug.py +++ b/tractor/_debug.py @@ -1,11 +1,12 @@ """ Multi-core debugging for da peeps! + """ import bdb import sys from functools import partial from contextlib import asynccontextmanager -from typing import Awaitable, Tuple, Optional, Callable, AsyncIterator +from typing import Tuple, Optional, Callable, AsyncIterator import tractor import trio @@ -30,16 +31,21 @@ log = get_logger(__name__) __all__ = ['breakpoint', 'post_mortem'] + +# TODO: wrap all these in a static global class: ``DebugLock`` maybe? + # placeholder for function to set a ``trio.Event`` on debugger exit _pdb_release_hook: Optional[Callable] = None # actor-wide variable pointing to current task name using debugger -_in_debug = False +_local_task_in_debug: Optional[str] = None + +# actor tree-wide actor uid that supposedly has the tty lock +_global_actor_in_debug: Optional[Tuple[str, str]] = None # lock in root actor preventing multi-access to local tty _debug_lock: trio.StrictFIFOLock = trio.StrictFIFOLock() -_debug_lock._uid = None -_pdb_complete: trio.Event = None +_pdb_complete: Optional[trio.Event] = None # XXX: set by the current task waiting on the root tty lock # and must be cancelled if this actor is cancelled via message @@ -62,19 +68,19 @@ class PdbwTeardown(pdbpp.Pdb): # TODO: figure out how to dissallow recursive .set_trace() entry # since that'll cause deadlock for us. def set_continue(self): - global _in_debug try: super().set_continue() finally: - _in_debug = False + global _local_task_in_debug + _local_task_in_debug = None _pdb_release_hook() def set_quit(self): - global _in_debug try: super().set_quit() finally: - _in_debug = False + global _local_task_in_debug + _local_task_in_debug = None _pdb_release_hook() @@ -120,21 +126,22 @@ async def _acquire_debug_lock(uid: Tuple[str, str]) -> AsyncIterator[None]: """Acquire a actor local FIFO lock meant to mutex entry to a local debugger entry point to avoid tty clobbering by multiple processes. """ - global _debug_lock + global _debug_lock, _global_actor_in_debug task_name = trio.lowlevel.current_task().name - log.runtime( + log.debug( f"Attempting to acquire TTY lock, remote task: {task_name}:{uid}") async with _debug_lock: - _debug_lock._uid = uid - log.runtime(f"TTY lock acquired, remote task: {task_name}:{uid}") + # _debug_lock._uid = uid + _global_actor_in_debug = uid + log.debug(f"TTY lock acquired, remote task: {task_name}:{uid}") yield - _debug_lock._uid = None - log.runtime(f"TTY lock released, remote task: {task_name}:{uid}") + _global_actor_in_debug = None + log.debug(f"TTY lock released, remote task: {task_name}:{uid}") # @contextmanager @@ -151,10 +158,10 @@ async def _acquire_debug_lock(uid: Tuple[str, str]) -> AsyncIterator[None]: @tractor.context async def _hijack_stdin_relay_to_child( - ctx: tractor.context, + ctx: tractor.Context, subactor_uid: Tuple[str, str] -) -> AsyncIterator[str]: +) -> None: global _pdb_complete @@ -168,7 +175,7 @@ async def _hijack_stdin_relay_to_child( f"remote task: {task_name}:{subactor_uid}" ) - log.runtime(f"Actor {subactor_uid} is WAITING on stdin hijack lock") + log.debug(f"Actor {subactor_uid} is WAITING on stdin hijack lock") async with _acquire_debug_lock(subactor_uid): @@ -176,30 +183,29 @@ async def _hijack_stdin_relay_to_child( # indicate to child that we've locked stdio await ctx.started('Locked') - log.runtime(f"Actor {subactor_uid} ACQUIRED stdin hijack lock") + log.runtime( # type: ignore + f"Actor {subactor_uid} ACQUIRED stdin hijack lock") # wait for unlock pdb by child async with ctx.open_stream() as stream: assert await stream.receive() == 'Unlock' - log.runtime( + log.debug( f"TTY lock released, remote task: {task_name}:{subactor_uid}") log.debug(f"Actor {subactor_uid} RELEASED stdin hijack lock") -# XXX: We only make this sync in case someone wants to -# overload the ``breakpoint()`` built-in. -async def _breakpoint(debug_func) -> Awaitable[None]: +async def _breakpoint(debug_func) -> None: """``tractor`` breakpoint entry for engaging pdb machinery in subactors. + """ actor = tractor.current_actor() task_name = trio.lowlevel.current_task().name - global _pdb_complete - global _pdb_release_hook - global _in_debug + global _pdb_complete, _pdb_release_hook + global _local_task_in_debug, _global_actor_in_debug async def wait_for_parent_stdin_hijack( task_status=trio.TASK_STATUS_IGNORED @@ -232,8 +238,8 @@ async def _breakpoint(debug_func) -> Awaitable[None]: finally: log.debug(f"Exiting debugger for actor {actor}") - global _in_debug - _in_debug = False + global _local_task_in_debug + _local_task_in_debug = None log.debug(f"Child {actor} released parent stdio lock") if not _pdb_complete or _pdb_complete.is_set(): @@ -241,8 +247,8 @@ async def _breakpoint(debug_func) -> Awaitable[None]: # TODO: need a more robust check for the "root" actor if actor._parent_chan and not is_root_process(): - if _in_debug: - if _in_debug == task_name: + if _local_task_in_debug: + if _local_task_in_debug == task_name: # this task already has the lock and is # likely recurrently entering a breakpoint return @@ -250,14 +256,14 @@ async def _breakpoint(debug_func) -> Awaitable[None]: # 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 {actor.uid} already has a debug lock, waiting...") + log.warning(f"{actor.uid} already has a debug lock, waiting...") + await _pdb_complete.wait() await trio.sleep(0.1) # mark local actor as "in debug mode" to avoid recurrent # entries/requests to the root process - _in_debug = task_name + _local_task_in_debug = task_name # assign unlock callback for debugger teardown hooks _pdb_release_hook = _pdb_complete.set @@ -276,7 +282,7 @@ async def _breakpoint(debug_func) -> Awaitable[None]: # TODO: wait, what about multiple root tasks acquiring # it though.. shrug? # root process (us) already has it; ignore - if _debug_lock._uid == actor.uid: + if _global_actor_in_debug == actor.uid: return # XXX: since we need to enter pdb synchronously below, @@ -284,15 +290,17 @@ async def _breakpoint(debug_func) -> Awaitable[None]: # callbacks. Can't think of a nicer way then this atm. await _debug_lock.acquire() - _debug_lock._uid = actor.uid + _global_actor_in_debug = actor.uid + _local_task_in_debug = task_name # the lock must be released on pdb completion def teardown(): - global _pdb_complete - global _debug_lock + global _pdb_complete, _debug_lock + global _global_actor_in_debug, _local_task_in_debug _debug_lock.release() - _debug_lock._uid = None + _global_actor_in_debug = None + _local_task_in_debug = None _pdb_complete.set() _pdb_release_hook = teardown @@ -330,8 +338,8 @@ def _set_trace(actor=None): else: # we entered the global ``breakpoint()`` built-in from sync code - global _in_debug, _pdb_release_hook - _in_debug = 'sync' + global _local_task_in_debug, _pdb_release_hook + _local_task_in_debug = 'sync' def nuttin(): pass From 6f05f5d5e6349c78a7bc536a71ec036c3ee95503 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 13 Jun 2021 20:21:18 -0400 Subject: [PATCH 03/22] Wait for debugger lock task context termination --- tractor/_debug.py | 26 +++++++++++++++++++++++--- 1 file changed, 23 insertions(+), 3 deletions(-) diff --git a/tractor/_debug.py b/tractor/_debug.py index da05480..4bb67c7 100644 --- a/tractor/_debug.py +++ b/tractor/_debug.py @@ -161,7 +161,7 @@ async def _hijack_stdin_relay_to_child( ctx: tractor.Context, subactor_uid: Tuple[str, str] -) -> None: +) -> str: global _pdb_complete @@ -179,6 +179,7 @@ async def _hijack_stdin_relay_to_child( async with _acquire_debug_lock(subactor_uid): + # XXX: only shield the context sync step! with trio.CancelScope(shield=True): # indicate to child that we've locked stdio @@ -188,12 +189,22 @@ async def _hijack_stdin_relay_to_child( # wait for unlock pdb by child async with ctx.open_stream() as stream: - assert await stream.receive() == 'Unlock' + try: + assert await stream.receive() == 'pdb_unlock' + + except trio.BrokenResourceError: + # XXX: there may be a race with the portal teardown + # with the calling actor which we can safely ignore + # the alternative would be sending an ack message + # and allowing the client to wait for us to teardown + # first? + pass log.debug( f"TTY lock released, remote task: {task_name}:{subactor_uid}") log.debug(f"Actor {subactor_uid} RELEASED stdin hijack lock") + return "pdb_unlock_complete" async def _breakpoint(debug_func) -> None: @@ -233,8 +244,17 @@ async def _breakpoint(debug_func) -> None: # unblock local caller task_status.started() + # TODO: shielding currently can cause hangs... + # with trio.CancelScope(shield=True): + await _pdb_complete.wait() - await stream.send('Unlock') + await stream.send('pdb_unlock') + + # sync with callee termination + assert await ctx.result() == "pdb_unlock_complete" + + except tractor.ContextCancelled: + log.warning('Root actor cancelled debug lock') finally: log.debug(f"Exiting debugger for actor {actor}") From 49d439b681aa47cc7b0421d577e61e403dc7d888 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 27 Jun 2021 00:45:59 -0400 Subject: [PATCH 04/22] Add some brief todo notes on idea of shielded breakpoint --- tractor/_debug.py | 21 +++++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/tractor/_debug.py b/tractor/_debug.py index 4bb67c7..777e1b9 100644 --- a/tractor/_debug.py +++ b/tractor/_debug.py @@ -207,11 +207,24 @@ async def _hijack_stdin_relay_to_child( return "pdb_unlock_complete" -async def _breakpoint(debug_func) -> None: - """``tractor`` breakpoint entry for engaging pdb machinery - in subactors. +async def _breakpoint( + + debug_func, + + # TODO: + # shield: bool = False + +) -> None: + '''``tractor`` breakpoint entry for engaging pdb machinery + in the root or a subactor. + + ''' + # 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): - """ actor = tractor.current_actor() task_name = trio.lowlevel.current_task().name From 63bdddd0c933e79ca472b020da109ec68eb97b20 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 10 May 2021 07:46:16 -0400 Subject: [PATCH 05/22] Add debug example that causes pdb stdin clobbering --- .../root_timeout_while_child_crashed.py | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) create mode 100644 examples/debugging/root_timeout_while_child_crashed.py diff --git a/examples/debugging/root_timeout_while_child_crashed.py b/examples/debugging/root_timeout_while_child_crashed.py new file mode 100644 index 0000000..09a9003 --- /dev/null +++ b/examples/debugging/root_timeout_while_child_crashed.py @@ -0,0 +1,31 @@ + +import trio +import tractor + + +async def key_error(): + "Raise a ``NameError``" + return {}['doggy'] + + +async def main(): + """Root dies + + """ + async with tractor.open_nursery( + debug_mode=True, + loglevel='debug' + ) as n: + + # spawn both actors + portal = await n.run_in_actor(key_error) + + # XXX: originally a bug causes by this + # where root would enter debugger even + # though child should have it locked. + with trio.fail_after(1): + await trio.Event().wait() + + +if __name__ == '__main__': + trio.run(main) From 23a1622256b373b8d48fe2be86824dc1104b6083 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 10 May 2021 07:47:38 -0400 Subject: [PATCH 06/22] Don't kill root's immediate children when in debug If the root calls `trio.Process.kill()` on immediate child proc teardown when the child is using pdb, we can get stdstreams clobbering that results in a pdb++ repl where the user can't see what's been typed. Not killing such children on cancellation / error seems to resolve this issue whilst still giving reliable termination. For now, code that special path until a time it becomes a problem for ensuring zombie reaps. --- tractor/_spawn.py | 40 ++++++++++++++++++++++++++++++++++++++-- 1 file changed, 38 insertions(+), 2 deletions(-) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index 678250b..bc6ef2d 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -25,7 +25,10 @@ from . import _forkserver_override from ._state import ( current_actor, is_main_process, + is_root_process, + _runtime_vars, ) + from .log import get_logger from ._portal import Portal from ._actor import Actor, ActorFailure @@ -206,13 +209,46 @@ async def spawn_subactor( yield proc finally: + log.debug(f"Attempting to kill {proc}") # XXX: do this **after** cancellation/tearfown # to avoid killing the process too early # since trio does this internally on ``__aexit__()`` - log.debug(f"Attempting to kill {proc}") - await do_hard_kill(proc) + if ( + is_root_process() + + # XXX: basically the pre-closing of stdstreams in a + # root-processe's ``trio.Process.aclose()`` can clobber + # any existing debugger session so we avoid + and _runtime_vars['_debug_mode'] + ): + # XXX: this is ``trio.Process.aclose()`` minus + # the std-streams pre-closing steps and ``Process.kill()`` + # calls. + try: + await proc.wait() + finally: + if proc.returncode is None: + # XXX: skip this when in debug and a session might + # still be live + # proc.kill() + with trio.CancelScope(shield=True): + await proc.wait() + else: + # NOTE: this timeout used to do nothing since we were shielding + # the ``.wait()`` inside ``new_proc()`` which will pretty much + # never release until the process exits, now it acts as + # a hard-kill time ultimatum. + with trio.move_on_after(3) as cs: + + # NOTE: This ``__aexit__()`` shields internally. + async with proc: # calls ``trio.Process.aclose()`` + log.debug(f"Terminating {proc}") + + if cs.cancelled_caught: + log.critical(f"HARD KILLING {proc}") + proc.kill() async def new_proc( From 0f2a39a3111af7575f8ff8a83160c779c26cd330 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 30 May 2021 11:33:08 -0400 Subject: [PATCH 07/22] Catch and delay errors in the root if debugger is active --- tractor/_trionics.py | 25 +++++++++++++++++++++---- 1 file changed, 21 insertions(+), 4 deletions(-) diff --git a/tractor/_trionics.py b/tractor/_trionics.py index eea3aae..8f0bcf0 100644 --- a/tractor/_trionics.py +++ b/tractor/_trionics.py @@ -11,6 +11,7 @@ import warnings import trio from async_generator import asynccontextmanager +from . import _debug from ._state import current_actor, is_main_process from .log import get_logger, get_loglevel from ._actor import Actor @@ -368,10 +369,26 @@ async def open_nursery( async with open_root_actor(**kwargs) as actor: assert actor is current_actor() - async with _open_and_supervise_one_cancels_all_nursery( - actor - ) as anursery: - yield anursery + try: + async with _open_and_supervise_one_cancels_all_nursery( + actor + ) as anursery: + yield anursery + + except (Exception, trio.MultiError, trio.Cancelled): + # if we error in the root but the debugger is + # engaged we don't want to prematurely kill (and + # thus clobber access to) the local tty streams. + # instead try to wait for pdb to be released before + # tearing down. + if not _debug._pdb_complete.is_set(): + log.warning( + "Root has errored but pdb is active..waiting " + "on debug lock") + with trio.CancelScope(shield=True): + await _debug._pdb_complete.wait() + + raise else: # sub-nursery case From 37a1897c4725566b102b694a9bcc41ffc7ba572a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 2 Jun 2021 08:24:59 -0400 Subject: [PATCH 08/22] Don't shield debugger status wait; it causes hangs --- tractor/_trionics.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tractor/_trionics.py b/tractor/_trionics.py index 8f0bcf0..66080e0 100644 --- a/tractor/_trionics.py +++ b/tractor/_trionics.py @@ -385,8 +385,7 @@ async def open_nursery( log.warning( "Root has errored but pdb is active..waiting " "on debug lock") - with trio.CancelScope(shield=True): - await _debug._pdb_complete.wait() + await _debug._pdb_complete.wait() raise From 0fdcfa0ba108314211587edaef010dcee560d597 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 10 Jun 2021 14:02:12 -0400 Subject: [PATCH 09/22] Move debugger wait inside OCA nursery --- tractor/_trionics.py | 46 +++++++++++++++++++++++++------------------- 1 file changed, 26 insertions(+), 20 deletions(-) diff --git a/tractor/_trionics.py b/tractor/_trionics.py index 66080e0..33ad71a 100644 --- a/tractor/_trionics.py +++ b/tractor/_trionics.py @@ -12,7 +12,7 @@ import trio from async_generator import asynccontextmanager from . import _debug -from ._state import current_actor, is_main_process +from ._state import current_actor, is_main_process, is_root_process from .log import get_logger, get_loglevel from ._actor import Actor from ._portal import Portal @@ -260,6 +260,26 @@ async def _open_and_supervise_one_cancels_all_nursery( anursery._join_procs.set() except BaseException as err: + + if is_root_process() and ( + type(err) in { + Exception, trio.MultiError, trio.Cancelled + } + ): + # if we error in the root but the debugger is + # engaged we don't want to prematurely kill (and + # thus clobber access to) the local tty streams. + # instead try to wait for pdb to be released before + # tearing down. + debug_complete = _debug._pdb_complete + if debug_complete and not debug_complete.is_set(): + log.warning( + "Root has errored but pdb is active..waiting " + "on debug lock") + await _debug._pdb_complete.wait() + + # raise + # if the caller's scope errored then we activate our # one-cancels-all supervisor strategy (don't # worry more are coming). @@ -369,25 +389,11 @@ async def open_nursery( async with open_root_actor(**kwargs) as actor: assert actor is current_actor() - try: - async with _open_and_supervise_one_cancels_all_nursery( - actor - ) as anursery: - yield anursery - - except (Exception, trio.MultiError, trio.Cancelled): - # if we error in the root but the debugger is - # engaged we don't want to prematurely kill (and - # thus clobber access to) the local tty streams. - # instead try to wait for pdb to be released before - # tearing down. - if not _debug._pdb_complete.is_set(): - log.warning( - "Root has errored but pdb is active..waiting " - "on debug lock") - await _debug._pdb_complete.wait() - - raise + # try: + async with _open_and_supervise_one_cancels_all_nursery( + actor + ) as anursery: + yield anursery else: # sub-nursery case From 5b3894827f3565a6568b6a5aa5d7c808c4c4f6cf Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 26 Jun 2021 23:35:59 -0400 Subject: [PATCH 10/22] Move some infos to runtime level --- tractor/_spawn.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index bc6ef2d..4d6c512 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -147,7 +147,7 @@ async def cancel_on_completion( ) else: - log.info( + log.runtime( f"Cancelling {portal.channel.uid} gracefully " f"after result {result}") @@ -277,7 +277,7 @@ async def new_proc( subactor, parent_addr, ) as proc: - log.info(f"Started {proc}") + log.runtime(f"Started {proc}") # wait for actor to spawn and connect back to us # channel should have handshake completed by the @@ -432,7 +432,7 @@ async def mp_new_proc( if not proc.is_alive(): raise ActorFailure("Couldn't start sub-actor?") - log.info(f"Started {proc}") + log.runtime(f"Started {proc}") try: # wait for actor to spawn and connect back to us From ef89ed947a82f96c706237dc630bd2b34eb03d32 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 25 Jun 2021 20:52:08 -0400 Subject: [PATCH 11/22] Fix hard kill in debug mode; only do it when debug lock is empty --- tractor/_spawn.py | 61 +++++++++++++++++++++++++++++++---------------- 1 file changed, 40 insertions(+), 21 deletions(-) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index 4d6c512..dcbc49d 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -28,6 +28,7 @@ from ._state import ( is_root_process, _runtime_vars, ) +from ._debug import _global_actor_in_debug from .log import get_logger from ._portal import Portal @@ -154,6 +155,26 @@ async def cancel_on_completion( # cancel the process now that we have a final result await portal.cancel_actor() +async def do_hard_kill( + proc: trio.Process, +) -> None: + # NOTE: this timeout used to do nothing since we were shielding + # the ``.wait()`` inside ``new_proc()`` which will pretty much + # never release until the process exits, now it acts as + # a hard-kill time ultimatum. + with trio.move_on_after(3) as cs: + + # NOTE: This ``__aexit__()`` shields internally. + async with proc: # calls ``trio.Process.aclose()`` + log.debug(f"Terminating {proc}") + + if cs.cancelled_caught: + # XXX: should pretty much never get here unless we have + # to move the bits from ``proc.__aexit__()`` out and + # into here. + log.critical(f"HARD KILLING {proc}") + proc.kill() + async def do_hard_kill( proc: trio.Process, @@ -222,33 +243,31 @@ async def spawn_subactor( # root-processe's ``trio.Process.aclose()`` can clobber # any existing debugger session so we avoid and _runtime_vars['_debug_mode'] + and _global_actor_in_debug is not None ): - # XXX: this is ``trio.Process.aclose()`` minus - # the std-streams pre-closing steps and ``Process.kill()`` - # calls. + # XXX: this is ``trio.Process.aclose()`` MINUS the + # std-streams pre-closing steps inside ``proc.__aexit__()`` + # (see below) which incluses a ``Process.kill()`` call + + log.critical( + "Root process tty is locked in debug mode by " + f"{_global_actor_in_debug}. If the console is hanging, you " + "may need to trigger a KBI to kill any " + "not-fully-initialized" " subprocesses and allow errors " + "from `trio` to propagate" + ) try: + # one more graceful wait try can can be cancelled by KBI + # sent by user. await proc.wait() + finally: if proc.returncode is None: - # XXX: skip this when in debug and a session might - # still be live - # proc.kill() - with trio.CancelScope(shield=True): - await proc.wait() + # with trio.CancelScope(shield=True): + await do_hard_kill(proc) else: - # NOTE: this timeout used to do nothing since we were shielding - # the ``.wait()`` inside ``new_proc()`` which will pretty much - # never release until the process exits, now it acts as - # a hard-kill time ultimatum. - with trio.move_on_after(3) as cs: - - # NOTE: This ``__aexit__()`` shields internally. - async with proc: # calls ``trio.Process.aclose()`` - log.debug(f"Terminating {proc}") - - if cs.cancelled_caught: - log.critical(f"HARD KILLING {proc}") - proc.kill() + # with trio.CancelScope(shield=True): + await do_hard_kill(proc) async def new_proc( From be1fcb2a5b64086da1b7215979c24ce403fd4816 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 27 Jun 2021 11:59:21 -0400 Subject: [PATCH 12/22] Distinguish between a local pdb unlock and the tty unlock in root --- tractor/_debug.py | 40 ++++++++++++++++++++++++++-------------- 1 file changed, 26 insertions(+), 14 deletions(-) diff --git a/tractor/_debug.py b/tractor/_debug.py index 777e1b9..6623d6c 100644 --- a/tractor/_debug.py +++ b/tractor/_debug.py @@ -45,7 +45,8 @@ _global_actor_in_debug: Optional[Tuple[str, str]] = None # lock in root actor preventing multi-access to local tty _debug_lock: trio.StrictFIFOLock = trio.StrictFIFOLock() -_pdb_complete: Optional[trio.Event] = None +_local_pdb_complete: Optional[trio.Event] = None +_no_remote_has_tty: Optional[trio.Event] = None # XXX: set by the current task waiting on the root tty lock # and must be cancelled if this actor is cancelled via message @@ -130,14 +131,15 @@ async def _acquire_debug_lock(uid: Tuple[str, str]) -> AsyncIterator[None]: task_name = trio.lowlevel.current_task().name - log.debug( - f"Attempting to acquire TTY lock, remote task: {task_name}:{uid}") + log.pdb( + f"Attempting to acquire TTY lock, remote task: {task_name}:{uid}" + ) async with _debug_lock: - # _debug_lock._uid = uid _global_actor_in_debug = uid log.debug(f"TTY lock acquired, remote task: {task_name}:{uid}") + yield _global_actor_in_debug = None @@ -162,8 +164,17 @@ async def _hijack_stdin_relay_to_child( subactor_uid: Tuple[str, str] ) -> str: + '''Hijack the tty in the root process of an actor tree such that + the pdbpp debugger console can be allocated to a sub-actor for repl + bossing. - global _pdb_complete + ''' + global _no_remote_has_tty + + # mark the tty lock as being in use so that the runtime + # can try to avoid clobbering any connection from a child + # that's currently relying on it. + _no_remote_has_tty = trio.Event() task_name = trio.lowlevel.current_task().name @@ -184,7 +195,7 @@ async def _hijack_stdin_relay_to_child( # indicate to child that we've locked stdio await ctx.started('Locked') - log.runtime( # type: ignore + log.pdb( # type: ignore f"Actor {subactor_uid} ACQUIRED stdin hijack lock") # wait for unlock pdb by child @@ -204,6 +215,7 @@ async def _hijack_stdin_relay_to_child( f"TTY lock released, remote task: {task_name}:{subactor_uid}") log.debug(f"Actor {subactor_uid} RELEASED stdin hijack lock") + _no_remote_has_tty.set() return "pdb_unlock_complete" @@ -228,7 +240,7 @@ async def _breakpoint( actor = tractor.current_actor() task_name = trio.lowlevel.current_task().name - global _pdb_complete, _pdb_release_hook + global _local_pdb_complete, _pdb_release_hook global _local_task_in_debug, _global_actor_in_debug async def wait_for_parent_stdin_hijack( @@ -260,7 +272,7 @@ async def _breakpoint( # TODO: shielding currently can cause hangs... # with trio.CancelScope(shield=True): - await _pdb_complete.wait() + await _local_pdb_complete.wait() await stream.send('pdb_unlock') # sync with callee termination @@ -275,8 +287,8 @@ async def _breakpoint( _local_task_in_debug = None log.debug(f"Child {actor} released parent stdio lock") - if not _pdb_complete or _pdb_complete.is_set(): - _pdb_complete = trio.Event() + if not _local_pdb_complete or _local_pdb_complete.is_set(): + _local_pdb_complete = trio.Event() # TODO: need a more robust check for the "root" actor if actor._parent_chan and not is_root_process(): @@ -291,7 +303,7 @@ async def _breakpoint( # support for recursive entries to `tractor.breakpoint()` log.warning(f"{actor.uid} already has a debug lock, waiting...") - await _pdb_complete.wait() + await _local_pdb_complete.wait() await trio.sleep(0.1) # mark local actor as "in debug mode" to avoid recurrent @@ -299,7 +311,7 @@ async def _breakpoint( _local_task_in_debug = task_name # assign unlock callback for debugger teardown hooks - _pdb_release_hook = _pdb_complete.set + _pdb_release_hook = _local_pdb_complete.set # this **must** be awaited by the caller and is done using the # root nursery so that the debugger can continue to run without @@ -328,13 +340,13 @@ async def _breakpoint( # the lock must be released on pdb completion def teardown(): - global _pdb_complete, _debug_lock + global _local_pdb_complete, _debug_lock global _global_actor_in_debug, _local_task_in_debug _debug_lock.release() _global_actor_in_debug = None _local_task_in_debug = None - _pdb_complete.set() + _local_pdb_complete.set() _pdb_release_hook = teardown From 551816e80d8c08bb3ea5649a988beff2151bf2ca Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 29 Jun 2021 15:15:32 -0400 Subject: [PATCH 13/22] Solve the root-cancels-child-in-tty-lock race Finally this makes a cancelled root actor nursery not clobber child tasks which request and lock the root's tty for the debugger repl. Using an edge triggered event which is set after all fifo-lock-queued tasks are complete, we can be sure that no lingering child tasks are going to get interrupted during pdb use and tty lock acquisition. Further, even if new tasks do queue up to get the lock, the root will incrementally send cancel msgs to each sub-actor only once the tty is not locked by a (set of) child request task(s). Add shielding around all the critical sections where the child attempts to allocate the lock from the root such that it won't be disrupted from cancel messages from the root after the acquire lock transaction has started. --- tractor/_debug.py | 132 ++++++++++++++++++++++++++++++------------- tractor/_trionics.py | 70 ++++++++++++++++------- 2 files changed, 144 insertions(+), 58 deletions(-) diff --git a/tractor/_debug.py b/tractor/_debug.py index 6623d6c..7cc7d97 100644 --- a/tractor/_debug.py +++ b/tractor/_debug.py @@ -124,10 +124,11 @@ class PdbwTeardown(pdbpp.Pdb): @asynccontextmanager async def _acquire_debug_lock(uid: Tuple[str, str]) -> AsyncIterator[None]: - """Acquire a actor local FIFO lock meant to mutex entry to a local - debugger entry point to avoid tty clobbering by multiple processes. - """ - global _debug_lock, _global_actor_in_debug + '''Acquire a actor local FIFO lock meant to mutex entry to a local + debugger entry point to avoid tty clobbering a global root process. + + ''' + global _debug_lock, _global_actor_in_debug, _no_remote_has_tty task_name = trio.lowlevel.current_task().name @@ -135,15 +136,60 @@ async def _acquire_debug_lock(uid: Tuple[str, str]) -> AsyncIterator[None]: f"Attempting to acquire TTY lock, remote task: {task_name}:{uid}" ) - async with _debug_lock: + we_acquired = False + + if _no_remote_has_tty is None: + # mark the tty lock as being in use so that the runtime + # can try to avoid clobbering any connection from a child + # that's currently relying on it. + _no_remote_has_tty = trio.Event() + + try: + log.debug( + f"entering lock checkpoint, remote task: {task_name}:{uid}" + ) + we_acquired = True + await _debug_lock.acquire() + + # we_acquired = True _global_actor_in_debug = uid log.debug(f"TTY lock acquired, remote task: {task_name}:{uid}") - yield + # NOTE: critical section! + # this yield is unshielded. + # IF we received a cancel during the shielded lock + # entry of some next-in-queue requesting task, + # then the resumption here will result in that + # Cancelled being raised to our caller below! - _global_actor_in_debug = None - log.debug(f"TTY lock released, remote task: {task_name}:{uid}") + # in this case the finally below should trigger + # and the surrounding calle side context should cancel + # normally relaying back to the caller. + + yield _debug_lock + + finally: + # if _global_actor_in_debug == uid: + if we_acquired and _debug_lock.locked(): + _debug_lock.release() + + # IFF there are no more requesting tasks queued up fire, the + # "tty-unlocked" event thereby alerting any monitors of the lock that + # we are now back in the "tty unlocked" state. This is basically + # and edge triggered signal around an empty queue of sub-actor + # tasks that may have tried to acquire the lock. + stats = _debug_lock.statistics() + if ( + not stats.owner + ): + log.pdb(f"No more tasks waiting on tty lock! says {uid}") + _no_remote_has_tty.set() + _no_remote_has_tty = None + + _global_actor_in_debug = None + + log.debug(f"TTY lock released, remote task: {task_name}:{uid}") # @contextmanager @@ -169,53 +215,43 @@ async def _hijack_stdin_relay_to_child( bossing. ''' - global _no_remote_has_tty - - # mark the tty lock as being in use so that the runtime - # can try to avoid clobbering any connection from a child - # that's currently relying on it. - _no_remote_has_tty = trio.Event() - task_name = trio.lowlevel.current_task().name # TODO: when we get to true remote debugging # this will deliver stdin data? log.debug( - "Attempting to acquire TTY lock, " + "Attempting to acquire TTY lock\n" f"remote task: {task_name}:{subactor_uid}" ) log.debug(f"Actor {subactor_uid} is WAITING on stdin hijack lock") - async with _acquire_debug_lock(subactor_uid): + with trio.CancelScope(shield=True): - # XXX: only shield the context sync step! - with trio.CancelScope(shield=True): + async with _acquire_debug_lock(subactor_uid): # indicate to child that we've locked stdio await ctx.started('Locked') log.pdb( # type: ignore f"Actor {subactor_uid} ACQUIRED stdin hijack lock") - # wait for unlock pdb by child - async with ctx.open_stream() as stream: - try: - assert await stream.receive() == 'pdb_unlock' + # wait for unlock pdb by child + async with ctx.open_stream() as stream: + try: + assert await stream.receive() == 'pdb_unlock' - except trio.BrokenResourceError: - # XXX: there may be a race with the portal teardown - # with the calling actor which we can safely ignore - # the alternative would be sending an ack message - # and allowing the client to wait for us to teardown - # first? - pass + except trio.BrokenResourceError: + # XXX: there may be a race with the portal teardown + # with the calling actor which we can safely ignore + # the alternative would be sending an ack message + # and allowing the client to wait for us to teardown + # first? + pass log.debug( f"TTY lock released, remote task: {task_name}:{subactor_uid}") - log.debug(f"Actor {subactor_uid} RELEASED stdin hijack lock") - _no_remote_has_tty.set() return "pdb_unlock_complete" @@ -243,17 +279,21 @@ async def _breakpoint( global _local_pdb_complete, _pdb_release_hook global _local_task_in_debug, _global_actor_in_debug + await trio.lowlevel.checkpoint() + async def wait_for_parent_stdin_hijack( task_status=trio.TASK_STATUS_IGNORED ): global _debugger_request_cs - with trio.CancelScope() as cs: + with trio.CancelScope(shield=True) as cs: _debugger_request_cs = cs try: async with get_root() as portal: + log.error('got portal') + # this syncs to child's ``Context.started()`` call. async with portal.open_context( @@ -262,18 +302,22 @@ async def _breakpoint( ) as (ctx, val): + log.error('locked context') assert val == 'Locked' async with ctx.open_stream() as stream: + log.error('opened stream') # unblock local caller task_status.started() - # TODO: shielding currently can cause hangs... - # with trio.CancelScope(shield=True): + try: + await _local_pdb_complete.wait() - await _local_pdb_complete.wait() - await stream.send('pdb_unlock') + finally: + # TODO: shielding currently can cause hangs... + with trio.CancelScope(shield=True): + await stream.send('pdb_unlock') # sync with callee termination assert await ctx.result() == "pdb_unlock_complete" @@ -292,6 +336,7 @@ async def _breakpoint( # TODO: need a more robust check for the "root" actor if actor._parent_chan and not is_root_process(): + if _local_task_in_debug: if _local_task_in_debug == task_name: # this task already has the lock and is @@ -316,7 +361,13 @@ async def _breakpoint( # 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. - await actor._service_n.start(wait_for_parent_stdin_hijack) + + # NOTE: 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? + # actor._service_n.cancel_scope.shield = shield + with trio.CancelScope(shield=True): + await actor._service_n.start(wait_for_parent_stdin_hijack) elif is_root_process(): @@ -333,6 +384,11 @@ async def _breakpoint( # 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 _debug_lock.locked(): + log.warning( + 'Root actor attempting to acquire active tty lock' + f' owned by {_global_actor_in_debug}') + await _debug_lock.acquire() _global_actor_in_debug = actor.uid diff --git a/tractor/_trionics.py b/tractor/_trionics.py index 33ad71a..40abc2b 100644 --- a/tractor/_trionics.py +++ b/tractor/_trionics.py @@ -170,16 +170,25 @@ class ActorNursery: log.warning(f"Cancelling nursery in {self._actor.uid}") with trio.move_on_after(3) as cs: + async with trio.open_nursery() as nursery: + for subactor, proc, portal in self._children.values(): + + # TODO: are we ever even going to use this or + # is the spawning backend responsible for such + # things? I'm thinking latter. if hard_kill: proc.terminate() + else: if portal is None: # actor hasn't fully spawned yet event = self._actor._peer_connected[subactor.uid] log.warning( f"{subactor.uid} wasn't finished spawning?") + await event.wait() + # channel/portal should now be up _, _, portal = self._children[subactor.uid] @@ -239,6 +248,7 @@ async def _open_and_supervise_one_cancels_all_nursery( # As such if the strategy propagates any error(s) upwards # the above "daemon actor" nursery will be notified. async with trio.open_nursery() as ria_nursery: + anursery = ActorNursery( actor, ria_nursery, @@ -249,41 +259,53 @@ async def _open_and_supervise_one_cancels_all_nursery( # spawning of actors happens in the caller's scope # after we yield upwards yield anursery - log.debug( + + log.runtime( f"Waiting on subactors {anursery._children} " "to complete" ) # Last bit before first nursery block ends in the case # where we didn't error in the caller's scope - log.debug("Waiting on all subactors to complete") + + # signal all process monitor tasks to conduct + # hard join phase. anursery._join_procs.set() except BaseException as err: - if is_root_process() and ( - type(err) in { - Exception, trio.MultiError, trio.Cancelled - } - ): - # if we error in the root but the debugger is - # engaged we don't want to prematurely kill (and - # thus clobber access to) the local tty streams. - # instead try to wait for pdb to be released before - # tearing down. - debug_complete = _debug._pdb_complete - if debug_complete and not debug_complete.is_set(): - log.warning( - "Root has errored but pdb is active..waiting " - "on debug lock") - await _debug._pdb_complete.wait() + # If we error in the root but the debugger is + # engaged we don't want to prematurely kill (and + # thus clobber access to) the local tty since it + # will make the pdb repl unusable. + # Instead try to wait for pdb to be released before + # tearing down. + if is_root_process(): + log.exception(f"we're root with {err}") - # raise + # wait to see if a sub-actor task + # will be scheduled and grab the tty + # lock on the next tick + # await trio.testing.wait_all_tasks_blocked() + + debug_complete = _debug._no_remote_has_tty + if ( + debug_complete and + not debug_complete.is_set() + ): + log.warning( + 'Root has errored but pdb is in use by ' + f'child {_debug._global_actor_in_debug}\n' + 'Waiting on tty lock to release..') + + with trio.CancelScope(shield=True): + await debug_complete.wait() # if the caller's scope errored then we activate our # one-cancels-all supervisor strategy (don't # worry more are coming). anursery._join_procs.set() + try: # XXX: hypothetically an error could be # raised and then a cancel signal shows up @@ -322,7 +344,15 @@ async def _open_and_supervise_one_cancels_all_nursery( # ria_nursery scope end # XXX: do we need a `trio.Cancelled` catch here as well? - except (Exception, trio.MultiError, trio.Cancelled) as err: + # this is the catch around the ``.run_in_actor()`` nursery + except ( + + Exception, + trio.MultiError, + trio.Cancelled + + ) as err: + # If actor-local error was raised while waiting on # ".run_in_actor()" actors then we also want to cancel all # remaining sub-actors (due to our lone strategy: From 44bfacc0c2c97f59ad28c715f2fc1d7a62b726ce Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 30 Jun 2021 08:44:28 -0400 Subject: [PATCH 14/22] Comment hard-kill-sidestep for now since nursery version covers it? --- tractor/_spawn.py | 63 +++++++++++++++++++++++++---------------------- 1 file changed, 33 insertions(+), 30 deletions(-) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index dcbc49d..232cd79 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -157,6 +157,7 @@ async def cancel_on_completion( async def do_hard_kill( proc: trio.Process, + ) -> None: # NOTE: this timeout used to do nothing since we were shielding # the ``.wait()`` inside ``new_proc()`` which will pretty much @@ -230,44 +231,46 @@ async def spawn_subactor( yield proc finally: - log.debug(f"Attempting to kill {proc}") + log.runtime(f"Attempting to kill {proc}") # XXX: do this **after** cancellation/tearfown # to avoid killing the process too early # since trio does this internally on ``__aexit__()`` - if ( - is_root_process() + # if ( + # is_root_process() - # XXX: basically the pre-closing of stdstreams in a - # root-processe's ``trio.Process.aclose()`` can clobber - # any existing debugger session so we avoid - and _runtime_vars['_debug_mode'] - and _global_actor_in_debug is not None - ): - # XXX: this is ``trio.Process.aclose()`` MINUS the - # std-streams pre-closing steps inside ``proc.__aexit__()`` - # (see below) which incluses a ``Process.kill()`` call + # # XXX: basically the pre-closing of stdstreams in a + # # root-processe's ``trio.Process.aclose()`` can clobber + # # any existing debugger session so we avoid + # and _runtime_vars['_debug_mode'] + # and _global_actor_in_debug is not None + # ): + # # XXX: this is ``trio.Process.aclose()`` MINUS the + # # std-streams pre-closing steps inside ``proc.__aexit__()`` + # # (see below) which incluses a ``Process.kill()`` call - log.critical( - "Root process tty is locked in debug mode by " - f"{_global_actor_in_debug}. If the console is hanging, you " - "may need to trigger a KBI to kill any " - "not-fully-initialized" " subprocesses and allow errors " - "from `trio` to propagate" - ) - try: - # one more graceful wait try can can be cancelled by KBI - # sent by user. - await proc.wait() + # log.error( + # "Root process tty is locked in debug mode by " + # f"{_global_actor_in_debug}. If the console is hanging, you " + # "may need to trigger a KBI to kill any " + # "not-fully-initialized" " subprocesses and allow errors " + # "from `trio` to propagate" + # ) + # try: + # # one more graceful wait try can can be cancelled by KBI + # # sent by user. + # await proc.wait() - finally: - if proc.returncode is None: - # with trio.CancelScope(shield=True): - await do_hard_kill(proc) - else: - # with trio.CancelScope(shield=True): - await do_hard_kill(proc) + # finally: + # if proc.returncode is None: + # # with trio.CancelScope(shield=True): + # # await proc.wait() + + # await do_hard_kill(proc) + # else: + + await do_hard_kill(proc) async def new_proc( From 09f00a5a00ad35ab1303c714b83a2a9740e98e08 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 30 Jun 2021 08:45:43 -0400 Subject: [PATCH 15/22] Go back to only logging tbs on no debugger --- tractor/_root.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tractor/_root.py b/tractor/_root.py index 8391f4c..b9306b6 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -174,8 +174,11 @@ async def open_root_actor( yield actor except (Exception, trio.MultiError) as err: - logger.exception("Actor crashed:") - await _debug._maybe_enter_pm(err) + # with trio.CancelScope(shield=True): + entered = await _debug._maybe_enter_pm(err) + + if not entered: + logger.exception("Root actor crashed:") # always re-raise raise From 632c666a7d6c46e7b8bffee4c1dc3e71ac7d2b65 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 30 Jun 2021 13:49:56 -0400 Subject: [PATCH 16/22] Adjust debug tests to accomodate no more root clobbering We may get multiple re-entries to debugger by `bp_forever` sub-actor now since the root will incrementally try to cancel it only when the tty lock is not held. --- tests/test_debugger.py | 48 ++++++++++++++++++++++++++++++++---------- 1 file changed, 37 insertions(+), 11 deletions(-) diff --git a/tests/test_debugger.py b/tests/test_debugger.py index 8b9220b..6f8d3a5 100644 --- a/tests/test_debugger.py +++ b/tests/test_debugger.py @@ -317,32 +317,58 @@ def test_multi_daemon_subactors(spawn, loglevel): next_msg = name_error_msg elif name_error_msg in before: - next_msg = None + next_msg = bp_forever_msg else: raise ValueError("Neither log msg was found !?") - child.sendline('c') + # NOTE: previously since we did not have clobber prevention + # in the root actor this final resume could result in the debugger + # tearing down since both child actors would be cancelled and it was + # unlikely that `bp_forever` would re-acquire the tty loack again. + # Now, we should have a final resumption in the root plus a possible + # second entry by `bp_forever`. - # first name_error failure + child.sendline('c') child.expect(r"\(Pdb\+\+\)") before = str(child.before.decode()) - if next_msg: - assert next_msg in before + assert next_msg in before - child.sendline('c') + # XXX: hoorayy the root clobering the child here was fixed! + # IMO, this demonstrates the true power of SC system design. - child.expect(r"\(Pdb\+\+\)") - before = str(child.before.decode()) - assert "tractor._exceptions.RemoteActorError: ('name_error'" in before + # now the root actor won't clobber the bp_forever child + # during it's first access to the debug lock, but will instead + # wait for the lock to release, by the edge triggered + # ``_debug._no_remote_has_tty`` event before sending cancel messages + # (via portals) to its underlings B) + + # at some point here there should have been some warning msg from + # the root announcing it avoided a clobber of the child's lock, but + # it seems unreliable in testing here to gnab it: + # assert "in use by child ('bp_forever'," in before + + # wait for final error in root + while True: + + child.sendline('c') + child.expect(r"\(Pdb\+\+\)") + before = str(child.before.decode()) + try: + + # root error should be packed as remote error + assert "_exceptions.RemoteActorError: ('name_error'" in before + break + + except AssertionError: + assert bp_forever_msg in before try: child.sendline('c') child.expect(pexpect.EOF) except pexpect.exceptions.TIMEOUT: - # Failed to exit using continue..? child.sendline('q') child.expect(pexpect.EOF) @@ -397,7 +423,7 @@ def test_multi_nested_subactors_error_through_nurseries(spawn): child = spawn('multi_nested_subactors_error_up_through_nurseries') # startup time can be iffy - time.sleep(1) + # time.sleep(1) for i in range(12): try: From 13b76c943999c744e46210bb177b470b1d6ae147 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 30 Jun 2021 16:48:32 -0400 Subject: [PATCH 17/22] Add fast fail test using the context api --- .../fast_error_in_root_after_spawn.py | 53 +++++++++++++++++++ tests/test_debugger.py | 18 +++++++ 2 files changed, 71 insertions(+) create mode 100644 examples/debugging/fast_error_in_root_after_spawn.py diff --git a/examples/debugging/fast_error_in_root_after_spawn.py b/examples/debugging/fast_error_in_root_after_spawn.py new file mode 100644 index 0000000..d22833f --- /dev/null +++ b/examples/debugging/fast_error_in_root_after_spawn.py @@ -0,0 +1,53 @@ +''' +fast fail test with a context. +ensure the partially initialized sub-actor process +doesn't cause a hang on error/cancel of the parent +nrusery. + +''' +import trio +import tractor + + +@tractor.context +async def sleep( + ctx: tractor.Context, +): + await trio.sleep(0.5) + await ctx.started() + await trio.sleep_forever() + + +async def open_ctx( + n: tractor._trionics.ActorNursery +): + + # spawn both actors + portal = await n.start_actor( + name='sleeper', + enable_modules=[__name__], + ) + + async with portal.open_context( + sleep, + ) as (ctx, first): + assert first is None + + +async def main(): + + async with tractor.open_nursery( + debug_mode=True, + loglevel='runtime', + ) as an: + + async with trio.open_nursery() as n: + n.start_soon(open_ctx, an) + + await trio.sleep(0.2) + await trio.sleep(0.1) + assert 0 + + +if __name__ == '__main__': + trio.run(main) diff --git a/tests/test_debugger.py b/tests/test_debugger.py index 6f8d3a5..e58db41 100644 --- a/tests/test_debugger.py +++ b/tests/test_debugger.py @@ -505,3 +505,21 @@ def test_root_nursery_cancels_before_child_releases_tty_lock( assert "tractor._exceptions.RemoteActorError: ('spawner0'" in before assert "tractor._exceptions.RemoteActorError: ('name_error'" in before assert "NameError: name 'doggypants' is not defined" in before + + +def test_root_cancels_child_context_during_startup( + spawn, +): + '''Verify a fast fail in the root doesn't lock up the child reaping + and all while using the new context api. + + ''' + child = spawn('fast_error_in_root_after_spawn') + + child.expect(r"\(Pdb\+\+\)") + + before = str(child.before.decode()) + assert "AssertionError" in before + + child.sendline('c') + child.expect(pexpect.EOF) From b3d28a1ee42a8824e05af3f47d40bf03c47dbd12 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 5 Jul 2021 17:09:10 -0400 Subject: [PATCH 18/22] Drop debugger path and duplicate func from rebasing --- tractor/_spawn.py | 57 ----------------------------------------------- 1 file changed, 57 deletions(-) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index 232cd79..e54a652 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -25,10 +25,7 @@ from . import _forkserver_override from ._state import ( current_actor, is_main_process, - is_root_process, - _runtime_vars, ) -from ._debug import _global_actor_in_debug from .log import get_logger from ._portal import Portal @@ -155,27 +152,6 @@ async def cancel_on_completion( # cancel the process now that we have a final result await portal.cancel_actor() -async def do_hard_kill( - proc: trio.Process, - -) -> None: - # NOTE: this timeout used to do nothing since we were shielding - # the ``.wait()`` inside ``new_proc()`` which will pretty much - # never release until the process exits, now it acts as - # a hard-kill time ultimatum. - with trio.move_on_after(3) as cs: - - # NOTE: This ``__aexit__()`` shields internally. - async with proc: # calls ``trio.Process.aclose()`` - log.debug(f"Terminating {proc}") - - if cs.cancelled_caught: - # XXX: should pretty much never get here unless we have - # to move the bits from ``proc.__aexit__()`` out and - # into here. - log.critical(f"HARD KILLING {proc}") - proc.kill() - async def do_hard_kill( proc: trio.Process, @@ -237,39 +213,6 @@ async def spawn_subactor( # to avoid killing the process too early # since trio does this internally on ``__aexit__()`` - # if ( - # is_root_process() - - # # XXX: basically the pre-closing of stdstreams in a - # # root-processe's ``trio.Process.aclose()`` can clobber - # # any existing debugger session so we avoid - # and _runtime_vars['_debug_mode'] - # and _global_actor_in_debug is not None - # ): - # # XXX: this is ``trio.Process.aclose()`` MINUS the - # # std-streams pre-closing steps inside ``proc.__aexit__()`` - # # (see below) which incluses a ``Process.kill()`` call - - # log.error( - # "Root process tty is locked in debug mode by " - # f"{_global_actor_in_debug}. If the console is hanging, you " - # "may need to trigger a KBI to kill any " - # "not-fully-initialized" " subprocesses and allow errors " - # "from `trio` to propagate" - # ) - # try: - # # one more graceful wait try can can be cancelled by KBI - # # sent by user. - # await proc.wait() - - # finally: - # if proc.returncode is None: - # # with trio.CancelScope(shield=True): - # # await proc.wait() - - # await do_hard_kill(proc) - # else: - await do_hard_kill(proc) From 0afa7f0f8e857bfa55bc2f13f2e4d15cf3039748 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 31 Jul 2021 12:50:58 -0400 Subject: [PATCH 19/22] Fix lock context manager return type --- tractor/_debug.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tractor/_debug.py b/tractor/_debug.py index 7cc7d97..cef3d8e 100644 --- a/tractor/_debug.py +++ b/tractor/_debug.py @@ -123,7 +123,9 @@ class PdbwTeardown(pdbpp.Pdb): @asynccontextmanager -async def _acquire_debug_lock(uid: Tuple[str, str]) -> AsyncIterator[None]: +async def _acquire_debug_lock( + uid: Tuple[str, str] +) -> AsyncIterator[trio.StrictFIFOLock]: '''Acquire a actor local FIFO lock meant to mutex entry to a local debugger entry point to avoid tty clobbering a global root process. From 6006adc0de5c26fb4ff16a6d26c263ccf85a8e9e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 31 Jul 2021 13:56:26 -0400 Subject: [PATCH 20/22] Hide `_invoke()` tb, move actor error to exceptions mod --- tractor/_actor.py | 10 ++++------ tractor/_exceptions.py | 7 ++++++- tractor/_spawn.py | 3 ++- 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/tractor/_actor.py b/tractor/_actor.py index c45449d..0dbaede 100644 --- a/tractor/_actor.py +++ b/tractor/_actor.py @@ -41,10 +41,6 @@ from . import _mp_fixup_main log = get_logger('tractor') -class ActorFailure(Exception): - "General actor failure" - - async def _invoke( actor: 'Actor', @@ -56,8 +52,10 @@ async def _invoke( Union[trio.CancelScope, BaseException] ] = trio.TASK_STATUS_IGNORED, ): - """Invoke local func and deliver result(s) over provided channel. - """ + '''Invoke local func and deliver result(s) over provided channel. + + ''' + __tracebackhide__ = True treat_as_gen = False # possible a traceback (not sure what typing is for this..) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 30c872b..9c3edac 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -12,6 +12,10 @@ import trio _this_mod = importlib.import_module(__name__) +class ActorFailure(Exception): + "General actor failure" + + class RemoteActorError(Exception): # TODO: local recontruction of remote exception deats "Remote actor exception bundled locally" @@ -40,6 +44,7 @@ class InternalActorError(RemoteActorError): class TransportClosed(trio.ClosedResourceError): "Underlying channel transport was closed prior to use" + class ContextCancelled(RemoteActorError): "Inter-actor task context cancelled itself on the callee side." @@ -58,7 +63,7 @@ class NoRuntime(RuntimeError): def pack_error( exc: BaseException, - tb = None, + tb=None, ) -> Dict[str, Any]: """Create an "error message" for tranmission over diff --git a/tractor/_spawn.py b/tractor/_spawn.py index e54a652..ae1c708 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -29,8 +29,9 @@ from ._state import ( from .log import get_logger from ._portal import Portal -from ._actor import Actor, ActorFailure +from ._actor import Actor from ._entry import _mp_main +from ._exceptions import ActorFailure log = get_logger('tractor') From f173012fea121bb1af454f84dfadf64e7c1684e9 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 31 Jul 2021 15:01:26 -0400 Subject: [PATCH 21/22] Handle repeat child tty-acquires race --- tests/test_debugger.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/test_debugger.py b/tests/test_debugger.py index e58db41..277663c 100644 --- a/tests/test_debugger.py +++ b/tests/test_debugger.py @@ -497,7 +497,12 @@ def test_root_nursery_cancels_before_child_releases_tty_lock( child.sendline('c') - child.expect(pexpect.EOF) + while True: + try: + child.expect(pexpect.EOF) + break + except pexpect.exceptions.TIMEOUT: + print('child was ablel to grab tty lock again?') if not timed_out_early: From 674fbbc6b32b1bd439ddfec82e686e8cb45825ce Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 1 Aug 2021 10:43:21 -0400 Subject: [PATCH 22/22] Docs and comments tidying --- .../fast_error_in_root_after_spawn.py | 7 ++- .../root_timeout_while_child_crashed.py | 6 +- tests/test_debugger.py | 13 ++-- tractor/_debug.py | 62 ++++++++----------- tractor/_root.py | 2 +- tractor/_trionics.py | 7 ++- 6 files changed, 42 insertions(+), 55 deletions(-) diff --git a/examples/debugging/fast_error_in_root_after_spawn.py b/examples/debugging/fast_error_in_root_after_spawn.py index d22833f..044815b 100644 --- a/examples/debugging/fast_error_in_root_after_spawn.py +++ b/examples/debugging/fast_error_in_root_after_spawn.py @@ -1,8 +1,9 @@ ''' -fast fail test with a context. -ensure the partially initialized sub-actor process +Fast fail test with a context. + +Ensure the partially initialized sub-actor process doesn't cause a hang on error/cancel of the parent -nrusery. +nursery. ''' import trio diff --git a/examples/debugging/root_timeout_while_child_crashed.py b/examples/debugging/root_timeout_while_child_crashed.py index 09a9003..e313672 100644 --- a/examples/debugging/root_timeout_while_child_crashed.py +++ b/examples/debugging/root_timeout_while_child_crashed.py @@ -20,9 +20,9 @@ async def main(): # spawn both actors portal = await n.run_in_actor(key_error) - # XXX: originally a bug causes by this - # where root would enter debugger even - # though child should have it locked. + # XXX: originally a bug caused by this is where root would enter + # the debugger and clobber the tty used by the repl even though + # child should have it locked. with trio.fail_after(1): await trio.Event().wait() diff --git a/tests/test_debugger.py b/tests/test_debugger.py index 277663c..a3eb040 100644 --- a/tests/test_debugger.py +++ b/tests/test_debugger.py @@ -325,7 +325,7 @@ def test_multi_daemon_subactors(spawn, loglevel): # NOTE: previously since we did not have clobber prevention # in the root actor this final resume could result in the debugger # tearing down since both child actors would be cancelled and it was - # unlikely that `bp_forever` would re-acquire the tty loack again. + # unlikely that `bp_forever` would re-acquire the tty lock again. # Now, we should have a final resumption in the root plus a possible # second entry by `bp_forever`. @@ -335,7 +335,7 @@ def test_multi_daemon_subactors(spawn, loglevel): assert next_msg in before - # XXX: hoorayy the root clobering the child here was fixed! + # XXX: hooray the root clobbering the child here was fixed! # IMO, this demonstrates the true power of SC system design. # now the root actor won't clobber the bp_forever child @@ -412,9 +412,9 @@ def test_multi_subactors_root_errors(spawn): def test_multi_nested_subactors_error_through_nurseries(spawn): """Verify deeply nested actors that error trigger debugger entries at each actor nurserly (level) all the way up the tree. - """ - # NOTE: previously, inside this script was a a bug where if the + """ + # NOTE: previously, inside this script was a bug where if the # parent errors before a 2-levels-lower actor has released the lock, # the parent tries to cancel it but it's stuck in the debugger? # A test (below) has now been added to explicitly verify this is @@ -422,9 +422,6 @@ def test_multi_nested_subactors_error_through_nurseries(spawn): child = spawn('multi_nested_subactors_error_up_through_nurseries') - # startup time can be iffy - # time.sleep(1) - for i in range(12): try: child.expect(r"\(Pdb\+\+\)") @@ -502,7 +499,7 @@ def test_root_nursery_cancels_before_child_releases_tty_lock( child.expect(pexpect.EOF) break except pexpect.exceptions.TIMEOUT: - print('child was ablel to grab tty lock again?') + print('child was able to grab tty lock again?') if not timed_out_early: diff --git a/tractor/_debug.py b/tractor/_debug.py index cef3d8e..4fc7495 100644 --- a/tractor/_debug.py +++ b/tractor/_debug.py @@ -66,7 +66,7 @@ class PdbwTeardown(pdbpp.Pdb): # override the pdbpp config with our coolio one DefaultConfig = TractorConfig - # TODO: figure out how to dissallow recursive .set_trace() entry + # TODO: figure out how to disallow recursive .set_trace() entry # since that'll cause deadlock for us. def set_continue(self): try: @@ -125,9 +125,14 @@ class PdbwTeardown(pdbpp.Pdb): @asynccontextmanager async def _acquire_debug_lock( uid: Tuple[str, str] + ) -> AsyncIterator[trio.StrictFIFOLock]: - '''Acquire a actor local FIFO lock meant to mutex entry to a local - debugger entry point to avoid tty clobbering a global root process. + '''Acquire a root-actor local FIFO lock which tracks mutex access of + the process tree's global debugger breakpoint. + + This lock avoids tty clobbering (by preventing multiple processes + reading from stdstreams) and ensures multi-actor, sequential access + to the ``pdb`` repl. ''' global _debug_lock, _global_actor_in_debug, _no_remote_has_tty @@ -153,21 +158,18 @@ async def _acquire_debug_lock( we_acquired = True await _debug_lock.acquire() - # we_acquired = True - _global_actor_in_debug = uid log.debug(f"TTY lock acquired, remote task: {task_name}:{uid}") - # NOTE: critical section! - # this yield is unshielded. - # IF we received a cancel during the shielded lock - # entry of some next-in-queue requesting task, - # then the resumption here will result in that - # Cancelled being raised to our caller below! + # NOTE: critical section: this yield is unshielded! - # in this case the finally below should trigger - # and the surrounding calle side context should cancel - # normally relaying back to the caller. + # IF we received a cancel during the shielded lock entry of some + # next-in-queue requesting task, then the resumption here will + # result in that ``trio.Cancelled`` being raised to our caller + # (likely from ``_hijack_stdin_for_child()`` below)! In + # this case the ``finally:`` below should trigger and the + # surrounding caller side context should cancel normally + # relaying back to the caller. yield _debug_lock @@ -194,19 +196,8 @@ async def _acquire_debug_lock( log.debug(f"TTY lock released, remote task: {task_name}:{uid}") -# @contextmanager -# def _disable_sigint(): -# try: -# # disable sigint handling while in debug -# prior_handler = signal.signal(signal.SIGINT, handler) -# yield -# finally: -# # restore SIGINT handling -# signal.signal(signal.SIGINT, prior_handler) - - @tractor.context -async def _hijack_stdin_relay_to_child( +async def _hijack_stdin_for_child( ctx: tractor.Context, subactor_uid: Tuple[str, str] @@ -235,8 +226,7 @@ async def _hijack_stdin_relay_to_child( # indicate to child that we've locked stdio await ctx.started('Locked') - log.pdb( # type: ignore - f"Actor {subactor_uid} ACQUIRED stdin hijack lock") + log.pdb(f"Actor {subactor_uid} ACQUIRED stdin hijack lock") # wait for unlock pdb by child async with ctx.open_stream() as stream: @@ -245,14 +235,13 @@ async def _hijack_stdin_relay_to_child( except trio.BrokenResourceError: # XXX: there may be a race with the portal teardown - # with the calling actor which we can safely ignore - # the alternative would be sending an ack message + # with the calling actor which we can safely ignore. + # The alternative would be sending an ack message # and allowing the client to wait for us to teardown # first? pass - log.debug( - f"TTY lock released, remote task: {task_name}:{subactor_uid}") + log.debug(f"TTY lock released, remote task: {task_name}:{subactor_uid}") return "pdb_unlock_complete" @@ -299,7 +288,7 @@ async def _breakpoint( # this syncs to child's ``Context.started()`` call. async with portal.open_context( - tractor._debug._hijack_stdin_relay_to_child, + tractor._debug._hijack_stdin_for_child, subactor_uid=actor.uid, ) as (ctx, val): @@ -377,8 +366,7 @@ async def _breakpoint( # may have the tty locked prior global _debug_lock - # TODO: wait, what about multiple root tasks acquiring - # it though.. shrug? + # TODO: wait, what about multiple root tasks acquiring it though? # root process (us) already has it; ignore if _global_actor_in_debug == actor.uid: return @@ -408,8 +396,8 @@ async def _breakpoint( _pdb_release_hook = teardown - # block here one (at the appropriate frame *up* where - # ``breakpoint()`` was awaited and begin handling stdio + # 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) diff --git a/tractor/_root.py b/tractor/_root.py index b9306b6..bfaaf4f 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -174,7 +174,7 @@ async def open_root_actor( yield actor except (Exception, trio.MultiError) as err: - # with trio.CancelScope(shield=True): + entered = await _debug._maybe_enter_pm(err) if not entered: diff --git a/tractor/_trionics.py b/tractor/_trionics.py index 40abc2b..2eb64d1 100644 --- a/tractor/_trionics.py +++ b/tractor/_trionics.py @@ -283,9 +283,10 @@ async def _open_and_supervise_one_cancels_all_nursery( if is_root_process(): log.exception(f"we're root with {err}") - # wait to see if a sub-actor task - # will be scheduled and grab the tty - # lock on the next tick + # 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? # await trio.testing.wait_all_tasks_blocked() debug_complete = _debug._no_remote_has_tty