Compare commits

...

6 Commits

Author SHA1 Message Date
Tyler Goodlet db58f6e1b5 Woops, fix `_post_mortem()` type sig..
We're passing a `extra_frames_up_when_async=2` now (from prior attempt
to hide `CancelScope.__exit__()` when `shield=True`) and thus both
`debug_func`s must accept it 🤦

On the brighter side found out that the `TypeError` from the call-sig
mismatch was actually being swallowed entirely so add some
`.exception()` msgs for such cases to at least alert the dev they broke
stuff XD
2025-03-16 23:24:52 -04:00
Tyler Goodlet 76b7006977 Add `shield: bool` support to `.pause()`
It's been on the todo for a while and I've given up trying to properly
hide the `trio.CancelScope.__exit__()` frame for now instead opting to
just `log.pdb()` a big apology XD

Users can obvi still just not use the flag and wrap `tractor.pause()` in
their own cs block if they want to avoid having to hit `'up'` in the pdb
REPL if needed in a cancelled task-scope.

Impl deatz:
- factor orig `.pause()` impl into new `._pause()` so that we can more tersely
  wrap the original content depending on `shield: bool` input; only open
  the cancel-scope when shield is set to avoid aforemented extra strack
  frame annoyance.
- pass through `shield` to underlying `_pause` and `debug_func()` so we
  can actually know when so log our apology.
- add a buncha notes to new `.pause()` wrapper regarding the inability
  to hide the cancel-scope `.__exit__()`, inluding that overriding the
  code in `trio._core._run.CancelScope` doesn't seem to solve the issue
  either..

Unrelated `maybe_wait_for_debugger()` tweaks:
- don't read `Lock.global_actor_in_debug` more then needed, rename local
  read var to `in_debug` (since it can also hold the root actor uid, not
  just sub-actors).
- shield the `await debug_complete.wait()` since ideally we avoid the
  root cancellation child-actors in debug even when the root calls this
  func in a cancelled scope.
2025-03-16 23:24:52 -04:00
Tyler Goodlet bd1885bce1 Mk debugger tests work for arbitrary pre-REPL format
Since this was changed as part of overall project wide logging format
updates, and i ended up changing the both the crash and pause `.pdb()`
msgs to include some multi-line-ascii-"stuff", might as well make the
pre-prompt checks in the test suite more flexible to match.

As such, this exposes 2 new constants inside the `.devx._debug` mod:
- `._pause_msg: str` for the pre `tractor.pause()` header emitted via
  `log.pdb()` and,
- `._crash_msg: str` for the pre `._post_mortem()` equiv when handling
  errors in debug mode.

Adjust the test suite to use these values and thus make us more capable
to absorb changes in the future as well:
- add a new `in_prompt_msg()` predicate, very similar to `assert_before()`
  but minus `assert`s which takes in a `parts: list[str]` to match
  in the pre-prompt stdout.
- delegate to `in_prompt_msg()` in `assert_before()` since it was mostly
  duplicate minus `assert`.
- adjust all previous `<patt> in before` asserts to instead use
  `in_prompt_msg()` with separated pre-prompt-header vs. actor-name
  `parts`.
- use new `._pause/crash_msg` values in all such calls including any
  `assert_before()` cases.
2025-03-16 23:24:51 -04:00
Tyler Goodlet 066a35322e Support `maybe_wait_for_debugger(header_msg: str)`
Allow callers to stick in a header to the `.pdb()` level emitted msg(s)
such that any "waiting status" content is only shown if the caller
actually get's blocked waiting for the debug lock; use it inside the
`._spawn` sub-process reaper call.

Also, return early if `Lock.global_actor_in_debug == None` and thus
only enter the poll loop when actually needed, consequently raise
if we fall through the loop without acquisition.
2025-03-16 23:22:40 -04:00
Tyler Goodlet 2ebc30d708 Fix `.devx.maybe_wait_for_debugger()` polling deats
When entered by the root actor avoid excessive polling cycles by,
- blocking on the `Lock.no_remote_has_tty: trio.Event` and breaking
  *immediately* when set (though we should really also lock
  it from the root right?) to avoid extra loops..
- shielding the `await trio.sleep(poll_delay)` call to avoid any local
  cancellation causing the (presumably root-actor task) caller to move
  on (possibly to cancel its children) and instead to continue
  poll-blocking until the lock is actually released by its user.
- `break` the poll loop immediately if no remote locker is detected.
- use `.pdb()` level for reporting lock state changes.

Also add a #TODO to handle calls by non-root actors as it pertains to
2025-03-16 23:22:40 -04:00
Tyler Goodlet 57a5b7eb6f Add `stackscope` tree pprinter triggered by SIGUSR1
Can be optionally enabled via a new `enable_stack_on_sig()` which will
swap in the SIGUSR1 handler. Much thanks to @oremanj for writing this
amazing project, it's thus far helped me fix some very subtle hangs
inside our new IPC-context cancellation machinery that would have
otherwise taken much more manual pdb-ing and hair pulling XD

Full credit for `dump_task_tree()` goes to the original project author
with some minor tweaks as was handed to me via the trio-general matrix
room B)

Slight changes from orig version:
- use a `log.pdb()` emission to pprint to console
- toss in an ex sh CLI cmd to trigger the dump from another terminal
  using `kill` + `pgrep`.
2025-03-16 23:22:40 -04:00
6 changed files with 551 additions and 210 deletions

View File

@ -10,6 +10,7 @@ TODO:
- wonder if any of it'll work on OS X? - wonder if any of it'll work on OS X?
""" """
from functools import partial
import itertools import itertools
from typing import Optional from typing import Optional
import platform import platform
@ -26,6 +27,10 @@ from pexpect.exceptions import (
from tractor._testing import ( from tractor._testing import (
examples_dir, examples_dir,
) )
from tractor.devx._debug import (
_pause_msg,
_crash_msg,
)
from conftest import ( from conftest import (
_ci_env, _ci_env,
) )
@ -123,20 +128,52 @@ def expect(
raise raise
def in_prompt_msg(
prompt: str,
parts: list[str],
pause_on_false: bool = False,
print_prompt_on_false: bool = True,
) -> bool:
'''
Predicate check if (the prompt's) std-streams output has all
`str`-parts in it.
Can be used in test asserts for bulk matching expected
log/REPL output for a given `pdb` interact point.
'''
for part in parts:
if part not in prompt:
if pause_on_false:
import pdbp
pdbp.set_trace()
if print_prompt_on_false:
print(prompt)
return False
return True
def assert_before( def assert_before(
child, child,
patts: list[str], patts: list[str],
**kwargs,
) -> None: ) -> None:
before = str(child.before.decode()) # as in before the prompt end
before: str = str(child.before.decode())
assert in_prompt_msg(
prompt=before,
parts=patts,
for patt in patts: **kwargs
try: )
assert patt in before
except AssertionError:
print(before)
raise
@pytest.fixture( @pytest.fixture(
@ -195,7 +232,10 @@ def test_root_actor_error(spawn, user_in_out):
before = str(child.before.decode()) before = str(child.before.decode())
# make sure expected logging and error arrives # make sure expected logging and error arrives
assert "Attaching to pdb in crashed actor: ('root'" in before assert in_prompt_msg(
before,
[_crash_msg, "('root'"]
)
assert 'AssertionError' in before assert 'AssertionError' in before
# send user command # send user command
@ -332,7 +372,10 @@ def test_subactor_error(
child.expect(PROMPT) child.expect(PROMPT)
before = str(child.before.decode()) before = str(child.before.decode())
assert "Attaching to pdb in crashed actor: ('name_error'" in before assert in_prompt_msg(
before,
[_crash_msg, "('name_error'"]
)
if do_next: if do_next:
child.sendline('n') child.sendline('n')
@ -353,9 +396,15 @@ def test_subactor_error(
before = str(child.before.decode()) before = str(child.before.decode())
# root actor gets debugger engaged # root actor gets debugger engaged
assert "Attaching to pdb in crashed actor: ('root'" in before assert in_prompt_msg(
before,
[_crash_msg, "('root'"]
)
# error is a remote error propagated from the subactor # error is a remote error propagated from the subactor
assert "RemoteActorError: ('name_error'" in before assert in_prompt_msg(
before,
[_crash_msg, "('name_error'"]
)
# another round # another round
if ctlc: if ctlc:
@ -380,7 +429,10 @@ def test_subactor_breakpoint(
child.expect(PROMPT) child.expect(PROMPT)
before = str(child.before.decode()) before = str(child.before.decode())
assert "Attaching pdb to actor: ('breakpoint_forever'" in before assert in_prompt_msg(
before,
[_pause_msg, "('breakpoint_forever'"]
)
# do some "next" commands to demonstrate recurrent breakpoint # do some "next" commands to demonstrate recurrent breakpoint
# entries # entries
@ -396,7 +448,10 @@ def test_subactor_breakpoint(
child.sendline('continue') child.sendline('continue')
child.expect(PROMPT) child.expect(PROMPT)
before = str(child.before.decode()) before = str(child.before.decode())
assert "Attaching pdb to actor: ('breakpoint_forever'" in before assert in_prompt_msg(
before,
[_pause_msg, "('breakpoint_forever'"]
)
if ctlc: if ctlc:
do_ctlc(child) do_ctlc(child)
@ -441,7 +496,10 @@ def test_multi_subactors(
child.expect(PROMPT) child.expect(PROMPT)
before = str(child.before.decode()) before = str(child.before.decode())
assert "Attaching pdb to actor: ('breakpoint_forever'" in before assert in_prompt_msg(
before,
[_pause_msg, "('breakpoint_forever'"]
)
if ctlc: if ctlc:
do_ctlc(child) do_ctlc(child)
@ -461,7 +519,10 @@ def test_multi_subactors(
# first name_error failure # first name_error failure
child.expect(PROMPT) child.expect(PROMPT)
before = str(child.before.decode()) before = str(child.before.decode())
assert "Attaching to pdb in crashed actor: ('name_error'" in before assert in_prompt_msg(
before,
[_crash_msg, "('name_error'"]
)
assert "NameError" in before assert "NameError" in before
if ctlc: if ctlc:
@ -487,7 +548,10 @@ def test_multi_subactors(
child.sendline('c') child.sendline('c')
child.expect(PROMPT) child.expect(PROMPT)
before = str(child.before.decode()) before = str(child.before.decode())
assert "Attaching pdb to actor: ('breakpoint_forever'" in before assert in_prompt_msg(
before,
[_pause_msg, "('breakpoint_forever'"]
)
if ctlc: if ctlc:
do_ctlc(child) do_ctlc(child)
@ -527,17 +591,21 @@ def test_multi_subactors(
child.expect(PROMPT) child.expect(PROMPT)
before = str(child.before.decode()) before = str(child.before.decode())
assert_before(child, [ assert_before(
# debugger attaches to root child, [
"Attaching to pdb in crashed actor: ('root'", # debugger attaches to root
# "Attaching to pdb in crashed actor: ('root'",
_crash_msg,
"('root'",
# expect a multierror with exceptions for each sub-actor # expect a multierror with exceptions for each sub-actor
"RemoteActorError: ('breakpoint_forever'", "RemoteActorError: ('breakpoint_forever'",
"RemoteActorError: ('name_error'", "RemoteActorError: ('name_error'",
"RemoteActorError: ('spawn_error'", "RemoteActorError: ('spawn_error'",
"RemoteActorError: ('name_error_1'", "RemoteActorError: ('name_error_1'",
'bdb.BdbQuit', 'bdb.BdbQuit',
]) ]
)
if ctlc: if ctlc:
do_ctlc(child) do_ctlc(child)
@ -574,15 +642,22 @@ def test_multi_daemon_subactors(
# the root's tty lock first so anticipate either crash # the root's tty lock first so anticipate either crash
# message on the first entry. # message on the first entry.
bp_forever_msg = "Attaching pdb to actor: ('bp_forever'" bp_forev_parts = [_pause_msg, "('bp_forever'"]
bp_forev_in_msg = partial(
in_prompt_msg,
parts=bp_forev_parts,
)
name_error_msg = "NameError: name 'doggypants' is not defined" name_error_msg = "NameError: name 'doggypants' is not defined"
name_error_parts = [name_error_msg]
before = str(child.before.decode()) before = str(child.before.decode())
if bp_forever_msg in before:
next_msg = name_error_msg if bp_forev_in_msg(prompt=before):
next_parts = name_error_parts
elif name_error_msg in before: elif name_error_msg in before:
next_msg = bp_forever_msg next_parts = bp_forev_parts
else: else:
raise ValueError("Neither log msg was found !?") raise ValueError("Neither log msg was found !?")
@ -599,7 +674,10 @@ def test_multi_daemon_subactors(
child.sendline('c') child.sendline('c')
child.expect(PROMPT) child.expect(PROMPT)
assert_before(child, [next_msg]) assert_before(
child,
next_parts,
)
# XXX: hooray the root clobbering the child here was fixed! # XXX: hooray the root clobbering the child here was fixed!
# IMO, this demonstrates the true power of SC system design. # IMO, this demonstrates the true power of SC system design.
@ -623,9 +701,15 @@ def test_multi_daemon_subactors(
child.expect(PROMPT) child.expect(PROMPT)
try: try:
assert_before(child, [bp_forever_msg]) assert_before(
child,
bp_forev_parts,
)
except AssertionError: except AssertionError:
assert_before(child, [name_error_msg]) assert_before(
child,
name_error_parts,
)
else: else:
if ctlc: if ctlc:
@ -637,7 +721,10 @@ def test_multi_daemon_subactors(
child.sendline('c') child.sendline('c')
child.expect(PROMPT) child.expect(PROMPT)
assert_before(child, [name_error_msg]) assert_before(
child,
name_error_parts,
)
# wait for final error in root # wait for final error in root
# where it crashs with boxed error # where it crashs with boxed error
@ -647,7 +734,7 @@ def test_multi_daemon_subactors(
child.expect(PROMPT) child.expect(PROMPT)
assert_before( assert_before(
child, child,
[bp_forever_msg] bp_forev_parts
) )
except AssertionError: except AssertionError:
break break
@ -656,7 +743,9 @@ def test_multi_daemon_subactors(
child, child,
[ [
# boxed error raised in root task # boxed error raised in root task
"Attaching to pdb in crashed actor: ('root'", # "Attaching to pdb in crashed actor: ('root'",
_crash_msg,
"('root'",
"_exceptions.RemoteActorError: ('name_error'", "_exceptions.RemoteActorError: ('name_error'",
] ]
) )
@ -770,7 +859,7 @@ def test_multi_nested_subactors_error_through_nurseries(
child = spawn('multi_nested_subactors_error_up_through_nurseries') child = spawn('multi_nested_subactors_error_up_through_nurseries')
timed_out_early: bool = False # timed_out_early: bool = False
for send_char in itertools.cycle(['c', 'q']): for send_char in itertools.cycle(['c', 'q']):
try: try:
@ -871,11 +960,14 @@ def test_root_nursery_cancels_before_child_releases_tty_lock(
if not timed_out_early: if not timed_out_early:
before = str(child.before.decode()) before = str(child.before.decode())
assert_before(child, [ assert_before(
"tractor._exceptions.RemoteActorError: ('spawner0'", child,
"tractor._exceptions.RemoteActorError: ('name_error'", [
"NameError: name 'doggypants' is not defined", "tractor._exceptions.RemoteActorError: ('spawner0'",
]) "tractor._exceptions.RemoteActorError: ('name_error'",
"NameError: name 'doggypants' is not defined",
],
)
def test_root_cancels_child_context_during_startup( def test_root_cancels_child_context_during_startup(
@ -909,8 +1001,10 @@ def test_different_debug_mode_per_actor(
# only one actor should enter the debugger # only one actor should enter the debugger
before = str(child.before.decode()) before = str(child.before.decode())
assert "Attaching to pdb in crashed actor: ('debugged_boi'" in before assert in_prompt_msg(
assert "RuntimeError" in before before,
[_crash_msg, "('debugged_boi'", "RuntimeError"],
)
if ctlc: if ctlc:
do_ctlc(child) do_ctlc(child)

View File

@ -551,13 +551,14 @@ async def trio_proc(
with trio.move_on_after(0.5): with trio.move_on_after(0.5):
await proc.wait() await proc.wait()
log.pdb(
'Delaying subproc reaper while debugger locked..'
)
await maybe_wait_for_debugger( await maybe_wait_for_debugger(
child_in_debug=_runtime_vars.get( child_in_debug=_runtime_vars.get(
'_debug_mode', False '_debug_mode', False
), ),
header_msg=(
'Delaying subproc reaper while debugger locked..\n'
),
# TODO: need a diff value then default? # TODO: need a diff value then default?
# poll_steps=9999999, # poll_steps=9999999,
) )

View File

@ -21,27 +21,17 @@ and working with/on the actor runtime.
""" """
from ._debug import ( from ._debug import (
maybe_wait_for_debugger, maybe_wait_for_debugger as maybe_wait_for_debugger,
acquire_debug_lock, acquire_debug_lock as acquire_debug_lock,
breakpoint, breakpoint as breakpoint,
pause, pause as pause,
pause_from_sync, pause_from_sync as pause_from_sync,
shield_sigint_handler, shield_sigint_handler as shield_sigint_handler,
MultiActorPdb, MultiActorPdb as MultiActorPdb,
open_crash_handler, open_crash_handler as open_crash_handler,
maybe_open_crash_handler, maybe_open_crash_handler as maybe_open_crash_handler,
post_mortem, post_mortem as post_mortem,
)
from ._stackscope import (
enable_stack_on_sig as enable_stack_on_sig,
) )
__all__ = [
'maybe_wait_for_debugger',
'acquire_debug_lock',
'breakpoint',
'pause',
'pause_from_sync',
'shield_sigint_handler',
'MultiActorPdb',
'open_crash_handler',
'maybe_open_crash_handler',
'post_mortem',
]

View File

@ -1,18 +1,19 @@
# tractor: structured concurrent "actors". # tractor: structured concurrent "actors".
# Copyright 2018-eternity Tyler Goodlet. # Copyright 2018-eternity Tyler Goodlet.
# This program is free software: you can redistribute it and/or modify # This program is free software: you can redistribute it and/or
# it under the terms of the GNU Affero General Public License as published by # modify it under the terms of the GNU Affero General Public License
# the Free Software Foundation, either version 3 of the License, or # as published by the Free Software Foundation, either version 3 of
# (at your option) any later version. # the License, or (at your option) any later version.
# This program is distributed in the hope that it will be useful, # This program is distributed in the hope that it will be useful, but
# but WITHOUT ANY WARRANTY; without even the implied warranty of # WITHOUT ANY WARRANTY; without even the implied warranty of
# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the # MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
# GNU Affero General Public License for more details. # Affero General Public License for more details.
# You should have received a copy of the GNU Affero General Public License # You should have received a copy of the GNU Affero General Public
# along with this program. If not, see <https://www.gnu.org/licenses/>. # License along with this program. If not, see
# <https://www.gnu.org/licenses/>.
""" """
Multi-core debugging for da peeps! Multi-core debugging for da peeps!
@ -20,18 +21,19 @@ Multi-core debugging for da peeps!
""" """
from __future__ import annotations from __future__ import annotations
import bdb import bdb
import os
import sys
import signal
from functools import (
partial,
cached_property,
)
from contextlib import ( from contextlib import (
asynccontextmanager as acm, asynccontextmanager as acm,
contextmanager as cm, contextmanager as cm,
nullcontext, nullcontext,
) )
from functools import (
partial,
cached_property,
)
import os
import signal
import sys
import traceback
from typing import ( from typing import (
Any, Any,
Callable, Callable,
@ -43,6 +45,7 @@ from types import FrameType
import pdbp import pdbp
import tractor import tractor
import trio import trio
from trio.lowlevel import current_task
from trio_typing import ( from trio_typing import (
TaskStatus, TaskStatus,
# Task, # Task,
@ -50,6 +53,7 @@ from trio_typing import (
from ..log import get_logger from ..log import get_logger
from .._state import ( from .._state import (
current_actor,
is_root_process, is_root_process,
debug_mode, debug_mode,
) )
@ -91,12 +95,12 @@ class Lock:
# and must be cancelled if this actor is cancelled via IPC # and must be cancelled if this actor is cancelled via IPC
# request-message otherwise deadlocks with the parent actor may # request-message otherwise deadlocks with the parent actor may
# ensure # ensure
_debugger_request_cs: trio.CancelScope | None = None _debugger_request_cs: trio.CancelScope|None = None
# NOTE: set only in the root actor for the **local** root spawned task # NOTE: set only in the root actor for the **local** root spawned task
# which has acquired the lock (i.e. this is on the callee side of # which has acquired the lock (i.e. this is on the callee side of
# the `lock_tty_for_child()` context entry). # the `lock_tty_for_child()` context entry).
_root_local_task_cs_in_debug: trio.CancelScope | None = None _root_local_task_cs_in_debug: trio.CancelScope|None = None
# actor tree-wide actor uid that supposedly has the tty lock # actor tree-wide actor uid that supposedly has the tty lock
global_actor_in_debug: tuple[str, str] = None global_actor_in_debug: tuple[str, str] = None
@ -238,7 +242,7 @@ async def _acquire_debug_lock_from_root_task(
to the ``pdb`` repl. to the ``pdb`` repl.
''' '''
task_name: str = trio.lowlevel.current_task().name task_name: str = current_task().name
we_acquired: bool = False we_acquired: bool = False
log.runtime( log.runtime(
@ -323,8 +327,7 @@ async def lock_tty_for_child(
highly reliable at releasing the mutex complete! highly reliable at releasing the mutex complete!
''' '''
task_name = trio.lowlevel.current_task().name task_name: str = current_task().name
if tuple(subactor_uid) in Lock._blocked: if tuple(subactor_uid) in Lock._blocked:
log.warning( log.warning(
f'Actor {subactor_uid} is blocked from acquiring debug lock\n' f'Actor {subactor_uid} is blocked from acquiring debug lock\n'
@ -407,11 +410,13 @@ async def wait_for_parent_stdin_hijack(
assert val == 'Locked' assert val == 'Locked'
async with ctx.open_stream() as stream: async with ctx.open_stream() as stream:
# unblock local caller
try: try:
# unblock local caller
assert Lock.local_pdb_complete assert Lock.local_pdb_complete
task_status.started(cs) task_status.started(cs)
# wait for local task to exit and
# release the REPL
await Lock.local_pdb_complete.wait() await Lock.local_pdb_complete.wait()
finally: finally:
@ -468,7 +473,7 @@ def shield_sigint_handler(
uid_in_debug: tuple[str, str] | None = Lock.global_actor_in_debug uid_in_debug: tuple[str, str] | None = Lock.global_actor_in_debug
actor = tractor.current_actor() actor = current_actor()
# print(f'{actor.uid} in HANDLER with ') # print(f'{actor.uid} in HANDLER with ')
def do_cancel(): def do_cancel():
@ -607,17 +612,22 @@ def shield_sigint_handler(
# https://github.com/prompt-toolkit/python-prompt-toolkit/blob/c2c6af8a0308f9e5d7c0e28cb8a02963fe0ce07a/prompt_toolkit/patch_stdout.py # https://github.com/prompt-toolkit/python-prompt-toolkit/blob/c2c6af8a0308f9e5d7c0e28cb8a02963fe0ce07a/prompt_toolkit/patch_stdout.py
_pause_msg: str = 'Attaching to pdb REPL in actor'
def _set_trace( def _set_trace(
actor: tractor.Actor | None = None, actor: tractor.Actor | None = None,
pdb: MultiActorPdb | None = None, pdb: MultiActorPdb | None = None,
shield: bool = False, shield: bool = False,
extra_frames_up_when_async: int = 1,
): ):
__tracebackhide__: bool = True __tracebackhide__: bool = True
actor: tractor.Actor = actor or tractor.current_actor() actor: tractor.Actor = actor or current_actor()
# start 2 levels up in user code # always start 1 level up from THIS in user code.
frame: FrameType | None = sys._getframe() frame: FrameType|None
if frame: if frame := sys._getframe():
frame: FrameType = frame.f_back # type: ignore frame: FrameType = frame.f_back # type: ignore
if ( if (
@ -625,17 +635,39 @@ def _set_trace(
and ( and (
pdb pdb
and actor is not None and actor is not None
) or shield )
# or shield
): ):
msg: str = _pause_msg
if shield:
# log.warning(
msg = (
'\n\n'
' ------ - ------\n'
'Debugger invoked with `shield=True` so an extra\n'
'`trio.CancelScope.__exit__()` frame is shown..\n'
'\n'
'Try going up one frame to see your pause point!\n'
'\n'
' SORRY we need to fix this!\n'
' ------ - ------\n\n'
) + msg
# pdbp.set_trace() # pdbp.set_trace()
log.pdb(f"\nAttaching pdb to actor: {actor.uid}\n") # TODO: maybe print the actor supervion tree up to the
# root here? Bo
log.pdb(
f'{msg}\n'
'|\n'
f'|_ {actor.uid}\n'
)
# no f!#$&* idea, but when we're in async land # no f!#$&* idea, but when we're in async land
# we need 2x frames up? # we need 2x frames up?
frame = frame.f_back for i in range(extra_frames_up_when_async):
# frame = frame.f_back frame: FrameType = frame.f_back
log.debug(
# if shield: f'Going up frame {i} -> {frame}\n'
# frame = frame.f_back )
else: else:
pdb, undo_sigint = mk_mpdb() pdb, undo_sigint = mk_mpdb()
@ -645,10 +677,9 @@ def _set_trace(
Lock.local_task_in_debug = 'sync' Lock.local_task_in_debug = 'sync'
pdb.set_trace(frame=frame) pdb.set_trace(frame=frame)
# undo_
async def pause( async def _pause(
debug_func: Callable = _set_trace, debug_func: Callable = _set_trace,
release_lock_signal: trio.Event | None = None, release_lock_signal: trio.Event | None = None,
@ -662,30 +693,22 @@ async def pause(
# be no way to override it?.. # be no way to override it?..
# shield: bool = False, # shield: bool = False,
# TODO: shield: bool = False,
# shield: bool = False
task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED
) -> None: ) -> None:
''' '''
A pause point (more commonly known as a "breakpoint") interrupt Inner impl for `pause()` to avoid the `trio.CancelScope.__exit__()`
instruction for engaging a blocking debugger instance to stack frame when not shielded (since apparently i can't figure out
conduct manual console-based-REPL-interaction from within how to hide it using the normal mechanisms..)
`tractor`'s async runtime, normally from some single-threaded
and currently executing actor-hosted-`trio`-task in some
(remote) process.
NOTE: we use the semantics "pause" since it better encompasses Hopefully we won't need this in the long run.
the entirety of the necessary global-runtime-state-mutation any
actor-task must access and lock in order to get full isolated
control over the process tree's root TTY:
https://en.wikipedia.org/wiki/Breakpoint
''' '''
# __tracebackhide__ = True __tracebackhide__: bool = True
actor = tractor.current_actor() actor = current_actor()
pdb, undo_sigint = mk_mpdb() pdb, undo_sigint = mk_mpdb()
task_name = trio.lowlevel.current_task().name task_name: str = trio.lowlevel.current_task().name
if ( if (
not Lock.local_pdb_complete not Lock.local_pdb_complete
@ -693,24 +716,11 @@ async def pause(
): ):
Lock.local_pdb_complete = trio.Event() Lock.local_pdb_complete = trio.Event()
# if shield:
debug_func = partial( debug_func = partial(
debug_func, debug_func,
# shield=shield,
) )
# def _exit(self, *args, **kwargs): # TODO: need a more robust check for the "root" actor
# __tracebackhide__: bool = True
# super().__exit__(*args, **kwargs)
# trio.CancelScope.__exit__.__tracebackhide__ = True
# import types
# with trio.CancelScope(shield=shield) as cs:
# cs.__exit__ = types.MethodType(_exit, cs)
# cs.__exit__.__tracebackhide__ = True
# TODO: need a more robust check for the "root" actor
if ( if (
not is_root_process() not is_root_process()
and actor._parent_chan # a connected child and actor._parent_chan # a connected child
@ -798,37 +808,131 @@ async def pause(
Lock.repl = pdb Lock.repl = pdb
try: try:
if debug_func is None: # TODO: do we want to support using this **just** for the
# locking / common code (prolly to help address #320)?
#
# if debug_func is None:
# assert release_lock_signal, ( # assert release_lock_signal, (
# 'Must pass `release_lock_signal: trio.Event` if no ' # 'Must pass `release_lock_signal: trio.Event` if no '
# 'trace func provided!' # 'trace func provided!'
# ) # )
print(f"{actor.uid} ENTERING WAIT") # print(f"{actor.uid} ENTERING WAIT")
task_status.started()
# with trio.CancelScope(shield=True): # with trio.CancelScope(shield=True):
# await release_lock_signal.wait() # await release_lock_signal.wait()
else: # else:
# block here one (at the appropriate frame *up*) where # block here one (at the appropriate frame *up*) where
# ``breakpoint()`` was awaited and begin handling stdio. # ``breakpoint()`` was awaited and begin handling stdio.
log.debug("Entering the synchronous world of pdb") log.debug('Entering sync world of the `pdb` REPL..')
debug_func(actor, pdb) try:
debug_func(
actor,
pdb,
extra_frames_up_when_async=2,
shield=shield,
)
except BaseException:
log.exception(
'Failed to invoke internal `debug_func = '
f'{debug_func.func.__name__}`\n'
)
raise
except bdb.BdbQuit: except bdb.BdbQuit:
Lock.release() Lock.release()
raise raise
# XXX: apparently we can't do this without showing this frame except BaseException:
# in the backtrace on first entry to the REPL? Seems like an odd log.exception(
# behaviour that should have been fixed by now. This is also why 'Failed to engage debugger via `_pause()` ??\n'
# we scrapped all the @cm approaches that were tried previously. )
# finally: raise
# __tracebackhide__ = True
# # frame = sys._getframe() # XXX: apparently we can't do this without showing this frame
# # last_f = frame.f_back # in the backtrace on first entry to the REPL? Seems like an odd
# # last_f.f_globals['__tracebackhide__'] = True # behaviour that should have been fixed by now. This is also why
# # signal.signal = pdbp.hideframe(signal.signal) # 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 = pdbp.hideframe(signal.signal)
async def pause(
debug_func: Callable = _set_trace,
release_lock_signal: trio.Event | None = None,
# TODO: allow caller to pause despite task cancellation,
# exactly the same as wrapping with:
# with CancelScope(shield=True):
# await pause()
# => the REMAINING ISSUE is that the scope's .__exit__() frame
# is always show in the debugger on entry.. and there seems to
# be no way to override it?..
# shield: bool = False,
shield: bool = False,
task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED
) -> None:
'''
A pause point (more commonly known as a "breakpoint") interrupt
instruction for engaging a blocking debugger instance to
conduct manual console-based-REPL-interaction from within
`tractor`'s async runtime, normally from some single-threaded
and currently executing actor-hosted-`trio`-task in some
(remote) process.
NOTE: we use the semantics "pause" since it better encompasses
the entirety of the necessary global-runtime-state-mutation any
actor-task must access and lock in order to get full isolated
control over the process tree's root TTY:
https://en.wikipedia.org/wiki/Breakpoint
'''
__tracebackhide__: bool = True
if shield:
# NOTE XXX: even hard coding this inside the `class CancelScope:`
# doesn't seem to work for me!?
# ^ XXX ^
# def _exit(self, *args, **kwargs):
# __tracebackhide__: bool = True
# super().__exit__(*args, **kwargs)
trio.CancelScope.__enter__.__tracebackhide__ = True
trio.CancelScope.__exit__.__tracebackhide__ = True
# import types
# with trio.CancelScope(shield=shield) as cs:
# cs.__exit__ = types.MethodType(_exit, cs)
# cs.__exit__.__tracebackhide__ = True
with trio.CancelScope(shield=shield) as cs:
# setattr(cs.__exit__.__func__, '__tracebackhide__', True)
# setattr(cs.__enter__.__func__, '__tracebackhide__', True)
# NOTE: so the caller can always cancel even if shielded
task_status.started(cs)
return await _pause(
debug_func=debug_func,
release_lock_signal=release_lock_signal,
shield=True,
task_status=task_status,
)
else:
return await _pause(
debug_func=debug_func,
release_lock_signal=release_lock_signal,
shield=False,
task_status=task_status,
)
# TODO: allow pausing from sync code. # TODO: allow pausing from sync code.
@ -836,7 +940,7 @@ async def pause(
# runtime aware version which takes care of all . # runtime aware version which takes care of all .
def pause_from_sync() -> None: def pause_from_sync() -> None:
print("ENTER SYNC PAUSE") print("ENTER SYNC PAUSE")
actor: tractor.Actor = tractor.current_actor( actor: tractor.Actor = current_actor(
err_on_no_runtime=False, err_on_no_runtime=False,
) )
if actor: if actor:
@ -907,9 +1011,18 @@ async def breakpoint(**kwargs):
await pause(**kwargs) await pause(**kwargs)
_crash_msg: str = (
'Attaching to pdb REPL in crashed actor'
)
def _post_mortem( def _post_mortem(
actor: tractor.Actor, actor: tractor.Actor,
pdb: MultiActorPdb, pdb: MultiActorPdb,
shield: bool = False,
# only for compat with `._set_trace()`..
extra_frames_up_when_async=0,
) -> None: ) -> None:
''' '''
@ -917,20 +1030,28 @@ def _post_mortem(
debugger instance. debugger instance.
''' '''
log.pdb(f"\nAttaching to pdb in crashed actor: {actor.uid}\n") # TODO: print the actor supervion tree up to the root
# here! Bo
log.pdb(
f'{_crash_msg}\n'
'|\n'
f'|_ {actor.uid}\n'
)
# TODO: you need ``pdbpp`` master (at least this commit # TODO: only replacing this to add the
# https://github.com/pdbpp/pdbpp/commit/b757794857f98d53e3ebbe70879663d7d843a6c2) # `end=''` to the print XD
# to fix this and avoid the hang it causes. See issue: # pdbp.xpm(Pdb=lambda: pdb)
# https://github.com/pdbpp/pdbpp/issues/480 info = sys.exc_info()
# TODO: help with a 3.10+ major release if/when it arrives. print(traceback.format_exc(), end='')
pdbp.post_mortem(
pdbp.xpm(Pdb=lambda: pdb) t=info[2],
Pdb=lambda: pdb,
)
post_mortem = partial( post_mortem = partial(
pause, pause,
_post_mortem, debug_func=_post_mortem,
) )
@ -971,9 +1092,10 @@ async def acquire_debug_lock(
''' '''
Grab root's debug lock on entry, release on exit. Grab root's debug lock on entry, release on exit.
This helper is for actor's who don't actually need This helper is for actor's who don't actually need to acquired
to acquired the debugger but want to wait until the the debugger but want to wait until the lock is free in the
lock is free in the process-tree root. process-tree root such that they don't clobber an ongoing pdb
REPL session in some peer or child!
''' '''
if not debug_mode(): if not debug_mode():
@ -994,14 +1116,18 @@ async def maybe_wait_for_debugger(
poll_delay: float = 0.1, poll_delay: float = 0.1,
child_in_debug: bool = False, child_in_debug: bool = False,
) -> None: header_msg: str = '',
) -> bool: # was locked and we polled?
if ( if (
not debug_mode() not debug_mode()
and not child_in_debug and not child_in_debug
): ):
return return False
msg: str = header_msg
if ( if (
is_root_process() is_root_process()
): ):
@ -1011,45 +1137,98 @@ async def maybe_wait_for_debugger(
# will make the pdb repl unusable. # will make the pdb repl unusable.
# Instead try to wait for pdb to be released before # Instead try to wait for pdb to be released before
# tearing down. # tearing down.
sub_in_debug: tuple[str, str] | None = None in_debug: tuple[str, str]|None = Lock.global_actor_in_debug
debug_complete: trio.Event|None = Lock.no_remote_has_tty
for _ in range(poll_steps): if in_debug == current_actor().uid:
log.debug(
msg
+
'Root already owns the TTY LOCK'
)
return True
if Lock.global_actor_in_debug: elif in_debug:
sub_in_debug = tuple(Lock.global_actor_in_debug) msg += (
f'Debug `Lock` in use by subactor: {in_debug}\n'
log.debug('Root polling for debug') )
# TODO: could this make things more deterministic?
with trio.CancelScope(shield=True): # wait to see if a sub-actor task will be
await trio.sleep(poll_delay) # scheduled and grab the tty lock on the next
# tick?
# TODO: could this make things more deterministic? wait # XXX => but it doesn't seem to work..
# to see if a sub-actor task will be scheduled and grab # await trio.testing.wait_all_tasks_blocked(cushion=0)
# the tty lock on the next tick?
# XXX: doesn't seem to work
# await trio.testing.wait_all_tasks_blocked(cushion=0)
debug_complete = Lock.no_remote_has_tty
if (
debug_complete
and sub_in_debug is not None
and not debug_complete.is_set()
):
log.pdb(
'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()
await trio.sleep(poll_delay)
continue
else: else:
log.debug( log.debug(
'Root acquired TTY LOCK' msg
+
'Root immediately acquired debug TTY LOCK'
) )
return False
for istep in range(poll_steps):
if (
debug_complete
and not debug_complete.is_set()
and in_debug is not None
):
log.pdb(
msg
+
'Root is waiting on tty lock to release..\n'
)
with trio.CancelScope(shield=True):
await debug_complete.wait()
log.pdb(
f'Child subactor released debug lock\n'
f'|_{in_debug}\n'
)
# is no subactor locking debugger currently?
if (
in_debug is None
and (
debug_complete is None
or debug_complete.is_set()
)
):
log.pdb(
msg
+
'Root acquired tty lock!'
)
break
else:
# TODO: don't need this right?
# await trio.lowlevel.checkpoint()
log.debug(
'Root polling for debug:\n'
f'poll step: {istep}\n'
f'poll delya: {poll_delay}'
)
with trio.CancelScope(shield=True):
await trio.sleep(poll_delay)
continue
# fallthrough on failure to acquire..
# else:
# raise RuntimeError(
# msg
# +
# 'Root actor failed to acquire debug lock?'
# )
return True
# else:
# # TODO: non-root call for #320?
# this_uid: tuple[str, str] = current_actor().uid
# async with acquire_debug_lock(
# subactor_uid=this_uid,
# ):
# pass
return False
# TODO: better naming and what additionals? # TODO: better naming and what additionals?
# - [ ] optional runtime plugging? # - [ ] optional runtime plugging?

View File

@ -0,0 +1,84 @@
# tractor: structured concurrent "actors".
# Copyright 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/>.
'''
The fundamental cross process SC abstraction: an inter-actor,
cancel-scope linked task "context".
A ``Context`` is very similar to the ``trio.Nursery.cancel_scope`` built
into each ``trio.Nursery`` except it links the lifetimes of memory space
disjoint, parallel executing tasks in separate actors.
'''
from signal import (
signal,
SIGUSR1,
)
import trio
@trio.lowlevel.disable_ki_protection
def dump_task_tree() -> None:
import stackscope
from tractor.log import get_console_log
tree_str: str = str(
stackscope.extract(
trio.lowlevel.current_root_task(),
recurse_child_tasks=True
)
)
log = get_console_log('cancel')
log.pdb(
f'Dumping `stackscope` tree:\n\n'
f'{tree_str}\n'
)
# import logging
# try:
# with open("/dev/tty", "w") as tty:
# tty.write(tree_str)
# except BaseException:
# logging.getLogger(
# "task_tree"
# ).exception("Error printing task tree")
def signal_handler(sig: int, frame: object) -> None:
import traceback
try:
trio.lowlevel.current_trio_token(
).run_sync_soon(dump_task_tree)
except RuntimeError:
# not in async context -- print a normal traceback
traceback.print_stack()
def enable_stack_on_sig(
sig: int = SIGUSR1
) -> None:
'''
Enable `stackscope` tracing on reception of a signal; by
default this is SIGUSR1.
'''
signal(
sig,
signal_handler,
)
# NOTE: not the above can be triggered from
# a (xonsh) shell using:
# kill -SIGUSR1 @$(pgrep -f '<cmd>')

View File

@ -23,10 +23,6 @@ Currently popular frameworks supported are:
""" """
from __future__ import annotations from __future__ import annotations
from contextlib import (
# asynccontextmanager as acm,
contextmanager as cm,
)
from typing import ( from typing import (
Any, Any,
Callable, Callable,
@ -36,9 +32,6 @@ from typing_extensions import Annotated
import typer import typer
from ._debug import open_crash_handler
_runtime_vars: dict[str, Any] = {} _runtime_vars: dict[str, Any] = {}