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 | ||||
| 
 | ||||
| 
 | ||||
| def sync_pause(): | ||||
| 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( | ||||
|  | @ -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__], | ||||
|  |  | |||
|  | @ -1027,3 +1027,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) | ||||
|  |  | |||
|  | @ -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 | ||||
|     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' | ||||
|         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. | ||||
|  |  | |||
|  | @ -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, | ||||
|  | @ -337,27 +336,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, | ||||
|  | @ -385,7 +363,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..)? | ||||
|  |  | |||
|  | @ -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,6 +126,32 @@ class Lock: | |||
| 
 | ||||
|     @classmethod | ||||
|     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( | ||||
|                 signal.SIGINT, | ||||
|                 shield_sigint_handler, | ||||
|  | @ -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: | ||||
|             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() | ||||
|     __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() | ||||
|     else:  # we are presumably the `trio.run()` + main thread | ||||
|         greenback.await_( | ||||
|             pause( | ||||
|                 debug_func=None, | ||||
|             pdb=db, | ||||
|             undo_sigint=undo_sigint, | ||||
|                 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' | ||||
|  |  | |||
		Loading…
	
		Reference in New Issue