Add tractor.pause_from_sync() using the amazing greenback! #1

Open
goodboy wants to merge 3 commits from pause_from_sync_w_greenback into mv_to_new_trio_py3.11
5 changed files with 454 additions and 154 deletions

View File

@ -0,0 +1,73 @@
import trio
import tractor
def sync_pause(
use_builtin: bool = True,
error: bool = False,
):
if use_builtin:
breakpoint()
else:
tractor.pause_from_sync()
if error:
raise RuntimeError('yoyo sync code error')
@tractor.context
async def start_n_sync_pause(
ctx: tractor.Context,
):
# sync to requesting peer
await ctx.started()
actor: tractor.Actor = tractor.current_actor()
print(f'entering SYNC PAUSE in {actor.uid}')
sync_pause()
print(f'back from SYNC PAUSE in {actor.uid}')
async def main() -> None:
async with tractor.open_nursery(
debug_mode=True,
) as an:
p: tractor.Portal = await an.start_actor(
'subactor',
enable_modules=[__name__],
# infect_asyncio=True,
debug_mode=True,
loglevel='cancel',
)
# TODO: 3 sub-actor usage cases:
# -[ ] via a `.run_in_actor()` call
# -[ ] via a `.run()`
# -[ ] via a `.open_context()`
#
async with p.open_context(
start_n_sync_pause,
) as (ctx, first):
assert first is None
await tractor.pause()
sync_pause()
# TODO: make this work!!
await trio.to_thread.run_sync(
sync_pause,
abandon_on_cancel=False,
)
await ctx.cancel()
# TODO: case where we cancel from trio-side while asyncio task
# has debugger lock?
await p.cancel_actor()
if __name__ == '__main__':
trio.run(main)

View File

@ -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)

View File

@ -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
builtin_bp_handler = sys.breakpointhook if (
orig_bp_path: str | None = os.environ.get('PYTHONBREAKPOINT', None) await _debug.maybe_init_greenback(
os.environ['PYTHONBREAKPOINT'] = 'tractor.devx._debug.pause_from_sync' raise_not_found=False,
)
):
builtin_bp_handler = sys.breakpointhook
orig_bp_path: str|None = os.environ.get(
'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.

View File

@ -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..)?

View File

@ -33,35 +33,46 @@ 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 sniffio
import tractor import tractor
import trio import trio
from trio.lowlevel import current_task from trio.lowlevel import current_task
from trio_typing import ( from trio import (
TaskStatus, TaskStatus,
# 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,10 +127,36 @@ class Lock:
@classmethod @classmethod
def shield_sigint(cls): def shield_sigint(cls):
cls._orig_sigint_handler = signal.signal( '''
signal.SIGINT, Shield out SIGINT handling (which by default triggers
shield_sigint_handler, `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_main_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(
signal.SIGINT,
shield_sigint_handler,
)
@classmethod @classmethod
@pdbp.hideframe # XXX NOTE XXX see below in `.pause_from_sync()` @pdbp.hideframe # XXX NOTE XXX see below in `.pause_from_sync()`
@ -127,13 +164,52 @@ 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_main_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_main_trio_thread(cls) -> bool:
'''
Check if we're the "main" thread (as in the first one
started by cpython) AND that it is ALSO the thread that
called `trio.run()` and not some thread spawned with
`trio.to_thread.run_sync()`.
'''
return (
# TODO: since this is private, @oremanj says
# we should just copy the impl for now..
trio._util.is_main_thread()
and
sniffio.current_async_library() == 'trio'
)
# XXX apparently unreliable..see ^
# (
# threading.current_thread()
# is not threading.main_thread()
# )
@classmethod @classmethod
def release(cls): def release(cls):
try: try:
cls._debug_lock.release() if not cls.is_main_trio_thread():
trio.from_thread.run_sync(
cls._debug_lock.release
)
else:
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
@ -400,7 +476,6 @@ async def wait_for_parent_stdin_hijack(
# this syncs to child's ``Context.started()`` call. # this syncs to child's ``Context.started()`` call.
async with portal.open_context( async with portal.open_context(
lock_tty_for_child, lock_tty_for_child,
subactor_uid=actor_uid, subactor_uid=actor_uid,
@ -438,11 +513,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
@ -451,7 +546,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(
@ -464,17 +559,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: Actor = current_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
@ -509,7 +603,7 @@ def shield_sigint_handler(
return do_cancel() return do_cancel()
# only set in the actor actually running the REPL # only set in the actor actually running the REPL
pdb_obj: MultiActorPdb | None = Lock.repl pdb_obj: MultiActorPdb|None = Lock.repl
# root actor branch that reports whether or not a child # root actor branch that reports whether or not a child
# has locked debugger. # has locked debugger.
@ -616,14 +710,20 @@ _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, 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
@ -669,20 +769,17 @@ 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)
async def _pause( async def _pause(
debug_func: Callable = _set_trace, debug_func: Callable = _set_trace,
release_lock_signal: trio.Event | None = None,
# NOTE: must be passed in the `.pause_from_sync()` case!
pdb: MultiActorPdb|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:
@ -691,9 +788,9 @@ async def _pause(
# => the REMAINING ISSUE is that the scope's .__exit__() frame # => the REMAINING ISSUE is that the scope's .__exit__() frame
# is always show in the debugger on entry.. and there seems to # is always show in the debugger on entry.. and there seems to
# be no way to override it?.. # be no way to override it?..
# shield: bool = False, #
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:
@ -705,10 +802,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()
pdb, undo_sigint = mk_mpdb() 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
@ -716,9 +819,13 @@ async def _pause(
): ):
Lock.local_pdb_complete = trio.Event() Lock.local_pdb_complete = trio.Event()
debug_func = partial( if debug_func is not None:
debug_func, debug_func = partial(
) debug_func,
)
if pdb is None:
pdb: MultiActorPdb = mk_mpdb()
# TODO: need a more robust check for the "root" actor # TODO: need a more robust check for the "root" actor
if ( if (
@ -767,6 +874,7 @@ async def _pause(
actor.uid, actor.uid,
) )
Lock.repl = pdb Lock.repl = pdb
except RuntimeError: except RuntimeError:
Lock.release() Lock.release()
@ -811,32 +919,26 @@ async def _pause(
# TODO: do we want to support using this **just** for the # TODO: do we want to support using this **just** for the
# locking / common code (prolly to help address #320)? # locking / common code (prolly to help address #320)?
# #
# if debug_func is None: if debug_func is None:
# assert release_lock_signal, ( task_status.started(Lock)
# 'Must pass `release_lock_signal: trio.Event` if no '
# 'trace func provided!'
# )
# print(f"{actor.uid} ENTERING WAIT")
# with trio.CancelScope(shield=True):
# 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 sync world of the `pdb` REPL..') log.debug('Entering sync world of the `pdb` REPL..')
try: try:
debug_func( debug_func(
actor, actor,
pdb, pdb,
extra_frames_up_when_async=2, extra_frames_up_when_async=2,
shield=shield, shield=shield,
) )
except BaseException: except BaseException:
log.exception( log.exception(
'Failed to invoke internal `debug_func = ' 'Failed to invoke internal `debug_func = '
f'{debug_func.func.__name__}`\n' f'{debug_func.func.__name__}`\n'
) )
raise raise
except bdb.BdbQuit: except bdb.BdbQuit:
Lock.release() Lock.release()
@ -862,8 +964,7 @@ async def _pause(
async def pause( async def pause(
debug_func: Callable = _set_trace, debug_func: Callable|None = _set_trace,
release_lock_signal: trio.Event | 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:
@ -872,10 +973,11 @@ async def pause(
# => the REMAINING ISSUE is that the scope's .__exit__() frame # => the REMAINING ISSUE is that the scope's .__exit__() frame
# is always show in the debugger on entry.. and there seems to # is always show in the debugger on entry.. and there seems to
# be no way to override it?.. # be no way to override it?..
# shield: bool = False, #
shield: bool = False, shield: bool = False,
task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED,
**_pause_kwargs,
) -> None: ) -> None:
''' '''
@ -920,89 +1022,158 @@ async def pause(
task_status.started(cs) task_status.started(cs)
return await _pause( return await _pause(
debug_func=debug_func, debug_func=debug_func,
release_lock_signal=release_lock_signal,
shield=True, shield=True,
task_status=task_status, task_status=task_status,
**_pause_kwargs
) )
else: else:
return await _pause( return await _pause(
debug_func=debug_func, debug_func=debug_func,
release_lock_signal=release_lock_signal,
shield=False, shield=False,
task_status=task_status, task_status=task_status,
**_pause_kwargs
) )
_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() -> None: def pause_from_sync(
print("ENTER SYNC PAUSE") hide_tb: bool = False,
) -> None:
__tracebackhide__: bool = hide_tb
actor: tractor.Actor = current_actor( actor: tractor.Actor = current_actor(
err_on_no_runtime=False, err_on_no_runtime=False,
) )
if actor: log.debug(
try: f'{actor.uid}: JUST ENTERED `tractor.pause_from_sync()`'
import greenback f'|_{actor}\n'
# __tracebackhide__ = True )
if not actor:
raise RuntimeError(
'Not inside the `tractor`-runtime?\n'
'`tractor.pause_from_sync()` is not functional without a wrapping\n'
'- `async with tractor.open_nursery()` or,\n'
'- `async with tractor.open_root_actor()`\n'
)
# raises on not-found by default
greenback: ModuleType = maybe_import_greenback()
mdb: MultiActorPdb = mk_mpdb()
# task_can_release_tty_lock = trio.Event() # run async task which will lock out the root proc's TTY.
if not Lock.is_main_trio_thread():
# spawn bg task which will lock out the TTY, we poll # TODO: we could also check for a non-`.to_thread` context
# just below until the release event is reporting that task as # using `trio.from_thread.check_cancelled()` (says
# waiting.. not the most ideal but works for now ;) # oremanj) wherein we get the following outputs:
greenback.await_( #
actor._service_n.start(partial( # `RuntimeError`: non-`.to_thread` spawned thread
pause, # noop: non-cancelled `.to_thread`
debug_func=None, # `trio.Cancelled`: cancelled `.to_thread`
# release_lock_signal=task_can_release_tty_lock, #
)) 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)
except ModuleNotFoundError: else: # we are presumably the `trio.run()` + main thread
log.warning('NO GREENBACK FOUND') greenback.await_(
else: pause(
log.warning('Not inside actor-runtime') debug_func=None,
pdb=mdb,
hide_tb=hide_tb,
)
)
Lock.local_task_in_debug: str = current_task().name
db, undo_sigint = mk_mpdb() # TODO: ensure we aggressively make the user aware about
Lock.local_task_in_debug = 'sync' # entering the global ``breakpoint()`` built-in from sync
# db.config.enable_hidden_frames = True
# we entered the global ``breakpoint()`` built-in from sync
# code? # code?
frame: FrameType | None = sys._getframe() _set_trace(
# print(f'FRAME: {str(frame)}') actor=actor,
# assert not db._is_hidden(frame) pdb=mdb,
hide_tb=hide_tb,
extra_frames_up_when_async=1,
frame: FrameType = frame.f_back # type: ignore # TODO? will we ever need it?
# print(f'FRAME: {str(frame)}') # -> the gb._await() won't be affected by cancellation?
# if not db._is_hidden(frame): # shield=shield,
# pdbp.set_trace() )
# db._hidden_frames.append( # LEGACY NOTE on next LOC's frame showing weirdness..
# (frame, frame.f_lineno) #
# ) # XXX NOTE XXX no other LOC can be here without it
db.set_trace(frame=frame) # showing up in the REPL's last stack frame !?!
# NOTE XXX: see the `@pdbp.hideframe` decoration # -[ ] tried to use `@pdbp.hideframe` decoration but
# on `Lock.unshield_sigint()`.. I have NO CLUE why # still doesn't work
# 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?
# undo_sigint()
# Lock.global_actor_in_debug = actor.uid
# Lock.release()
# task_can_release_tty_lock.set()
# 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'