Drop all the `@cm.__exit__()` override attempts..
None of it worked (you still will see `.__exit__()` frames on debugger entry - you'd think this would have been solved by now but, shrug) so instead wrap the debugger entry-point in a `try:` and put the SIGINT handler restoration inside `MultiActorPdb` teardown hooks. This seems to restore the UX as it was prior but with also giving the desired SIGINT override handler behaviour.310_plus
							parent
							
								
									9a1dadecff
								
							
						
					
					
						commit
						aa09a31d25
					
				| 
						 | 
					@ -24,16 +24,12 @@ import sys
 | 
				
			||||||
import signal
 | 
					import signal
 | 
				
			||||||
from functools import partial
 | 
					from functools import partial
 | 
				
			||||||
from contextlib import asynccontextmanager as acm
 | 
					from contextlib import asynccontextmanager as acm
 | 
				
			||||||
from contextlib import contextmanager as cm
 | 
					 | 
				
			||||||
from contextlib import _GeneratorContextManager
 | 
					 | 
				
			||||||
import contextlib
 | 
					 | 
				
			||||||
from typing import (
 | 
					from typing import (
 | 
				
			||||||
    Tuple,
 | 
					    Tuple,
 | 
				
			||||||
    Optional,
 | 
					    Optional,
 | 
				
			||||||
    Callable,
 | 
					    Callable,
 | 
				
			||||||
    AsyncIterator,
 | 
					    AsyncIterator,
 | 
				
			||||||
    AsyncGenerator,
 | 
					    AsyncGenerator,
 | 
				
			||||||
    Iterator,
 | 
					 | 
				
			||||||
)
 | 
					)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
import tractor
 | 
					import tractor
 | 
				
			||||||
| 
						 | 
					@ -58,19 +54,6 @@ except ImportError:
 | 
				
			||||||
log = get_logger(__name__)
 | 
					log = get_logger(__name__)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
class noexittbGCM(_GeneratorContextManager):
 | 
					 | 
				
			||||||
    # @pdb.hideframe
 | 
					 | 
				
			||||||
    def __exit__(self, type, value, traceback):
 | 
					 | 
				
			||||||
        __tracebackhide__ = True
 | 
					 | 
				
			||||||
        # try:
 | 
					 | 
				
			||||||
        return super().__exit__(type, value, traceback)
 | 
					 | 
				
			||||||
        # except:
 | 
					 | 
				
			||||||
        #     print('EXITED')
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
contextlib._GeneratorContextManager = noexittbGCM
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
__all__ = ['breakpoint', 'post_mortem']
 | 
					__all__ = ['breakpoint', 'post_mortem']
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
| 
						 | 
					@ -103,9 +86,11 @@ class TractorConfig(pdbpp.DefaultConfig):
 | 
				
			||||||
    enable_hidden_frames = False
 | 
					    enable_hidden_frames = False
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
class PdbwTeardown(pdbpp.Pdb):
 | 
					class MultiActorPdb(pdbpp.Pdb):
 | 
				
			||||||
    """Add teardown hooks to the regular ``pdbpp.Pdb``.
 | 
					    '''
 | 
				
			||||||
    """
 | 
					    Add teardown hooks to the regular ``pdbpp.Pdb``.
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					    '''
 | 
				
			||||||
    # override the pdbpp config with our coolio one
 | 
					    # override the pdbpp config with our coolio one
 | 
				
			||||||
    DefaultConfig = TractorConfig
 | 
					    DefaultConfig = TractorConfig
 | 
				
			||||||
 | 
					
 | 
				
			||||||
| 
						 | 
					@ -172,7 +157,8 @@ async def _acquire_debug_lock(
 | 
				
			||||||
    uid: Tuple[str, str]
 | 
					    uid: Tuple[str, str]
 | 
				
			||||||
 | 
					
 | 
				
			||||||
) -> AsyncIterator[trio.StrictFIFOLock]:
 | 
					) -> AsyncIterator[trio.StrictFIFOLock]:
 | 
				
			||||||
    '''Acquire a root-actor local FIFO lock which tracks mutex access of
 | 
					    '''
 | 
				
			||||||
 | 
					    Acquire a root-actor local FIFO lock which tracks mutex access of
 | 
				
			||||||
    the process tree's global debugger breakpoint.
 | 
					    the process tree's global debugger breakpoint.
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    This lock avoids tty clobbering (by preventing multiple processes
 | 
					    This lock avoids tty clobbering (by preventing multiple processes
 | 
				
			||||||
| 
						 | 
					@ -269,52 +255,68 @@ async def _hijack_stdin_for_child(
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    log.debug(f"Actor {subactor_uid} is WAITING on stdin hijack lock")
 | 
					    log.debug(f"Actor {subactor_uid} is WAITING on stdin hijack lock")
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    with (
 | 
					    orig_handler = signal.signal(
 | 
				
			||||||
        trio.CancelScope(shield=True),
 | 
					        signal.SIGINT,
 | 
				
			||||||
        # disable_sigint(),
 | 
					        shield_sigint,
 | 
				
			||||||
    ):
 | 
					        # partial(shield_sigint, pdb=pdb),
 | 
				
			||||||
 | 
					    )
 | 
				
			||||||
 | 
					#     try:
 | 
				
			||||||
 | 
					#         yield
 | 
				
			||||||
 | 
					    try:
 | 
				
			||||||
 | 
					        with (
 | 
				
			||||||
 | 
					            trio.CancelScope(shield=True),
 | 
				
			||||||
 | 
					            # disable_sigint(),
 | 
				
			||||||
 | 
					        ):
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        try:
 | 
					            try:
 | 
				
			||||||
            lock = None
 | 
					                lock = None
 | 
				
			||||||
            async with _acquire_debug_lock(subactor_uid) as lock:
 | 
					                async with _acquire_debug_lock(subactor_uid) as lock:
 | 
				
			||||||
 | 
					
 | 
				
			||||||
                # indicate to child that we've locked stdio
 | 
					                    # indicate to child that we've locked stdio
 | 
				
			||||||
                await ctx.started('Locked')
 | 
					                    await ctx.started('Locked')
 | 
				
			||||||
                log.debug(f"Actor {subactor_uid} acquired stdin hijack lock")
 | 
					                    log.debug(
 | 
				
			||||||
 | 
					                        f"Actor {subactor_uid} acquired stdin hijack lock"
 | 
				
			||||||
 | 
					                    )
 | 
				
			||||||
 | 
					
 | 
				
			||||||
                # wait for unlock pdb by child
 | 
					                    # wait for unlock pdb by child
 | 
				
			||||||
                async with ctx.open_stream() as stream:
 | 
					                    async with ctx.open_stream() as stream:
 | 
				
			||||||
                    assert await stream.receive() == 'pdb_unlock'
 | 
					                        assert await stream.receive() == 'pdb_unlock'
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        except (
 | 
					            except (
 | 
				
			||||||
            # BaseException,
 | 
					                # BaseException,
 | 
				
			||||||
            trio.MultiError,
 | 
					                trio.MultiError,
 | 
				
			||||||
            Exception,
 | 
					                Exception,
 | 
				
			||||||
            # trio.BrokenResourceError,
 | 
					                # trio.BrokenResourceError,
 | 
				
			||||||
            # trio.Cancelled,  # by local cancellation
 | 
					                # trio.Cancelled,  # by local cancellation
 | 
				
			||||||
            # trio.ClosedResourceError,  # by self._rx_chan
 | 
					                # trio.ClosedResourceError,  # by self._rx_chan
 | 
				
			||||||
            # ContextCancelled,
 | 
					                # ContextCancelled,
 | 
				
			||||||
            # ConnectionResetError,
 | 
					                # ConnectionResetError,
 | 
				
			||||||
        ) as err:
 | 
					            ) as err:
 | 
				
			||||||
 | 
					
 | 
				
			||||||
            # XXX: there may be a race with the portal teardown
 | 
					                # XXX: there may be a race with the portal teardown
 | 
				
			||||||
            # with the calling actor which we can safely ignore.
 | 
					                # with the calling actor which we can safely ignore.
 | 
				
			||||||
            # The alternative would be sending an ack message
 | 
					                # The alternative would be sending an ack message
 | 
				
			||||||
            # and allowing the client to wait for us to teardown
 | 
					                # and allowing the client to wait for us to teardown
 | 
				
			||||||
            # first?
 | 
					                # first?
 | 
				
			||||||
            if lock and lock.locked():
 | 
					                if lock and lock.locked():
 | 
				
			||||||
                lock.release()
 | 
					                    lock.release()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
            if isinstance(err, trio.Cancelled):
 | 
					                if isinstance(err, trio.Cancelled):
 | 
				
			||||||
                raise
 | 
					                    raise
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        finally:
 | 
					            finally:
 | 
				
			||||||
            log.runtime(
 | 
					                log.runtime(
 | 
				
			||||||
                "TTY lock released, remote task:"
 | 
					                    "TTY lock released, remote task:"
 | 
				
			||||||
                f"{task_name}:{subactor_uid}"
 | 
					                    f"{task_name}:{subactor_uid}"
 | 
				
			||||||
            )
 | 
					                )
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    return "pdb_unlock_complete"
 | 
					        return "pdb_unlock_complete"
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					    finally:
 | 
				
			||||||
 | 
					        signal.signal(
 | 
				
			||||||
 | 
					            signal.SIGINT,
 | 
				
			||||||
 | 
					            orig_handler
 | 
				
			||||||
 | 
					        )
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
async def wait_for_parent_stdin_hijack(
 | 
					async def wait_for_parent_stdin_hijack(
 | 
				
			||||||
| 
						 | 
					@ -361,12 +363,14 @@ async def wait_for_parent_stdin_hijack(
 | 
				
			||||||
 | 
					
 | 
				
			||||||
                        finally:
 | 
					                        finally:
 | 
				
			||||||
                            # TODO: shielding currently can cause hangs...
 | 
					                            # TODO: shielding currently can cause hangs...
 | 
				
			||||||
                            with trio.CancelScope(shield=True):
 | 
					                            # with trio.CancelScope(shield=True):
 | 
				
			||||||
                                await stream.send('pdb_unlock')
 | 
					                            await stream.send('pdb_unlock')
 | 
				
			||||||
 | 
					
 | 
				
			||||||
                        # sync with callee termination
 | 
					                        # sync with callee termination
 | 
				
			||||||
                        assert await ctx.result() == "pdb_unlock_complete"
 | 
					                        assert await ctx.result() == "pdb_unlock_complete"
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					                log.pdb('unlocked context')
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        except tractor.ContextCancelled:
 | 
					        except tractor.ContextCancelled:
 | 
				
			||||||
            log.warning('Root actor cancelled debug lock')
 | 
					            log.warning('Root actor cancelled debug lock')
 | 
				
			||||||
 | 
					
 | 
				
			||||||
| 
						 | 
					@ -398,6 +402,15 @@ async def _breakpoint(
 | 
				
			||||||
    # scope here???
 | 
					    # scope here???
 | 
				
			||||||
    # with trio.CancelScope(shield=shield):
 | 
					    # with trio.CancelScope(shield=shield):
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					    pdb = MultiActorPdb()
 | 
				
			||||||
 | 
					    signal.signal = pdbpp.hideframe(signal.signal)
 | 
				
			||||||
 | 
					    orig_handler = signal.signal(
 | 
				
			||||||
 | 
					        signal.SIGINT,
 | 
				
			||||||
 | 
					        partial(shield_sigint, pdb_obj=pdb),
 | 
				
			||||||
 | 
					    )
 | 
				
			||||||
 | 
					    pdb.allow_kbdint = True
 | 
				
			||||||
 | 
					    pdb.nosigint = True
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    actor = tractor.current_actor()
 | 
					    actor = tractor.current_actor()
 | 
				
			||||||
    task_name = trio.lowlevel.current_task().name
 | 
					    task_name = trio.lowlevel.current_task().name
 | 
				
			||||||
 | 
					
 | 
				
			||||||
| 
						 | 
					@ -414,9 +427,12 @@ async def _breakpoint(
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        if _local_task_in_debug:
 | 
					        if _local_task_in_debug:
 | 
				
			||||||
            if _local_task_in_debug == task_name:
 | 
					            if _local_task_in_debug == task_name:
 | 
				
			||||||
 | 
					                print("LOCAL TASK ALREADY IN DEBUG")
 | 
				
			||||||
                # this task already has the lock and is
 | 
					                # this task already has the lock and is
 | 
				
			||||||
                # likely recurrently entering a breakpoint
 | 
					                # likely recurrently entering a breakpoint
 | 
				
			||||||
                return
 | 
					                return
 | 
				
			||||||
 | 
					            else:
 | 
				
			||||||
 | 
					                print("NOT LOCAL TASK ALREADY IN DEBUG")
 | 
				
			||||||
 | 
					
 | 
				
			||||||
            # if **this** actor is already in debug mode block here
 | 
					            # if **this** actor is already in debug mode block here
 | 
				
			||||||
            # waiting for the control to be released - this allows
 | 
					            # waiting for the control to be released - this allows
 | 
				
			||||||
| 
						 | 
					@ -430,8 +446,18 @@ async def _breakpoint(
 | 
				
			||||||
        # entries/requests to the root process
 | 
					        # entries/requests to the root process
 | 
				
			||||||
        _local_task_in_debug = task_name
 | 
					        _local_task_in_debug = task_name
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					        def child_release_hook():
 | 
				
			||||||
 | 
					            # _local_task_in_debug = None
 | 
				
			||||||
 | 
					            _local_pdb_complete.set()
 | 
				
			||||||
 | 
					            # restore original sigint handler
 | 
				
			||||||
 | 
					            signal.signal(
 | 
				
			||||||
 | 
					                signal.SIGINT,
 | 
				
			||||||
 | 
					                orig_handler
 | 
				
			||||||
 | 
					            )
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        # assign unlock callback for debugger teardown hooks
 | 
					        # assign unlock callback for debugger teardown hooks
 | 
				
			||||||
        _pdb_release_hook = _local_pdb_complete.set
 | 
					        # _pdb_release_hook = _local_pdb_complete.set
 | 
				
			||||||
 | 
					        _pdb_release_hook = child_release_hook
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        # this **must** be awaited by the caller and is done using the
 | 
					        # this **must** be awaited by the caller and is done using the
 | 
				
			||||||
        # root nursery so that the debugger can continue to run without
 | 
					        # root nursery so that the debugger can continue to run without
 | 
				
			||||||
| 
						 | 
					@ -496,42 +522,49 @@ async def _breakpoint(
 | 
				
			||||||
            _local_task_in_debug = None
 | 
					            _local_task_in_debug = None
 | 
				
			||||||
            _local_pdb_complete.set()
 | 
					            _local_pdb_complete.set()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					            # restore original sigint handler
 | 
				
			||||||
 | 
					            signal.signal(
 | 
				
			||||||
 | 
					                signal.SIGINT,
 | 
				
			||||||
 | 
					                orig_handler
 | 
				
			||||||
 | 
					            )
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        _pdb_release_hook = teardown
 | 
					        _pdb_release_hook = teardown
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    # block here one (at the appropriate frame *up*) where
 | 
					    frame = sys._getframe()
 | 
				
			||||||
    # ``breakpoint()`` was awaited and begin handling stdio.
 | 
					    last_f = frame.f_back
 | 
				
			||||||
    log.debug("Entering the synchronous world of pdb")
 | 
					    last_f.f_globals['__tracebackhide__'] = True
 | 
				
			||||||
 | 
					 | 
				
			||||||
    debug_func(actor)
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
@cm
 | 
					 | 
				
			||||||
@pdb.hideframe
 | 
					 | 
				
			||||||
def _open_pdb() -> Iterator[PdbwTeardown]:
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
    # XXX: setting these flags on the pdb instance are absolutely
 | 
					 | 
				
			||||||
    # critical to having ctrl-c work in the ``trio`` standard way!  The
 | 
					 | 
				
			||||||
    # stdlib's pdb supports entering the current sync frame on a SIGINT,
 | 
					 | 
				
			||||||
    # with ``trio`` we pretty much never want this and if we did we can
 | 
					 | 
				
			||||||
    # handle it in the ``tractor`` task runtime.
 | 
					 | 
				
			||||||
    __tracebackhide__ = True
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
    pdb = PdbwTeardown()
 | 
					 | 
				
			||||||
    pdb.allow_kbdint = True
 | 
					 | 
				
			||||||
    pdb.nosigint = True
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
    try:
 | 
					    try:
 | 
				
			||||||
        yield pdb
 | 
					        # block here one (at the appropriate frame *up*) where
 | 
				
			||||||
 | 
					        # ``breakpoint()`` was awaited and begin handling stdio.
 | 
				
			||||||
 | 
					        log.debug("Entering the synchronous world of pdb")
 | 
				
			||||||
 | 
					        debug_func(actor, pdb)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    except bdb.BdbQuit:
 | 
					    except bdb.BdbQuit:
 | 
				
			||||||
        if _pdb_release_hook:
 | 
					        if _pdb_release_hook:
 | 
				
			||||||
            _pdb_release_hook()
 | 
					            _pdb_release_hook()
 | 
				
			||||||
        raise
 | 
					        raise
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					    # XXX: apparently we can't do this without showing this frame
 | 
				
			||||||
 | 
					    # in the backtrace on first entry to the REPL? Seems like an odd
 | 
				
			||||||
 | 
					    # behaviour that should have been fixed by now. This is also why
 | 
				
			||||||
 | 
					    # we scrapped all the @cm approaches that were tried previously.
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					    # finally:
 | 
				
			||||||
 | 
					    #     __tracebackhide__ = True
 | 
				
			||||||
 | 
					    #     # frame = sys._getframe()
 | 
				
			||||||
 | 
					    #     # last_f = frame.f_back
 | 
				
			||||||
 | 
					    #     # last_f.f_globals['__tracebackhide__'] = True
 | 
				
			||||||
 | 
					    #     # signal.signal = pdbpp.hideframe(signal.signal)
 | 
				
			||||||
 | 
					    #     signal.signal(
 | 
				
			||||||
 | 
					    #         signal.SIGINT,
 | 
				
			||||||
 | 
					    #         orig_handler
 | 
				
			||||||
 | 
					    #     )
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
def shield_sigint(
 | 
					def shield_sigint(
 | 
				
			||||||
    signum: int,
 | 
					    signum: int,
 | 
				
			||||||
    frame: 'frame',  # type: ignore # noqa
 | 
					    frame: 'frame',  # type: ignore # noqa
 | 
				
			||||||
    pdb: Optional[PdbwTeardown] = None,
 | 
					    pdb_obj: Optional[MultiActorPdb] = None,
 | 
				
			||||||
    *args,
 | 
					    *args,
 | 
				
			||||||
 | 
					
 | 
				
			||||||
) -> None:
 | 
					) -> None:
 | 
				
			||||||
| 
						 | 
					@ -543,6 +576,12 @@ def shield_sigint(
 | 
				
			||||||
    is always cancelled on ctrl-c.
 | 
					    is always cancelled on ctrl-c.
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    '''
 | 
					    '''
 | 
				
			||||||
 | 
					    __tracebackhide__ = True
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					    frame = sys._getframe()
 | 
				
			||||||
 | 
					    last_f = frame.f_back
 | 
				
			||||||
 | 
					    last_f.f_globals['__tracebackhide__'] = True
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    global _local_task_in_debug, _global_actor_in_debug
 | 
					    global _local_task_in_debug, _global_actor_in_debug
 | 
				
			||||||
    in_debug = _global_actor_in_debug
 | 
					    in_debug = _global_actor_in_debug
 | 
				
			||||||
 | 
					
 | 
				
			||||||
| 
						 | 
					@ -598,91 +637,56 @@ def shield_sigint(
 | 
				
			||||||
            raise KeyboardInterrupt
 | 
					            raise KeyboardInterrupt
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    # maybe redraw/print last REPL output to console
 | 
					    # maybe redraw/print last REPL output to console
 | 
				
			||||||
    if pdb:
 | 
					    if pdb_obj:
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        # TODO: make this work like sticky mode where if there is output
 | 
					        # TODO: make this work like sticky mode where if there is output
 | 
				
			||||||
        # detected as written to the tty we redraw this part underneath
 | 
					        # detected as written to the tty we redraw this part underneath
 | 
				
			||||||
        # and erase the past draw of this same bit above?
 | 
					        # and erase the past draw of this same bit above?
 | 
				
			||||||
        # pdb.sticky = True
 | 
					        # pdb_obj.sticky = True
 | 
				
			||||||
        # pdb._print_if_sticky()
 | 
					        # pdb_obj._print_if_sticky()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        # also see these links for an approach from ``ptk``:
 | 
					        # also see these links for an approach from ``ptk``:
 | 
				
			||||||
        # https://github.com/goodboy/tractor/issues/130#issuecomment-663752040
 | 
					        # https://github.com/goodboy/tractor/issues/130#issuecomment-663752040
 | 
				
			||||||
        # https://github.com/prompt-toolkit/python-prompt-toolkit/blob/c2c6af8a0308f9e5d7c0e28cb8a02963fe0ce07a/prompt_toolkit/patch_stdout.py
 | 
					        # https://github.com/prompt-toolkit/python-prompt-toolkit/blob/c2c6af8a0308f9e5d7c0e28cb8a02963fe0ce07a/prompt_toolkit/patch_stdout.py
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        pdb.do_longlist(None)
 | 
					        pdb_obj.do_longlist(None)
 | 
				
			||||||
        print(pdb.prompt, end='', flush=True)
 | 
					        print(pdb_obj.prompt, end='', flush=True)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@pdb.hideframe
 | 
					 | 
				
			||||||
@cm
 | 
					 | 
				
			||||||
def disable_sigint(
 | 
					 | 
				
			||||||
    pdb: Optional[PdbwTeardown] = None
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
) -> Iterator[None]:
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
    __tracebackhide__ = True
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
    # ensure the ``contextlib.contextmanager`` frame inside the wrapping
 | 
					 | 
				
			||||||
    # ``.__exit__()`` method isn't shown either.
 | 
					 | 
				
			||||||
    # frame = sys._getframe()
 | 
					 | 
				
			||||||
    # last_f = frame.f_back
 | 
					 | 
				
			||||||
    # last_f.f_globals['__tracebackhide__'] = True
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
    # NOTE: this seems like a form of cpython bug wherein
 | 
					 | 
				
			||||||
    # it's likely that ``functools.WRAPPER_ASSIGNMENTS`` should
 | 
					 | 
				
			||||||
    # probably contain this attr name?
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
    # for manual debugging if necessary
 | 
					 | 
				
			||||||
    # pdb.set_trace()
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
    orig_handler = signal.signal(
 | 
					 | 
				
			||||||
        signal.SIGINT,
 | 
					 | 
				
			||||||
        partial(shield_sigint, pdb=pdb),
 | 
					 | 
				
			||||||
    )
 | 
					 | 
				
			||||||
    try:
 | 
					 | 
				
			||||||
        yield
 | 
					 | 
				
			||||||
    finally:
 | 
					 | 
				
			||||||
        signal.signal(
 | 
					 | 
				
			||||||
            signal.SIGINT,
 | 
					 | 
				
			||||||
            orig_handler
 | 
					 | 
				
			||||||
        )
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
@pdb.hideframe
 | 
					 | 
				
			||||||
def _set_trace(
 | 
					def _set_trace(
 | 
				
			||||||
    actor: Optional[tractor.Actor] = None
 | 
					    actor: Optional[tractor.Actor] = None,
 | 
				
			||||||
 | 
					    pdb: Optional[MultiActorPdb] = None,
 | 
				
			||||||
):
 | 
					):
 | 
				
			||||||
    __tracebackhide__ = True
 | 
					    __tracebackhide__ = True
 | 
				
			||||||
    actor = actor or tractor.current_actor()
 | 
					    actor = actor or tractor.current_actor()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    with (
 | 
					    # frame = sys._getframe()
 | 
				
			||||||
        _open_pdb() as pdb,
 | 
					    # last_f = frame.f_back
 | 
				
			||||||
        disable_sigint(pdb=pdb),
 | 
					    # last_f.f_globals['__tracebackhide__'] = True
 | 
				
			||||||
    ):
 | 
					 | 
				
			||||||
        if actor is not None:
 | 
					 | 
				
			||||||
            log.pdb(f"\nAttaching pdb to actor: {actor.uid}\n")
 | 
					 | 
				
			||||||
 | 
					
 | 
				
			||||||
            pdb.set_trace(
 | 
					    if actor is not None:
 | 
				
			||||||
                # start 2 levels up in user code
 | 
					        log.pdb(f"\nAttaching pdb to actor: {actor.uid}\n")
 | 
				
			||||||
                frame=sys._getframe().f_back.f_back,
 | 
					 | 
				
			||||||
            )
 | 
					 | 
				
			||||||
 | 
					
 | 
				
			||||||
        else:
 | 
					        pdb.set_trace(
 | 
				
			||||||
            # we entered the global ``breakpoint()`` built-in from sync code?
 | 
					            # start 2 levels up in user code
 | 
				
			||||||
            assert 0, 'Woa this path finally triggered?'
 | 
					            frame=sys._getframe().f_back.f_back,
 | 
				
			||||||
            global _local_task_in_debug, _pdb_release_hook
 | 
					        )
 | 
				
			||||||
            _local_task_in_debug = 'sync'
 | 
					 | 
				
			||||||
 | 
					
 | 
				
			||||||
            def nuttin():
 | 
					    else:
 | 
				
			||||||
                pass
 | 
					        # we entered the global ``breakpoint()`` built-in from sync code?
 | 
				
			||||||
 | 
					        assert 0, 'Woa this path finally triggered?'
 | 
				
			||||||
 | 
					        global _local_task_in_debug, _pdb_release_hook
 | 
				
			||||||
 | 
					        _local_task_in_debug = 'sync'
 | 
				
			||||||
 | 
					
 | 
				
			||||||
            _pdb_release_hook = nuttin
 | 
					        def nuttin():
 | 
				
			||||||
 | 
					            pass
 | 
				
			||||||
 | 
					
 | 
				
			||||||
            pdb.set_trace(
 | 
					        _pdb_release_hook = nuttin
 | 
				
			||||||
                # start 2 levels up in user code
 | 
					
 | 
				
			||||||
                frame=sys._getframe().f_back.f_back,
 | 
					        pdb.set_trace(
 | 
				
			||||||
            )
 | 
					            # start 2 levels up in user code
 | 
				
			||||||
 | 
					            frame=sys._getframe().f_back.f_back,
 | 
				
			||||||
 | 
					        )
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
breakpoint = partial(
 | 
					breakpoint = partial(
 | 
				
			||||||
| 
						 | 
					@ -691,20 +695,18 @@ breakpoint = partial(
 | 
				
			||||||
)
 | 
					)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@pdb.hideframe
 | 
					def _post_mortem(
 | 
				
			||||||
def _post_mortem(actor):
 | 
					    actor: tractor.Actor,
 | 
				
			||||||
 | 
					    pdb: MultiActorPdb,
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    __tracebackhide__ = True
 | 
					) -> None:
 | 
				
			||||||
 | 
					    '''
 | 
				
			||||||
 | 
					    Enter the ``pdbpp`` port mortem entrypoint using our custom
 | 
				
			||||||
 | 
					    debugger instance.
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    with (
 | 
					    '''
 | 
				
			||||||
        # noop()
 | 
					    log.pdb(f"\nAttaching to pdb in crashed actor: {actor.uid}\n")
 | 
				
			||||||
        _open_pdb() as pdb,
 | 
					    pdbpp.xpm(Pdb=lambda: pdb)
 | 
				
			||||||
        # disable_sigint(pdb=pdb),
 | 
					 | 
				
			||||||
    ):
 | 
					 | 
				
			||||||
        log.pdb(f"\nAttaching to pdb in crashed actor: {actor.uid}\n")
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
        # custom Pdb post-mortem entry
 | 
					 | 
				
			||||||
        pdbpp.xpm(Pdb=lambda: pdb)
 | 
					 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
post_mortem = partial(
 | 
					post_mortem = partial(
 | 
				
			||||||
| 
						 | 
					
 | 
				
			||||||
		Loading…
	
		Reference in New Issue