forked from goodboy/tractor
Refine and test `tractor.pause_from_sync()`
Now supports use from any `trio` task, any sync thread started with `trio.to_thread.run_sync()` AND also via `breakpoint()` builtin API! The only bit missing now is support for `asyncio` tasks when in infected mode.. Bo `greenback` setup/API adjustments: - move `._rpc.maybe_import_gb()` to -> `devx._debug` and factor out the cached import checking into a sync func whilst placing the async `.ensure_portal()` bootstrapping into a new async `maybe_init_greenback()`. - use the new init-er func inside `open_root_actor()` with the output predicating whether we override the `breakpoint()` hook. core `devx._debug` implementation deatz: - make `mk_mpdb()` only return the `pdp.Pdb` subtype instance since the sigint unshielding func is now accessible from the `Lock` singleton from anywhere. - add non-main thread support (at least for `trio.to_thread` use cases) to our `Lock` with a new `.is_trio_thread()` predicate that delegates directly to `trio`'s internal version. - do `Lock.is_trio_thread()` checks inside any methods which require special provisions when invoked from a non-main `trio` thread: - `.[un]shield_sigint()` methods since `signal.signal` usage is only allowed from cpython's main thread. - `.release()` since `trio.StrictFIFOLock` can only be called from a `trio` task. - rework `.pause_from_sync()` itself to directly call `._set_trace()` and don't bother with `greenback._await()` when we're already calling it from a `.to_thread.run_sync()` thread, oh and try to use the thread/task name when setting `Lock.local_task_in_debug`. - make it an RTE for now if you try to use `.pause_from_sync()` from any infected-`asyncio` task, but support is (hopefully) coming soon! For testing we add a new `test_debugger.py::test_pause_from_sync()` which includes a ctrl-c parametrization around the `examples/debugging/sync_bp.py` script which includes all currently supported/working usages: - `tractor.pause_from_sync()`. - via `breakpoint()` overload. - from a `trio.to_thread.run_sync()` spawn.pause_from_sync_w_greenback
parent
c04d77a3c9
commit
4f863a6989
|
@ -2,9 +2,19 @@ import trio
|
||||||
import tractor
|
import tractor
|
||||||
|
|
||||||
|
|
||||||
def sync_pause():
|
def sync_pause(
|
||||||
|
use_builtin: bool = True,
|
||||||
|
error: bool = False,
|
||||||
|
):
|
||||||
|
if use_builtin:
|
||||||
|
breakpoint()
|
||||||
|
|
||||||
|
else:
|
||||||
tractor.pause_from_sync()
|
tractor.pause_from_sync()
|
||||||
|
|
||||||
|
if error:
|
||||||
|
raise RuntimeError('yoyo sync code error')
|
||||||
|
|
||||||
|
|
||||||
@tractor.context
|
@tractor.context
|
||||||
async def start_n_sync_pause(
|
async def start_n_sync_pause(
|
||||||
|
@ -21,16 +31,10 @@ async def start_n_sync_pause(
|
||||||
|
|
||||||
async def main() -> None:
|
async def main() -> None:
|
||||||
|
|
||||||
from tractor._rpc import maybe_import_gb
|
|
||||||
|
|
||||||
async with tractor.open_nursery(
|
async with tractor.open_nursery(
|
||||||
debug_mode=True,
|
debug_mode=True,
|
||||||
) as an:
|
) as an:
|
||||||
|
|
||||||
# TODO: where to put this?
|
|
||||||
# => just inside `open_root_actor()` yah?
|
|
||||||
await maybe_import_gb()
|
|
||||||
|
|
||||||
p: tractor.Portal = await an.start_actor(
|
p: tractor.Portal = await an.start_actor(
|
||||||
'subactor',
|
'subactor',
|
||||||
enable_modules=[__name__],
|
enable_modules=[__name__],
|
||||||
|
|
|
@ -1027,3 +1027,67 @@ def test_different_debug_mode_per_actor(
|
||||||
# instead crashed completely
|
# instead crashed completely
|
||||||
assert "tractor._exceptions.RemoteActorError: ('crash_boi'" in before
|
assert "tractor._exceptions.RemoteActorError: ('crash_boi'" in before
|
||||||
assert "RuntimeError" in before
|
assert "RuntimeError" in before
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
def test_pause_from_sync(
|
||||||
|
spawn,
|
||||||
|
ctlc: bool
|
||||||
|
):
|
||||||
|
'''
|
||||||
|
Verify we can use the `pdbp` REPL from sync functions AND from
|
||||||
|
any thread spawned with `trio.to_thread.run_sync()`.
|
||||||
|
|
||||||
|
`examples/debugging/sync_bp.py`
|
||||||
|
|
||||||
|
'''
|
||||||
|
child = spawn('sync_bp')
|
||||||
|
child.expect(PROMPT)
|
||||||
|
assert_before(
|
||||||
|
child,
|
||||||
|
[
|
||||||
|
'`greenback` portal opened!',
|
||||||
|
# pre-prompt line
|
||||||
|
_pause_msg, "('root'",
|
||||||
|
]
|
||||||
|
)
|
||||||
|
if ctlc:
|
||||||
|
do_ctlc(child)
|
||||||
|
child.sendline('c')
|
||||||
|
child.expect(PROMPT)
|
||||||
|
|
||||||
|
# XXX shouldn't see gb loaded again
|
||||||
|
before = str(child.before.decode())
|
||||||
|
assert not in_prompt_msg(
|
||||||
|
before,
|
||||||
|
['`greenback` portal opened!'],
|
||||||
|
)
|
||||||
|
assert_before(
|
||||||
|
child,
|
||||||
|
[_pause_msg, "('root'",],
|
||||||
|
)
|
||||||
|
|
||||||
|
if ctlc:
|
||||||
|
do_ctlc(child)
|
||||||
|
child.sendline('c')
|
||||||
|
child.expect(PROMPT)
|
||||||
|
assert_before(
|
||||||
|
child,
|
||||||
|
[_pause_msg, "('subactor'",],
|
||||||
|
)
|
||||||
|
|
||||||
|
if ctlc:
|
||||||
|
do_ctlc(child)
|
||||||
|
child.sendline('c')
|
||||||
|
child.expect(PROMPT)
|
||||||
|
# non-main thread case
|
||||||
|
# TODO: should we agument the pre-prompt msg in this case?
|
||||||
|
assert_before(
|
||||||
|
child,
|
||||||
|
[_pause_msg, "('root'",],
|
||||||
|
)
|
||||||
|
|
||||||
|
if ctlc:
|
||||||
|
do_ctlc(child)
|
||||||
|
child.sendline('c')
|
||||||
|
child.expect(pexpect.EOF)
|
||||||
|
|
|
@ -94,12 +94,24 @@ async def open_root_actor(
|
||||||
Runtime init entry point for ``tractor``.
|
Runtime init entry point for ``tractor``.
|
||||||
|
|
||||||
'''
|
'''
|
||||||
|
# TODO: stick this in a `@cm` defined in `devx._debug`?
|
||||||
|
#
|
||||||
# Override the global debugger hook to make it play nice with
|
# Override the global debugger hook to make it play nice with
|
||||||
# ``trio``, see much discussion in:
|
# ``trio``, see much discussion in:
|
||||||
# https://github.com/python-trio/trio/issues/1155#issuecomment-742964018
|
# https://github.com/python-trio/trio/issues/1155#issuecomment-742964018
|
||||||
|
if (
|
||||||
|
await _debug.maybe_init_greenback(
|
||||||
|
raise_not_found=False,
|
||||||
|
)
|
||||||
|
):
|
||||||
builtin_bp_handler = sys.breakpointhook
|
builtin_bp_handler = sys.breakpointhook
|
||||||
orig_bp_path: str | None = os.environ.get('PYTHONBREAKPOINT', None)
|
orig_bp_path: str|None = os.environ.get(
|
||||||
os.environ['PYTHONBREAKPOINT'] = 'tractor.devx._debug.pause_from_sync'
|
'PYTHONBREAKPOINT',
|
||||||
|
None,
|
||||||
|
)
|
||||||
|
os.environ['PYTHONBREAKPOINT'] = (
|
||||||
|
'tractor.devx._debug.pause_from_sync'
|
||||||
|
)
|
||||||
|
|
||||||
# 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.
|
||||||
|
|
|
@ -26,7 +26,6 @@ from contextlib import (
|
||||||
from functools import partial
|
from functools import partial
|
||||||
import inspect
|
import inspect
|
||||||
from pprint import pformat
|
from pprint import pformat
|
||||||
from types import ModuleType
|
|
||||||
from typing import (
|
from typing import (
|
||||||
Any,
|
Any,
|
||||||
Callable,
|
Callable,
|
||||||
|
@ -337,27 +336,6 @@ async def _errors_relayed_via_ipc(
|
||||||
actor._ongoing_rpc_tasks.set()
|
actor._ongoing_rpc_tasks.set()
|
||||||
|
|
||||||
|
|
||||||
_gb_mod: ModuleType|None|False = None
|
|
||||||
|
|
||||||
|
|
||||||
async def maybe_import_gb():
|
|
||||||
global _gb_mod
|
|
||||||
if _gb_mod is False:
|
|
||||||
return
|
|
||||||
|
|
||||||
try:
|
|
||||||
import greenback
|
|
||||||
_gb_mod = greenback
|
|
||||||
await greenback.ensure_portal()
|
|
||||||
|
|
||||||
except ModuleNotFoundError:
|
|
||||||
log.debug(
|
|
||||||
'`greenback` is not installed.\n'
|
|
||||||
'No sync debug support!\n'
|
|
||||||
)
|
|
||||||
_gb_mod = False
|
|
||||||
|
|
||||||
|
|
||||||
async def _invoke(
|
async def _invoke(
|
||||||
|
|
||||||
actor: Actor,
|
actor: Actor,
|
||||||
|
@ -385,7 +363,9 @@ async def _invoke(
|
||||||
treat_as_gen: bool = False
|
treat_as_gen: bool = False
|
||||||
|
|
||||||
if _state.debug_mode():
|
if _state.debug_mode():
|
||||||
await maybe_import_gb()
|
# XXX for .pause_from_sync()` usage we need to make sure
|
||||||
|
# `greenback` is boostrapped in the subactor!
|
||||||
|
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..)?
|
||||||
|
|
|
@ -33,14 +33,19 @@ from functools import (
|
||||||
import os
|
import os
|
||||||
import signal
|
import signal
|
||||||
import sys
|
import sys
|
||||||
|
import threading
|
||||||
import traceback
|
import traceback
|
||||||
from typing import (
|
from typing import (
|
||||||
Any,
|
Any,
|
||||||
Callable,
|
Callable,
|
||||||
AsyncIterator,
|
AsyncIterator,
|
||||||
AsyncGenerator,
|
AsyncGenerator,
|
||||||
|
TYPE_CHECKING,
|
||||||
|
)
|
||||||
|
from types import (
|
||||||
|
FrameType,
|
||||||
|
ModuleType,
|
||||||
)
|
)
|
||||||
from types import FrameType
|
|
||||||
|
|
||||||
import pdbp
|
import pdbp
|
||||||
import tractor
|
import tractor
|
||||||
|
@ -51,17 +56,22 @@ from trio import (
|
||||||
# Task,
|
# Task,
|
||||||
)
|
)
|
||||||
|
|
||||||
from ..log import get_logger
|
from tractor.log import get_logger
|
||||||
from .._state import (
|
from tractor._state import (
|
||||||
current_actor,
|
current_actor,
|
||||||
is_root_process,
|
is_root_process,
|
||||||
debug_mode,
|
debug_mode,
|
||||||
)
|
)
|
||||||
from .._exceptions import (
|
from tractor._exceptions import (
|
||||||
is_multi_cancelled,
|
is_multi_cancelled,
|
||||||
ContextCancelled,
|
ContextCancelled,
|
||||||
)
|
)
|
||||||
from .._ipc import Channel
|
from tractor._ipc import Channel
|
||||||
|
|
||||||
|
if TYPE_CHECKING:
|
||||||
|
from tractor._runtime import (
|
||||||
|
Actor,
|
||||||
|
)
|
||||||
|
|
||||||
log = get_logger(__name__)
|
log = get_logger(__name__)
|
||||||
|
|
||||||
|
@ -116,6 +126,32 @@ class Lock:
|
||||||
|
|
||||||
@classmethod
|
@classmethod
|
||||||
def shield_sigint(cls):
|
def shield_sigint(cls):
|
||||||
|
'''
|
||||||
|
Shield out SIGINT handling (which by default triggers
|
||||||
|
`trio.Task` cancellation) in subactors when the `pdb` REPL
|
||||||
|
is active.
|
||||||
|
|
||||||
|
Avoids cancellation of the current actor (task) when the
|
||||||
|
user mistakenly sends ctl-c or a signal is received from
|
||||||
|
an external request; explicit runtime cancel requests are
|
||||||
|
allowed until the use exits the REPL session using
|
||||||
|
'continue' or 'quit', at which point the orig SIGINT
|
||||||
|
handler is restored.
|
||||||
|
|
||||||
|
'''
|
||||||
|
#
|
||||||
|
# XXX detect whether we're running from a non-main thread
|
||||||
|
# in which case schedule the SIGINT shielding override
|
||||||
|
# to in the main thread.
|
||||||
|
# https://docs.python.org/3/library/signal.html#signals-and-threads
|
||||||
|
if not cls.is_trio_thread():
|
||||||
|
cls._orig_sigint_handler: Callable = trio.from_thread.run_sync(
|
||||||
|
signal.signal,
|
||||||
|
signal.SIGINT,
|
||||||
|
shield_sigint_handler,
|
||||||
|
)
|
||||||
|
|
||||||
|
else:
|
||||||
cls._orig_sigint_handler = signal.signal(
|
cls._orig_sigint_handler = signal.signal(
|
||||||
signal.SIGINT,
|
signal.SIGINT,
|
||||||
shield_sigint_handler,
|
shield_sigint_handler,
|
||||||
|
@ -127,13 +163,46 @@ class Lock:
|
||||||
# always restore ``trio``'s sigint handler. see notes below in
|
# always restore ``trio``'s sigint handler. see notes below in
|
||||||
# the pdb factory about the nightmare that is that code swapping
|
# the pdb factory about the nightmare that is that code swapping
|
||||||
# out the handler when the repl activates...
|
# out the handler when the repl activates...
|
||||||
signal.signal(signal.SIGINT, cls._trio_handler)
|
if not cls.is_trio_thread():
|
||||||
|
trio.from_thread.run_sync(
|
||||||
|
signal.signal,
|
||||||
|
signal.SIGINT,
|
||||||
|
cls._trio_handler,
|
||||||
|
)
|
||||||
|
else:
|
||||||
|
signal.signal(
|
||||||
|
signal.SIGINT,
|
||||||
|
cls._trio_handler,
|
||||||
|
)
|
||||||
|
|
||||||
cls._orig_sigint_handler = None
|
cls._orig_sigint_handler = None
|
||||||
|
|
||||||
|
@classmethod
|
||||||
|
def is_trio_thread(self) -> bool:
|
||||||
|
'''
|
||||||
|
Check if we're the "main" thread (as in the first one
|
||||||
|
started by cpython) and presume that it is the thread that
|
||||||
|
called `trio.run()` and not some thread spawned with
|
||||||
|
`trio.to_thread.run_sync()`.
|
||||||
|
|
||||||
|
'''
|
||||||
|
return trio._util.is_main_thread()
|
||||||
|
# XXX apparently unreliable..see ^
|
||||||
|
# (
|
||||||
|
# threading.current_thread()
|
||||||
|
# is not threading.main_thread()
|
||||||
|
# )
|
||||||
|
|
||||||
@classmethod
|
@classmethod
|
||||||
def release(cls):
|
def release(cls):
|
||||||
try:
|
try:
|
||||||
|
if not cls.is_trio_thread():
|
||||||
|
trio.from_thread.run_sync(
|
||||||
|
cls._debug_lock.release
|
||||||
|
)
|
||||||
|
else:
|
||||||
cls._debug_lock.release()
|
cls._debug_lock.release()
|
||||||
|
|
||||||
except RuntimeError:
|
except RuntimeError:
|
||||||
# uhhh makes no sense but been seeing the non-owner
|
# uhhh makes no sense but been seeing the non-owner
|
||||||
# release error even though this is definitely the task
|
# release error even though this is definitely the task
|
||||||
|
@ -437,11 +506,31 @@ async def wait_for_parent_stdin_hijack(
|
||||||
log.debug('Exiting debugger from child')
|
log.debug('Exiting debugger from child')
|
||||||
|
|
||||||
|
|
||||||
def mk_mpdb() -> tuple[MultiActorPdb, Callable]:
|
def mk_mpdb() -> MultiActorPdb:
|
||||||
|
'''
|
||||||
|
Deliver a new `MultiActorPdb`: a multi-process safe `pdbp`
|
||||||
|
REPL using the magic of SC!
|
||||||
|
|
||||||
|
Our `pdb.Pdb` subtype accomplishes multi-process safe debugging
|
||||||
|
by:
|
||||||
|
|
||||||
|
- mutexing access to the root process' TTY & stdstreams
|
||||||
|
via an IPC managed `Lock` singleton per process tree.
|
||||||
|
|
||||||
|
- temporarily overriding any subactor's SIGINT handler to shield during
|
||||||
|
live REPL sessions in sub-actors such that cancellation is
|
||||||
|
never (mistakenly) triggered by a ctrl-c and instead only
|
||||||
|
by either explicit requests in the runtime or
|
||||||
|
|
||||||
|
'''
|
||||||
pdb = MultiActorPdb()
|
pdb = MultiActorPdb()
|
||||||
# signal.signal = pdbp.hideframe(signal.signal)
|
|
||||||
|
|
||||||
|
# Always shield out SIGINTs for subactors when REPL is active.
|
||||||
|
#
|
||||||
|
# XXX detect whether we're running from a non-main thread
|
||||||
|
# in which case schedule the SIGINT shielding override
|
||||||
|
# to in the main thread.
|
||||||
|
# https://docs.python.org/3/library/signal.html#signals-and-threads
|
||||||
Lock.shield_sigint()
|
Lock.shield_sigint()
|
||||||
|
|
||||||
# XXX: These are the important flags mentioned in
|
# XXX: These are the important flags mentioned in
|
||||||
|
@ -450,7 +539,7 @@ def mk_mpdb() -> tuple[MultiActorPdb, Callable]:
|
||||||
pdb.allow_kbdint = True
|
pdb.allow_kbdint = True
|
||||||
pdb.nosigint = True
|
pdb.nosigint = True
|
||||||
|
|
||||||
return pdb, Lock.unshield_sigint
|
return pdb
|
||||||
|
|
||||||
|
|
||||||
def shield_sigint_handler(
|
def shield_sigint_handler(
|
||||||
|
@ -463,17 +552,16 @@ def shield_sigint_handler(
|
||||||
'''
|
'''
|
||||||
Specialized, debugger-aware SIGINT handler.
|
Specialized, debugger-aware SIGINT handler.
|
||||||
|
|
||||||
In childred we always ignore to avoid deadlocks since cancellation
|
In childred we always ignore/shield for SIGINT to avoid
|
||||||
should always be managed by the parent supervising actor. The root
|
deadlocks since cancellation should always be managed by the
|
||||||
is always cancelled on ctrl-c.
|
supervising parent actor. The root actor-proces is always
|
||||||
|
cancelled on ctrl-c.
|
||||||
|
|
||||||
'''
|
'''
|
||||||
__tracebackhide__ = True
|
__tracebackhide__: bool = True
|
||||||
|
|
||||||
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 = current_actor()
|
actor: Actor = current_actor()
|
||||||
# print(f'{actor.uid} in HANDLER with ')
|
|
||||||
|
|
||||||
def do_cancel():
|
def do_cancel():
|
||||||
# If we haven't tried to cancel the runtime then do that instead
|
# If we haven't tried to cancel the runtime then do that instead
|
||||||
|
@ -620,9 +708,15 @@ def _set_trace(
|
||||||
shield: bool = False,
|
shield: bool = False,
|
||||||
|
|
||||||
extra_frames_up_when_async: int = 1,
|
extra_frames_up_when_async: int = 1,
|
||||||
|
hide_tb: bool = True,
|
||||||
):
|
):
|
||||||
__tracebackhide__: bool = True
|
__tracebackhide__: bool = hide_tb
|
||||||
actor: tractor.Actor = actor or current_actor()
|
|
||||||
|
actor: tractor.Actor = (
|
||||||
|
actor
|
||||||
|
or
|
||||||
|
current_actor()
|
||||||
|
)
|
||||||
|
|
||||||
# always start 1 level up from THIS in user code.
|
# always start 1 level up from THIS in user code.
|
||||||
frame: FrameType|None
|
frame: FrameType|None
|
||||||
|
@ -668,13 +762,8 @@ def _set_trace(
|
||||||
f'Going up frame {i} -> {frame}\n'
|
f'Going up frame {i} -> {frame}\n'
|
||||||
)
|
)
|
||||||
|
|
||||||
else:
|
# engage ze REPL
|
||||||
pdb, undo_sigint = mk_mpdb()
|
# B~()
|
||||||
|
|
||||||
# we entered the global ``breakpoint()`` built-in from sync
|
|
||||||
# code?
|
|
||||||
Lock.local_task_in_debug = 'sync'
|
|
||||||
|
|
||||||
pdb.set_trace(frame=frame)
|
pdb.set_trace(frame=frame)
|
||||||
|
|
||||||
|
|
||||||
|
@ -684,7 +773,6 @@ async def _pause(
|
||||||
|
|
||||||
# NOTE: must be passed in the `.pause_from_sync()` case!
|
# NOTE: must be passed in the `.pause_from_sync()` case!
|
||||||
pdb: MultiActorPdb|None = None,
|
pdb: MultiActorPdb|None = None,
|
||||||
undo_sigint: Callable|None = None,
|
|
||||||
|
|
||||||
# TODO: allow caller to pause despite task cancellation,
|
# TODO: allow caller to pause despite task cancellation,
|
||||||
# exactly the same as wrapping with:
|
# exactly the same as wrapping with:
|
||||||
|
@ -695,6 +783,7 @@ async def _pause(
|
||||||
# be no way to override it?..
|
# be no way to override it?..
|
||||||
#
|
#
|
||||||
shield: bool = False,
|
shield: bool = False,
|
||||||
|
hide_tb: bool = True,
|
||||||
task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED
|
task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED
|
||||||
|
|
||||||
) -> None:
|
) -> None:
|
||||||
|
@ -706,9 +795,16 @@ async def _pause(
|
||||||
Hopefully we won't need this in the long run.
|
Hopefully we won't need this in the long run.
|
||||||
|
|
||||||
'''
|
'''
|
||||||
__tracebackhide__: bool = True
|
__tracebackhide__: bool = hide_tb
|
||||||
actor = current_actor()
|
actor: Actor = current_actor()
|
||||||
|
try:
|
||||||
task_name: str = trio.lowlevel.current_task().name
|
task_name: str = trio.lowlevel.current_task().name
|
||||||
|
except RuntimeError as rte:
|
||||||
|
if actor.is_infected_aio():
|
||||||
|
raise RuntimeError(
|
||||||
|
'`tractor.pause[_from_sync]()` not yet supported '
|
||||||
|
'for infected `asyncio` mode!'
|
||||||
|
) from rte
|
||||||
|
|
||||||
if (
|
if (
|
||||||
not Lock.local_pdb_complete
|
not Lock.local_pdb_complete
|
||||||
|
@ -722,8 +818,7 @@ async def _pause(
|
||||||
)
|
)
|
||||||
|
|
||||||
if pdb is None:
|
if pdb is None:
|
||||||
assert undo_sigint is None, 'You must pass both!?!'
|
pdb: MultiActorPdb = mk_mpdb()
|
||||||
pdb, undo_sigint = mk_mpdb()
|
|
||||||
|
|
||||||
# TODO: need a more robust check for the "root" actor
|
# TODO: need a more robust check for the "root" actor
|
||||||
if (
|
if (
|
||||||
|
@ -766,7 +861,6 @@ async def _pause(
|
||||||
# ```
|
# ```
|
||||||
# but not entirely sure if that's a sane way to implement it?
|
# but not entirely sure if that's a sane way to implement it?
|
||||||
try:
|
try:
|
||||||
print("ACQUIRING TTY LOCK from CHILD")
|
|
||||||
with trio.CancelScope(shield=True):
|
with trio.CancelScope(shield=True):
|
||||||
await actor._service_n.start(
|
await actor._service_n.start(
|
||||||
wait_for_parent_stdin_hijack,
|
wait_for_parent_stdin_hijack,
|
||||||
|
@ -786,13 +880,11 @@ async def _pause(
|
||||||
raise
|
raise
|
||||||
|
|
||||||
elif is_root_process():
|
elif is_root_process():
|
||||||
print("ROOT TTY LOCK BRANCH")
|
|
||||||
|
|
||||||
# we also wait in the root-parent for any child that
|
# we also wait in the root-parent for any child that
|
||||||
# may have the tty locked prior
|
# may have the tty locked prior
|
||||||
# TODO: wait, what about multiple root tasks acquiring it though?
|
# TODO: wait, what about multiple root tasks acquiring it though?
|
||||||
if Lock.global_actor_in_debug == actor.uid:
|
if Lock.global_actor_in_debug == actor.uid:
|
||||||
print("ROOT ALREADY HAS TTY?")
|
|
||||||
# re-entrant root process already has it: noop.
|
# re-entrant root process already has it: noop.
|
||||||
return
|
return
|
||||||
|
|
||||||
|
@ -806,14 +898,11 @@ async def _pause(
|
||||||
|
|
||||||
# must shield here to avoid hitting a ``Cancelled`` and
|
# must shield here to avoid hitting a ``Cancelled`` and
|
||||||
# a child getting stuck bc we clobbered the tty
|
# a child getting stuck bc we clobbered the tty
|
||||||
print("ACQUIRING TTY LOCK from ROOT")
|
|
||||||
with trio.CancelScope(shield=True):
|
with trio.CancelScope(shield=True):
|
||||||
await Lock._debug_lock.acquire()
|
await Lock._debug_lock.acquire()
|
||||||
else:
|
else:
|
||||||
# may be cancelled
|
# may be cancelled
|
||||||
print("ROOT TRYING LOCK ACQUIRE")
|
|
||||||
await Lock._debug_lock.acquire()
|
await Lock._debug_lock.acquire()
|
||||||
print("ROOT LOCKED TTY")
|
|
||||||
|
|
||||||
Lock.global_actor_in_debug = actor.uid
|
Lock.global_actor_in_debug = actor.uid
|
||||||
Lock.local_task_in_debug = task_name
|
Lock.local_task_in_debug = task_name
|
||||||
|
@ -825,7 +914,6 @@ async def _pause(
|
||||||
#
|
#
|
||||||
if debug_func is None:
|
if debug_func is None:
|
||||||
task_status.started(Lock)
|
task_status.started(Lock)
|
||||||
print("ROOT .started(Lock) now!")
|
|
||||||
|
|
||||||
else:
|
else:
|
||||||
# block here one (at the appropriate frame *up*) where
|
# block here one (at the appropriate frame *up*) where
|
||||||
|
@ -940,20 +1028,74 @@ async def pause(
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
||||||
|
_gb_mod: None|ModuleType|False = None
|
||||||
|
|
||||||
|
|
||||||
|
def maybe_import_greenback(
|
||||||
|
raise_not_found: bool = True,
|
||||||
|
force_reload: bool = False,
|
||||||
|
|
||||||
|
) -> ModuleType|False:
|
||||||
|
# be cached-fast on module-already-inited
|
||||||
|
global _gb_mod
|
||||||
|
|
||||||
|
if _gb_mod is False:
|
||||||
|
return False
|
||||||
|
|
||||||
|
elif (
|
||||||
|
_gb_mod is not None
|
||||||
|
and not force_reload
|
||||||
|
):
|
||||||
|
return _gb_mod
|
||||||
|
|
||||||
|
try:
|
||||||
|
import greenback
|
||||||
|
_gb_mod = greenback
|
||||||
|
return greenback
|
||||||
|
|
||||||
|
except ModuleNotFoundError as mnf:
|
||||||
|
log.debug(
|
||||||
|
'`greenback` is not installed.\n'
|
||||||
|
'No sync debug support!\n'
|
||||||
|
)
|
||||||
|
_gb_mod = False
|
||||||
|
|
||||||
|
if raise_not_found:
|
||||||
|
raise RuntimeError(
|
||||||
|
'The `greenback` lib is required to use `tractor.pause_from_sync()`!\n'
|
||||||
|
'https://github.com/oremanj/greenback\n'
|
||||||
|
) from mnf
|
||||||
|
|
||||||
|
return False
|
||||||
|
|
||||||
|
|
||||||
|
async def maybe_init_greenback(
|
||||||
|
**kwargs,
|
||||||
|
) -> None|ModuleType:
|
||||||
|
|
||||||
|
if mod := maybe_import_greenback(**kwargs):
|
||||||
|
await mod.ensure_portal()
|
||||||
|
log.info(
|
||||||
|
'`greenback` portal opened!\n'
|
||||||
|
'Sync debug support activated!\n'
|
||||||
|
)
|
||||||
|
return mod
|
||||||
|
|
||||||
|
return None
|
||||||
|
|
||||||
|
|
||||||
# TODO: allow pausing from sync code.
|
# TODO: allow pausing from sync code.
|
||||||
# normally by remapping python's builtin breakpoint() hook to this
|
# normally by remapping python's builtin breakpoint() hook to this
|
||||||
# runtime aware version which takes care of all .
|
# runtime aware version which takes care of all .
|
||||||
def pause_from_sync(
|
def pause_from_sync(
|
||||||
hide_tb: bool = True
|
hide_tb: bool = False,
|
||||||
) -> None:
|
) -> None:
|
||||||
|
|
||||||
__tracebackhide__: bool = hide_tb
|
__tracebackhide__: bool = hide_tb
|
||||||
actor: tractor.Actor = current_actor(
|
actor: tractor.Actor = current_actor(
|
||||||
err_on_no_runtime=False,
|
err_on_no_runtime=False,
|
||||||
)
|
)
|
||||||
print(
|
log.debug(
|
||||||
f'{actor.uid}: JUST ENTERED `tractor.pause_from_sync()`'
|
f'{actor.uid}: JUST ENTERED `tractor.pause_from_sync()`'
|
||||||
f'|_{actor}\n'
|
f'|_{actor}\n'
|
||||||
)
|
)
|
||||||
|
@ -965,73 +1107,57 @@ def pause_from_sync(
|
||||||
'- `async with tractor.open_root_actor()`\n'
|
'- `async with tractor.open_root_actor()`\n'
|
||||||
)
|
)
|
||||||
|
|
||||||
try:
|
# raises on not-found by default
|
||||||
import greenback
|
greenback: ModuleType = maybe_import_greenback()
|
||||||
except ModuleNotFoundError:
|
mdb: MultiActorPdb = mk_mpdb()
|
||||||
raise RuntimeError(
|
|
||||||
'The `greenback` lib is required to use `tractor.pause_from_sync()`!\n'
|
# run async task which will lock out the root proc's TTY.
|
||||||
'https://github.com/oremanj/greenback\n'
|
if not Lock.is_trio_thread():
|
||||||
|
trio.from_thread.run(
|
||||||
|
partial(
|
||||||
|
pause,
|
||||||
|
debug_func=None,
|
||||||
|
pdb=mdb,
|
||||||
|
hide_tb=hide_tb,
|
||||||
)
|
)
|
||||||
|
)
|
||||||
|
# TODO: maybe the `trio.current_task()` id/name if avail?
|
||||||
|
Lock.local_task_in_debug: str = str(threading.current_thread().name)
|
||||||
|
|
||||||
# out = greenback.await_(
|
else: # we are presumably the `trio.run()` + main thread
|
||||||
# actor._service_n.start(partial(
|
|
||||||
# pause,
|
|
||||||
# debug_func=None,
|
|
||||||
# release_lock_signal=task_can_release_tty_lock,
|
|
||||||
# ))
|
|
||||||
# )
|
|
||||||
|
|
||||||
# spawn bg task which will lock out the TTY, we poll
|
|
||||||
# just below until the release event is reporting that task as
|
|
||||||
# waiting.. not the most ideal but works for now ;)
|
|
||||||
db, undo_sigint = mk_mpdb()
|
|
||||||
greenback.await_(
|
greenback.await_(
|
||||||
pause(
|
pause(
|
||||||
debug_func=None,
|
debug_func=None,
|
||||||
pdb=db,
|
pdb=mdb,
|
||||||
undo_sigint=undo_sigint,
|
hide_tb=hide_tb,
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
Lock.local_task_in_debug: str = current_task().name
|
||||||
Lock.local_task_in_debug = 'sync'
|
|
||||||
|
|
||||||
# TODO: ensure we aggressively make the user aware about
|
# TODO: ensure we aggressively make the user aware about
|
||||||
# entering the global ``breakpoint()`` built-in from sync
|
# entering the global ``breakpoint()`` built-in from sync
|
||||||
# code?
|
# code?
|
||||||
frame: FrameType | None = sys._getframe()
|
_set_trace(
|
||||||
frame: FrameType = frame.f_back # type: ignore
|
actor=actor,
|
||||||
|
pdb=mdb,
|
||||||
# db.config.enable_hidden_frames = True
|
hide_tb=hide_tb,
|
||||||
# assert not db._is_hidden(frame)
|
extra_frames_up_when_async=1,
|
||||||
# print(f'FRAME: {str(frame)}')
|
|
||||||
# if not db._is_hidden(frame):
|
|
||||||
# pdbp.set_trace()
|
|
||||||
# db._hidden_frames.append(
|
|
||||||
# (frame, frame.f_lineno)
|
|
||||||
# )
|
|
||||||
db.set_trace(frame=frame)
|
|
||||||
|
|
||||||
|
# TODO? will we ever need it?
|
||||||
|
# -> the gb._await() won't be affected by cancellation?
|
||||||
|
# shield=shield,
|
||||||
|
)
|
||||||
|
# LEGACY NOTE on next LOC's frame showing weirdness..
|
||||||
|
#
|
||||||
# XXX NOTE XXX no other LOC can be here without it
|
# XXX NOTE XXX no other LOC can be here without it
|
||||||
# showing up in the REPL's last stack frame !?!
|
# showing up in the REPL's last stack frame !?!
|
||||||
# -[ ] tried to use `@pdbp.hideframe` decoration but
|
# -[ ] tried to use `@pdbp.hideframe` decoration but
|
||||||
# still doesn't work
|
# still doesn't work
|
||||||
#
|
|
||||||
# FROM BEFORE: on `Lock.unshield_sigint()`.. I have NO CLUE why
|
|
||||||
# the next instruction's def frame is being shown
|
|
||||||
# in the tb but it seems to be something wonky with
|
|
||||||
# the way `pdb` core works?
|
|
||||||
#
|
|
||||||
# NOTE: not needed any more anyway since it's all in
|
|
||||||
# `Lock.release()` now!
|
|
||||||
# undo_sigint()
|
|
||||||
|
|
||||||
|
|
||||||
# using the "pause" semantics instead since
|
|
||||||
# that better covers actually somewhat "pausing the runtime"
|
|
||||||
# for this particular paralell task to do debugging B)
|
|
||||||
# pp = pause # short-hand for "pause point"
|
|
||||||
|
|
||||||
|
|
||||||
|
# NOTE prefer a new "pause" semantic since it better describes
|
||||||
|
# "pausing the actor's runtime" for this particular
|
||||||
|
# paralell task to do debugging in a REPL.
|
||||||
async def breakpoint(**kwargs):
|
async def breakpoint(**kwargs):
|
||||||
log.warning(
|
log.warning(
|
||||||
'`tractor.breakpoint()` is deprecated!\n'
|
'`tractor.breakpoint()` is deprecated!\n'
|
||||||
|
|
Loading…
Reference in New Issue