Compare commits

..

1 Commits

Author SHA1 Message Date
Tyler Goodlet e696caf810 Mega-refactor on `._invoke()` targeting `@context`s
Since eventually we want to implement all other RPC "func types" as
contexts underneath this starts the rework to move all the other cases
into a separate func not only to simplify the main `._invoke()` body but
also as a reminder of the intention to do it XD

Details of re-factor:
- add a new `._invoke_non_context()` which just moves all the old blocks
  for non-context handling to a single def.
- factor what was basically just the `finally:` block handler (doing all
  the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()`
  with that content packed into the post-`yield` (also with a `hide_tb:
  bool` flag added of course).
  * include a `debug_kbis: bool` for when needed.
- since the `@context` block is the only type left in the main
  `_invoke()` body, de-dent it so it's more grok-able B)

Obviously this patch also includes a few improvements regarding
context-cancellation-semantics (for the `context` RPC case) on the
callee side in order to match previous changes to the `Context` api:
- always setting any ctxc as the `Context._local_error`.
- using the new convenience `.maybe_raise()` topically (for now).
- avoiding any previous reliance on `Context.cancelled_caught` for
  anything public of meaning.

Further included is more logging content updates:
- being pedantic in `.cancel()` msgs about whether termination is caused
  by error or ctxc.
- optional `._invoke()` traceback hiding via a `hide_tb: bool`.
- simpler log headers throughout instead leveraging new `.__repr__()` on
  primitives.
- buncha `<= <actor-uid>` sent some message emissions.
- simplified handshake statuses reporting.

Other subsys api changes we need to match:
- change to `Channel.transport`.
- avoiding any `local_nursery: ActorNursery` waiting when the
  `._implicit_runtime_started` is set.

And yes, lotsa more comments for #TODOs dawg.. since there's always
somethin!
2024-03-02 19:26:40 -05:00
11 changed files with 144 additions and 232 deletions

View File

@ -329,7 +329,7 @@ async def inf_streamer(
# close out the stream gracefully # close out the stream gracefully
except trio.ClosedResourceError: except trio.ClosedResourceError:
print('transport closed on streamer side!') print('msgstream closed on streamer side!')
assert stream.closed assert stream.closed
break break
else: else:

View File

@ -10,6 +10,7 @@ from contextlib import asynccontextmanager as acm
import pytest import pytest
import trio import trio
from trio_typing import TaskStatus
import tractor import tractor
from tractor import RemoteActorError from tractor import RemoteActorError
from async_generator import aclosing from async_generator import aclosing

View File

@ -10,13 +10,12 @@ 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 os import path from os import path
from typing import Optional from typing import Optional
import platform import platform
import pathlib import pathlib
# import sys import sys
import time import time
import pytest import pytest
@ -26,10 +25,6 @@ from pexpect.exceptions import (
EOF, EOF,
) )
from tractor.devx._debug import (
_pause_msg,
_crash_msg,
)
from conftest import ( from conftest import (
examples_dir, examples_dir,
_ci_env, _ci_env,
@ -128,52 +123,20 @@ 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:
# as in before the prompt end before = str(child.before.decode())
before: str = str(child.before.decode())
assert in_prompt_msg(
prompt=before,
parts=patts,
**kwargs for patt in patts:
) try:
assert patt in before
except AssertionError:
print(before)
raise
@pytest.fixture( @pytest.fixture(
@ -232,10 +195,7 @@ 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 in_prompt_msg( assert "Attaching to pdb in crashed actor: ('root'" in before
before,
[_crash_msg, "('root'"]
)
assert 'AssertionError' in before assert 'AssertionError' in before
# send user command # send user command
@ -372,10 +332,7 @@ def test_subactor_error(
child.expect(PROMPT) child.expect(PROMPT)
before = str(child.before.decode()) before = str(child.before.decode())
assert in_prompt_msg( assert "Attaching to pdb in crashed actor: ('name_error'" in before
before,
[_crash_msg, "('name_error'"]
)
if do_next: if do_next:
child.sendline('n') child.sendline('n')
@ -396,15 +353,9 @@ 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 in_prompt_msg( assert "Attaching to pdb in crashed actor: ('root'" in before
before,
[_crash_msg, "('root'"]
)
# error is a remote error propagated from the subactor # error is a remote error propagated from the subactor
assert in_prompt_msg( assert "RemoteActorError: ('name_error'" in before
before,
[_crash_msg, "('name_error'"]
)
# another round # another round
if ctlc: if ctlc:
@ -429,10 +380,7 @@ def test_subactor_breakpoint(
child.expect(PROMPT) child.expect(PROMPT)
before = str(child.before.decode()) before = str(child.before.decode())
assert in_prompt_msg( assert "Attaching pdb to actor: ('breakpoint_forever'" in before
before,
[_pause_msg, "('breakpoint_forever'"]
)
# do some "next" commands to demonstrate recurrent breakpoint # do some "next" commands to demonstrate recurrent breakpoint
# entries # entries
@ -448,10 +396,7 @@ 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 in_prompt_msg( assert "Attaching pdb to actor: ('breakpoint_forever'" in before
before,
[_pause_msg, "('breakpoint_forever'"]
)
if ctlc: if ctlc:
do_ctlc(child) do_ctlc(child)
@ -496,10 +441,7 @@ def test_multi_subactors(
child.expect(PROMPT) child.expect(PROMPT)
before = str(child.before.decode()) before = str(child.before.decode())
assert in_prompt_msg( assert "Attaching pdb to actor: ('breakpoint_forever'" in before
before,
[_pause_msg, "('breakpoint_forever'"]
)
if ctlc: if ctlc:
do_ctlc(child) do_ctlc(child)
@ -519,10 +461,7 @@ 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 in_prompt_msg( assert "Attaching to pdb in crashed actor: ('name_error'" in before
before,
[_crash_msg, "('name_error'"]
)
assert "NameError" in before assert "NameError" in before
if ctlc: if ctlc:
@ -548,10 +487,7 @@ 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 in_prompt_msg( assert "Attaching pdb to actor: ('breakpoint_forever'" in before
before,
[_pause_msg, "('breakpoint_forever'"]
)
if ctlc: if ctlc:
do_ctlc(child) do_ctlc(child)
@ -591,12 +527,9 @@ def test_multi_subactors(
child.expect(PROMPT) child.expect(PROMPT)
before = str(child.before.decode()) before = str(child.before.decode())
assert_before( assert_before(child, [
child, [
# debugger attaches to root # debugger attaches to root
# "Attaching to pdb in crashed actor: ('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'",
@ -604,8 +537,7 @@ def test_multi_subactors(
"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)
@ -642,22 +574,15 @@ 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_forev_parts = [_pause_msg, "('bp_forever'"] bp_forever_msg = "Attaching pdb to actor: ('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:
if bp_forev_in_msg(prompt=before): next_msg = name_error_msg
next_parts = name_error_parts
elif name_error_msg in before: elif name_error_msg in before:
next_parts = bp_forev_parts next_msg = bp_forever_msg
else: else:
raise ValueError("Neither log msg was found !?") raise ValueError("Neither log msg was found !?")
@ -674,10 +599,7 @@ def test_multi_daemon_subactors(
child.sendline('c') child.sendline('c')
child.expect(PROMPT) child.expect(PROMPT)
assert_before( assert_before(child, [next_msg])
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.
@ -701,15 +623,9 @@ def test_multi_daemon_subactors(
child.expect(PROMPT) child.expect(PROMPT)
try: try:
assert_before( assert_before(child, [bp_forever_msg])
child,
bp_forev_parts,
)
except AssertionError: except AssertionError:
assert_before( assert_before(child, [name_error_msg])
child,
name_error_parts,
)
else: else:
if ctlc: if ctlc:
@ -721,10 +637,7 @@ def test_multi_daemon_subactors(
child.sendline('c') child.sendline('c')
child.expect(PROMPT) child.expect(PROMPT)
assert_before( assert_before(child, [name_error_msg])
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
@ -734,7 +647,7 @@ def test_multi_daemon_subactors(
child.expect(PROMPT) child.expect(PROMPT)
assert_before( assert_before(
child, child,
bp_forev_parts [bp_forever_msg]
) )
except AssertionError: except AssertionError:
break break
@ -743,9 +656,7 @@ 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'",
] ]
) )
@ -859,7 +770,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:
@ -960,14 +871,11 @@ 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( assert_before(child, [
child,
[
"tractor._exceptions.RemoteActorError: ('spawner0'", "tractor._exceptions.RemoteActorError: ('spawner0'",
"tractor._exceptions.RemoteActorError: ('name_error'", "tractor._exceptions.RemoteActorError: ('name_error'",
"NameError: name 'doggypants' is not defined", "NameError: name 'doggypants' is not defined",
], ])
)
def test_root_cancels_child_context_during_startup( def test_root_cancels_child_context_during_startup(
@ -1001,10 +909,8 @@ 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 in_prompt_msg( assert "Attaching to pdb in crashed actor: ('debugged_boi'" in before
before, assert "RuntimeError" in before
[_crash_msg, "('debugged_boi'", "RuntimeError"],
)
if ctlc: if ctlc:
do_ctlc(child) do_ctlc(child)

View File

@ -134,7 +134,7 @@ def test_rpc_errors(
value = err.value value = err.value
# might get multiple `trio.Cancelled`s as well inside an inception # might get multiple `trio.Cancelled`s as well inside an inception
if isinstance(value, ExceptionGroup): if isinstance(value, trio.MultiError):
value = next(itertools.dropwhile( value = next(itertools.dropwhile(
lambda exc: not isinstance(exc, tractor.RemoteActorError), lambda exc: not isinstance(exc, tractor.RemoteActorError),
value.exceptions value.exceptions

View File

@ -116,18 +116,14 @@ def _trio_main(
if actor.loglevel is not None: if actor.loglevel is not None:
get_console_log(actor.loglevel) get_console_log(actor.loglevel)
import os import os
actor_info: str = ( log.info(
'Started new trio process:\n'
f'|_{actor}\n' f'|_{actor}\n'
f' uid: {actor.uid}\n' f' uid: {actor.uid}\n'
f' pid: {os.getpid()}\n' f' pid: {os.getpid()}\n'
f' parent_addr: {parent_addr}\n' f' parent_addr: {parent_addr}\n'
f' loglevel: {actor.loglevel}\n' f' loglevel: {actor.loglevel}\n'
) )
log.info(
'Started new trio process:\n'
+
actor_info
)
try: try:
if infect_asyncio: if infect_asyncio:
@ -137,14 +133,8 @@ def _trio_main(
trio.run(trio_main) trio.run(trio_main)
except KeyboardInterrupt: except KeyboardInterrupt:
log.cancel( log.cancel(
'Actor received KBI\n' f'@{actor.uid} received KBI'
+
actor_info
) )
finally: finally:
log.info( log.info(f"Actor {actor.uid} terminated")
'Actor terminated\n'
+
actor_info
)

View File

@ -257,7 +257,7 @@ class Portal:
return False return False
reminfo: str = ( reminfo: str = (
f'`Portal.cancel_actor()` => {self.channel.uid}\n' f'{self.channel.uid}\n'
f' |_{chan}\n' f' |_{chan}\n'
) )
log.cancel( log.cancel(
@ -949,13 +949,9 @@ class Portal:
# CASE 1 # CASE 1
else: else:
outcome_str: str = ctx.repr_outcome(
show_error_fields=True,
# type_only=True,
)
log.cancel( log.cancel(
f'Context terminated due to local scope error:\n\n' f'Context terminated due to local scope error:\n'
f'{ctx.chan.uid} => {outcome_str}\n' f'{etype.__name__}\n'
) )
# FINALLY, remove the context from runtime tracking and # FINALLY, remove the context from runtime tracking and

View File

@ -342,9 +342,7 @@ async def open_root_actor(
# for an in nurseries: # for an in nurseries:
# tempn.start_soon(an.exited.wait) # tempn.start_soon(an.exited.wait)
logger.info( logger.cancel("Shutting down root actor")
'Closing down root actor'
)
await actor.cancel(None) # self cancel await actor.cancel(None) # self cancel
finally: finally:
_state._current_actor = None _state._current_actor = None

View File

@ -627,7 +627,7 @@ async def _invoke(
# -[ ] textwrap.indent() it correctly! # -[ ] textwrap.indent() it correctly!
# -[ ] BUT we need to wait until # -[ ] BUT we need to wait until
# the state is filled out before emitting # the state is filled out before emitting
# this msg right ow its kinda empty? bleh.. # this msg right?
# #
# f' |_{ctx}' # f' |_{ctx}'
) )
@ -638,6 +638,7 @@ async def _invoke(
msg += ( msg += (
# '------ - ------\n' # '------ - ------\n'
# 'IPC msg:\n' # 'IPC msg:\n'
# f'\n{ctx._cancel_msg}'
f'\n\n{ctx._cancel_msg}' f'\n\n{ctx._cancel_msg}'
) )
@ -672,13 +673,28 @@ async def _invoke(
# if a remote error was set then likely the # if a remote error was set then likely the
# exception group was raised due to that, so # exception group was raised due to that, so
# and we instead raise that error immediately! # and we instead raise that error immediately!
# if (
# # ctx.side == 'caller'
# # ctx._portal
# # and (re := ctx._remote_error)
# re := ctx._remote_error
# ):
# if re := ctx._remote_error:
# ctx._maybe_raise_remote_err(re)
ctx.maybe_raise() ctx.maybe_raise()
# maybe TODO: pack in come kinda # if isinstance(scope_error, KeyboardInterrupt):
# `trio.Cancelled.__traceback__` here so they can be # await pause()
# unwrapped and displayed on the caller side? no se..
# maybe TODO: pack in
# ``trio.Cancelled.__traceback__`` here so they can
# be unwrapped and displayed on the caller side?
raise raise
# except BaseException as scope_error:
# ctx._local_error =
# else:
# `@context` entrypoint task bookeeping. # `@context` entrypoint task bookeeping.
# i.e. only pop the context tracking if used ;) # i.e. only pop the context tracking if used ;)
finally: finally:
@ -691,6 +707,13 @@ async def _invoke(
(chan.uid, cid) (chan.uid, cid)
) )
# ctx.maybe_error
# # ctx._local_error
# # or ctx._remote_error
# )
# if not final_err and ctx.canceller:
# await pause()
merr: Exception|None = ctx.maybe_error merr: Exception|None = ctx.maybe_error
( (
@ -1094,6 +1117,7 @@ class Actor:
# append new channel # append new channel
# TODO: can we just use list-ref directly? # TODO: can we just use list-ref directly?
chans.append(chan) chans.append(chan)
# self._peers[uid].append(chan)
log.runtime(con_msg) log.runtime(con_msg)
@ -1224,6 +1248,7 @@ class Actor:
f'uid: {chan.uid}\n' f'uid: {chan.uid}\n'
f'|_{pformat(chan)}\n' f'|_{pformat(chan)}\n'
) )
# chans = self._peers.get(chan.uid)
chans.remove(chan) chans.remove(chan)
# TODO: do we need to be this pedantic? # TODO: do we need to be this pedantic?

View File

@ -21,17 +21,30 @@ and working with/on the actor runtime.
""" """
from ._debug import ( from ._debug import (
maybe_wait_for_debugger as maybe_wait_for_debugger, maybe_wait_for_debugger,
acquire_debug_lock as acquire_debug_lock, acquire_debug_lock,
breakpoint as breakpoint, breakpoint,
pause as pause, pause,
pause_from_sync as pause_from_sync, pause_from_sync,
shield_sigint_handler as shield_sigint_handler, shield_sigint_handler,
MultiActorPdb as MultiActorPdb, MultiActorPdb,
open_crash_handler as open_crash_handler, open_crash_handler,
maybe_open_crash_handler as maybe_open_crash_handler, maybe_open_crash_handler,
post_mortem as post_mortem, post_mortem,
) )
from ._stackscope import ( from ._stackscope import (
enable_stack_on_sig as enable_stack_on_sig, 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

@ -21,19 +21,18 @@ 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,
@ -612,9 +611,6 @@ 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,
@ -636,13 +632,7 @@ def _set_trace(
) or shield ) or shield
): ):
# pdbp.set_trace() # pdbp.set_trace()
# TODO: maybe print the actor supervion tree up to the log.pdb(f"\nAttaching pdb to actor: {actor.uid}\n")
# root here? Bo
log.pdb(
f'{_pause_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 frame = frame.f_back
@ -921,11 +911,6 @@ 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,
@ -936,23 +921,15 @@ def _post_mortem(
debugger instance. debugger instance.
''' '''
# TODO: print the actor supervion tree up to the root log.pdb(f"\nAttaching to pdb in crashed actor: {actor.uid}\n")
# here! Bo
log.pdb(
f'{_crash_msg}\n'
'|\n'
f'|_ {actor.uid}\n'
)
# TODO: only replacing this to add the # TODO: you need ``pdbpp`` master (at least this commit
# `end=''` to the print XD # https://github.com/pdbpp/pdbpp/commit/b757794857f98d53e3ebbe70879663d7d843a6c2)
# pdbp.xpm(Pdb=lambda: pdb) # to fix this and avoid the hang it causes. See issue:
info = sys.exc_info() # https://github.com/pdbpp/pdbpp/issues/480
print(traceback.format_exc(), end='') # TODO: help with a 3.10+ major release if/when it arrives.
pdbp.post_mortem(
t=info[2], pdbp.xpm(Pdb=lambda: pdb)
Pdb=lambda: pdb,
)
post_mortem = partial( post_mortem = partial(
@ -1024,13 +1001,13 @@ async def maybe_wait_for_debugger(
header_msg: str = '', header_msg: str = '',
) -> bool: # was locked and we polled? ) -> None:
if ( if (
not debug_mode() not debug_mode()
and not child_in_debug and not child_in_debug
): ):
return False return
msg: str = header_msg msg: str = header_msg
@ -1048,7 +1025,8 @@ async def maybe_wait_for_debugger(
if sub_in_debug := Lock.global_actor_in_debug: if sub_in_debug := Lock.global_actor_in_debug:
msg += ( msg += (
f'Debug `Lock` in use by subactor: {sub_in_debug}\n' 'Debug `Lock` in use by subactor\n'
f'|_{sub_in_debug}\n'
) )
# TODO: could this make things more deterministic? # TODO: could this make things more deterministic?
# wait to see if a sub-actor task will be # wait to see if a sub-actor task will be
@ -1057,12 +1035,12 @@ async def maybe_wait_for_debugger(
# XXX => but it doesn't seem to work.. # XXX => but it doesn't seem to work..
# await trio.testing.wait_all_tasks_blocked(cushion=0) # await trio.testing.wait_all_tasks_blocked(cushion=0)
else: else:
log.debug( log.pdb(
msg msg
+ +
'Root immediately acquired debug TTY LOCK' 'Root immediately acquired debug TTY LOCK'
) )
return False return
for istep in range(poll_steps): for istep in range(poll_steps):
@ -1112,13 +1090,12 @@ async def maybe_wait_for_debugger(
continue continue
# fallthrough on failure to acquire.. # fallthrough on failure to acquire..
# else: else:
# raise RuntimeError( raise RuntimeError(
# msg msg
# + +
# 'Root actor failed to acquire debug lock?' 'Root actor failed to acquire debug lock?'
# ) )
return True
# else: # else:
# # TODO: non-root call for #320? # # TODO: non-root call for #320?
@ -1127,7 +1104,6 @@ async def maybe_wait_for_debugger(
# subactor_uid=this_uid, # subactor_uid=this_uid,
# ): # ):
# pass # pass
return False
# TODO: better naming and what additionals? # TODO: better naming and what additionals?
# - [ ] optional runtime plugging? # - [ ] optional runtime plugging?

View File

@ -23,6 +23,10 @@ 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,
@ -32,6 +36,9 @@ 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] = {}