forked from goodboy/tractor
1
0
Fork 0

Better subproc supervisor logging, todo for #320

Given i just similarly revamped a buncha `._runtime` log msg formatting,
might as well do something similar inside the spawning machinery such
that groking teardown sequences of each supervising task is much more
sane XD

Mostly this includes doing similar `'<field>: <value>\n'` multi-line
formatting when reporting various subproc supervision steps as well as
showing a detailed `trio.Process.__repr__()` as appropriate.

Also adds a detailed #TODO according to the needs of #320 for which
we're going to need some internal mechanism for intermediary parent
actors to determine if a given debug tty locker (sub-actor) is one of
*their* (transitive) children and thus stall the normal
cancellation/teardown sequence until that locker is complete.
modden_spawn_from_client_req
Tyler Goodlet 2024-02-20 13:12:51 -05:00
parent 81f8e2d4ac
commit 0268b2ce91
1 changed files with 83 additions and 35 deletions

View File

@ -35,7 +35,7 @@ from exceptiongroup import BaseExceptionGroup
import trio import trio
from trio_typing import TaskStatus from trio_typing import TaskStatus
from .devx._debug import ( from .devx import (
maybe_wait_for_debugger, maybe_wait_for_debugger,
acquire_debug_lock, acquire_debug_lock,
) )
@ -144,7 +144,7 @@ async def exhaust_portal(
# XXX: streams should never be reaped here since they should # XXX: streams should never be reaped here since they should
# always be established and shutdown using a context manager api # always be established and shutdown using a context manager api
final = await portal.result() final: Any = await portal.result()
except ( except (
Exception, Exception,
@ -152,13 +152,23 @@ async def exhaust_portal(
) as err: ) as err:
# we reraise in the parent task via a ``BaseExceptionGroup`` # we reraise in the parent task via a ``BaseExceptionGroup``
return err return err
except trio.Cancelled as err: except trio.Cancelled as err:
# lol, of course we need this too ;P # lol, of course we need this too ;P
# TODO: merge with above? # TODO: merge with above?
log.warning(f"Cancelled result waiter for {portal.actor.uid}") log.warning(
'Cancelled portal result waiter task:\n'
f'uid: {portal.channel.uid}\n'
f'error: {err}\n'
)
return err return err
else: else:
log.debug(f"Returning final result: {final}") log.debug(
f'Returning final result from portal:\n'
f'uid: {portal.channel.uid}\n'
f'result: {final}\n'
)
return final return final
@ -170,26 +180,34 @@ async def cancel_on_completion(
) -> None: ) -> None:
''' '''
Cancel actor gracefully once it's "main" portal's Cancel actor gracefully once its "main" portal's
result arrives. result arrives.
Should only be called for actors spawned with `run_in_actor()`. Should only be called for actors spawned via the
`Portal.run_in_actor()` API.
=> and really this API will be deprecated and should be
re-implemented as a `.hilevel.one_shot_task_nursery()`..)
''' '''
# if this call errors we store the exception for later # if this call errors we store the exception for later
# in ``errors`` which will be reraised inside # in ``errors`` which will be reraised inside
# an exception group and we still send out a cancel request # an exception group and we still send out a cancel request
result = await exhaust_portal(portal, actor) result: Any|Exception = await exhaust_portal(portal, actor)
if isinstance(result, Exception): if isinstance(result, Exception):
errors[actor.uid] = result errors[actor.uid]: Exception = result
log.warning( log.warning(
f"Cancelling {portal.channel.uid} after error {result}" 'Cancelling subactor due to error:\n'
f'uid: {portal.channel.uid}\n'
f'error: {result}\n'
) )
else: else:
log.runtime( log.runtime(
f"Cancelling {portal.channel.uid} gracefully " 'Cancelling subactor gracefully:\n'
f"after result {result}") f'uid: {portal.channel.uid}\n'
f'result: {result}\n'
)
# cancel the process now that we have a final result # cancel the process now that we have a final result
await portal.cancel_actor() await portal.cancel_actor()
@ -219,11 +237,14 @@ async def do_hard_kill(
to be handled. to be handled.
''' '''
log.cancel(
'Terminating sub-proc:\n'
f'|_{proc}\n'
)
# NOTE: this timeout used to do nothing since we were shielding # NOTE: this timeout used to do nothing since we were shielding
# the ``.wait()`` inside ``new_proc()`` which will pretty much # the ``.wait()`` inside ``new_proc()`` which will pretty much
# never release until the process exits, now it acts as # never release until the process exits, now it acts as
# a hard-kill time ultimatum. # a hard-kill time ultimatum.
log.debug(f"Terminating {proc}")
with trio.move_on_after(terminate_after) as cs: with trio.move_on_after(terminate_after) as cs:
# NOTE: code below was copied verbatim from the now deprecated # NOTE: code below was copied verbatim from the now deprecated
@ -260,7 +281,10 @@ async def do_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:
log.critical(f"#ZOMBIE_LORD_IS_HERE: {proc}") log.critical(
'Well, the #ZOMBIE_LORD_IS_HERE# to collect\n'
f'|_{proc}\n'
)
proc.kill() proc.kill()
@ -281,10 +305,16 @@ async def soft_wait(
join/reap on an actor-runtime-in-process. join/reap on an actor-runtime-in-process.
''' '''
uid = portal.channel.uid uid: tuple[str, str] = portal.channel.uid
try: try:
log.cancel(f'Soft waiting on actor:\n{uid}') log.cancel(
'Soft waiting on sub-actor proc:\n'
f'uid: {uid}\n'
f'|_{proc}\n'
)
# wait on sub-proc to signal termination
await wait_func(proc) await wait_func(proc)
except trio.Cancelled: except trio.Cancelled:
# if cancelled during a soft wait, cancel the child # if cancelled during a soft wait, cancel the child
# actor before entering the hard reap sequence # actor before entering the hard reap sequence
@ -296,8 +326,8 @@ async def soft_wait(
async def cancel_on_proc_deth(): async def cancel_on_proc_deth():
''' '''
Cancel the actor cancel request if we detect that "Cancel the (actor) cancel" request if we detect
that the process terminated. that that the underlying sub-process terminated.
''' '''
await wait_func(proc) await wait_func(proc)
@ -314,10 +344,10 @@ async def soft_wait(
if proc.poll() is None: # type: ignore if proc.poll() is None: # type: ignore
log.warning( log.warning(
'Actor still alive after cancel request:\n' 'Subactor still alive after cancel request?\n\n'
f'{uid}' f'uid: {uid}\n'
f'|_{proc}\n'
) )
n.cancel_scope.cancel() n.cancel_scope.cancel()
raise raise
@ -341,7 +371,7 @@ async def new_proc(
) -> None: ) -> None:
# lookup backend spawning target # lookup backend spawning target
target = _methods[_spawn_method] target: Callable = _methods[_spawn_method]
# mark the new actor with the global spawn method # mark the new actor with the global spawn method
subactor._spawn_method = _spawn_method subactor._spawn_method = _spawn_method
@ -491,8 +521,9 @@ async def trio_proc(
# cancel result waiter that may have been spawned in # cancel result waiter that may have been spawned in
# tandem if not done already # tandem if not done already
log.cancel( log.cancel(
"Cancelling existing result waiter task for " 'Cancelling existing result waiter task for '
f"{subactor.uid}") f'{subactor.uid}'
)
nursery.cancel_scope.cancel() nursery.cancel_scope.cancel()
finally: finally:
@ -510,7 +541,16 @@ async def trio_proc(
with trio.move_on_after(0.5): with trio.move_on_after(0.5):
await proc.wait() await proc.wait()
if is_root_process(): log.pdb(
'Delaying subproc reaper while debugger locked..'
)
await maybe_wait_for_debugger(
child_in_debug=_runtime_vars.get(
'_debug_mode', False
),
# TODO: need a diff value then default?
# poll_steps=9999999,
)
# TODO: solve the following issue where we need # TODO: solve the following issue where we need
# to do a similar wait like this but in an # to do a similar wait like this but in an
# "intermediary" parent actor that itself isn't # "intermediary" parent actor that itself isn't
@ -518,10 +558,18 @@ async def trio_proc(
# to hold off on relaying SIGINT until that child # to hold off on relaying SIGINT until that child
# is complete. # is complete.
# https://github.com/goodboy/tractor/issues/320 # https://github.com/goodboy/tractor/issues/320
await maybe_wait_for_debugger( # -[ ] we need to handle non-root parent-actors specially
child_in_debug=_runtime_vars.get( # by somehow determining if a child is in debug and then
'_debug_mode', False), # avoiding cancel/kill of said child by this
) # (intermediary) parent until such a time as the root says
# the pdb lock is released and we are good to tear down
# (our children)..
#
# -[ ] so maybe something like this where we try to
# acquire the lock and get notified of who has it,
# check that uid against our known children?
# this_uid: tuple[str, str] = current_actor().uid
# await acquire_debug_lock(this_uid)
if proc.poll() is None: if proc.poll() is None:
log.cancel(f"Attempting to hard kill {proc}") log.cancel(f"Attempting to hard kill {proc}")