From 8731ab3134832efd3d477ccb791c18edf688a478 Mon Sep 17 00:00:00 2001
From: Tyler Goodlet <jgbt@protonmail.com>
Date: Fri, 22 Mar 2024 16:41:49 -0400
Subject: [PATCH] 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.
---
 examples/debugging/sync_bp.py |  20 ++-
 tests/test_debugger.py        |  64 +++++++
 tractor/_root.py              |  18 +-
 tractor/_rpc.py               |  26 +--
 tractor/devx/_debug.py        | 330 +++++++++++++++++++++++-----------
 5 files changed, 322 insertions(+), 136 deletions(-)

diff --git a/examples/debugging/sync_bp.py b/examples/debugging/sync_bp.py
index 49f4d9aa..23469d6c 100644
--- a/examples/debugging/sync_bp.py
+++ b/examples/debugging/sync_bp.py
@@ -2,8 +2,18 @@ import trio
 import tractor
 
 
-def sync_pause():
-    tractor.pause_from_sync()
+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
@@ -21,16 +31,10 @@ async def start_n_sync_pause(
 
 async def main() -> None:
 
-    from tractor._rpc import maybe_import_gb
-
     async with tractor.open_nursery(
         debug_mode=True,
     ) as an:
 
-        # TODO: where to put this?
-        # => just inside `open_root_actor()` yah?
-        await maybe_import_gb()
-
         p: tractor.Portal  = await an.start_actor(
             'subactor',
             enable_modules=[__name__],
diff --git a/tests/test_debugger.py b/tests/test_debugger.py
index 3fcf71f9..0de2020d 100644
--- a/tests/test_debugger.py
+++ b/tests/test_debugger.py
@@ -1025,3 +1025,67 @@ def test_different_debug_mode_per_actor(
     # instead crashed completely
     assert "tractor._exceptions.RemoteActorError: ('crash_boi'" 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)
diff --git a/tractor/_root.py b/tractor/_root.py
index e1a7fb6c..c9faeb04 100644
--- a/tractor/_root.py
+++ b/tractor/_root.py
@@ -94,12 +94,24 @@ async def open_root_actor(
     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
     # ``trio``, see much discussion in:
     # https://github.com/python-trio/trio/issues/1155#issuecomment-742964018
-    builtin_bp_handler = sys.breakpointhook
-    orig_bp_path: str | None = os.environ.get('PYTHONBREAKPOINT', None)
-    os.environ['PYTHONBREAKPOINT'] = 'tractor.devx._debug.pause_from_sync'
+    if (
+        await _debug.maybe_init_greenback(
+            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
     # on our debugger lock state.
diff --git a/tractor/_rpc.py b/tractor/_rpc.py
index e50c80dd..b108fdda 100644
--- a/tractor/_rpc.py
+++ b/tractor/_rpc.py
@@ -26,7 +26,6 @@ from contextlib import (
 from functools import partial
 import inspect
 from pprint import pformat
-from types import ModuleType
 from typing import (
     Any,
     Callable,
@@ -332,27 +331,6 @@ async def _errors_relayed_via_ipc(
                 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(
 
     actor: Actor,
@@ -380,7 +358,9 @@ async def _invoke(
     treat_as_gen: bool = False
 
     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
     # (not sure what typing is for this..)?
diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py
index 105d2ca4..770995a2 100644
--- a/tractor/devx/_debug.py
+++ b/tractor/devx/_debug.py
@@ -33,14 +33,19 @@ from functools import (
 import os
 import signal
 import sys
+import threading
 import traceback
 from typing import (
     Any,
     Callable,
     AsyncIterator,
     AsyncGenerator,
+    TYPE_CHECKING,
+)
+from types import (
+    FrameType,
+    ModuleType,
 )
-from types import FrameType
 
 import pdbp
 import tractor
@@ -51,17 +56,22 @@ from trio import (
     # Task,
 )
 
-from ..log import get_logger
-from .._state import (
+from tractor.log import get_logger
+from tractor._state import (
     current_actor,
     is_root_process,
     debug_mode,
 )
-from .._exceptions import (
+from tractor._exceptions import (
     is_multi_cancelled,
     ContextCancelled,
 )
-from .._ipc import Channel
+from tractor._ipc import Channel
+
+if TYPE_CHECKING:
+    from tractor._runtime import (
+        Actor,
+    )
 
 log = get_logger(__name__)
 
@@ -116,10 +126,36 @@ class Lock:
 
     @classmethod
     def shield_sigint(cls):
-        cls._orig_sigint_handler = signal.signal(
-            signal.SIGINT,
-            shield_sigint_handler,
-        )
+        '''
+        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(
+                signal.SIGINT,
+                shield_sigint_handler,
+            )
 
     @classmethod
     @pdbp.hideframe  # XXX NOTE XXX see below in `.pause_from_sync()`
@@ -127,13 +163,46 @@ class Lock:
         # always restore ``trio``'s sigint handler. see notes below in
         # the pdb factory about the nightmare that is that code swapping
         # 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
 
+    @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
     def release(cls):
         try:
-            cls._debug_lock.release()
+            if not cls.is_trio_thread():
+                trio.from_thread.run_sync(
+                    cls._debug_lock.release
+                )
+            else:
+                cls._debug_lock.release()
+
         except RuntimeError:
             # uhhh makes no sense but been seeing the non-owner
             # 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')
 
 
-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()
-    # 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()
 
     # XXX: These are the important flags mentioned in
@@ -450,7 +539,7 @@ def mk_mpdb() -> tuple[MultiActorPdb, Callable]:
     pdb.allow_kbdint = True
     pdb.nosigint = True
 
-    return pdb, Lock.unshield_sigint
+    return pdb
 
 
 def shield_sigint_handler(
@@ -463,17 +552,16 @@ def shield_sigint_handler(
     '''
     Specialized, debugger-aware SIGINT handler.
 
-    In childred we always ignore to avoid deadlocks since cancellation
-    should always be managed by the parent supervising actor. The root
-    is always cancelled on ctrl-c.
+    In childred we always ignore/shield for SIGINT to avoid
+    deadlocks since cancellation should always be managed by the
+    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()
-    # print(f'{actor.uid} in HANDLER with ')
+    actor: Actor = current_actor()
 
     def do_cancel():
         # If we haven't tried to cancel the runtime then do that instead
@@ -508,7 +596,7 @@ def shield_sigint_handler(
                 return do_cancel()
 
     # 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
     # has locked debugger.
@@ -615,14 +703,20 @@ _pause_msg: str = 'Attaching to pdb REPL in actor'
 
 
 def _set_trace(
-    actor: tractor.Actor | None = None,
-    pdb: MultiActorPdb | None = None,
+    actor: tractor.Actor|None = None,
+    pdb: MultiActorPdb|None = None,
     shield: bool = False,
 
     extra_frames_up_when_async: int = 1,
+    hide_tb: bool = True,
 ):
-    __tracebackhide__: bool = True
-    actor: tractor.Actor = actor or current_actor()
+    __tracebackhide__: bool = hide_tb
+
+    actor: tractor.Actor = (
+        actor
+        or
+        current_actor()
+    )
 
     # always start 1 level up from THIS in user code.
     frame: FrameType|None
@@ -668,13 +762,8 @@ def _set_trace(
                 f'Going up frame {i} -> {frame}\n'
             )
 
-    else:
-        pdb, undo_sigint = mk_mpdb()
-
-        # we entered the global ``breakpoint()`` built-in from sync
-        # code?
-        Lock.local_task_in_debug = 'sync'
-
+    # engage ze REPL
+    # B~()
     pdb.set_trace(frame=frame)
 
 
@@ -684,7 +773,6 @@ async def _pause(
 
     # NOTE: must be passed in the `.pause_from_sync()` case!
     pdb: MultiActorPdb|None = None,
-    undo_sigint: Callable|None = None,
 
     # TODO: allow caller to pause despite task cancellation,
     # exactly the same as wrapping with:
@@ -695,6 +783,7 @@ async def _pause(
     # be no way to override it?..
     #
     shield: bool = False,
+    hide_tb: bool = True,
     task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED
 
 ) -> None:
@@ -706,9 +795,16 @@ async def _pause(
     Hopefully we won't need this in the long run.
 
     '''
-    __tracebackhide__: bool = True
-    actor = current_actor()
-    task_name: str = trio.lowlevel.current_task().name
+    __tracebackhide__: bool = hide_tb
+    actor: Actor = current_actor()
+    try:
+        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 (
         not Lock.local_pdb_complete
@@ -722,8 +818,7 @@ async def _pause(
         )
 
     if pdb is None:
-        assert undo_sigint is None, 'You must pass both!?!'
-        pdb, undo_sigint = mk_mpdb()
+        pdb: MultiActorPdb = mk_mpdb()
 
     # TODO: need a more robust check for the "root" actor
     if (
@@ -766,7 +861,6 @@ async def _pause(
         # ```
         # but not entirely sure if that's a sane way to implement it?
         try:
-            print("ACQUIRING TTY LOCK from CHILD")
             with trio.CancelScope(shield=True):
                 await actor._service_n.start(
                     wait_for_parent_stdin_hijack,
@@ -786,13 +880,11 @@ async def _pause(
             raise
 
     elif is_root_process():
-        print("ROOT TTY LOCK BRANCH")
 
         # we also wait in the root-parent for any child that
         # may have the tty locked prior
         # TODO: wait, what about multiple root tasks acquiring it though?
         if Lock.global_actor_in_debug == actor.uid:
-            print("ROOT ALREADY HAS TTY?")
             # re-entrant root process already has it: noop.
             return
 
@@ -806,14 +898,11 @@ async def _pause(
 
             # must shield here to avoid hitting a ``Cancelled`` and
             # a child getting stuck bc we clobbered the tty
-            print("ACQUIRING TTY LOCK from ROOT")
             with trio.CancelScope(shield=True):
                 await Lock._debug_lock.acquire()
         else:
             # may be cancelled
-            print("ROOT TRYING LOCK ACQUIRE")
             await Lock._debug_lock.acquire()
-            print("ROOT LOCKED TTY")
 
         Lock.global_actor_in_debug = actor.uid
         Lock.local_task_in_debug = task_name
@@ -825,7 +914,6 @@ async def _pause(
         #
         if debug_func is None:
             task_status.started(Lock)
-            print("ROOT .started(Lock) now!")
 
         else:
             # 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.
 # normally by remapping python's builtin breakpoint() hook to this
 # runtime aware version which takes care of all .
 def pause_from_sync(
-    hide_tb: bool = True
+    hide_tb: bool = False,
 ) -> None:
 
     __tracebackhide__: bool = hide_tb
     actor: tractor.Actor = current_actor(
         err_on_no_runtime=False,
     )
-    print(
+    log.debug(
         f'{actor.uid}: JUST ENTERED `tractor.pause_from_sync()`'
         f'|_{actor}\n'
     )
@@ -965,73 +1107,57 @@ def pause_from_sync(
             '- `async with tractor.open_root_actor()`\n'
         )
 
-    try:
-        import greenback
-    except ModuleNotFoundError:
-        raise RuntimeError(
-            'The `greenback` lib is required to use `tractor.pause_from_sync()`!\n'
-            'https://github.com/oremanj/greenback\n'
+    # raises on not-found by default
+    greenback: ModuleType = maybe_import_greenback()
+    mdb: MultiActorPdb = mk_mpdb()
+
+    # run async task which will lock out the root proc's TTY.
+    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_(
-    #     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_(
-        pause(
-            debug_func=None,
-            pdb=db,
-            undo_sigint=undo_sigint,
+    else:  # we are presumably the `trio.run()` + main thread
+        greenback.await_(
+            pause(
+                debug_func=None,
+                pdb=mdb,
+                hide_tb=hide_tb,
+            )
         )
-    )
-
-    Lock.local_task_in_debug = 'sync'
+        Lock.local_task_in_debug: str = current_task().name
 
     # TODO: ensure we aggressively make the user aware about
     # entering the global ``breakpoint()`` built-in from sync
     # code?
-    frame: FrameType | None = sys._getframe()
-    frame: FrameType = frame.f_back  # type: ignore
-
-    # db.config.enable_hidden_frames = True
-    # assert not db._is_hidden(frame)
-    # 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)
+    _set_trace(
+        actor=actor,
+        pdb=mdb,
+        hide_tb=hide_tb,
+        extra_frames_up_when_async=1,
 
+        # 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
     # showing up in the REPL's last stack frame !?!
     # -[ ] tried to use `@pdbp.hideframe` decoration but
     #   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):
     log.warning(
         '`tractor.breakpoint()` is deprecated!\n'