Make `request_root_stdio_lock()` post-mortem-able
Finally got this working so that if/when an internal bug is introduced to this request task-func, we can actually REPL-debug the lock request task itself B) As in, if the subactor's lock request task internally errors we, - ensure the task always terminates (by calling `DebugStatus.release()`) and explicitly reports (via a `log.exception()`) the internal error. - capture the error instance and set as a new `DebugStatus.req_err` and always check for it on final teardown - in which case we also, - ensure it's reraised from a new `DebugRequestError`. - unhide the stack frames for `_pause()`, `_enter_repl_sync()` so that the dev can upward inspect the `_pause()` call stack sanely. Supporting internal impl changes, - add `DebugStatus.cancel()` and `.req_err`. - don't ever cancel the request task from `PdbREPL.set_[continue/quit]()` only when there's some internal error that would likely result in a hang and stale lock state with the root. - only release the root's lock when the current ask is also the owner (avoids bad release errors). - also show internal `._pause()`-related frames on any `repl_err`. Other temp-dev-tweaks, - make pld-dec change log msgs info level again while solving this final context-vars race stuff.. - drop the debug pld-dec instance match asserts for now since the problem is already caught (and now debug-able B) by an attr-error on the decoded-as-`dict` started msg, and instead add in a `log.exception()` trace to see which task is triggering the case where the debug `MsgDec` isn't set correctly vs. when we think it's being applied.runtime_to_msgspec
parent
31de5f6648
commit
b23780c102
|
@ -234,49 +234,26 @@ class Lock:
|
|||
cls,
|
||||
force: bool = False,
|
||||
):
|
||||
lock: trio.StrictFIFOLock = cls._debug_lock
|
||||
try:
|
||||
if lock.locked():
|
||||
lock: trio.StrictFIFOLock = cls._debug_lock
|
||||
owner: Task = lock.statistics().owner
|
||||
if (
|
||||
lock.locked()
|
||||
and
|
||||
owner is current_task()
|
||||
# ^-NOTE-^ if not will raise a RTE..
|
||||
):
|
||||
if not DebugStatus.is_main_trio_thread():
|
||||
trio.from_thread.run_sync(
|
||||
cls._debug_lock.release
|
||||
)
|
||||
else:
|
||||
cls._debug_lock.release()
|
||||
message: str = 'TTY lock released for child\n'
|
||||
|
||||
message: str = 'TTY lock released for child\n'
|
||||
else:
|
||||
message: str = 'TTY lock not held by any child\n'
|
||||
|
||||
except RuntimeError as rte:
|
||||
message: str = (
|
||||
'TTY lock FAILED to release for child??\n'
|
||||
f'{current_task()}\n'
|
||||
)
|
||||
log.exception(message)
|
||||
|
||||
# uhhh makes no sense but been seeing the non-owner
|
||||
# release error even though this is definitely the task
|
||||
# that locked?
|
||||
owner = cls._debug_lock.statistics().owner
|
||||
# if (
|
||||
# owner
|
||||
# and
|
||||
# cls.remote_task_in_debug is None
|
||||
# ):
|
||||
# raise RuntimeError(
|
||||
# 'Stale `Lock` detected, no remote task active!?\n'
|
||||
# f'|_{owner}\n'
|
||||
# # f'{cls}'
|
||||
# ) from rte
|
||||
|
||||
if owner:
|
||||
raise rte
|
||||
|
||||
# OW suppress, can't member why tho .. XD
|
||||
# something somethin corrupts a cancel-scope
|
||||
# somewhere..
|
||||
|
||||
finally:
|
||||
# IFF there are no more requesting tasks queued up fire, the
|
||||
# "tty-unlocked" event thereby alerting any monitors of the lock that
|
||||
|
@ -518,11 +495,23 @@ async def lock_tty_for_child(
|
|||
locked=False,
|
||||
)
|
||||
|
||||
except BaseException:
|
||||
log.exception(
|
||||
'Errored during root TTY-lock dialog?\n'
|
||||
'Forcing release since an internal error caused this!\n'
|
||||
except BaseException as req_err:
|
||||
message: str = (
|
||||
'Forcing `Lock.release()` since likely an internal error!\n'
|
||||
)
|
||||
if isinstance(req_err, trio.Cancelled):
|
||||
log.cancel(
|
||||
'Cancelled during root TTY-lock dialog?\n'
|
||||
+
|
||||
message
|
||||
)
|
||||
else:
|
||||
log.exception(
|
||||
'Errored during root TTY-lock dialog?\n'
|
||||
+
|
||||
message
|
||||
)
|
||||
|
||||
Lock.release(force=True)
|
||||
raise
|
||||
|
||||
|
@ -555,6 +544,7 @@ class DebugStatus:
|
|||
repl_release: trio.Event|None = None
|
||||
req_finished: trio.Event|None = None
|
||||
lock_status: LockStatus|None = None
|
||||
req_err: BaseException|None = None
|
||||
|
||||
_orig_sigint_handler: Callable|None = None
|
||||
_trio_handler: (
|
||||
|
@ -693,28 +683,37 @@ class DebugStatus:
|
|||
# is not threading.main_thread()
|
||||
# )
|
||||
|
||||
@classmethod
|
||||
def cancel(cls) -> bool:
|
||||
if (req_cs := cls.req_cs):
|
||||
req_cs.cancel()
|
||||
return True
|
||||
|
||||
return False
|
||||
|
||||
@classmethod
|
||||
@pdbp.hideframe
|
||||
def release(
|
||||
cls,
|
||||
cancel_req_task: bool = True,
|
||||
cancel_req_task: bool = False,
|
||||
):
|
||||
repl_release: trio.Event = cls.repl_release
|
||||
try:
|
||||
# sometimes the task might already be terminated in
|
||||
# which case this call will raise an RTE?
|
||||
if cls.repl_release is not None:
|
||||
cls.repl_release.set()
|
||||
if repl_release is not None:
|
||||
repl_release.set()
|
||||
|
||||
finally:
|
||||
# if req_ctx := cls.req_ctx:
|
||||
# req_ctx._scope.cancel()
|
||||
|
||||
if (
|
||||
cancel_req_task
|
||||
and
|
||||
(req_cs := cls.req_cs)
|
||||
):
|
||||
req_cs.cancel()
|
||||
if cancel_req_task:
|
||||
cancelled: bool = cls.cancel()
|
||||
if not cancelled:
|
||||
log.warning(
|
||||
'Failed to cancel request task!?\n'
|
||||
f'{cls.repl_task}\n'
|
||||
)
|
||||
|
||||
# restore original sigint handler
|
||||
cls.unshield_sigint()
|
||||
|
@ -759,16 +758,19 @@ class PdbREPL(pdbp.Pdb):
|
|||
status = DebugStatus
|
||||
|
||||
# NOTE: see details in stdlib's `bdb.py`
|
||||
def user_exception(self, frame, exc_info):
|
||||
'''
|
||||
Called when we stop on an exception.
|
||||
'''
|
||||
log.warning(
|
||||
'Exception during REPL sesh\n\n'
|
||||
f'{frame}\n\n'
|
||||
f'{exc_info}\n\n'
|
||||
)
|
||||
# def user_exception(self, frame, exc_info):
|
||||
# '''
|
||||
# Called when we stop on an exception.
|
||||
# '''
|
||||
# log.warning(
|
||||
# 'Exception during REPL sesh\n\n'
|
||||
# f'{frame}\n\n'
|
||||
# f'{exc_info}\n\n'
|
||||
# )
|
||||
|
||||
# NOTE: this actually hooks but i don't see anyway to detect
|
||||
# if an error was caught.. this is why currently we just always
|
||||
# call `DebugStatus.release` inside `_post_mortem()`.
|
||||
# def preloop(self):
|
||||
# print('IN PRELOOP')
|
||||
# super().preloop()
|
||||
|
@ -804,10 +806,7 @@ class PdbREPL(pdbp.Pdb):
|
|||
try:
|
||||
super().set_quit()
|
||||
finally:
|
||||
DebugStatus.release(
|
||||
cancel_req_task=False,
|
||||
)
|
||||
|
||||
DebugStatus.release()
|
||||
if (
|
||||
is_root_process()
|
||||
and
|
||||
|
@ -863,7 +862,6 @@ def apply_debug_pldec() -> _codec.MsgCodec:
|
|||
(only in the current task).
|
||||
|
||||
'''
|
||||
|
||||
from tractor.msg import (
|
||||
_ops as msgops,
|
||||
)
|
||||
|
@ -874,8 +872,12 @@ def apply_debug_pldec() -> _codec.MsgCodec:
|
|||
with msgops.limit_plds(
|
||||
spec=__pld_spec__,
|
||||
) as debug_dec:
|
||||
assert debug_dec is msgops.current_pldrx().pld_dec
|
||||
log.runtime(
|
||||
assert (
|
||||
debug_dec
|
||||
is
|
||||
msgops.current_pldrx().pld_dec
|
||||
)
|
||||
log.info(
|
||||
'Applied `.devx._debug` pld-spec\n\n'
|
||||
f'{debug_dec}\n'
|
||||
)
|
||||
|
@ -887,11 +889,12 @@ def apply_debug_pldec() -> _codec.MsgCodec:
|
|||
and
|
||||
plrx.pld_dec is orig_pldec
|
||||
)
|
||||
log.runtime(
|
||||
log.info(
|
||||
'Reverted to previous pld-spec\n\n'
|
||||
f'{orig_pldec}\n'
|
||||
)
|
||||
|
||||
|
||||
async def request_root_stdio_lock(
|
||||
actor_uid: tuple[str, str],
|
||||
task_uid: tuple[str, int],
|
||||
|
@ -911,6 +914,10 @@ async def request_root_stdio_lock(
|
|||
entering the REPL at the same time.
|
||||
|
||||
'''
|
||||
|
||||
log.pdb(
|
||||
'Initing stdio-lock request task with root actor'
|
||||
)
|
||||
# TODO: likely we can implement this mutex more generally as
|
||||
# a `._sync.Lock`?
|
||||
# -[ ] simply add the wrapping needed for the debugger specifics?
|
||||
|
@ -923,6 +930,8 @@ async def request_root_stdio_lock(
|
|||
DebugStatus.req_finished = trio.Event()
|
||||
try:
|
||||
from tractor._discovery import get_root
|
||||
from tractor.msg import _ops as msgops
|
||||
debug_dec: msgops.MsgDec
|
||||
with (
|
||||
# NOTE: we need this to ensure that this task exits
|
||||
# BEFORE the REPl instance raises an error like
|
||||
|
@ -953,12 +962,13 @@ async def request_root_stdio_lock(
|
|||
#
|
||||
apply_debug_pldec() as debug_dec,
|
||||
):
|
||||
log.critical(
|
||||
'Request cancel-scope is:\n\n'
|
||||
f'{pformat_cs(req_cs, var_name="req_cs")}\n\n'
|
||||
|
||||
)
|
||||
# XXX: was orig for debugging cs stack corruption..
|
||||
# log.info(
|
||||
# 'Request cancel-scope is:\n\n'
|
||||
# f'{pformat_cs(req_cs, var_name="req_cs")}\n\n'
|
||||
# )
|
||||
DebugStatus.req_cs = req_cs
|
||||
req_ctx: Context|None = None
|
||||
try:
|
||||
# TODO: merge into single async with ?
|
||||
async with get_root() as portal:
|
||||
|
@ -966,31 +976,37 @@ async def request_root_stdio_lock(
|
|||
async with portal.open_context(
|
||||
lock_tty_for_child,
|
||||
subactor_task_uid=task_uid,
|
||||
) as (ctx, status):
|
||||
) as (req_ctx, status):
|
||||
|
||||
DebugStatus.req_ctx = ctx
|
||||
DebugStatus.req_ctx = req_ctx
|
||||
|
||||
# sanity checks on pld-spec limit state
|
||||
assert debug_dec
|
||||
# curr_pldrx: msgops.PldRx = msgops.current_pldrx()
|
||||
# assert (
|
||||
# curr_pldrx.pld_dec is debug_dec
|
||||
# )
|
||||
|
||||
from tractor.msg import (
|
||||
_ops as msgops,
|
||||
)
|
||||
assert (
|
||||
msgops.current_pldrx().pld_dec is debug_dec
|
||||
)
|
||||
log.debug(
|
||||
'Subactor locked TTY with msg\n\n'
|
||||
f'{status}\n'
|
||||
)
|
||||
|
||||
# mk_pdb().set_trace()
|
||||
assert status.subactor_uid == actor_uid
|
||||
assert status.cid
|
||||
try:
|
||||
assert status.subactor_uid == actor_uid
|
||||
assert status.cid
|
||||
except AttributeError:
|
||||
log.exception('failed pldspec asserts!')
|
||||
raise
|
||||
|
||||
# set last rxed lock dialog status.
|
||||
DebugStatus.lock_status = status
|
||||
|
||||
async with ctx.open_stream() as stream:
|
||||
async with req_ctx.open_stream() as stream:
|
||||
|
||||
assert DebugStatus.repl_release
|
||||
task_status.started(ctx)
|
||||
task_status.started(req_ctx)
|
||||
|
||||
# wait for local task to exit its
|
||||
# `PdbREPL.interaction()`, call
|
||||
|
@ -1006,25 +1022,25 @@ async def request_root_stdio_lock(
|
|||
|
||||
# sync with child-side root locker task
|
||||
# completion
|
||||
status: LockStatus = await ctx.result()
|
||||
status: LockStatus = await req_ctx.result()
|
||||
assert not status.locked
|
||||
DebugStatus.lock_status = status
|
||||
|
||||
log.pdb(
|
||||
'TTY lock was released for subactor with msg\n\n'
|
||||
f'{status}\n\n'
|
||||
f'Exitting {ctx.side!r}-side of locking ctx'
|
||||
f'Exitting {req_ctx.side!r}-side of locking req_ctx'
|
||||
)
|
||||
|
||||
except (
|
||||
tractor.ContextCancelled,
|
||||
trio.Cancelled,
|
||||
):
|
||||
log.exception(
|
||||
'Debug lock request CANCELLED?\n\n'
|
||||
f'{pformat_cs(req_cs, var_name="req_cs")}\n\n'
|
||||
f'{pformat_cs(ctx._scope, var_name="ctx._scope")}\n\n'
|
||||
f'{ctx}'
|
||||
log.cancel(
|
||||
'Debug lock request was CANCELLED?\n\n'
|
||||
f'{req_ctx}\n'
|
||||
# f'{pformat_cs(req_cs, var_name="req_cs")}\n\n'
|
||||
# f'{pformat_cs(req_ctx._scope, var_name="req_ctx._scope")}\n\n'
|
||||
)
|
||||
raise
|
||||
|
||||
|
@ -1033,11 +1049,11 @@ async def request_root_stdio_lock(
|
|||
):
|
||||
log.exception(
|
||||
'Failed during root TTY-lock dialog?\n'
|
||||
f'{ctx}\n'
|
||||
f'{req_ctx}\n'
|
||||
|
||||
f'Cancelling IPC ctx!\n'
|
||||
)
|
||||
await ctx.cancel()
|
||||
await req_ctx.cancel()
|
||||
raise
|
||||
|
||||
|
||||
|
@ -1047,13 +1063,26 @@ async def request_root_stdio_lock(
|
|||
):
|
||||
log.cancel(
|
||||
'Debug lock request CANCELLED?\n'
|
||||
f'{ctx}\n'
|
||||
f'{req_ctx}\n'
|
||||
)
|
||||
raise
|
||||
|
||||
except BaseException:
|
||||
log.exception('Errored during root TTY-lock dialog?')
|
||||
raise
|
||||
except BaseException as req_err:
|
||||
# log.error('Failed to request root stdio-lock?')
|
||||
DebugStatus.req_err = req_err
|
||||
DebugStatus.release()
|
||||
|
||||
# TODO: how to dev a test that ensures we actually drop
|
||||
# into THIS internal frame on any internal error in the above
|
||||
# code?
|
||||
# -[ ] eg. on failed pld_dec assert above we should be able
|
||||
# to REPL pm it.
|
||||
# -[ ]FURTHER, after we 'continue', we should be able to
|
||||
# ctl-c out of the currently hanging task!
|
||||
raise DebugRequestError(
|
||||
'Failed to lock stdio from subactor IPC ctx!\n\n'
|
||||
f'req_ctx: {req_ctx}\n'
|
||||
) from req_err
|
||||
|
||||
finally:
|
||||
log.debug('Exiting debugger TTY lock request func from child')
|
||||
|
@ -1369,6 +1398,13 @@ def shield_sigint_handler(
|
|||
_pause_msg: str = 'Attaching to pdb REPL in actor'
|
||||
|
||||
|
||||
class DebugRequestError(RuntimeError):
|
||||
'''
|
||||
Failed to request stdio lock from root actor!
|
||||
|
||||
'''
|
||||
|
||||
|
||||
async def _pause(
|
||||
|
||||
debug_func: Callable|None,
|
||||
|
@ -1480,15 +1516,18 @@ async def _pause(
|
|||
raise
|
||||
|
||||
except BaseException:
|
||||
__tracebackhide__: bool = False
|
||||
log.exception(
|
||||
'Failed to invoke internal `debug_func = '
|
||||
f'{debug_func.func.__name__}`\n'
|
||||
)
|
||||
# NOTE: OW this is ONLY called from the
|
||||
# `.set_continue/next` hooks!
|
||||
DebugStatus.release()
|
||||
DebugStatus.release(cancel_req_task=True)
|
||||
|
||||
raise
|
||||
|
||||
repl_err: BaseException|None = None
|
||||
try:
|
||||
if is_root_process():
|
||||
|
||||
|
@ -1584,43 +1623,45 @@ async def _pause(
|
|||
# actor._service_n.cancel_scope.shield = shield
|
||||
# ```
|
||||
# but not entirely sure if that's a sane way to implement it?
|
||||
try:
|
||||
# NOTE spawn the stdio locker request task inside the
|
||||
# current `Context._scope_nursery` to entsure that
|
||||
# the request never can outlive the task's (parent)
|
||||
# lifetime.
|
||||
curr_ctx: Context = current_ipc_ctx()
|
||||
# TODO: see `_errors_relayed_via_ipc()` where we
|
||||
# should dynamically open a `debug_tn` for use here,
|
||||
# BUT it needs to be outside the normal error
|
||||
# catching and `_maybe_enter_debugger()` call!
|
||||
# ctx: Context = await curr_ctx._debug_tn.start(
|
||||
ctx: Context = await actor._service_n.start(
|
||||
request_root_stdio_lock,
|
||||
actor.uid,
|
||||
(task.name, id(task)), # task uuid (effectively)
|
||||
)
|
||||
# our locker task should be the one in ctx
|
||||
# with the root actor
|
||||
assert (
|
||||
ctx
|
||||
is
|
||||
DebugStatus.req_ctx
|
||||
is not
|
||||
curr_ctx
|
||||
)
|
||||
|
||||
# enter REPL
|
||||
_enter_repl_sync(debug_func)
|
||||
# NOTE currently we spawn the lock request task inside this
|
||||
# subactor's global `Actor._service_n` so that the
|
||||
# lifetime of the lock-request can outlive the current
|
||||
# `._pause()` scope while the user steps through their
|
||||
# application code and when they finally exit the
|
||||
# session, via 'continue' or 'quit' cmds, the `PdbREPL`
|
||||
# will manually call `DebugStatus.release()` to release
|
||||
# the lock session with the root actor.
|
||||
#
|
||||
# TODO: ideally we can add a tighter scope for this
|
||||
# request task likely by conditionally opening a "debug
|
||||
# nursery" inside `_errors_relayed_via_ipc()`, see the
|
||||
# todo in tht module, but
|
||||
# -[ ] it needs to be outside the normal crash handling
|
||||
# `_maybe_enter_debugger()` block-call.
|
||||
# -[ ] we probably only need to allocate the nursery when
|
||||
# we detect the runtime is already in debug mode.
|
||||
#
|
||||
# ctx: Context = await curr_ctx._debug_tn.start(
|
||||
req_ctx: Context = await actor._service_n.start(
|
||||
request_root_stdio_lock,
|
||||
actor.uid,
|
||||
(task.name, id(task)), # task uuid (effectively)
|
||||
)
|
||||
# XXX sanity, our locker task should be the one which
|
||||
# entered a new IPC ctx with the root actor, NOT the one
|
||||
# that exists around the task calling into `._pause()`.
|
||||
curr_ctx: Context = current_ipc_ctx()
|
||||
assert (
|
||||
req_ctx
|
||||
is
|
||||
DebugStatus.req_ctx
|
||||
is not
|
||||
curr_ctx
|
||||
)
|
||||
|
||||
except RuntimeError:
|
||||
if actor._cancel_called:
|
||||
# service nursery won't be usable and we
|
||||
# don't want to lock up the root either way since
|
||||
# we're in (the midst of) cancellation.
|
||||
return
|
||||
|
||||
raise
|
||||
# enter REPL
|
||||
_enter_repl_sync(debug_func)
|
||||
|
||||
# TODO: prolly factor this plus the similar block from
|
||||
# `_enter_repl_sync()` into a common @cm?
|
||||
|
@ -1629,13 +1670,31 @@ async def _pause(
|
|||
log.devx(
|
||||
'REPL for pdb was quit!\n'
|
||||
)
|
||||
|
||||
# when the actor is mid-runtime cancellation the
|
||||
# `Actor._service_n` might get closed before we can spawn
|
||||
# the request task, so just ignore expected RTE.
|
||||
elif (
|
||||
isinstance(repl_err, RuntimeError)
|
||||
and
|
||||
actor._cancel_called
|
||||
):
|
||||
# service nursery won't be usable and we
|
||||
# don't want to lock up the root either way since
|
||||
# we're in (the midst of) cancellation.
|
||||
log.warning(
|
||||
'Service nursery likely closed due to actor-runtime cancellation..\n'
|
||||
'Ignoring failed debugger lock request task spawn..\n'
|
||||
)
|
||||
return
|
||||
|
||||
else:
|
||||
log.exception(
|
||||
'Failed to engage debugger via `_pause()` ??\n'
|
||||
)
|
||||
mk_pdb().set_trace()
|
||||
|
||||
DebugStatus.release()
|
||||
DebugStatus.release(cancel_req_task=True)
|
||||
|
||||
# sanity checks for ^ on request/status teardown
|
||||
assert DebugStatus.repl is None
|
||||
assert DebugStatus.repl_task is None
|
||||
|
@ -1645,6 +1704,16 @@ async def _pause(
|
|||
|
||||
raise
|
||||
|
||||
finally:
|
||||
# always show frame when request fails due to internal
|
||||
# failure in the above code (including an `BdbQuit`).
|
||||
if (
|
||||
DebugStatus.req_err
|
||||
or
|
||||
repl_err
|
||||
):
|
||||
__tracebackhide__: bool = False
|
||||
|
||||
|
||||
def _set_trace(
|
||||
repl: PdbREPL, # passed by `_pause()`
|
||||
|
@ -1703,7 +1772,7 @@ async def pause(
|
|||
https://en.wikipedia.org/wiki/Breakpoint
|
||||
|
||||
'''
|
||||
__tracebackhide__: bool = True
|
||||
__tracebackhide__: bool = hide_tb
|
||||
|
||||
# always start 1 level up from THIS in user code since normally
|
||||
# `tractor.pause()` is called explicitly by use-app code thus
|
||||
|
@ -1885,12 +1954,15 @@ def pause_from_sync(
|
|||
# NOTE prefer a new "pause" semantic since it better describes
|
||||
# "pausing the actor's runtime" for this particular
|
||||
# paralell task to do debugging in a REPL.
|
||||
async def breakpoint(**kwargs):
|
||||
async def breakpoint(
|
||||
hide_tb: bool = True,
|
||||
**kwargs,
|
||||
):
|
||||
log.warning(
|
||||
'`tractor.breakpoint()` is deprecated!\n'
|
||||
'Please use `tractor.pause()` instead!\n'
|
||||
)
|
||||
__tracebackhide__: bool = True
|
||||
__tracebackhide__: bool = hide_tb
|
||||
await pause(
|
||||
api_frame=inspect.currentframe(),
|
||||
**kwargs,
|
||||
|
@ -1951,6 +2023,7 @@ def _post_mortem(
|
|||
# frame=None,
|
||||
traceback=tb,
|
||||
)
|
||||
# XXX NOTE XXX: absolutely required to avoid hangs!
|
||||
# Since we presume the post-mortem was enaged to a task-ending
|
||||
# error, we MUST release the local REPL request so that not other
|
||||
# local task nor the root remains blocked!
|
||||
|
|
Loading…
Reference in New Issue