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..044815b --- /dev/null +++ b/examples/debugging/fast_error_in_root_after_spawn.py @@ -0,0 +1,54 @@ +''' +Fast fail test with a context. + +Ensure the partially initialized sub-actor process +doesn't cause a hang on error/cancel of the parent +nursery. + +''' +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/examples/debugging/root_timeout_while_child_crashed.py b/examples/debugging/root_timeout_while_child_crashed.py new file mode 100644 index 0000000..e313672 --- /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 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() + + +if __name__ == '__main__': + trio.run(main) diff --git a/tests/test_debugger.py b/tests/test_debugger.py index 8b9220b..a3eb040 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 lock 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: hooray the root clobbering 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) @@ -386,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 @@ -396,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\+\+\)") @@ -471,7 +494,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 able to grab tty lock again?') if not timed_out_early: @@ -479,3 +507,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) 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/_debug.py b/tractor/_debug.py index c76422a..4fc7495 100644 --- a/tractor/_debug.py +++ b/tractor/_debug.py @@ -1,13 +1,13 @@ """ 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 -from async_generator import aclosing import tractor import trio @@ -31,14 +31,22 @@ 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() +_debug_lock: trio.StrictFIFOLock = trio.StrictFIFOLock() +_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 @@ -58,22 +66,22 @@ 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): - 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() @@ -115,147 +123,283 @@ 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. - """ +async def _acquire_debug_lock( + uid: Tuple[str, str] + +) -> AsyncIterator[trio.StrictFIFOLock]: + '''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 + task_name = trio.lowlevel.current_task().name + + log.pdb( + f"Attempting to acquire TTY lock, remote task: {task_name}:{uid}" + ) + + 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"Attempting to acquire TTY lock, remote task: {task_name}:{uid}") + f"entering lock checkpoint, remote task: {task_name}:{uid}" + ) + we_acquired = True await _debug_lock.acquire() + _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 ``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 finally: - _debug_lock.release() + # 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 -# 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_for_child( - -async def _hijack_stdin_relay_to_child( + ctx: tractor.Context, subactor_uid: Tuple[str, str] -) -> AsyncIterator[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. + + ''' + 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") - async with _acquire_debug_lock(subactor_uid): - log.warning(f"Actor {subactor_uid} ACQUIRED stdin hijack lock") + # this will deliver stdin data? - # with _disable_sigint(): + log.debug( + "Attempting to acquire TTY lock\n" + f"remote task: {task_name}:{subactor_uid}" + ) - # indicate to child that we've locked stdio - yield 'Locked' + log.debug(f"Actor {subactor_uid} is WAITING on stdin hijack lock") - # wait for cancellation of stream by child - # indicating debugger is dis-engaged - await trio.sleep_forever() + with trio.CancelScope(shield=True): - log.debug(f"Actor {subactor_uid} RELEASED stdin hijack lock") + async with _acquire_debug_lock(subactor_uid): + + # indicate to child that we've locked stdio + await ctx.started('Locked') + log.pdb(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' + + 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}") + + return "pdb_unlock_complete" -# XXX: We only make this sync in case someone wants to -# overload the ``breakpoint()`` built-in. -def _breakpoint(debug_func) -> Awaitable[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() - do_unlock = trio.Event() + task_name = trio.lowlevel.current_task().name + + 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: - 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: + log.error('got portal') - assert val == 'Locked' - task_status.started() + # this syncs to child's ``Context.started()`` call. + async with portal.open_context( - # with trio.CancelScope(shield=True): - await do_unlock.wait() + tractor._debug._hijack_stdin_for_child, + subactor_uid=actor.uid, + + ) 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() + + try: + await _local_pdb_complete.wait() + + 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" + + except tractor.ContextCancelled: + log.warning('Root actor cancelled debug lock') - # trigger cancellation of remote stream - break 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") - 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 _local_pdb_complete or _local_pdb_complete.is_set(): + _local_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(): - # 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 _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 - # 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) + # if **this** actor is already in debug mode block here + # waiting for the control to be released - this allows + # support for recursive entries to `tractor.breakpoint()` + log.warning(f"{actor.uid} already has a debug lock, waiting...") - # assign unlock callback for debugger teardown hooks - global _pdb_release_hook - _pdb_release_hook = do_unlock.set + await _local_pdb_complete.wait() + await trio.sleep(0.1) - # mark local actor as "in debug mode" to avoid recurrent - # entries/requests to the root process - _in_debug = task_name + # mark local actor as "in debug mode" to avoid recurrent + # entries/requests to the root process + _local_task_in_debug = task_name - # this **must** be awaited by the caller and is done using the - # root nursery so that the debugger can continue to run without - # being restricted by the scope of a new task nursery. + # assign unlock callback for debugger teardown hooks + _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 + # being restricted by the scope of a new task nursery. + + # 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(): - # 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 - return - await _debug_lock.acquire() - _pdb_release_hook = _debug_lock.release + elif is_root_process(): - # 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) + # we also wait in the root-parent for any child that + # may have the tty locked prior + global _debug_lock - # user code **must** await this! - return _bp() + # 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 + + # 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 + _local_task_in_debug = task_name + + # the lock must be released on pdb completion + def teardown(): + 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 + _local_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(): @@ -285,8 +429,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 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/_root.py b/tractor/_root.py index 8391f4c..bfaaf4f 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) + + entered = await _debug._maybe_enter_pm(err) + + if not entered: + logger.exception("Root actor crashed:") # always re-raise raise diff --git a/tractor/_spawn.py b/tractor/_spawn.py index 678250b..ae1c708 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -26,10 +26,12 @@ from ._state import ( current_actor, is_main_process, ) + 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') @@ -144,7 +146,7 @@ async def cancel_on_completion( ) else: - log.info( + log.runtime( f"Cancelling {portal.channel.uid} gracefully " f"after result {result}") @@ -206,12 +208,12 @@ async def spawn_subactor( yield proc finally: + 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__()`` - log.debug(f"Attempting to kill {proc}") await do_hard_kill(proc) @@ -241,7 +243,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 @@ -396,7 +398,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 diff --git a/tractor/_trionics.py b/tractor/_trionics.py index eea3aae..2eb64d1 100644 --- a/tractor/_trionics.py +++ b/tractor/_trionics.py @@ -11,7 +11,8 @@ import warnings import trio from async_generator import asynccontextmanager -from ._state import current_actor, is_main_process +from . import _debug +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 @@ -169,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] @@ -238,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, @@ -248,21 +259,54 @@ 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 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}") + + # 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 + 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 @@ -301,7 +345,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: @@ -368,6 +420,7 @@ 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: