forked from goodboy/tractor
1
0
Fork 0
tractor/tractor/_debug.py

922 lines
29 KiB
Python
Raw Normal View History

Re-license code base for distribution under AGPL This commit obviously denotes a re-license of all applicable parts of the code base. Acknowledgement of this change was completed in #274 by the majority of the current set of contributors. From here henceforth all changes will be AGPL licensed and distributed. This is purely an effort to maintain the same copy-left policy whilst closing the (perceived) SaaS loophole the GPL allows for. It is merely for this loophole: to avoid code hiding by any potential "network providers" who are attempting to use the project to make a profit without either compensating the authors or re-distributing their changes. I thought quite a bit about this change and can't see a reason not to close the SaaS loophole in our current license. We still are (hard) copy-left and I plan to keep the code base this way for a couple reasons: - The code base produces income/profit through parent projects and is demonstrably of high value. - I believe firms should not get free lunch for the sake of "contributions from their employees" or "usage as a service" which I have found to be a dubious argument at best. - If a firm who intends to profit from the code base wants to use it they can propose a secondary commercial license to purchase with the proceeds going to the project's authors under some form of well defined contract. - Many successful projects like Qt use this model; I see no reason it can't work in this case until such a time as the authors feel it should be loosened. There has been detailed discussion in #103 on licensing alternatives. The main point of this AGPL change is to protect the code base for the time being from exploitation while it grows and as we move into the next phase of development which will include extension into the multi-host distributed software space.
2021-12-13 18:08:32 +00:00
# tractor: structured concurrent "actors".
# Copyright 2018-eternity Tyler Goodlet.
# This program is free software: you can redistribute it and/or modify
# it under the terms of the GNU Affero General Public License as published by
# the Free Software Foundation, either version 3 of the License, or
# (at your option) any later version.
# This program is distributed in the hope that it will be useful,
# but WITHOUT ANY WARRANTY; without even the implied warranty of
# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
# GNU Affero General Public License for more details.
# You should have received a copy of the GNU Affero General Public License
# along with this program. If not, see <https://www.gnu.org/licenses/>.
"""
Multi-core debugging for da peeps!
2021-05-12 16:01:43 +00:00
"""
from __future__ import annotations
import bdb
import sys
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
import signal
from functools import partial
2021-10-14 03:32:02 +00:00
from contextlib import asynccontextmanager as acm
from typing import (
Tuple,
Optional,
Callable,
AsyncIterator,
AsyncGenerator,
)
2022-02-09 15:04:37 +00:00
from types import FrameType
import tractor
import trio
2021-10-14 15:38:44 +00:00
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 ._ipc import Channel
try:
# wtf: only exported when installed in dev mode?
import pdbpp
except ImportError:
2020-10-13 15:03:55 +00:00
# pdbpp is installed in regular mode...it monkey patches stuff
import pdb
2022-02-16 18:07:21 +00:00
xpm = getattr(pdb, 'xpm', None)
assert xpm, "pdbpp is not installed?" # type: ignore
pdbpp = pdb
log = get_logger(__name__)
__all__ = ['breakpoint', 'post_mortem']
2021-05-12 16:01:43 +00:00
# TODO: wrap all these in a static global class: ``DebugLock`` maybe?
# placeholder for function to set a ``trio.Event`` on debugger exit
2020-07-30 14:42:22 +00:00
_pdb_release_hook: Optional[Callable] = None
# actor-wide variable pointing to current task name using debugger
2021-05-12 16:01:43 +00:00
_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 = 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
# otherwise deadlocks with the parent actor may ensure
2020-10-13 15:03:55 +00:00
_debugger_request_cs: Optional[trio.CancelScope] = None
2020-07-30 14:42:22 +00:00
class TractorConfig(pdbpp.DefaultConfig):
"""Custom ``pdbpp`` goodness.
"""
2020-08-07 03:47:43 +00:00
# sticky_by_default = True
enable_hidden_frames = False
2020-07-30 14:42:22 +00:00
class MultiActorPdb(pdbpp.Pdb):
'''
Add teardown hooks to the regular ``pdbpp.Pdb``.
'''
2020-07-30 14:42:22 +00:00
# override the pdbpp config with our coolio one
DefaultConfig = TractorConfig
2021-08-01 14:43:21 +00:00
# TODO: figure out how to disallow recursive .set_trace() entry
# since that'll cause deadlock for us.
def set_continue(self):
try:
super().set_continue()
finally:
global _local_task_in_debug, _pdb_release_hook
2021-05-12 16:01:43 +00:00
_local_task_in_debug = None
if _pdb_release_hook:
_pdb_release_hook()
def set_quit(self):
try:
super().set_quit()
finally:
global _local_task_in_debug, _pdb_release_hook
2021-05-12 16:01:43 +00:00
_local_task_in_debug = None
if _pdb_release_hook:
_pdb_release_hook()
# TODO: will be needed whenever we get to true remote debugging.
# XXX see https://github.com/goodboy/tractor/issues/130
2020-07-30 14:42:22 +00:00
# # TODO: is there some way to determine this programatically?
# _pdb_exit_patterns = tuple(
# str.encode(patt + "\n") for patt in (
# 'c', 'cont', 'continue', 'q', 'quit')
# )
# def subactoruid2proc(
# actor: 'Actor', # noqa
# uid: Tuple[str, str]
# ) -> trio.Process:
# n = actor._actoruid2nursery[uid]
# _, proc, _ = n._children[uid]
# return proc
# async def hijack_stdin():
# log.info(f"Hijacking stdin from {actor.uid}")
# trap std in and relay to subproc
# async_stdin = trio.wrap_file(sys.stdin)
# async with aclosing(async_stdin):
# async for msg in async_stdin:
2021-06-30 17:47:07 +00:00
# log.runtime(f"Stdin input:\n{msg}")
# # encode to bytes
# bmsg = str.encode(msg)
# # relay bytes to subproc over pipe
# # await proc.stdin.send_all(bmsg)
# if bmsg in _pdb_exit_patterns:
# log.info("Closing stdin hijack")
# break
2021-10-14 03:32:02 +00:00
@acm
2021-07-31 16:50:58 +00:00
async def _acquire_debug_lock(
uid: Tuple[str, str]
2021-08-01 14:43:21 +00:00
2021-07-31 16:50:58 +00:00
) -> AsyncIterator[trio.StrictFIFOLock]:
'''
Acquire a root-actor local FIFO lock which tracks mutex access of
2021-08-01 14:43:21 +00:00
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
2021-01-09 01:45:19 +00:00
task_name = trio.lowlevel.current_task().name
2020-12-26 20:11:18 +00:00
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
log.runtime(
f"Attempting to acquire TTY lock, remote task: {task_name}:{uid}"
)
we_acquired = False
try:
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
log.runtime(
f"entering lock checkpoint, remote task: {task_name}:{uid}"
)
we_acquired = True
await _debug_lock.acquire()
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()
2021-05-12 16:01:43 +00:00
_global_actor_in_debug = uid
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
log.runtime(f"TTY lock acquired, remote task: {task_name}:{uid}")
2021-08-01 14:43:21 +00:00
# NOTE: critical section: this yield is unshielded!
2021-08-01 14:43:21 +00:00
# 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:
# if _global_actor_in_debug == uid:
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
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
):
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
log.runtime(f"No more tasks waiting on tty lock! says {uid}")
if _no_remote_has_tty is not None:
_no_remote_has_tty.set()
_no_remote_has_tty = None
2020-12-26 20:11:18 +00:00
_global_actor_in_debug = None
log.runtime(
f"TTY lock released, remote task: {task_name}:{uid}"
)
@tractor.context
2021-08-01 14:43:21 +00:00
async def _hijack_stdin_for_child(
2021-05-12 16:01:43 +00:00
ctx: tractor.Context,
subactor_uid: Tuple[str, str]
) -> str:
2021-10-08 22:13:55 +00:00
'''
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.
NOTE: this task is invoked in the root actor-process of the actor
tree. It is meant to be invoked as an rpc-task which should be
highly reliable at cleaning out the tty-lock state when complete!
'''
task_name = trio.lowlevel.current_task().name
# TODO: when we get to true remote debugging
# this will deliver stdin data?
log.debug(
"Attempting to acquire TTY lock\n"
f"remote task: {task_name}:{subactor_uid}"
)
2021-05-12 16:01:43 +00:00
log.debug(f"Actor {subactor_uid} is WAITING on stdin hijack lock")
orig_handler = signal.signal(
signal.SIGINT,
shield_sigint,
)
try:
with (
trio.CancelScope(shield=True),
):
# try:
# lock = None
async with _acquire_debug_lock(subactor_uid): # as lock:
# indicate to child that we've locked stdio
await ctx.started('Locked')
log.debug(
f"Actor {subactor_uid} acquired stdin hijack lock"
)
# wait for unlock pdb by child
async with ctx.open_stream() as stream:
assert await stream.receive() == 'pdb_unlock'
# except (
# BaseException,
# # trio.MultiError,
# # Exception,
# # trio.BrokenResourceError,
# # trio.Cancelled, # by local cancellation
# # trio.ClosedResourceError, # by self._rx_chan
# # ContextCancelled,
# # ConnectionResetError,
# ):
# # 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?
# if lock and lock.locked():
# try:
# lock.release()
# except RuntimeError:
# log.exception(f"we don't own the tty lock?")
# # if isinstance(err, trio.Cancelled):
# raise
# finally:
# log.runtime(
# "TTY lock released, remote task:"
# f"{task_name}:{subactor_uid}"
# )
return "pdb_unlock_complete"
finally:
signal.signal(
signal.SIGINT,
orig_handler
)
async def wait_for_parent_stdin_hijack(
actor_uid: Tuple[str, str],
task_status: TaskStatus[trio.CancelScope] = trio.TASK_STATUS_IGNORED
):
'''
2021-10-14 15:38:44 +00:00
Connect to the root actor via a ctx and invoke a task which locks
a root-local TTY lock.
This function is used by any sub-actor to acquire mutex access to
pdb and the root's TTY for interactive debugging (see below inside
``_breakpoint()``). It can be used to ensure that an intermediate
nursery-owning actor does not clobber its children if they are in
debug (see below inside ``maybe_wait_for_debugger()``).
'''
global _debugger_request_cs
with trio.CancelScope(shield=True) as cs:
_debugger_request_cs = cs
try:
async with get_root() as portal:
# this syncs to child's ``Context.started()`` call.
async with portal.open_context(
tractor._debug._hijack_stdin_for_child,
subactor_uid=actor_uid,
) as (ctx, val):
log.pdb('locked context')
assert val == 'Locked'
async with ctx.open_stream() as stream:
# unblock local caller
try:
assert _local_pdb_complete
task_status.started(cs)
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"
log.pdb('unlocked context')
except tractor.ContextCancelled:
log.warning('Root actor cancelled debug lock')
finally:
log.debug(f"Exiting debugger for actor {actor_uid}")
global _local_task_in_debug
_local_task_in_debug = None
log.debug(f"Child {actor_uid} released parent stdio lock")
2022-02-09 15:04:37 +00:00
def mk_mpdb() -> tuple[MultiActorPdb, Callable]:
2022-02-09 12:51:34 +00:00
pdb = MultiActorPdb()
# signal.signal = pdbpp.hideframe(signal.signal)
2022-02-09 12:51:34 +00:00
orig_handler = signal.signal(
signal.SIGINT,
partial(shield_sigint, pdb_obj=pdb),
)
# XXX: These are the important flags mentioned in
# https://github.com/python-trio/trio/issues/1155
# which resolve the traceback spews to console.
2022-02-09 12:51:34 +00:00
pdb.allow_kbdint = True
pdb.nosigint = True
# TODO: add this as method on our pdb obj?
def undo_sigint():
# restore original sigint handler
signal.signal(
signal.SIGINT,
orig_handler
)
return pdb, undo_sigint
async def _breakpoint(
debug_func,
# TODO:
# shield: bool = False
) -> None:
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
'''
breakpoint entry for engaging pdb machinery in the root or
a subactor.
'''
__tracebackhide__ = True
2022-02-09 12:51:34 +00:00
pdb, undo_sigint = mk_mpdb()
actor = tractor.current_actor()
task_name = trio.lowlevel.current_task().name
global _local_pdb_complete, _pdb_release_hook
2021-05-12 16:01:43 +00:00
global _local_task_in_debug, _global_actor_in_debug
# 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):
# await trio.lowlevel.checkpoint()
if not _local_pdb_complete or _local_pdb_complete.is_set():
_local_pdb_complete = trio.Event()
# TODO: need a more robust check for the "root" actor
if actor._parent_chan and not is_root_process():
2021-05-12 16:01:43 +00:00
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()`
2021-05-12 16:01:43 +00:00
log.warning(f"{actor.uid} already has a debug lock, waiting...")
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
2021-05-12 16:01:43 +00:00
_local_task_in_debug = task_name
def child_release_hook():
try:
# sometimes the ``trio`` might already be termianated in
# which case this call will raise.
_local_pdb_complete.set()
finally:
# restore original sigint handler
undo_sigint()
# should always be cleared in the hijack hook aboved right?
# _local_task_in_debug = None
# assign unlock callback for debugger teardown hooks
# _pdb_release_hook = _local_pdb_complete.set
_pdb_release_hook = child_release_hook
# 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
try:
with trio.CancelScope(shield=True):
await actor._service_n.start(
wait_for_parent_stdin_hijack,
actor.uid,
)
except RuntimeError:
child_release_hook()
raise
elif is_root_process():
# we also wait in the root-parent for any child that
# may have the tty locked prior
global _debug_lock
2021-08-01 14:43:21 +00:00
# TODO: wait, what about multiple root tasks acquiring it though?
# root process (us) already has it; ignore
2021-05-12 16:01:43 +00:00
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 shield-acquire active tty lock'
f' owned by {_global_actor_in_debug}')
2021-10-08 22:13:55 +00:00
# must shield here to avoid hitting a ``Cancelled`` and
# a child getting stuck bc we clobbered the tty
with trio.CancelScope(shield=True):
await _debug_lock.acquire()
else:
# may be cancelled
await _debug_lock.acquire()
2021-05-12 16:01:43 +00:00
_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
2021-05-12 16:01:43 +00:00
global _global_actor_in_debug, _local_task_in_debug
try:
_debug_lock.release()
except RuntimeError:
# uhhh makes no sense but been seeing the non-owner
# release error even though this is definitely the task
# that locked?
owner = _debug_lock.statistics().owner
if owner:
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
raise
2021-05-12 16:01:43 +00:00
_global_actor_in_debug = None
_local_task_in_debug = None
try:
# sometimes the ``trio`` might already be termianated in
# which case this call will raise.
_local_pdb_complete.set()
finally:
# restore original sigint handler
undo_sigint()
_pdb_release_hook = teardown
2022-02-09 15:04:37 +00:00
# frame = sys._getframe()
# last_f = frame.f_back
# last_f.f_globals['__tracebackhide__'] = True
try:
# 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, pdb)
except bdb.BdbQuit:
2022-01-23 22:33:09 +00:00
if _pdb_release_hook:
_pdb_release_hook()
raise
# XXX: apparently we can't do this without showing this frame
# in the backtrace on first entry to the REPL? Seems like an odd
# behaviour that should have been fixed by now. This is also why
# we scrapped all the @cm approaches that were tried previously.
# finally:
# __tracebackhide__ = True
# # frame = sys._getframe()
# # last_f = frame.f_back
# # last_f.f_globals['__tracebackhide__'] = True
# # signal.signal = pdbpp.hideframe(signal.signal)
# signal.signal(
# signal.SIGINT,
# orig_handler
# )
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
def shield_sigint(
signum: int,
frame: 'frame', # type: ignore # noqa
pdb_obj: Optional[MultiActorPdb] = None,
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
*args,
) -> None:
'''
Specialized debugger compatible SIGINT handler.
In childred we always ignore to avoid deadlocks since cancellation
should always be managed by the parent supervising actor. The root
is always cancelled on ctrl-c.
'''
__tracebackhide__ = True
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
global _local_task_in_debug, _global_actor_in_debug
uid_in_debug = _global_actor_in_debug
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
actor = tractor.current_actor()
def do_cancel():
# If we haven't tried to cancel the runtime then do that instead
# of raising a KBI (which may non-gracefully destroy
# a ``trio.run()``).
if not actor._cancel_called:
actor.cancel_soon()
# If the runtime is already cancelled it likely means the user
# hit ctrl-c again because teardown didn't full take place in
# which case we do the "hard" raising of a local KBI.
else:
raise KeyboardInterrupt
any_connected = False
if uid_in_debug is not None:
# try to see if the supposed (sub)actor in debug still
# has an active connection to *this* actor, and if not
# it's likely they aren't using the TTY lock / debugger
# and we should propagate SIGINT normally.
chans = actor._peers.get(tuple(uid_in_debug))
if chans:
any_connected = any(chan.connected() for chan in chans)
if not any_connected:
log.warning(
'A global actor reported to be in debug '
'but no connection exists for this child:\n'
f'{uid_in_debug}\n'
'Allowing SIGINT propagation..'
)
return do_cancel()
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
# root actor branch that reports whether or not a child
# has locked debugger.
if (
is_root_process()
and uid_in_debug is not None
# XXX: only if there is an existing connection to the
# (sub-)actor in debug do we ignore SIGINT in this
# parent! Otherwise we may hang waiting for an actor
# which has already terminated to unlock.
and any_connected
):
name = uid_in_debug[0]
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
if name != 'root':
log.pdb(
f"Ignoring SIGINT while child in debug mode: `{uid_in_debug}`"
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
)
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
else:
log.pdb(
"Ignoring SIGINT while in debug mode"
)
# child actor that has locked the debugger
elif (
not is_root_process()
):
chan: Channel = actor._parent_chan
if not chan or not chan.connected():
log.warning(
'A global actor reported to be in debug '
'but no connection exists for its parent:\n'
f'{uid_in_debug}\n'
'Allowing SIGINT propagation..'
)
return do_cancel()
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
task = _local_task_in_debug
if task:
log.pdb(
f"Ignoring SIGINT while task in debug mode: `{task}`"
)
# TODO: how to handle the case of an intermediary-child actor
# that **is not** marked in debug mode?
# elif debug_mode():
else:
log.pdb(
"Ignoring SIGINT since debug mode is enabled"
)
# NOTE: currently (at least on ``fancycompleter`` 0.9.2)
# it lookks to be that the last command that was run (eg. ll)
# will be repeated by default.
# TODO: maybe redraw/print last REPL output to console
# if pdb_obj:
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
# TODO: make this work like sticky mode where if there is output
# detected as written to the tty we redraw this part underneath
# and erase the past draw of this same bit above?
# pdb_obj.sticky = True
# pdb_obj._print_if_sticky()
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
# also see these links for an approach from ``ptk``:
# https://github.com/goodboy/tractor/issues/130#issuecomment-663752040
# https://github.com/prompt-toolkit/python-prompt-toolkit/blob/c2c6af8a0308f9e5d7c0e28cb8a02963fe0ce07a/prompt_toolkit/patch_stdout.py
# XXX: lol, see ``pdbpp`` issue:
# https://github.com/pdbpp/pdbpp/issues/496
# TODO: pretty sure this is what we should expect to have to run
# in total but for now we're just going to wait until `pdbpp`
# figures out it's own stuff on 3.10 (and maybe we'll help).
# pdb_obj.do_longlist(None)
# XXX: we were doing this but it shouldn't be required..
# print(pdb_obj.prompt, end='', flush=True)
Refine the handler for child vs. root cases This gets very close to avoiding any possible hangs to do with tty locking and SIGINT handling minus a special case that will be detailed below. Summary of implementation changes: - convert `_mk_pdb()` -> `with _open_pdb() as pdb:` which implicitly handles the `bdb.BdbQuit` case such that debugger teardown hooks are always called. - rename the handler to `shield_sigint()` and handle a variety of new cases: * the root is in debug but hasn't been cancelled -> call `Actor.cancel_soon()` * the root is in debug but *has* been called (`Actor.cancel_soon()` already called) -> raise KBI * a child is in debug *and* has a task locking the debugger -> ignore SIGINT in child *and* the root actor. - if the debugger instance is provided to the handler at acquire time, on SIGINT handling completion re-print the last pdb++ REPL output so that the user realizes they are still actively in debug. - ignore the unlock case where a race condition of "no task" holding the lock causes the `RuntimeError` normally associated with the "wrong task" doing so (not sure if this is a `trio` bug?). - change debug logs to runtime level. Unhandled case(s): - a child is maybe in debug mode but does not itself have any task using the debugger. * ToDo: we need a way to decide what to do with "intermediate" child actors who themselves either are not in `debug_mode=True` but have children who *are* such that a SIGINT won't cause cancellation of that child-as-parent-of-another-child **iff** any of their children are in in debug mode.
2022-01-23 22:04:49 +00:00
2022-01-23 22:33:09 +00:00
def _set_trace(
2022-02-09 15:04:37 +00:00
actor: Optional[tractor._actor.Actor] = None,
pdb: Optional[MultiActorPdb] = None,
):
__tracebackhide__ = True
actor = actor or tractor.current_actor()
2022-02-09 15:04:37 +00:00
# XXX: on latest ``pdbpp`` i guess we don't need this?
# frame = sys._getframe()
# last_f = frame.f_back
# last_f.f_globals['__tracebackhide__'] = True
2022-02-09 15:04:37 +00:00
# start 2 levels up in user code
2022-07-11 13:42:26 +00:00
frame: Optional[FrameType] = sys._getframe()
2022-02-09 15:04:37 +00:00
if frame:
frame = frame.f_back # type: ignore
2022-02-09 15:04:37 +00:00
2022-07-11 13:42:26 +00:00
if frame and pdb and actor is not None:
log.pdb(f"\nAttaching pdb to actor: {actor.uid}\n")
2022-07-11 13:42:26 +00:00
# no f!#$&* idea, but when we're in async land
# we need 2x frames up?
frame = frame.f_back
else:
2022-02-09 12:51:34 +00:00
pdb, undo_sigint = mk_mpdb()
# we entered the global ``breakpoint()`` built-in from sync code?
global _local_task_in_debug, _pdb_release_hook
_local_task_in_debug = 'sync'
2022-02-09 15:04:37 +00:00
pdb.set_trace(frame=frame)
breakpoint = partial(
_breakpoint,
_set_trace,
)
def _post_mortem(
2022-02-09 15:04:37 +00:00
actor: tractor._actor.Actor,
pdb: MultiActorPdb,
) -> None:
'''
Enter the ``pdbpp`` port mortem entrypoint using our custom
debugger instance.
'''
log.pdb(f"\nAttaching to pdb in crashed actor: {actor.uid}\n")
# XXX: on py3.10 if you don't have latest ``pdbpp`` installed.
# The exception looks something like:
# Traceback (most recent call last):
# File ".../tractor/_debug.py", line 729, in _post_mortem
# for _ in range(100):
# File "../site-packages/pdb.py", line 1227, in xpm
# post_mortem(info[2], Pdb)
# File "../site-packages/pdb.py", line 1175, in post_mortem
# p.interaction(None, t)
# File "../site-packages/pdb.py", line 216, in interaction
# ret = self.setup(frame, traceback)
# File "../site-packages/pdb.py", line 259, in setup
# ret = super(Pdb, self).setup(frame, tb)
# File "/usr/lib/python3.10/pdb.py", line 217, in setup
# self.curframe = self.stack[self.curindex][0]
# IndexError: list index out of range
# NOTE: you need ``pdbpp`` master (at least this commit
# https://github.com/pdbpp/pdbpp/commit/b757794857f98d53e3ebbe70879663d7d843a6c2)
# to fix this and avoid the hang it causes XD.
# see also: https://github.com/pdbpp/pdbpp/issues/480
pdbpp.xpm(Pdb=lambda: pdb)
post_mortem = partial(
_breakpoint,
_post_mortem,
)
async def _maybe_enter_pm(err):
if (
debug_mode()
# NOTE: don't enter debug mode recursively after quitting pdb
# Iow, don't re-enter the repl if the `quit` command was issued
# by the user.
and not isinstance(err, bdb.BdbQuit)
# XXX: if the error is the likely result of runtime-wide
# cancellation, we don't want to enter the debugger since
# there's races between when the parent actor has killed all
# comms and when the child tries to contact said parent to
# acquire the tty lock.
# Really we just want to mostly avoid catching KBIs here so there
# might be a simpler check we can do?
and not is_multi_cancelled(err)
):
2020-12-26 20:11:18 +00:00
log.debug("Actor crashed, entering debug mode")
await post_mortem()
2020-12-26 20:11:18 +00:00
return True
else:
return False
2021-10-08 22:13:55 +00:00
2021-10-14 03:32:02 +00:00
@acm
async def acquire_debug_lock(
2021-10-14 17:46:27 +00:00
subactor_uid: Tuple[str, str],
) -> AsyncGenerator[None, tuple]:
2021-10-14 03:32:02 +00:00
'''
Grab root's debug lock on entry, release on exit.
2021-11-04 15:47:28 +00:00
This helper is for actor's who don't actually need
to acquired the debugger but want to wait until the
lock is free in the tree root.
2021-10-14 03:32:02 +00:00
'''
if not debug_mode():
2021-11-04 15:47:28 +00:00
yield None
return
2021-10-14 03:32:02 +00:00
async with trio.open_nursery() as n:
cs = await n.start(
wait_for_parent_stdin_hijack,
subactor_uid,
2021-10-14 03:32:02 +00:00
)
yield None
2021-10-14 03:32:02 +00:00
cs.cancel()
async def maybe_wait_for_debugger(
poll_steps: int = 2,
poll_delay: float = 0.1,
child_in_debug: bool = False,
2021-10-14 03:32:02 +00:00
) -> None:
2021-10-08 22:13:55 +00:00
if not debug_mode() and not child_in_debug:
2021-10-14 03:32:02 +00:00
return
2021-10-08 22:13:55 +00:00
if (
is_root_process()
):
2021-10-14 03:32:02 +00:00
global _no_remote_has_tty, _global_actor_in_debug, _wait_all_tasks_lock
2021-10-08 22:13:55 +00:00
2021-10-14 03:32:02 +00:00
# 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.
2021-10-08 22:13:55 +00:00
sub_in_debug = None
2021-10-14 03:32:02 +00:00
for _ in range(poll_steps):
2021-10-08 22:13:55 +00:00
if _global_actor_in_debug:
sub_in_debug = tuple(_global_actor_in_debug)
log.debug(
'Root polling for debug')
with trio.CancelScope(shield=True):
2021-10-14 03:32:02 +00:00
await trio.sleep(poll_delay)
2021-10-08 22:13:55 +00:00
# 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?
# XXX: doesn't seem to work
# await trio.testing.wait_all_tasks_blocked(cushion=0)
2021-10-08 22:13:55 +00:00
debug_complete = _no_remote_has_tty
if (
(debug_complete and
not debug_complete.is_set())
):
log.debug(
2021-10-08 22:13:55 +00:00
'Root has errored but pdb is in use by '
f'child {sub_in_debug}\n'
'Waiting on tty lock to release..')
await debug_complete.wait()
2021-10-14 03:32:02 +00:00
await trio.sleep(poll_delay)
2021-10-08 22:13:55 +00:00
continue
else:
log.debug(
'Root acquired TTY LOCK'
2021-10-08 22:13:55 +00:00
)