From 1c480e6c9205800f6f2883e2655bfe53316c0dbf Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 11 Oct 2022 12:17:35 -0400 Subject: [PATCH 1/8] Add `Context` cancel message and debug toggle flag In the case of a callee-side context cancelling itself it can be handy to let the caller-side task know (even if through logging) that the cancel was due to some known reason. Make `.cancel()` accept such a message on the callee side and have it included in the `._runtime._invoke()` raised `ContextCancelled` emission. Also add a `Context._trigger_debugger_on_cancel: bool` flag which can be set to `False` to avoid the debugger post-mortem crash mode from engaging on cross-context tasks which cancel themselves for a known reason (as is needed for blocked tasks in the debug TTY-lock machinery). --- tractor/_streaming.py | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/tractor/_streaming.py b/tractor/_streaming.py index 4500ec0..9a75464 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -371,6 +371,8 @@ class Context: # status flags _cancel_called: bool = False + _cancel_msg: Optional[str] = None + _trigger_debugger_on_cancel: bool = True _started_called: bool = False _started_received: bool = False _stream_opened: bool = False @@ -452,7 +454,11 @@ class Context: if not self._scope_nursery._closed: # type: ignore self._scope_nursery.start_soon(raiser) - async def cancel(self) -> None: + async def cancel( + self, + msg: Optional[str] = None, + + ) -> None: ''' Cancel this inter-actor-task context. @@ -461,6 +467,8 @@ class Context: ''' side = 'caller' if self._portal else 'callee' + if msg: + assert side == 'callee', 'Only callee side can provide cancel msg' log.cancel(f'Cancelling {side} side of context to {self.chan.uid}') @@ -497,8 +505,10 @@ class Context: log.cancel( "Timed out on cancelling remote task " f"{cid} for {self._portal.channel.uid}") + + # callee side remote task else: - # callee side remote task + self._cancel_msg = msg # TODO: should we have an explicit cancel message # or is relaying the local `trio.Cancelled` as an From 3398153c5224991c43f28702dd09bfadb6ef23f6 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 11 Oct 2022 13:35:55 -0400 Subject: [PATCH 2/8] Add timeout around `trio`-callee-task --- tests/test_infected_asyncio.py | 31 ++++++++++++++++--------------- 1 file changed, 16 insertions(+), 15 deletions(-) diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 976741d..30ddfb6 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -203,24 +203,25 @@ def test_context_spawns_aio_task_that_errors( ''' async def main(): - async with tractor.open_nursery() as n: - p = await n.start_actor( - 'aio_daemon', - enable_modules=[__name__], - infect_asyncio=True, - # debug_mode=True, - loglevel='cancel', - ) - async with p.open_context( - trio_ctx, - ) as (ctx, first): + with trio.fail_after(2): + async with tractor.open_nursery() as n: + p = await n.start_actor( + 'aio_daemon', + enable_modules=[__name__], + infect_asyncio=True, + # debug_mode=True, + loglevel='cancel', + ) + async with p.open_context( + trio_ctx, + ) as (ctx, first): - assert first == 'start' + assert first == 'start' - if parent_cancels: - await p.cancel_actor() + if parent_cancels: + await p.cancel_actor() - await trio.sleep_forever() + await trio.sleep_forever() with pytest.raises(RemoteActorError) as excinfo: trio.run(main) From 30ea7a06b0a5f4fd3635ed0f10c13e57c6a19baf Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 11 Oct 2022 13:48:27 -0400 Subject: [PATCH 3/8] Avoid inf nursery hang by reversing `async with` ordering --- tests/test_infected_asyncio.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 30ddfb6..ac8cbcd 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -170,11 +170,11 @@ async def trio_ctx( # message. with trio.fail_after(2): async with ( + trio.open_nursery() as n, + tractor.to_asyncio.open_channel_from( sleep_and_err, ) as (first, chan), - - trio.open_nursery() as n, ): assert first == 'start' From 734d8dd6633a2f370720ed25e5043728d3384ed5 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 11 Oct 2022 14:03:49 -0400 Subject: [PATCH 4/8] Move `trio` scope outside first inter-task-chan receive --- tractor/to_asyncio.py | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index aeb376a..be3ac8d 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -466,11 +466,11 @@ async def open_channel_from( ): # sync to a "started()"-like first delivered value from the # ``asyncio`` task. - first = await chan.receive() - - # deliver stream handle upward try: with chan._trio_cs: + first = await chan.receive() + + # deliver stream handle upward yield first, chan finally: chan._trio_exited = True @@ -491,16 +491,18 @@ def run_as_asyncio_guest( SC semantics. ''' - # Uh, oh. :o + # Uh, oh. + # + # :o # It looks like your event loop has caught a case of the ``trio``s. # :() - # Don't worry, we've heard you'll barely notice. You might hallucinate - # a few more propagating errors and feel like your digestion has - # slowed but if anything get's too bad your parents will know about - # it. + # Don't worry, we've heard you'll barely notice. You might + # hallucinate a few more propagating errors and feel like your + # digestion has slowed but if anything get's too bad your parents + # will know about it. # :) From fb721f36ef0800fcc13c6787d83679cd3c447176 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 11 Oct 2022 15:22:19 -0400 Subject: [PATCH 5/8] Support debug-lock blocking, use on no-more IPC This is a lingering debugger locking race case we needed to handle: - child crashes acquires TTY lock in root and attaches to `pdb` - child IPC goes down such that all channels to the root are broken / non-functional. - root is stuck thinking the child is still in debug even though it can't be contacted and the child actor machinery hasn't been cancelled by its parent. - root get's stuck in deadlock with child since it won't send a cancel request until the child is finished debugging, but the child can't unlock the debugger bc IPC is down. To avoid this scenario add debug lock blocking list via `._debug.Lock._blocked: set[tuple]` which holds actor uids for any actor that is detected by the root as having no transport channel connections with said root (of which at least one should exist if this sub-actor at some point acquired the debug lock). The root consequently checks this list for any actor that tries to (re)acquire the lock and blocks with a `ContextCancelled`. When a debug condition is tested in `._runtime._invoke` the context's `._enter_debugger_on_cancel` which is set to `False` if the actor is on the block list in which case the post-mortem entry is skipped. Further this adds a root-locking-task side cancel scope to `Lock._root_local_task_cs_in_debug` which can be cancelled by the root runtime when a stale lock is detected after all IPC channels for the actor have been torn down. NOTE: right now we're NOT doing this since it seems to cause test failures likely due because it may cause pre-mature cancellation and maybe needs a bit more experimenting? --- tractor/_debug.py | 57 ++++++++++++++++++++++++++++++++++--------- tractor/_runtime.py | 44 ++++++++++++++++++++++++++++++--- tractor/_streaming.py | 2 +- 3 files changed, 87 insertions(+), 16 deletions(-) diff --git a/tractor/_debug.py b/tractor/_debug.py index 46b69f0..751c646 100644 --- a/tractor/_debug.py +++ b/tractor/_debug.py @@ -38,8 +38,14 @@ from trio_typing import TaskStatus from .log import get_logger from ._discovery import get_root -from ._state import is_root_process, debug_mode -from ._exceptions import is_multi_cancelled +from ._state import ( + is_root_process, + debug_mode, +) +from ._exceptions import ( + is_multi_cancelled, + ContextCancelled, +) from ._ipc import Channel @@ -72,6 +78,18 @@ class Lock: # actor-wide variable pointing to current task name using debugger local_task_in_debug: Optional[str] = None + # NOTE: set by the current task waiting on the root tty lock from + # the CALLER side of the `lock_tty_for_child()` context entry-call + # and must be cancelled if this actor is cancelled via IPC + # request-message otherwise deadlocks with the parent actor may + # ensure + _debugger_request_cs: Optional[trio.CancelScope] = None + + # NOTE: set only in the root actor for the **local** root spawned task + # which has acquired the lock (i.e. this is on the callee side of + # the `lock_tty_for_child()` context entry). + _root_local_task_cs_in_debug: Optional[trio.CancelScope] = None + # actor tree-wide actor uid that supposedly has the tty lock global_actor_in_debug: Optional[tuple[str, str]] = None @@ -81,12 +99,8 @@ class Lock: # lock in root actor preventing multi-access to local tty _debug_lock: trio.StrictFIFOLock = trio.StrictFIFOLock() - # XXX: set by the current task waiting on the root tty lock - # and must be cancelled if this actor is cancelled via message - # otherwise deadlocks with the parent actor may ensure - _debugger_request_cs: Optional[trio.CancelScope] = None - _orig_sigint_handler: Optional[Callable] = None + _blocked: set[tuple[str, str]] = set() @classmethod def shield_sigint(cls): @@ -196,6 +210,12 @@ async def _acquire_debug_lock_from_root_task( f"entering lock checkpoint, remote task: {task_name}:{uid}" ) we_acquired = True + + # NOTE: if the surrounding cancel scope from the + # `lock_tty_for_child()` caller is cancelled, this line should + # unblock and NOT leave us in some kind of + # a "child-locked-TTY-but-child-is-uncontactable-over-IPC" + # condition. await Lock._debug_lock.acquire() if Lock.no_remote_has_tty is None: @@ -267,6 +287,15 @@ async def lock_tty_for_child( ''' task_name = trio.lowlevel.current_task().name + if tuple(subactor_uid) in Lock._blocked: + log.warning( + f'Actor {subactor_uid} is blocked from acquiring debug lock\n' + f"remote task: {task_name}:{subactor_uid}" + ) + ctx._enter_debugger_on_cancel = False + await ctx.cancel(f'Debug lock blocked for {subactor_uid}') + return 'pdb_lock_blocked' + # TODO: when we get to true remote debugging # this will deliver stdin data? @@ -280,8 +309,9 @@ async def lock_tty_for_child( try: with ( - trio.CancelScope(shield=True), + trio.CancelScope(shield=True) as debug_lock_cs, ): + Lock._root_local_task_cs_in_debug = debug_lock_cs async with _acquire_debug_lock_from_root_task(subactor_uid): # indicate to child that we've locked stdio @@ -297,6 +327,7 @@ async def lock_tty_for_child( return "pdb_unlock_complete" finally: + Lock._root_local_task_cs_in_debug = None Lock.unshield_sigint() @@ -353,7 +384,7 @@ async def wait_for_parent_stdin_hijack( log.pdb('unlocked context') - except tractor.ContextCancelled: + except ContextCancelled: log.warning('Root actor cancelled debug lock') finally: @@ -721,9 +752,11 @@ async def _maybe_enter_pm(err): and not is_multi_cancelled(err) ): log.debug("Actor crashed, entering debug mode") - await post_mortem() - Lock.release() - return True + try: + await post_mortem() + finally: + Lock.release() + return True else: return False diff --git a/tractor/_runtime.py b/tractor/_runtime.py index c2f9551..e9a5099 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -234,6 +234,9 @@ async def _invoke( f'{ctx.chan.uid}' ) + if ctx._cancel_msg: + msg += f' with msg:\n{ctx._cancel_msg}' + # task-contex was cancelled so relay to the cancel to caller raise ContextCancelled( msg, @@ -275,8 +278,16 @@ async def _invoke( # if not is_multi_cancelled(err) and ( entered_debug: bool = False - if not isinstance(err, ContextCancelled) or ( - isinstance(err, ContextCancelled) and ctx._cancel_called + if ( + not isinstance(err, ContextCancelled) + or ( + isinstance(err, ContextCancelled) + and ctx._cancel_called + + # if the root blocks the debugger lock request from a child + # we will get a remote-cancelled condition. + and ctx._enter_debugger_on_cancel + ) ): # XXX: is there any case where we'll want to debug IPC # disconnects as a default? @@ -286,7 +297,6 @@ async def _invoke( # recovery logic - the only case is some kind of strange bug # in our transport layer itself? Going to keep this # open ended for now. - entered_debug = await _debug._maybe_enter_pm(err) if not entered_debug: @@ -702,6 +712,34 @@ class Actor: # if chan.uid == uid: # self._contexts.pop((uid, cid)) + # NOTE: block this actor from acquiring the + # debugger-TTY-lock since we have no way to know if we + # cancelled it and further there is no way to ensure the + # lock will be released if acquired due to having no + # more active IPC channels. + if ( + _state.is_root_process() + ): + pdb_lock = _debug.Lock + log.cancel( + f'{uid} is now blocked from debugger lock' + ) + log.runtime(f"{uid} blocked from pdb locking") + pdb_lock._blocked.add(uid) + + # if a now stale local task has the TTY lock still + # we cancel it to allow servicing other requests for + # the lock. + if ( + pdb_lock._root_local_task_cs_in_debug + and not pdb_lock._root_local_task_cs_in_debug.cancel_called + ): + log.warning( + f'STALE DEBUG LOCK DETECTED FOR {uid}' + ) + # TODO: figure out why this breaks tests.. + # pdb_lock._root_local_task_cs_in_debug.cancel() + log.runtime(f"Peers is {self._peers}") # No more channels to other actors (at all) registered diff --git a/tractor/_streaming.py b/tractor/_streaming.py index 9a75464..bb99dc5 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -372,7 +372,7 @@ class Context: # status flags _cancel_called: bool = False _cancel_msg: Optional[str] = None - _trigger_debugger_on_cancel: bool = True + _enter_debugger_on_cancel: bool = True _started_called: bool = False _started_received: bool = False _stream_opened: bool = False From 72fbda4cefabf698ec096ec0e272a6c81bed7320 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 12 Oct 2022 12:35:11 -0400 Subject: [PATCH 6/8] Add nooz file --- nooz/337.feature.rst | 41 +++++++++++++++++++++++++++++++++++++++++ 1 file changed, 41 insertions(+) create mode 100644 nooz/337.feature.rst diff --git a/nooz/337.feature.rst b/nooz/337.feature.rst new file mode 100644 index 0000000..6e3e903 --- /dev/null +++ b/nooz/337.feature.rst @@ -0,0 +1,41 @@ +Add support for debug-lock blocking using a ``._debug.Lock._blocked: +set[tuple]`` and add ids when no-more IPC connections with the +root actor are detected. + +This is an enhancement which (mostly) solves a lingering debugger +locking race case we needed to handle: + +- child crashes acquires TTY lock in root and attaches to ``pdb`` +- child IPC goes down such that all channels to the root are broken + / non-functional. +- root is stuck thinking the child is still in debug even though it + can't be contacted and the child actor machinery hasn't been + cancelled by its parent. +- root get's stuck in deadlock with child since it won't send a cancel + request until the child is finished debugging (to avoid clobbering + a child that is actually using the debugger), but the child can't + unlock the debugger bc IPC is down and it can't contact the root. + +To avoid this scenario add debug lock blocking list via +`._debug.Lock._blocked: set[tuple]` which holds actor uids for any actor +that is detected by the root as having no transport channel connections +(of which at least one should exist if this sub-actor at some point +acquired the debug lock). The root consequently checks this list for any +actor that tries to (re)acquire the lock and blocks with +a ``ContextCancelled``. Further, when a debug condition is tested in +``._runtime._invoke``, the context's ``._enter_debugger_on_cancel`` is +set to `False` if the actor was put on the block list then all +post-mortem / crash handling will be bypassed for that task. + +In theory this approach to block list management may cause problems +where some nested child actor acquires and releases the lock multiple +times and it gets stuck on the block list after the first use? If this +turns out to be an issue we can try changing the strat so blocks are +only added when the root has zero IPC peers left? + +Further, this adds a root-locking-task side cancel scope, +``Lock._root_local_task_cs_in_debug``, which can be ``.cancel()``-ed by the root +runtime when a stale lock is detected during the IPC channel testing. +However, right now we're NOT using this since it seems to cause test +failures likely due to causing pre-mature cancellation and maybe needs +a bit more experimenting? From b81b6be98af5a394865cb595b3223a4c4c888828 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 12 Oct 2022 12:35:35 -0400 Subject: [PATCH 7/8] Drop extra log msgs, some old commented code --- tractor/_runtime.py | 15 +++------------ 1 file changed, 3 insertions(+), 12 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index e9a5099..2ece834 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -708,24 +708,15 @@ class Actor: log.runtime(f"No more channels for {chan.uid}") self._peers.pop(uid, None) - # for (uid, cid) in self._contexts.copy(): - # if chan.uid == uid: - # self._contexts.pop((uid, cid)) - # NOTE: block this actor from acquiring the # debugger-TTY-lock since we have no way to know if we # cancelled it and further there is no way to ensure the # lock will be released if acquired due to having no # more active IPC channels. - if ( - _state.is_root_process() - ): + if _state.is_root_process(): pdb_lock = _debug.Lock - log.cancel( - f'{uid} is now blocked from debugger lock' - ) - log.runtime(f"{uid} blocked from pdb locking") pdb_lock._blocked.add(uid) + log.runtime(f"{uid} blocked from pdb locking") # if a now stale local task has the TTY lock still # we cancel it to allow servicing other requests for @@ -745,7 +736,7 @@ class Actor: # No more channels to other actors (at all) registered # as connected. if not self._peers: - log.runtime("Signalling no more peer channels") + log.runtime("Signalling no more peer channel connections") self._no_more_peers.set() # XXX: is this necessary (GC should do it)? From fc17f6790ee2dbe2fd1d3c99f3424e06303d64e2 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 12 Oct 2022 12:36:09 -0400 Subject: [PATCH 8/8] Bump `towncrier` alpha version --- pyproject.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pyproject.toml b/pyproject.toml index bbb9697..e52aa47 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -2,7 +2,7 @@ package = "tractor" filename = "NEWS.rst" directory = "nooz/" -version = "0.1.0a5" +version = "0.1.0a6" title_format = "tractor {version} ({project_date})" template = "nooz/_template.rst" all_bullets = true