Mk `.devx._debug` a sub-pkg `.devx.debug`

With plans for much factoring of the original module into sub-mods!
Adjust all imports and refs throughout to match.
repl_fixture
Tyler Goodlet 2025-05-13 12:13:12 -04:00
parent 09a61dbd8a
commit 18ae7b0048
21 changed files with 68 additions and 68 deletions

View File

@ -29,7 +29,7 @@ async def bp_then_error(
to_trio.send_nowait('start') to_trio.send_nowait('start')
# NOTE: what happens here inside the hook needs some refinement.. # NOTE: what happens here inside the hook needs some refinement..
# => seems like it's still `._debug._set_trace()` but # => seems like it's still `.debug._set_trace()` but
# we set `Lock.local_task_in_debug = 'sync'`, we probably want # we set `Lock.local_task_in_debug = 'sync'`, we probably want
# some further, at least, meta-data about the task/actor in debug # some further, at least, meta-data about the task/actor in debug
# in terms of making it clear it's `asyncio` mucking about. # in terms of making it clear it's `asyncio` mucking about.

View File

@ -18,14 +18,14 @@ async def main() -> None:
assert ( assert (
(pybp_var := os.environ['PYTHONBREAKPOINT']) (pybp_var := os.environ['PYTHONBREAKPOINT'])
== ==
'tractor.devx._debug._sync_pause_from_builtin' 'tractor.devx.debug._sync_pause_from_builtin'
) )
# TODO: an assert that verifies the hook has indeed been, hooked # TODO: an assert that verifies the hook has indeed been, hooked
# XD # XD
assert ( assert (
(pybp_hook := sys.breakpointhook) (pybp_hook := sys.breakpointhook)
is not tractor.devx._debug._set_trace is not tractor.devx.debug._set_trace
) )
print( print(

View File

@ -6,7 +6,7 @@ import tractor
# TODO: only import these when not running from test harness? # TODO: only import these when not running from test harness?
# can we detect `pexpect` usage maybe? # can we detect `pexpect` usage maybe?
# from tractor.devx._debug import ( # from tractor.devx.debug import (
# get_lock, # get_lock,
# get_debug_req, # get_debug_req,
# ) # )

View File

@ -16,7 +16,7 @@ from pexpect.spawnbase import SpawnBase
from tractor._testing import ( from tractor._testing import (
mk_cmd, mk_cmd,
) )
from tractor.devx._debug import ( from tractor.devx.debug import (
_pause_msg as _pause_msg, _pause_msg as _pause_msg,
_crash_msg as _crash_msg, _crash_msg as _crash_msg,
_repl_fail_msg as _repl_fail_msg, _repl_fail_msg as _repl_fail_msg,
@ -111,7 +111,7 @@ def ctlc(
# XXX: disable pygments highlighting for auto-tests # XXX: disable pygments highlighting for auto-tests
# since some envs (like actions CI) will struggle # since some envs (like actions CI) will struggle
# the the added color-char encoding.. # the the added color-char encoding..
from tractor.devx._debug import TractorConfig from tractor.devx.debug import TractorConfig
TractorConfig.use_pygements = False TractorConfig.use_pygements = False
yield use_ctlc yield use_ctlc

View File

@ -528,7 +528,7 @@ def test_multi_daemon_subactors(
# now the root actor won't clobber the bp_forever child # now the root actor won't clobber the bp_forever child
# during it's first access to the debug lock, but will instead # during it's first access to the debug lock, but will instead
# wait for the lock to release, by the edge triggered # wait for the lock to release, by the edge triggered
# ``devx._debug.Lock.no_remote_has_tty`` event before sending cancel messages # ``devx.debug.Lock.no_remote_has_tty`` event before sending cancel messages
# (via portals) to its underlings B) # (via portals) to its underlings B)
# at some point here there should have been some warning msg from # at some point here there should have been some warning msg from

View File

@ -889,7 +889,7 @@ async def manage_file(
# NOTE: turns out you don't even need to sched an aio task # NOTE: turns out you don't even need to sched an aio task
# since the original issue, even though seemingly was due to # since the original issue, even though seemingly was due to
# the guest-run being abandoned + a `._debug.pause()` inside # the guest-run being abandoned + a `.debug.pause()` inside
# `._runtime._async_main()` (which was originally trying to # `._runtime._async_main()` (which was originally trying to
# debug the `.lifetime_stack` not closing), IS NOT actually # debug the `.lifetime_stack` not closing), IS NOT actually
# the core issue? # the core issue?
@ -1101,7 +1101,7 @@ def test_sigint_closes_lifetime_stack(
# => completed using `.bestow_portal(task)` inside # => completed using `.bestow_portal(task)` inside
# `.to_asyncio._run_asyncio_task()` right? # `.to_asyncio._run_asyncio_task()` right?
# -[ ] translation func to get from `asyncio` task calling to # -[ ] translation func to get from `asyncio` task calling to
# `._debug.wait_for_parent_stdin_hijack()` which does root # `.debug.wait_for_parent_stdin_hijack()` which does root
# call to do TTY locking. # call to do TTY locking.
# #
def test_sync_breakpoint(): def test_sync_breakpoint():

View File

@ -292,7 +292,7 @@ class Context:
# - `._runtime._invoke()` will check this flag before engaging # - `._runtime._invoke()` will check this flag before engaging
# the crash handler REPL in such cases where the "callee" # the crash handler REPL in such cases where the "callee"
# raises the cancellation, # raises the cancellation,
# - `.devx._debug.lock_stdio_for_peer()` will set it to `False` if # - `.devx.debug.lock_stdio_for_peer()` will set it to `False` if
# the global tty-lock has been configured to filter out some # the global tty-lock has been configured to filter out some
# actors from being able to acquire the debugger lock. # actors from being able to acquire the debugger lock.
_enter_debugger_on_cancel: bool = True _enter_debugger_on_cancel: bool = True
@ -1234,8 +1234,8 @@ class Context:
# ?XXX, should already be set in `._deliver_msg()` right? # ?XXX, should already be set in `._deliver_msg()` right?
if self._outcome_msg is not Unresolved: if self._outcome_msg is not Unresolved:
# from .devx import _debug # from .devx import debug
# await _debug.pause() # await debug.pause()
assert self._outcome_msg is outcome_msg assert self._outcome_msg is outcome_msg
else: else:
self._outcome_msg = outcome_msg self._outcome_msg = outcome_msg
@ -2170,7 +2170,7 @@ async def open_context_from_portal(
# debugging the tractor-runtime itself using it's # debugging the tractor-runtime itself using it's
# own `.devx.` tooling! # own `.devx.` tooling!
# #
# await _debug.pause() # await debug.pause()
# CASE 2: context was cancelled by local task calling # CASE 2: context was cancelled by local task calling
# `.cancel()`, we don't raise and the exit block should # `.cancel()`, we don't raise and the exit block should
@ -2237,7 +2237,7 @@ async def open_context_from_portal(
# NOTE: `Context.cancel()` is conversely NEVER CALLED in # NOTE: `Context.cancel()` is conversely NEVER CALLED in
# the `ContextCancelled` "self cancellation absorbed" case # the `ContextCancelled` "self cancellation absorbed" case
# handled in the block above ^^^ !! # handled in the block above ^^^ !!
# await _debug.pause() # await debug.pause()
# log.cancel( # log.cancel(
match scope_err: match scope_err:
case trio.Cancelled: case trio.Cancelled:
@ -2252,11 +2252,11 @@ async def open_context_from_portal(
) )
if debug_mode(): if debug_mode():
# async with _debug.acquire_debug_lock(portal.actor.uid): # async with debug.acquire_debug_lock(portal.actor.uid):
# pass # pass
# TODO: factor ^ into below for non-root cases? # TODO: factor ^ into below for non-root cases?
# #
from .devx._debug import maybe_wait_for_debugger from .devx.debug import maybe_wait_for_debugger
was_acquired: bool = await maybe_wait_for_debugger( was_acquired: bool = await maybe_wait_for_debugger(
# header_msg=( # header_msg=(
# 'Delaying `ctx.cancel()` until debug lock ' # 'Delaying `ctx.cancel()` until debug lock '
@ -2319,8 +2319,8 @@ async def open_context_from_portal(
raise raise
# yes this worx! # yes this worx!
# from .devx import _debug # from .devx import debug
# await _debug.pause() # await debug.pause()
# an exception type boxed in a `RemoteActorError` # an exception type boxed in a `RemoteActorError`
# is returned (meaning it was obvi not raised) # is returned (meaning it was obvi not raised)
@ -2355,7 +2355,7 @@ async def open_context_from_portal(
# where the root is waiting on the lock to clear but the # where the root is waiting on the lock to clear but the
# child has already cleared it and clobbered IPC. # child has already cleared it and clobbered IPC.
if debug_mode(): if debug_mode():
from .devx._debug import maybe_wait_for_debugger from .devx.debug import maybe_wait_for_debugger
await maybe_wait_for_debugger() await maybe_wait_for_debugger()
# though it should be impossible for any tasks # though it should be impossible for any tasks

View File

@ -35,7 +35,7 @@ from .log import (
) )
from . import _state from . import _state
from .devx import ( from .devx import (
_debug, debug,
pformat, pformat,
) )
from .to_asyncio import run_as_asyncio_guest from .to_asyncio import run_as_asyncio_guest
@ -116,7 +116,7 @@ def _trio_main(
Entry point for a `trio_run_in_process` subactor. Entry point for a `trio_run_in_process` subactor.
''' '''
_debug.hide_runtime_frames() debug.hide_runtime_frames()
_state._current_actor = actor _state._current_actor = actor
trio_main = partial( trio_main = partial(

View File

@ -44,7 +44,7 @@ from ._runtime import (
# Arbiter as Registry, # Arbiter as Registry,
async_main, async_main,
) )
from .devx import _debug from .devx import debug
from . import _spawn from . import _spawn
from . import _state from . import _state
from . import log from . import log
@ -67,7 +67,7 @@ from ._exceptions import (
logger = log.get_logger('tractor') logger = log.get_logger('tractor')
# TODO: stick this in a `@acm` defined in `devx._debug`? # TODO: stick this in a `@acm` defined in `devx.debug`?
# -[ ] also maybe consider making this a `wrapt`-deco to # -[ ] also maybe consider making this a `wrapt`-deco to
# save an indent level? # save an indent level?
# #
@ -89,7 +89,7 @@ async def maybe_block_bp(
debug_mode debug_mode
and maybe_enable_greenback and maybe_enable_greenback
and ( and (
maybe_mod := await _debug.maybe_init_greenback( maybe_mod := await debug.maybe_init_greenback(
raise_not_found=False, raise_not_found=False,
) )
) )
@ -99,7 +99,7 @@ async def maybe_block_bp(
'Enabling `tractor.pause_from_sync()` support!\n' 'Enabling `tractor.pause_from_sync()` support!\n'
) )
os.environ['PYTHONBREAKPOINT'] = ( os.environ['PYTHONBREAKPOINT'] = (
'tractor.devx._debug._sync_pause_from_builtin' 'tractor.devx.debug._sync_pause_from_builtin'
) )
_state._runtime_vars['use_greenback'] = True _state._runtime_vars['use_greenback'] = True
bp_blocked = False bp_blocked = False
@ -178,7 +178,7 @@ async def open_root_actor(
hide_tb: bool = True, hide_tb: bool = True,
# XXX, proxied directly to `.devx._debug._maybe_enter_pm()` # XXX, proxied directly to `.devx.debug._maybe_enter_pm()`
# for REPL-entry logic. # for REPL-entry logic.
debug_filter: Callable[ debug_filter: Callable[
[BaseException|BaseExceptionGroup], [BaseException|BaseExceptionGroup],
@ -223,12 +223,12 @@ async def open_root_actor(
len(enable_transports) == 1 len(enable_transports) == 1
), 'No multi-tpt support yet!' ), 'No multi-tpt support yet!'
_debug.hide_runtime_frames() debug.hide_runtime_frames()
__tracebackhide__: bool = hide_tb __tracebackhide__: bool = hide_tb
# attempt to retreive ``trio``'s sigint handler and stash it # attempt to retreive ``trio``'s sigint handler and stash it
# on our debugger lock state. # on our debugger lock state.
_debug.DebugStatus._trio_handler = signal.getsignal(signal.SIGINT) debug.DebugStatus._trio_handler = signal.getsignal(signal.SIGINT)
# mark top most level process as root actor # mark top most level process as root actor
_state._runtime_vars['_is_root'] = True _state._runtime_vars['_is_root'] = True
@ -283,7 +283,7 @@ async def open_root_actor(
# expose internal debug module to every actor allowing for # expose internal debug module to every actor allowing for
# use of ``await tractor.pause()`` # use of ``await tractor.pause()``
enable_modules.append('tractor.devx._debug') enable_modules.append('tractor.devx.debug')
# if debug mode get's enabled *at least* use that level of # if debug mode get's enabled *at least* use that level of
# logging for some informative console prompts. # logging for some informative console prompts.
@ -465,7 +465,7 @@ async def open_root_actor(
# TODO, in beginning to handle the subsubactor with # TODO, in beginning to handle the subsubactor with
# crashed grandparent cases.. # crashed grandparent cases..
# #
# was_locked: bool = await _debug.maybe_wait_for_debugger( # was_locked: bool = await debug.maybe_wait_for_debugger(
# child_in_debug=True, # child_in_debug=True,
# ) # )
# XXX NOTE XXX see equiv note inside # XXX NOTE XXX see equiv note inside
@ -473,7 +473,7 @@ async def open_root_actor(
# non-root or root-that-opened-this-mahually case we # non-root or root-that-opened-this-mahually case we
# wait for the local actor-nursery to exit before # wait for the local actor-nursery to exit before
# exiting the transport channel handler. # exiting the transport channel handler.
entered: bool = await _debug._maybe_enter_pm( entered: bool = await debug._maybe_enter_pm(
err, err,
api_frame=inspect.currentframe(), api_frame=inspect.currentframe(),
debug_filter=debug_filter, debug_filter=debug_filter,

View File

@ -57,7 +57,7 @@ from ._exceptions import (
unpack_error, unpack_error,
) )
from .devx import ( from .devx import (
_debug, debug,
add_div, add_div,
) )
from . import _state from . import _state
@ -266,7 +266,7 @@ async def _errors_relayed_via_ipc(
# TODO: a debug nursery when in debug mode! # TODO: a debug nursery when in debug mode!
# async with maybe_open_debugger_nursery() as debug_tn: # async with maybe_open_debugger_nursery() as debug_tn:
# => see matching comment in side `._debug._pause()` # => see matching comment in side `.debug._pause()`
rpc_err: BaseException|None = None rpc_err: BaseException|None = None
try: try:
yield # run RPC invoke body yield # run RPC invoke body
@ -318,7 +318,7 @@ async def _errors_relayed_via_ipc(
'RPC task crashed, attempting to enter debugger\n' 'RPC task crashed, attempting to enter debugger\n'
f'|_{ctx}' f'|_{ctx}'
) )
entered_debug = await _debug._maybe_enter_pm( entered_debug = await debug._maybe_enter_pm(
err, err,
api_frame=inspect.currentframe(), api_frame=inspect.currentframe(),
) )
@ -462,7 +462,7 @@ async def _invoke(
): ):
# XXX for .pause_from_sync()` usage we need to make sure # XXX for .pause_from_sync()` usage we need to make sure
# `greenback` is boostrapped in the subactor! # `greenback` is boostrapped in the subactor!
await _debug.maybe_init_greenback() await debug.maybe_init_greenback()
# TODO: possibly a specially formatted traceback # TODO: possibly a specially formatted traceback
# (not sure what typing is for this..)? # (not sure what typing is for this..)?
@ -751,7 +751,7 @@ async def _invoke(
and 'Cancel scope stack corrupted' in scope_error.args[0] and 'Cancel scope stack corrupted' in scope_error.args[0]
): ):
log.exception('Cancel scope stack corrupted!?\n') log.exception('Cancel scope stack corrupted!?\n')
# _debug.mk_pdb().set_trace() # debug.mk_pdb().set_trace()
# always set this (child) side's exception as the # always set this (child) side's exception as the
# local error on the context # local error on the context
@ -779,7 +779,7 @@ async def _invoke(
# don't pop the local context until we know the # don't pop the local context until we know the
# associated child isn't in debug any more # associated child isn't in debug any more
await _debug.maybe_wait_for_debugger() await debug.maybe_wait_for_debugger()
ctx: Context = actor._contexts.pop(( ctx: Context = actor._contexts.pop((
chan.uid, chan.uid,
cid, cid,
@ -983,7 +983,7 @@ async def process_messages(
# XXX NOTE XXX don't start entire actor # XXX NOTE XXX don't start entire actor
# runtime cancellation if this actor is # runtime cancellation if this actor is
# currently in debug mode! # currently in debug mode!
pdb_complete: trio.Event|None = _debug.DebugStatus.repl_release pdb_complete: trio.Event|None = debug.DebugStatus.repl_release
if pdb_complete: if pdb_complete:
await pdb_complete.wait() await pdb_complete.wait()

View File

@ -96,7 +96,7 @@ from ._exceptions import (
MsgTypeError, MsgTypeError,
unpack_error, unpack_error,
) )
from .devx import _debug from .devx import debug
from ._discovery import get_registry from ._discovery import get_registry
from ._portal import Portal from ._portal import Portal
from . import _state from . import _state
@ -220,7 +220,7 @@ class Actor:
self._parent_main_data = _mp_fixup_main._mp_figure_out_main() self._parent_main_data = _mp_fixup_main._mp_figure_out_main()
# always include debugging tools module # always include debugging tools module
enable_modules.append('tractor.devx._debug') enable_modules.append('tractor.devx.debug')
self.enable_modules: dict[str, str] = {} self.enable_modules: dict[str, str] = {}
for name in enable_modules: for name in enable_modules:
@ -930,7 +930,7 @@ class Actor:
# kill any debugger request task to avoid deadlock # kill any debugger request task to avoid deadlock
# with the root actor in this tree # with the root actor in this tree
debug_req = _debug.DebugStatus debug_req = debug.DebugStatus
lock_req_ctx: Context = debug_req.req_ctx lock_req_ctx: Context = debug_req.req_ctx
if ( if (
lock_req_ctx lock_req_ctx
@ -940,7 +940,7 @@ class Actor:
msg += ( msg += (
f'\n' f'\n'
f'-> Cancelling active debugger request..\n' f'-> Cancelling active debugger request..\n'
f'|_{_debug.Lock.repr()}\n\n' f'|_{debug.Lock.repr()}\n\n'
f'|_{lock_req_ctx}\n\n' f'|_{lock_req_ctx}\n\n'
) )
# lock_req_ctx._scope.cancel() # lock_req_ctx._scope.cancel()
@ -1266,7 +1266,7 @@ async def async_main(
# attempt to retreive ``trio``'s sigint handler and stash it # attempt to retreive ``trio``'s sigint handler and stash it
# on our debugger state. # on our debugger state.
_debug.DebugStatus._trio_handler = signal.getsignal(signal.SIGINT) debug.DebugStatus._trio_handler = signal.getsignal(signal.SIGINT)
is_registered: bool = False is_registered: bool = False
try: try:
@ -1360,7 +1360,7 @@ async def async_main(
# try: # try:
# actor.load_modules() # actor.load_modules()
# except ModuleNotFoundError as err: # except ModuleNotFoundError as err:
# _debug.pause_from_sync() # debug.pause_from_sync()
# import pdbp; pdbp.set_trace() # import pdbp; pdbp.set_trace()
# raise # raise
@ -1393,7 +1393,7 @@ async def async_main(
# tranport address bind errors - normally it's # tranport address bind errors - normally it's
# something silly like the wrong socket-address # something silly like the wrong socket-address
# passed via a config or CLI Bo # passed via a config or CLI Bo
entered_debug: bool = await _debug._maybe_enter_pm( entered_debug: bool = await debug._maybe_enter_pm(
oserr, oserr,
) )
if not entered_debug: if not entered_debug:
@ -1431,7 +1431,7 @@ async def async_main(
waddr = wrap_address(addr) waddr = wrap_address(addr)
assert waddr.is_valid assert waddr.is_valid
except AssertionError: except AssertionError:
await _debug.pause() await debug.pause()
async with get_registry(addr) as reg_portal: async with get_registry(addr) as reg_portal:
for accept_addr in accept_addrs: for accept_addr in accept_addrs:
@ -1549,7 +1549,7 @@ async def async_main(
# prevents any `infected_aio` actor from continuing # prevents any `infected_aio` actor from continuing
# and any callbacks in the `ls` here WILL NOT be # and any callbacks in the `ls` here WILL NOT be
# called!! # called!!
# await _debug.pause(shield=True) # await debug.pause(shield=True)
ls.close() ls.close()
@ -1562,7 +1562,7 @@ async def async_main(
# #
# if actor.name == 'brokerd.ib': # if actor.name == 'brokerd.ib':
# with CancelScope(shield=True): # with CancelScope(shield=True):
# await _debug.breakpoint() # await debug.breakpoint()
# Unregister actor from the registry-sys / registrar. # Unregister actor from the registry-sys / registrar.
if ( if (
@ -1751,7 +1751,7 @@ class Arbiter(Actor):
waddr: Address = wrap_address(addr) waddr: Address = wrap_address(addr)
if not waddr.is_valid: if not waddr.is_valid:
# should never be 0-dynamic-os-alloc # should never be 0-dynamic-os-alloc
await _debug.pause() await debug.pause()
self._registry[uid] = addr self._registry[uid] = addr

View File

@ -34,7 +34,7 @@ from typing import (
import trio import trio
from trio import TaskStatus from trio import TaskStatus
from .devx._debug import ( from .devx.debug import (
maybe_wait_for_debugger, maybe_wait_for_debugger,
acquire_debug_lock, acquire_debug_lock,
) )

View File

@ -426,8 +426,8 @@ class MsgStream(trio.abc.Channel):
self._closed = re self._closed = re
# if caught_eoc: # if caught_eoc:
# # from .devx import _debug # # from .devx import debug
# # await _debug.pause() # # await debug.pause()
# with trio.CancelScope(shield=True): # with trio.CancelScope(shield=True):
# await rx_chan.aclose() # await rx_chan.aclose()

View File

@ -31,7 +31,7 @@ import warnings
import trio import trio
from .devx._debug import maybe_wait_for_debugger from .devx.debug import maybe_wait_for_debugger
from ._addr import ( from ._addr import (
UnwrappedAddress, UnwrappedAddress,
mk_uuid, mk_uuid,

View File

@ -26,7 +26,7 @@ import os
import pathlib import pathlib
import tractor import tractor
from tractor.devx._debug import ( from tractor.devx.debug import (
BoxedMaybeException, BoxedMaybeException,
) )
from .pytest import ( from .pytest import (

View File

@ -20,7 +20,7 @@ Runtime "developer experience" utils and addons to aid our
and working with/on the actor runtime. 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 as maybe_wait_for_debugger,
acquire_debug_lock as acquire_debug_lock, acquire_debug_lock as acquire_debug_lock,
breakpoint as breakpoint, breakpoint as breakpoint,

View File

@ -49,7 +49,7 @@ from tractor import (
_state, _state,
log as logmod, log as logmod,
) )
from tractor.devx import _debug from tractor.devx import debug
log = logmod.get_logger(__name__) log = logmod.get_logger(__name__)
@ -82,7 +82,7 @@ def dump_task_tree() -> None:
if ( if (
current_sigint_handler current_sigint_handler
is not is not
_debug.DebugStatus._trio_handler debug.DebugStatus._trio_handler
): ):
sigint_handler_report: str = ( sigint_handler_report: str = (
'The default `trio` SIGINT handler was replaced?!' 'The default `trio` SIGINT handler was replaced?!'

View File

@ -43,7 +43,7 @@ from trio import (
SocketListener, SocketListener,
) )
# from ..devx import _debug # from ..devx import debug
from .._exceptions import ( from .._exceptions import (
TransportClosed, TransportClosed,
) )
@ -107,7 +107,7 @@ async def handle_stream_from_peer(
server._no_more_peers = trio.Event() # unset by making new server._no_more_peers = trio.Event() # unset by making new
# TODO, debug_mode tooling for when hackin this lower layer? # TODO, debug_mode tooling for when hackin this lower layer?
# with _debug.maybe_open_crash_handler( # with debug.maybe_open_crash_handler(
# pdb=True, # pdb=True,
# ) as boxerr: # ) as boxerr:
@ -343,7 +343,7 @@ async def handle_stream_from_peer(
# - is root but `open_root_actor()` was # - is root but `open_root_actor()` was
# entered manually (in which case we do # entered manually (in which case we do
# the equiv wait there using the # the equiv wait there using the
# `devx._debug` sub-sys APIs). # `devx.debug` sub-sys APIs).
not local_nursery._implicit_runtime_started not local_nursery._implicit_runtime_started
): ):
log.runtime( log.runtime(
@ -456,8 +456,8 @@ async def handle_stream_from_peer(
and and
_state.is_debug_mode() _state.is_debug_mode()
): ):
from ..devx import _debug from ..devx import debug
pdb_lock = _debug.Lock pdb_lock = debug.Lock
pdb_lock._blocked.add(uid) pdb_lock._blocked.add(uid)
# TODO: NEEEDS TO BE TESTED! # TODO: NEEEDS TO BE TESTED!
@ -492,7 +492,7 @@ async def handle_stream_from_peer(
f'last disconnected child uid: {uid}\n' f'last disconnected child uid: {uid}\n'
f'locking child uid: {pdb_user_uid}\n' f'locking child uid: {pdb_user_uid}\n'
) )
await _debug.maybe_wait_for_debugger( await debug.maybe_wait_for_debugger(
child_in_debug=True child_in_debug=True
) )

View File

@ -608,7 +608,7 @@ async def drain_to_final_msg(
# #
# -[ ] make sure pause points work here for REPLing # -[ ] make sure pause points work here for REPLing
# the runtime itself; i.e. ensure there's no hangs! # the runtime itself; i.e. ensure there's no hangs!
# |_from tractor.devx._debug import pause # |_from tractor.devx.debug import pause
# await pause() # await pause()
# NOTE: we get here if the far end was # NOTE: we get here if the far end was

View File

@ -49,7 +49,7 @@ from tractor._state import (
_runtime_vars, _runtime_vars,
) )
from tractor._context import Unresolved from tractor._context import Unresolved
from tractor.devx import _debug from tractor.devx import debug
from tractor.log import ( from tractor.log import (
get_logger, get_logger,
StackLevelAdapter, StackLevelAdapter,
@ -479,7 +479,7 @@ def _run_asyncio_task(
if ( if (
debug_mode() debug_mode()
and and
(greenback := _debug.maybe_import_greenback( (greenback := debug.maybe_import_greenback(
force_reload=True, force_reload=True,
raise_not_found=False, raise_not_found=False,
)) ))
@ -841,7 +841,7 @@ async def translate_aio_errors(
except BaseException as _trio_err: except BaseException as _trio_err:
trio_err = chan._trio_err = _trio_err trio_err = chan._trio_err = _trio_err
# await tractor.pause(shield=True) # workx! # await tractor.pause(shield=True) # workx!
entered: bool = await _debug._maybe_enter_pm( entered: bool = await debug._maybe_enter_pm(
trio_err, trio_err,
api_frame=inspect.currentframe(), api_frame=inspect.currentframe(),
) )
@ -1406,7 +1406,7 @@ def run_as_asyncio_guest(
) )
# XXX make it obvi we know this isn't supported yet! # XXX make it obvi we know this isn't supported yet!
assert 0 assert 0
# await _debug.maybe_init_greenback( # await debug.maybe_init_greenback(
# force_reload=True, # force_reload=True,
# ) # )