Merge pull request #397 from goodboy/post_mortems

Fix root-actor crash handling despite runtime cancellation
main
Bd 2025-08-20 12:45:06 -04:00 committed by GitHub
commit c9a55c2d46
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
10 changed files with 255 additions and 72 deletions

View File

@ -0,0 +1,35 @@
import trio
import tractor
async def main():
async with tractor.open_root_actor(
debug_mode=True,
loglevel='cancel',
) as _root:
# manually trigger self-cancellation and wait
# for it to fully trigger.
_root.cancel_soon()
await _root._cancel_complete.wait()
print('root cancelled')
# now ensure we can still use the REPL
try:
await tractor.pause()
except trio.Cancelled as _taskc:
assert (root_cs := _root._root_tn.cancel_scope).cancel_called
# NOTE^^ above logic but inside `open_root_actor()` and
# passed to the `shield=` expression is effectively what
# we're testing here!
await tractor.pause(shield=root_cs.cancel_called)
# XXX, if shield logic *is wrong* inside `open_root_actor()`'s
# crash-handler block this should never be interacted,
# instead `trio.Cancelled` would be bubbled up: the original
# BUG.
assert 0
if __name__ == '__main__':
trio.run(main)

View File

@ -1,13 +1,13 @@
""" """
That "native" debug mode better work! That "native" debug mode better work!
All these tests can be understood (somewhat) by running the equivalent All these tests can be understood (somewhat) by running the
`examples/debugging/` scripts manually. equivalent `examples/debugging/` scripts manually.
TODO: TODO:
- none of these tests have been run successfully on windows yet but - none of these tests have been run successfully on windows yet but
there's been manual testing that verified it works. there's been manual testing that verified it works.
- wonder if any of it'll work on OS X? - wonder if any of it'll work on OS X?
""" """
from __future__ import annotations from __future__ import annotations
@ -925,6 +925,7 @@ def test_post_mortem_api(
"<Task 'name_error'", "<Task 'name_error'",
"NameError", "NameError",
"('child'", "('child'",
'getattr(doggypants)', # exc-LoC
] ]
) )
if ctlc: if ctlc:
@ -941,8 +942,8 @@ def test_post_mortem_api(
"<Task '__main__.main'", "<Task '__main__.main'",
"('root'", "('root'",
"NameError", "NameError",
"tractor.post_mortem()",
"src_uid=('child'", "src_uid=('child'",
"tractor.post_mortem()", # in `main()`-LoC
] ]
) )
if ctlc: if ctlc:
@ -960,6 +961,10 @@ def test_post_mortem_api(
"('root'", "('root'",
"NameError", "NameError",
"src_uid=('child'", "src_uid=('child'",
# raising line in `main()` but from crash-handling
# in `tractor.open_nursery()`.
'async with p.open_context(name_error) as (ctx, first):',
] ]
) )
if ctlc: if ctlc:
@ -1151,6 +1156,54 @@ def test_ctxep_pauses_n_maybe_ipc_breaks(
) )
def test_crash_handling_within_cancelled_root_actor(
spawn: PexpectSpawner,
):
'''
Ensure that when only a root-actor is started via `open_root_actor()`
we can crash-handle in debug-mode despite self-cancellation.
More-or-less ensures we conditionally shield the pause in
`._root.open_root_actor()`'s `await debug._maybe_enter_pm()`
call.
'''
child = spawn('root_self_cancelled_w_error')
child.expect(PROMPT)
assert_before(
child,
[
"Actor.cancel_soon()` was called!",
"root cancelled",
_pause_msg,
"('root'", # actor name
]
)
child.sendline('c')
child.expect(PROMPT)
assert_before(
child,
[
_crash_msg,
"('root'", # actor name
"AssertionError",
"assert 0",
]
)
child.sendline('c')
child.expect(EOF)
assert_before(
child,
[
"AssertionError",
"assert 0",
]
)
# TODO: better error for "non-ideal" usage from the root actor. # TODO: better error for "non-ideal" usage from the root actor.
# -[ ] if called from an async scope emit a message that suggests # -[ ] if called from an async scope emit a message that suggests
# using `await tractor.pause()` instead since it's less overhead # using `await tractor.pause()` instead since it's less overhead

View File

@ -481,10 +481,11 @@ async def open_root_actor(
collapse_eg(), collapse_eg(),
trio.open_nursery() as root_tn, trio.open_nursery() as root_tn,
# XXX, finally-footgun below? # ?TODO? finally-footgun below?
# -> see note on why shielding. # -> see note on why shielding.
# maybe_raise_from_masking_exc(), # maybe_raise_from_masking_exc(),
): ):
actor._root_tn = root_tn
# `_runtime.async_main()` creates an internal nursery # `_runtime.async_main()` creates an internal nursery
# and blocks here until any underlying actor(-process) # and blocks here until any underlying actor(-process)
# tree has terminated thereby conducting so called # tree has terminated thereby conducting so called
@ -523,6 +524,11 @@ async def open_root_actor(
err, err,
api_frame=inspect.currentframe(), api_frame=inspect.currentframe(),
debug_filter=debug_filter, debug_filter=debug_filter,
# XXX NOTE, required to debug root-actor
# crashes under cancellation conditions; so
# most of them!
shield=root_tn.cancel_scope.cancel_called,
) )
if ( if (
@ -562,6 +568,7 @@ async def open_root_actor(
f'{op_nested_actor_repr}' f'{op_nested_actor_repr}'
) )
# XXX, THIS IS A *finally-footgun*! # XXX, THIS IS A *finally-footgun*!
# (also mentioned in with-block above)
# -> though already shields iternally it can # -> though already shields iternally it can
# taskc here and mask underlying errors raised in # taskc here and mask underlying errors raised in
# the try-block above? # the try-block above?

View File

@ -384,7 +384,7 @@ async def _errors_relayed_via_ipc(
# RPC task bookeeping. # RPC task bookeeping.
# since RPC tasks are scheduled inside a flat # since RPC tasks are scheduled inside a flat
# `Actor._service_n`, we add "handles" to each such that # `Actor._service_tn`, we add "handles" to each such that
# they can be individually ccancelled. # they can be individually ccancelled.
finally: finally:
@ -462,7 +462,7 @@ async def _invoke(
connected IPC channel. connected IPC channel.
This is the core "RPC" `trio.Task` scheduling machinery used to start every This is the core "RPC" `trio.Task` scheduling machinery used to start every
remotely invoked function, normally in `Actor._service_n: Nursery`. remotely invoked function, normally in `Actor._service_tn: Nursery`.
''' '''
__tracebackhide__: bool = hide_tb __tracebackhide__: bool = hide_tb
@ -936,7 +936,7 @@ async def process_messages(
Receive (multiplexed) per-`Channel` RPC requests as msgs from Receive (multiplexed) per-`Channel` RPC requests as msgs from
remote processes; schedule target async funcs as local remote processes; schedule target async funcs as local
`trio.Task`s inside the `Actor._service_n: Nursery`. `trio.Task`s inside the `Actor._service_tn: Nursery`.
Depending on msg type, non-`cmd` (task spawning/starting) Depending on msg type, non-`cmd` (task spawning/starting)
request payloads (eg. `started`, `yield`, `return`, `error`) request payloads (eg. `started`, `yield`, `return`, `error`)
@ -961,7 +961,7 @@ async def process_messages(
''' '''
actor: Actor = _state.current_actor() actor: Actor = _state.current_actor()
assert actor._service_n # runtime state sanity assert actor._service_tn # runtime state sanity
# TODO: once `trio` get's an "obvious way" for req/resp we # TODO: once `trio` get's an "obvious way" for req/resp we
# should use it? # should use it?
@ -1172,7 +1172,7 @@ async def process_messages(
start_status += '->( scheduling new task..\n' start_status += '->( scheduling new task..\n'
log.runtime(start_status) log.runtime(start_status)
try: try:
ctx: Context = await actor._service_n.start( ctx: Context = await actor._service_tn.start(
partial( partial(
_invoke, _invoke,
actor, actor,
@ -1312,7 +1312,7 @@ async def process_messages(
) as err: ) as err:
if nursery_cancelled_before_task: if nursery_cancelled_before_task:
sn: Nursery = actor._service_n sn: Nursery = actor._service_tn
assert sn and sn.cancel_scope.cancel_called # sanity assert sn and sn.cancel_scope.cancel_called # sanity
log.cancel( log.cancel(
f'Service nursery cancelled before it handled {funcname}' f'Service nursery cancelled before it handled {funcname}'

View File

@ -35,6 +35,15 @@ for running all lower level spawning, supervision and msging layers:
SC-transitive RPC via scheduling of `trio` tasks. SC-transitive RPC via scheduling of `trio` tasks.
- registration of newly spawned actors with the discovery sys. - registration of newly spawned actors with the discovery sys.
Glossary:
--------
- tn: a `trio.Nursery` or "task nursery".
- an: an `ActorNursery` or "actor nursery".
- root: top/parent-most scope/task/process/actor (or other runtime
primitive) in a hierarchical tree.
- parent-ish: "higher-up" in the runtime-primitive hierarchy.
- child-ish: "lower-down" in the runtime-primitive hierarchy.
''' '''
from __future__ import annotations from __future__ import annotations
from contextlib import ( from contextlib import (
@ -76,6 +85,7 @@ from tractor.msg import (
) )
from .trionics import ( from .trionics import (
collapse_eg, collapse_eg,
maybe_open_nursery,
) )
from .ipc import ( from .ipc import (
Channel, Channel,
@ -173,9 +183,11 @@ class Actor:
msg_buffer_size: int = 2**6 msg_buffer_size: int = 2**6
# nursery placeholders filled in by `async_main()` after fork # nursery placeholders filled in by `async_main()`,
_service_n: Nursery|None = None # - after fork for subactors.
# - during boot for the root actor.
_root_tn: Nursery|None = None
_service_tn: Nursery|None = None
_ipc_server: _server.IPCServer|None = None _ipc_server: _server.IPCServer|None = None
@property @property
@ -1009,12 +1021,48 @@ class Actor:
the RPC service nursery. the RPC service nursery.
''' '''
assert self._service_n actor_repr: str = _pformat.nest_from_op(
self._service_n.start_soon( input_op='>c(',
text=self.pformat(),
nest_indent=1,
)
log.cancel(
'Actor.cancel_soon()` was called!\n'
f'>> scheduling `Actor.cancel()`\n'
f'{actor_repr}'
)
assert self._service_tn
self._service_tn.start_soon(
self.cancel, self.cancel,
None, # self cancel all rpc tasks None, # self cancel all rpc tasks
) )
# schedule a "canceller task" in the `._root_tn` once the
# `._service_tn` is fully shutdown; task waits for child-ish
# scopes to fully exit then finally cancels its parent,
# root-most, scope.
async def cancel_root_tn_after_services():
log.runtime(
'Waiting on service-tn to cancel..\n'
f'c>)\n'
f'|_{self._service_tn.cancel_scope!r}\n'
)
await self._cancel_complete.wait()
log.cancel(
f'`._service_tn` cancelled\n'
f'>c)\n'
f'|_{self._service_tn.cancel_scope!r}\n'
f'\n'
f'>> cancelling `._root_tn`\n'
f'c>(\n'
f' |_{self._root_tn.cancel_scope!r}\n'
)
self._root_tn.cancel_scope.cancel()
self._root_tn.start_soon(
cancel_root_tn_after_services
)
@property @property
def cancel_complete(self) -> bool: def cancel_complete(self) -> bool:
return self._cancel_complete.is_set() return self._cancel_complete.is_set()
@ -1119,8 +1167,8 @@ class Actor:
await ipc_server.wait_for_shutdown() await ipc_server.wait_for_shutdown()
# cancel all rpc tasks permanently # cancel all rpc tasks permanently
if self._service_n: if self._service_tn:
self._service_n.cancel_scope.cancel() self._service_tn.cancel_scope.cancel()
log_meth(msg) log_meth(msg)
self._cancel_complete.set() self._cancel_complete.set()
@ -1257,7 +1305,7 @@ class Actor:
''' '''
Cancel all ongoing RPC tasks owned/spawned for a given Cancel all ongoing RPC tasks owned/spawned for a given
`parent_chan: Channel` or simply all tasks (inside `parent_chan: Channel` or simply all tasks (inside
`._service_n`) when `parent_chan=None`. `._service_tn`) when `parent_chan=None`.
''' '''
tasks: dict = self._rpc_tasks tasks: dict = self._rpc_tasks
@ -1469,46 +1517,55 @@ async def async_main(
accept_addrs.append(addr.unwrap()) accept_addrs.append(addr.unwrap())
assert accept_addrs assert accept_addrs
# The "root" nursery ensures the channel with the immediate
# parent is kept alive as a resilient service until ya_root_tn: bool = bool(actor._root_tn)
# cancellation steps have (mostly) occurred in ya_service_tn: bool = bool(actor._service_tn)
# a deterministic way.
# NOTE, a top-most "root" nursery in each actor-process
# enables a lifetime priority for the IPC-channel connection
# with a sub-actor's immediate parent. I.e. this connection
# is kept alive as a resilient service connection until all
# other machinery has exited, cancellation of all
# embedded/child scopes have completed. This helps ensure
# a deterministic (and thus "graceful")
# first-class-supervision style teardown where a parent actor
# (vs. say peers) is always the last to be contacted before
# disconnect.
root_tn: trio.Nursery root_tn: trio.Nursery
async with ( async with (
collapse_eg(), collapse_eg(),
trio.open_nursery() as root_tn, maybe_open_nursery(
nursery=actor._root_tn,
) as root_tn,
): ):
# actor._root_n = root_tn if ya_root_tn:
# assert actor._root_n assert root_tn is actor._root_tn
else:
actor._root_tn = root_tn
ipc_server: _server.IPCServer ipc_server: _server.IPCServer
async with ( async with (
collapse_eg(), collapse_eg(),
trio.open_nursery() as service_nursery, maybe_open_nursery(
nursery=actor._service_tn,
) as service_tn,
_server.open_ipc_server( _server.open_ipc_server(
parent_tn=service_nursery, parent_tn=service_tn, # ?TODO, why can't this be the root-tn
stream_handler_tn=service_nursery, stream_handler_tn=service_tn,
) as ipc_server, ) as ipc_server,
# ) as actor._ipc_server,
# ^TODO? prettier?
): ):
# This nursery is used to handle all inbound if ya_service_tn:
# connections to us such that if the TCP server assert service_tn is actor._service_tn
# is killed, connections can continue to process else:
# in the background until this nursery is cancelled. # This nursery is used to handle all inbound
actor._service_n = service_nursery # connections to us such that if the TCP server
# is killed, connections can continue to process
# in the background until this nursery is cancelled.
actor._service_tn = service_tn
# set after allocate
actor._ipc_server = ipc_server actor._ipc_server = ipc_server
assert (
actor._service_n
and (
actor._service_n
is
actor._ipc_server._parent_tn
is
ipc_server._stream_handler_tn
)
)
# load exposed/allowed RPC modules # load exposed/allowed RPC modules
# XXX: do this **after** establishing a channel to the parent # XXX: do this **after** establishing a channel to the parent
@ -1534,10 +1591,11 @@ async def async_main(
# - root actor: the ``accept_addr`` passed to this method # - root actor: the ``accept_addr`` passed to this method
# TODO: why is this not with the root nursery? # TODO: why is this not with the root nursery?
# - see above that the `._service_tn` is what's used?
try: try:
eps: list = await ipc_server.listen_on( eps: list = await ipc_server.listen_on(
accept_addrs=accept_addrs, accept_addrs=accept_addrs,
stream_handler_nursery=service_nursery, stream_handler_nursery=service_tn,
) )
log.runtime( log.runtime(
f'Booted IPC server\n' f'Booted IPC server\n'
@ -1545,7 +1603,7 @@ async def async_main(
) )
assert ( assert (
(eps[0].listen_tn) (eps[0].listen_tn)
is not service_nursery is not service_tn
) )
except OSError as oserr: except OSError as oserr:
@ -1707,7 +1765,7 @@ async def async_main(
# XXX TODO but hard XXX # XXX TODO but hard XXX
# we can't actually do this bc the debugger uses the # we can't actually do this bc the debugger uses the
# _service_n to spawn the lock task, BUT, in theory if we had # _service_tn to spawn the lock task, BUT, in theory if we had
# the root nursery surround this finally block it might be # the root nursery surround this finally block it might be
# actually possible to debug THIS machinery in the same way # actually possible to debug THIS machinery in the same way
# as user task code? # as user task code?

View File

@ -297,6 +297,23 @@ async def hard_kill(
# zombies (as a feature) we ask the OS to do send in the # zombies (as a feature) we ask the OS to do send in the
# removal swad as the last resort. # removal swad as the last resort.
if cs.cancelled_caught: if cs.cancelled_caught:
# TODO? attempt at intermediary-rent-sub
# with child in debug lock?
# |_https://github.com/goodboy/tractor/issues/320
#
# if not is_root_process():
# log.warning(
# 'Attempting to acquire debug-REPL-lock before zombie reap!'
# )
# with trio.CancelScope(shield=True):
# async with debug.acquire_debug_lock(
# subactor_uid=current_actor().uid,
# ) as _ctx:
# log.warning(
# 'Acquired debug lock, child ready to be killed ??\n'
# )
# TODO: toss in the skynet-logo face as ascii art? # TODO: toss in the skynet-logo face as ascii art?
log.critical( log.critical(
# 'Well, the #ZOMBIE_LORD_IS_HERE# to collect\n' # 'Well, the #ZOMBIE_LORD_IS_HERE# to collect\n'

View File

@ -250,7 +250,7 @@ async def _maybe_enter_pm(
*, *,
tb: TracebackType|None = None, tb: TracebackType|None = None,
api_frame: FrameType|None = None, api_frame: FrameType|None = None,
hide_tb: bool = False, hide_tb: bool = True,
# only enter debugger REPL when returns `True` # only enter debugger REPL when returns `True`
debug_filter: Callable[ debug_filter: Callable[

View File

@ -58,6 +58,7 @@ from tractor._context import Context
from tractor import _state from tractor import _state
from tractor._exceptions import ( from tractor._exceptions import (
NoRuntime, NoRuntime,
InternalError,
) )
from tractor._state import ( from tractor._state import (
current_actor, current_actor,
@ -79,6 +80,9 @@ from ._sigint import (
sigint_shield as sigint_shield, sigint_shield as sigint_shield,
_ctlc_ignore_header as _ctlc_ignore_header _ctlc_ignore_header as _ctlc_ignore_header
) )
from ..pformat import (
ppfmt,
)
if TYPE_CHECKING: if TYPE_CHECKING:
from trio.lowlevel import Task from trio.lowlevel import Task
@ -477,12 +481,12 @@ async def _pause(
# we have to figure out how to avoid having the service nursery # we have to figure out how to avoid having the service nursery
# cancel on this task start? I *think* this works below: # cancel on this task start? I *think* this works below:
# ```python # ```python
# actor._service_n.cancel_scope.shield = shield # actor._service_tn.cancel_scope.shield = shield
# ``` # ```
# but not entirely sure if that's a sane way to implement it? # but not entirely sure if that's a sane way to implement it?
# NOTE currently we spawn the lock request task inside this # NOTE currently we spawn the lock request task inside this
# subactor's global `Actor._service_n` so that the # subactor's global `Actor._service_tn` so that the
# lifetime of the lock-request can outlive the current # lifetime of the lock-request can outlive the current
# `._pause()` scope while the user steps through their # `._pause()` scope while the user steps through their
# application code and when they finally exit the # application code and when they finally exit the
@ -506,7 +510,7 @@ async def _pause(
f'|_{task}\n' f'|_{task}\n'
) )
with trio.CancelScope(shield=shield): with trio.CancelScope(shield=shield):
req_ctx: Context = await actor._service_n.start( req_ctx: Context = await actor._service_tn.start(
partial( partial(
request_root_stdio_lock, request_root_stdio_lock,
actor_uid=actor.uid, actor_uid=actor.uid,
@ -540,7 +544,7 @@ async def _pause(
_repl_fail_report = None _repl_fail_report = None
# when the actor is mid-runtime cancellation the # when the actor is mid-runtime cancellation the
# `Actor._service_n` might get closed before we can spawn # `Actor._service_tn` might get closed before we can spawn
# the request task, so just ignore expected RTE. # the request task, so just ignore expected RTE.
elif ( elif (
isinstance(pause_err, RuntimeError) isinstance(pause_err, RuntimeError)
@ -985,7 +989,7 @@ def pause_from_sync(
# that output and assign the `repl` created above! # that output and assign the `repl` created above!
bg_task, _ = trio.from_thread.run( bg_task, _ = trio.from_thread.run(
afn=partial( afn=partial(
actor._service_n.start, actor._service_tn.start,
partial( partial(
_pause_from_bg_root_thread, _pause_from_bg_root_thread,
behalf_of_thread=thread, behalf_of_thread=thread,
@ -1153,9 +1157,10 @@ def pause_from_sync(
'use_greenback', 'use_greenback',
False, False,
): ):
raise RuntimeError( raise InternalError(
'`greenback` was never initialized in this actor!?\n\n' f'`greenback` was never initialized in this actor?\n'
f'{_state._runtime_vars}\n' f'\n'
f'{ppfmt(_state._runtime_vars)}\n'
) from rte ) from rte
raise raise

View File

@ -1001,7 +1001,11 @@ class Server(Struct):
partial( partial(
_serve_ipc_eps, _serve_ipc_eps,
server=self, server=self,
stream_handler_tn=stream_handler_nursery, stream_handler_tn=(
stream_handler_nursery
or
self._stream_handler_tn
),
listen_addrs=accept_addrs, listen_addrs=accept_addrs,
) )
) )
@ -1145,13 +1149,17 @@ async def open_ipc_server(
async with maybe_open_nursery( async with maybe_open_nursery(
nursery=parent_tn, nursery=parent_tn,
) as rent_tn: ) as parent_tn:
no_more_peers = trio.Event() no_more_peers = trio.Event()
no_more_peers.set() no_more_peers.set()
ipc_server = IPCServer( ipc_server = IPCServer(
_parent_tn=rent_tn, _parent_tn=parent_tn,
_stream_handler_tn=stream_handler_tn or rent_tn, _stream_handler_tn=(
stream_handler_tn
or
parent_tn
),
_no_more_peers=no_more_peers, _no_more_peers=no_more_peers,
) )
try: try:

View File

@ -204,7 +204,7 @@ class _Cache:
a kept-alive-while-in-use async resource. a kept-alive-while-in-use async resource.
''' '''
service_n: Optional[trio.Nursery] = None service_tn: Optional[trio.Nursery] = None
locks: dict[Hashable, trio.Lock] = {} locks: dict[Hashable, trio.Lock] = {}
users: int = 0 users: int = 0
values: dict[Any, Any] = {} values: dict[Any, Any] = {}
@ -294,15 +294,15 @@ async def maybe_open_context(
f'task: {task}\n' f'task: {task}\n'
f'task_tn: {task_tn}\n' f'task_tn: {task_tn}\n'
) )
service_n = tn service_tn = tn
else: else:
service_n: trio.Nursery = current_actor()._service_n service_tn: trio.Nursery = current_actor()._service_tn
# TODO: is there any way to allocate # TODO: is there any way to allocate
# a 'stays-open-till-last-task-finshed nursery? # a 'stays-open-till-last-task-finshed nursery?
# service_n: trio.Nursery # service_tn: trio.Nursery
# async with maybe_open_nursery(_Cache.service_n) as service_n: # async with maybe_open_nursery(_Cache.service_tn) as service_tn:
# _Cache.service_n = service_n # _Cache.service_tn = service_tn
cache_miss_ke: KeyError|None = None cache_miss_ke: KeyError|None = None
maybe_taskc: trio.Cancelled|None = None maybe_taskc: trio.Cancelled|None = None
@ -324,8 +324,8 @@ async def maybe_open_context(
mngr = acm_func(**kwargs) mngr = acm_func(**kwargs)
resources = _Cache.resources resources = _Cache.resources
assert not resources.get(ctx_key), f'Resource exists? {ctx_key}' assert not resources.get(ctx_key), f'Resource exists? {ctx_key}'
resources[ctx_key] = (service_n, trio.Event()) resources[ctx_key] = (service_tn, trio.Event())
yielded: Any = await service_n.start( yielded: Any = await service_tn.start(
_Cache.run_ctx, _Cache.run_ctx,
mngr, mngr,
ctx_key, ctx_key,