Compare commits

..

10 Commits

Author SHA1 Message Date
Tyler Goodlet fec2ba004c Bleh more nested actor hackin.. 2022-07-31 15:42:13 -04:00
Tyler Goodlet ba9c914221 Allow for tree death in CI? 2022-07-31 14:41:39 -04:00
Tyler Goodlet 30ee3f2dcc Just skip expecting attach msg in CI for ctlc cases 2022-07-29 20:52:54 -04:00
Tyler Goodlet 0b4fc4fc47 Use `assert_before` more extensively 2022-07-29 19:56:11 -04:00
Tyler Goodlet 6b8c193221 Add another name hack for CI race conditions.. 2022-07-29 18:57:44 -04:00
Tyler Goodlet 05167bdc70 Try less times for EOF 2022-07-29 17:51:33 -04:00
Tyler Goodlet fa21083b51 Factor sigint overriding into lock methods 2022-07-29 17:51:12 -04:00
Tyler Goodlet e6ad7a117b Move all module vars into a `Lock` type 2022-07-29 16:03:36 -04:00
Tyler Goodlet 4366873582 Handle missing prompt render case? 2022-07-29 15:59:10 -04:00
Tyler Goodlet 9e6a22e52e Drop pytest full-tracing in CI again 2022-07-29 13:00:54 -04:00
4 changed files with 216 additions and 185 deletions

View File

@ -74,7 +74,7 @@ jobs:
run: pip install -U . -r requirements-test.txt -r requirements-docs.txt --upgrade-strategy eager run: pip install -U . -r requirements-test.txt -r requirements-docs.txt --upgrade-strategy eager
- name: Run tests - name: Run tests
run: pytest tests/ --spawn-backend=${{ matrix.spawn_backend }} -rs -v --full-trace run: pytest tests/ --spawn-backend=${{ matrix.spawn_backend }} -rs -v
# We skip 3.10 on windows for now due to # We skip 3.10 on windows for now due to
# https://github.com/pytest-dev/pytest/issues/8733 # https://github.com/pytest-dev/pytest/issues/8733

View File

@ -18,6 +18,10 @@ import time
import pytest import pytest
import pexpect import pexpect
from pexpect.exceptions import (
TIMEOUT,
EOF,
)
from conftest import repodir from conftest import repodir
@ -372,10 +376,11 @@ def test_multi_subactors(
spawn, spawn,
ctlc: bool, ctlc: bool,
): ):
""" '''
Multiple subactors, both erroring and breakpointing as well as Multiple subactors, both erroring and
a nested subactor erroring. breakpointing as well as a nested subactor erroring.
"""
'''
child = spawn(r'multi_subactors') child = spawn(r'multi_subactors')
# scan for the pdbpp prompt # scan for the pdbpp prompt
@ -414,14 +419,16 @@ def test_multi_subactors(
# 2nd name_error failure # 2nd name_error failure
child.expect(r"\(Pdb\+\+\)") child.expect(r"\(Pdb\+\+\)")
assert_before(child, [ # XXX: lol honestly no idea why CI is cuck but
"Attaching to pdb in crashed actor: ('name_error_1'", # seems like this likely falls into our unhandled nested
"NameError", # case and isn't working in that env due to raciness..
]) from conftest import _ci_env
if not ctlc and _ci_env:
# before = str(child.before.decode()) name = 'name_error' if ctlc else 'name_error_1'
# assert "Attaching to pdb in crashed actor: ('name_error_1'" in before assert_before(child, [
# assert "NameError" in before f"Attaching to pdb in crashed actor: ('{name}'",
"NameError",
])
if ctlc: if ctlc:
do_ctlc(child) do_ctlc(child)
@ -451,11 +458,11 @@ def test_multi_subactors(
do_ctlc(child) do_ctlc(child)
# 2nd depth nursery should trigger # 2nd depth nursery should trigger
# child.sendline('c') if not ctlc:
# child.expect(r"\(Pdb\+\+\)") assert_before(child, [
# before = str(child.before.decode()) spawn_err,
assert spawn_err in before "RemoteActorError: ('name_error_1'",
assert "RemoteActorError: ('name_error_1'" in before ])
# now run some "continues" to show re-entries # now run some "continues" to show re-entries
for _ in range(5): for _ in range(5):
@ -464,31 +471,50 @@ def test_multi_subactors(
# quit the loop and expect parent to attach # quit the loop and expect parent to attach
child.sendline('q') child.sendline('q')
child.expect(r"\(Pdb\+\+\)")
before = str(child.before.decode())
# debugger attaches to root try:
assert "Attaching to pdb in crashed actor: ('root'" in before child.expect(r"\(Pdb\+\+\)")
# expect a multierror with exceptions for each sub-actor except TIMEOUT:
assert "RemoteActorError: ('breakpoint_forever'" in before if _ci_env and not ctlc:
assert "RemoteActorError: ('name_error'" in before raise
assert "RemoteActorError: ('spawn_error'" in before
assert "RemoteActorError: ('name_error_1'" in before # in ci seems like this can sometimes just result
assert 'bdb.BdbQuit' in before # in full tree death?
print('tree died?')
else:
before = str(child.before.decode())
assert_before(child, [
# debugger attaches to root
"Attaching to pdb in crashed actor: ('root'",
# expect a multierror with exceptions for each sub-actor
"RemoteActorError: ('breakpoint_forever'",
"RemoteActorError: ('name_error'",
"RemoteActorError: ('spawn_error'",
"RemoteActorError: ('name_error_1'",
'bdb.BdbQuit',
])
if ctlc: if ctlc:
do_ctlc(child) do_ctlc(child)
# process should exit # process should exit
child.sendline('c') child.sendline('c')
child.expect(pexpect.EOF)
try:
child.expect(pexpect.EOF)
except TIMEOUT:
child.expect(r"\(Pdb\+\+\)")
# repeat of previous multierror for final output # repeat of previous multierror for final output
before = str(child.before.decode()) assert_before(child, [
assert "RemoteActorError: ('breakpoint_forever'" in before "RemoteActorError: ('breakpoint_forever'",
assert "RemoteActorError: ('name_error'" in before "RemoteActorError: ('name_error'",
assert "RemoteActorError: ('spawn_error'" in before "RemoteActorError: ('spawn_error'",
assert "RemoteActorError: ('name_error_1'" in before "RemoteActorError: ('name_error_1'",
assert 'bdb.BdbQuit' in before 'bdb.BdbQuit',
])
def test_multi_daemon_subactors( def test_multi_daemon_subactors(
@ -544,7 +570,7 @@ def test_multi_daemon_subactors(
# now the root actor won't clobber the bp_forever child # now the root actor won't clobber the bp_forever child
# during it's first access to the debug lock, but will instead # during it's first access to the debug lock, but will instead
# wait for the lock to release, by the edge triggered # wait for the lock to release, by the edge triggered
# ``_debug._no_remote_has_tty`` event before sending cancel messages # ``_debug.Lock.no_remote_has_tty`` event before sending cancel messages
# (via portals) to its underlings B) # (via portals) to its underlings B)
# at some point here there should have been some warning msg from # at some point here there should have been some warning msg from
@ -577,7 +603,7 @@ def test_multi_daemon_subactors(
child.sendline('c') child.sendline('c')
child.expect(pexpect.EOF) child.expect(pexpect.EOF)
except pexpect.exceptions.TIMEOUT: except TIMEOUT:
# Failed to exit using continue..? # Failed to exit using continue..?
child.sendline('q') child.sendline('q')
child.expect(pexpect.EOF) child.expect(pexpect.EOF)
@ -607,32 +633,45 @@ def test_multi_subactors_root_errors(
# continue again to catch 2nd name error from # continue again to catch 2nd name error from
# actor 'name_error_1' (which is 2nd depth). # actor 'name_error_1' (which is 2nd depth).
child.sendline('c') child.sendline('c')
child.expect(r"\(Pdb\+\+\)") try:
before = str(child.before.decode()) child.expect(r"\(Pdb\+\+\)")
assert "Attaching to pdb in crashed actor: ('name_error_1'" in before except TIMEOUT:
assert "NameError" in before child.sendline('')
# XXX: lol honestly no idea why CI is cuck but
# seems like this likely falls into our unhandled nested
# case and isn't working in that env due to raciness..
from conftest import _ci_env
if not ctlc and _ci_env:
name = 'name_error' if ctlc else 'name_error_1'
assert_before(child, [
f"Attaching to pdb in crashed actor: ('{name}'",
"NameError",
])
if ctlc: if ctlc:
do_ctlc(child) do_ctlc(child)
child.sendline('c') child.sendline('c')
child.expect(r"\(Pdb\+\+\)") child.expect(r"\(Pdb\+\+\)")
before = str(child.before.decode()) assert_before(child, [
assert "Attaching to pdb in crashed actor: ('spawn_error'" in before "Attaching to pdb in crashed actor: ('spawn_error'",
# boxed error from previous step # boxed error from previous step
assert "RemoteActorError: ('name_error_1'" in before "RemoteActorError: ('name_error_1'",
assert "NameError" in before "NameError",
])
if ctlc: if ctlc:
do_ctlc(child) do_ctlc(child)
child.sendline('c') child.sendline('c')
child.expect(r"\(Pdb\+\+\)") child.expect(r"\(Pdb\+\+\)")
before = str(child.before.decode()) assert_before(child, [
assert "Attaching to pdb in crashed actor: ('root'" in before "Attaching to pdb in crashed actor: ('root'",
# boxed error from first level failure # boxed error from previous step
assert "RemoteActorError: ('name_error'" in before "RemoteActorError: ('name_error'",
assert "NameError" in before "NameError",
])
# warnings assert we probably don't need # warnings assert we probably don't need
# assert "Cancelling nursery in ('spawn_error'," in before # assert "Cancelling nursery in ('spawn_error'," in before
@ -676,7 +715,7 @@ def test_multi_nested_subactors_error_through_nurseries(
child.sendline('c') child.sendline('c')
time.sleep(0.1) time.sleep(0.1)
except pexpect.exceptions.EOF: except EOF:
# race conditions on how fast the continue is sent? # race conditions on how fast the continue is sent?
print(f"Failed early on {i}?") print(f"Failed early on {i}?")
@ -722,8 +761,8 @@ def test_root_nursery_cancels_before_child_releases_tty_lock(
child.expect(r"\(Pdb\+\+\)") child.expect(r"\(Pdb\+\+\)")
except ( except (
pexpect.exceptions.EOF, EOF,
pexpect.exceptions.TIMEOUT, TIMEOUT,
): ):
# races all over.. # races all over..
@ -744,11 +783,11 @@ def test_root_nursery_cancels_before_child_releases_tty_lock(
child.sendline('c') child.sendline('c')
time.sleep(0.1) time.sleep(0.1)
for i in range(10): for i in range(3):
try: try:
child.expect(pexpect.EOF) child.expect(pexpect.EOF)
break break
except pexpect.exceptions.TIMEOUT: except TIMEOUT:
child.sendline('c') child.sendline('c')
time.sleep(0.1) time.sleep(0.1)
print('child was able to grab tty lock again?') print('child was able to grab tty lock again?')

View File

@ -967,7 +967,7 @@ class Actor:
# don't start entire actor runtime # don't start entire actor runtime
# cancellation if this actor is in debug # cancellation if this actor is in debug
# mode # mode
pdb_complete = _debug._local_pdb_complete pdb_complete = _debug.Lock.local_pdb_complete
if pdb_complete: if pdb_complete:
await pdb_complete.wait() await pdb_complete.wait()
@ -1413,7 +1413,7 @@ class Actor:
# kill any debugger request task to avoid deadlock # kill any debugger request task to avoid deadlock
# with the root actor in this tree # with the root actor in this tree
dbcs = _debug._debugger_request_cs dbcs = _debug.Lock._debugger_request_cs
if dbcs is not None: if dbcs is not None:
log.cancel("Cancelling active debugger request") log.cancel("Cancelling active debugger request")
dbcs.cancel() dbcs.cancel()

View File

@ -60,26 +60,81 @@ log = get_logger(__name__)
__all__ = ['breakpoint', 'post_mortem'] __all__ = ['breakpoint', 'post_mortem']
# TODO: wrap all these in a static global class: ``DebugLock`` maybe? class Lock:
'''
Actor global debug lock state.
# placeholder for function to set a ``trio.Event`` on debugger exit Mostly to avoid a lot of ``global`` declarations for now XD.
_pdb_release_hook: Optional[Callable] = None
# actor-wide variable pointing to current task name using debugger '''
_local_task_in_debug: Optional[str] = None # placeholder for function to set a ``trio.Event`` on debugger exit
pdb_release_hook: Optional[Callable] = None
# actor tree-wide actor uid that supposedly has the tty lock # actor-wide variable pointing to current task name using debugger
_global_actor_in_debug: Optional[Tuple[str, str]] = None local_task_in_debug: Optional[str] = None
# lock in root actor preventing multi-access to local tty # actor tree-wide actor uid that supposedly has the tty lock
_debug_lock: trio.StrictFIFOLock = trio.StrictFIFOLock() global_actor_in_debug: Optional[Tuple[str, str]] = None
_local_pdb_complete: Optional[trio.Event] = None
_no_remote_has_tty: Optional[trio.Event] = None
# XXX: set by the current task waiting on the root tty lock local_pdb_complete: Optional[trio.Event] = None
# and must be cancelled if this actor is cancelled via message no_remote_has_tty: Optional[trio.Event] = None
# otherwise deadlocks with the parent actor may ensure
_debugger_request_cs: Optional[trio.CancelScope] = None # lock in root actor preventing multi-access to local tty
_debug_lock: trio.StrictFIFOLock = trio.StrictFIFOLock()
# XXX: set by the current task waiting on the root tty lock
# and must be cancelled if this actor is cancelled via message
# otherwise deadlocks with the parent actor may ensure
_debugger_request_cs: Optional[trio.CancelScope] = None
_orig_sigint_handler: Optional[Callable] = None
@classmethod
def shield_sigint(cls):
cls._orig_sigint_handler = signal.signal(
signal.SIGINT,
shield_sigint,
)
@classmethod
def unshield_sigint(cls):
if cls._orig_sigint_handler is not None:
# restore original sigint handler
signal.signal(
signal.SIGINT,
cls._orig_sigint_handler
)
cls._orig_sigint_handler = None
@classmethod
def maybe_release(cls):
cls.local_task_in_debug = None
if cls.pdb_release_hook:
cls.pdb_release_hook()
@classmethod
def root_release(cls):
try:
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
# that locked?
owner = cls._debug_lock.statistics().owner
if owner:
raise
cls.global_actor_in_debug = None
cls.local_task_in_debug = None
try:
# sometimes the ``trio`` might already be terminated in
# which case this call will raise.
cls.local_pdb_complete.set()
finally:
# restore original sigint handler
cls.unshield_sigint()
class TractorConfig(pdbpp.DefaultConfig): class TractorConfig(pdbpp.DefaultConfig):
@ -108,13 +163,13 @@ class MultiActorPdb(pdbpp.Pdb):
try: try:
super().set_continue() super().set_continue()
finally: finally:
maybe_release() Lock.maybe_release()
def set_quit(self): def set_quit(self):
try: try:
super().set_quit() super().set_quit()
finally: finally:
maybe_release() Lock.maybe_release()
# TODO: will be needed whenever we get to true remote debugging. # TODO: will be needed whenever we get to true remote debugging.
@ -153,14 +208,6 @@ class MultiActorPdb(pdbpp.Pdb):
# log.info("Closing stdin hijack") # log.info("Closing stdin hijack")
# break # break
# TODO: make this method on a global lock type!
def maybe_release():
global _local_task_in_debug, _pdb_release_hook
_local_task_in_debug = None
if _pdb_release_hook:
_pdb_release_hook()
@acm @acm
async def _acquire_debug_lock( async def _acquire_debug_lock(
uid: Tuple[str, str] uid: Tuple[str, str]
@ -175,8 +222,6 @@ async def _acquire_debug_lock(
to the ``pdb`` repl. to the ``pdb`` repl.
''' '''
global _debug_lock, _global_actor_in_debug, _no_remote_has_tty
task_name = trio.lowlevel.current_task().name task_name = trio.lowlevel.current_task().name
log.runtime( log.runtime(
@ -190,15 +235,15 @@ async def _acquire_debug_lock(
f"entering lock checkpoint, remote task: {task_name}:{uid}" f"entering lock checkpoint, remote task: {task_name}:{uid}"
) )
we_acquired = True we_acquired = True
await _debug_lock.acquire() await Lock._debug_lock.acquire()
if _no_remote_has_tty is None: if Lock.no_remote_has_tty is None:
# mark the tty lock as being in use so that the runtime # mark the tty lock as being in use so that the runtime
# can try to avoid clobbering any connection from a child # can try to avoid clobbering any connection from a child
# that's currently relying on it. # that's currently relying on it.
_no_remote_has_tty = trio.Event() Lock.no_remote_has_tty = trio.Event()
_global_actor_in_debug = uid Lock.global_actor_in_debug = uid
log.runtime(f"TTY lock acquired, remote task: {task_name}:{uid}") log.runtime(f"TTY lock acquired, remote task: {task_name}:{uid}")
# NOTE: critical section: this yield is unshielded! # NOTE: critical section: this yield is unshielded!
@ -211,32 +256,32 @@ async def _acquire_debug_lock(
# surrounding caller side context should cancel normally # surrounding caller side context should cancel normally
# relaying back to the caller. # relaying back to the caller.
yield _debug_lock yield Lock._debug_lock
finally: finally:
# if _global_actor_in_debug == uid: # if Lock.global_actor_in_debug == uid:
if ( if (
we_acquired we_acquired
and _debug_lock.locked() and Lock._debug_lock.locked()
): ):
_debug_lock.release() Lock._debug_lock.release()
# IFF there are no more requesting tasks queued up fire, the # IFF there are no more requesting tasks queued up fire, the
# "tty-unlocked" event thereby alerting any monitors of the lock that # "tty-unlocked" event thereby alerting any monitors of the lock that
# we are now back in the "tty unlocked" state. This is basically # we are now back in the "tty unlocked" state. This is basically
# and edge triggered signal around an empty queue of sub-actor # and edge triggered signal around an empty queue of sub-actor
# tasks that may have tried to acquire the lock. # tasks that may have tried to acquire the lock.
stats = _debug_lock.statistics() stats = Lock._debug_lock.statistics()
if ( if (
not stats.owner not stats.owner
): ):
log.runtime(f"No more tasks waiting on tty lock! says {uid}") log.runtime(f"No more tasks waiting on tty lock! says {uid}")
if _no_remote_has_tty is not None: if Lock.no_remote_has_tty is not None:
_no_remote_has_tty.set() Lock.no_remote_has_tty.set()
_no_remote_has_tty = None Lock.no_remote_has_tty = None
_global_actor_in_debug = None Lock.global_actor_in_debug = None
log.runtime( log.runtime(
f"TTY lock released, remote task: {task_name}:{uid}" f"TTY lock released, remote task: {task_name}:{uid}"
@ -271,11 +316,8 @@ 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")
Lock.shield_sigint()
orig_handler = signal.signal(
signal.SIGINT,
shield_sigint,
)
try: try:
with ( with (
trio.CancelScope(shield=True), trio.CancelScope(shield=True),
@ -327,10 +369,7 @@ async def _hijack_stdin_for_child(
return "pdb_unlock_complete" return "pdb_unlock_complete"
finally: finally:
signal.signal( Lock.unshield_sigint()
signal.SIGINT,
orig_handler
)
async def wait_for_parent_stdin_hijack( async def wait_for_parent_stdin_hijack(
@ -348,10 +387,8 @@ async def wait_for_parent_stdin_hijack(
debug (see below inside ``maybe_wait_for_debugger()``). debug (see below inside ``maybe_wait_for_debugger()``).
''' '''
global _debugger_request_cs
with trio.CancelScope(shield=True) as cs: with trio.CancelScope(shield=True) as cs:
_debugger_request_cs = cs Lock._debugger_request_cs = cs
try: try:
async with get_root() as portal: async with get_root() as portal:
@ -371,9 +408,9 @@ async def wait_for_parent_stdin_hijack(
# unblock local caller # unblock local caller
try: try:
assert _local_pdb_complete assert Lock.local_pdb_complete
task_status.started(cs) task_status.started(cs)
await _local_pdb_complete.wait() await Lock.local_pdb_complete.wait()
finally: finally:
# TODO: shielding currently can cause hangs... # TODO: shielding currently can cause hangs...
@ -390,8 +427,7 @@ async def wait_for_parent_stdin_hijack(
finally: finally:
log.pdb(f"Exiting debugger for actor {actor_uid}") log.pdb(f"Exiting debugger for actor {actor_uid}")
global _local_task_in_debug Lock.local_task_in_debug = None
_local_task_in_debug = None
log.pdb(f"Child {actor_uid} released parent stdio lock") log.pdb(f"Child {actor_uid} released parent stdio lock")
@ -399,10 +435,8 @@ def mk_mpdb() -> tuple[MultiActorPdb, Callable]:
pdb = MultiActorPdb() pdb = MultiActorPdb()
# signal.signal = pdbpp.hideframe(signal.signal) # signal.signal = pdbpp.hideframe(signal.signal)
orig_handler = signal.signal(
signal.SIGINT, Lock.shield_sigint()
partial(shield_sigint, pdb_obj=pdb),
)
# XXX: These are the important flags mentioned in # XXX: These are the important flags mentioned in
# https://github.com/python-trio/trio/issues/1155 # https://github.com/python-trio/trio/issues/1155
@ -410,15 +444,7 @@ def mk_mpdb() -> tuple[MultiActorPdb, Callable]:
pdb.allow_kbdint = True pdb.allow_kbdint = True
pdb.nosigint = True pdb.nosigint = True
# TODO: add this as method on our pdb obj? return pdb, Lock.unshield_sigint
def undo_sigint():
# restore original sigint handler
signal.signal(
signal.SIGINT,
orig_handler
)
return pdb, undo_sigint
async def _breakpoint( async def _breakpoint(
@ -440,9 +466,6 @@ async def _breakpoint(
actor = tractor.current_actor() actor = tractor.current_actor()
task_name = trio.lowlevel.current_task().name task_name = trio.lowlevel.current_task().name
global _local_pdb_complete, _pdb_release_hook
global _local_task_in_debug, _global_actor_in_debug
# TODO: is it possible to debug a trio.Cancelled except block? # TODO: is it possible to debug a trio.Cancelled except block?
# right now it seems like we can kinda do with by shielding # right now it seems like we can kinda do with by shielding
# around ``tractor.breakpoint()`` but not if we move the shielded # around ``tractor.breakpoint()`` but not if we move the shielded
@ -450,14 +473,14 @@ async def _breakpoint(
# with trio.CancelScope(shield=shield): # with trio.CancelScope(shield=shield):
# await trio.lowlevel.checkpoint() # await trio.lowlevel.checkpoint()
if not _local_pdb_complete or _local_pdb_complete.is_set(): if not Lock.local_pdb_complete or Lock.local_pdb_complete.is_set():
_local_pdb_complete = trio.Event() Lock.local_pdb_complete = trio.Event()
# TODO: need a more robust check for the "root" actor # TODO: need a more robust check for the "root" actor
if actor._parent_chan and not is_root_process(): if actor._parent_chan and not is_root_process():
if _local_task_in_debug: if Lock.local_task_in_debug:
if _local_task_in_debug == task_name: if Lock.local_task_in_debug == task_name:
# 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
@ -467,18 +490,18 @@ async def _breakpoint(
# support for recursive entries to `tractor.breakpoint()` # support for recursive entries to `tractor.breakpoint()`
log.warning(f"{actor.uid} already has a debug lock, waiting...") log.warning(f"{actor.uid} already has a debug lock, waiting...")
await _local_pdb_complete.wait() await Lock.local_pdb_complete.wait()
await trio.sleep(0.1) await trio.sleep(0.1)
# mark local actor as "in debug mode" to avoid recurrent # mark local actor as "in debug mode" to avoid recurrent
# entries/requests to the root process # entries/requests to the root process
_local_task_in_debug = task_name Lock.local_task_in_debug = task_name
def child_release(): def child_release():
try: try:
# sometimes the ``trio`` might already be termianated in # sometimes the ``trio`` might already be termianated in
# which case this call will raise. # which case this call will raise.
_local_pdb_complete.set() Lock.local_pdb_complete.set()
finally: finally:
# restore original sigint handler # restore original sigint handler
undo_sigint() undo_sigint()
@ -486,7 +509,7 @@ async def _breakpoint(
# _local_task_in_debug = None # _local_task_in_debug = None
# assign unlock callback for debugger teardown hooks # assign unlock callback for debugger teardown hooks
_pdb_release_hook = child_release Lock.pdb_release_hook = child_release
# 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
@ -503,66 +526,39 @@ async def _breakpoint(
actor.uid, actor.uid,
) )
except RuntimeError: except RuntimeError:
_pdb_release_hook() Lock.pdb_release_hook()
raise raise
elif is_root_process(): elif is_root_process():
# we also wait in the root-parent for any child that # we also wait in the root-parent for any child that
# may have the tty locked prior # may have the tty locked prior
global _debug_lock
# TODO: wait, what about multiple root tasks acquiring it though? # TODO: wait, what about multiple root tasks acquiring it though?
# root process (us) already has it; ignore # root process (us) already has it; ignore
if _global_actor_in_debug == actor.uid: if Lock.global_actor_in_debug == actor.uid:
return return
# XXX: since we need to enter pdb synchronously below, # XXX: since we need to enter pdb synchronously below,
# we have to release the lock manually from pdb completion # we have to release the lock manually from pdb completion
# callbacks. Can't think of a nicer way then this atm. # callbacks. Can't think of a nicer way then this atm.
if _debug_lock.locked(): if Lock._debug_lock.locked():
log.warning( log.warning(
'Root actor attempting to shield-acquire active tty lock' 'Root actor attempting to shield-acquire active tty lock'
f' owned by {_global_actor_in_debug}') f' owned by {Lock.global_actor_in_debug}')
# must shield here to avoid hitting a ``Cancelled`` and # must shield here to avoid hitting a ``Cancelled`` and
# a child getting stuck bc we clobbered the tty # a child getting stuck bc we clobbered the tty
with trio.CancelScope(shield=True): with trio.CancelScope(shield=True):
await _debug_lock.acquire() await Lock._debug_lock.acquire()
else: else:
# may be cancelled # may be cancelled
await _debug_lock.acquire() await Lock._debug_lock.acquire()
_global_actor_in_debug = actor.uid Lock.global_actor_in_debug = actor.uid
_local_task_in_debug = task_name Lock.local_task_in_debug = task_name
# the lock must be released on pdb completion # the lock must be released on pdb completion
def root_release(): Lock.pdb_release_hook = Lock.root_release
global _local_pdb_complete, _debug_lock
global _global_actor_in_debug, _local_task_in_debug
try:
_debug_lock.release()
except RuntimeError:
# uhhh makes no sense but been seeing the non-owner
# release error even though this is definitely the task
# that locked?
owner = _debug_lock.statistics().owner
if owner:
raise
_global_actor_in_debug = None
_local_task_in_debug = None
try:
# sometimes the ``trio`` might already be termianated in
# which case this call will raise.
_local_pdb_complete.set()
finally:
# restore original sigint handler
undo_sigint()
_pdb_release_hook = root_release
try: try:
# block here one (at the appropriate frame *up*) where # block here one (at the appropriate frame *up*) where
@ -571,7 +567,7 @@ async def _breakpoint(
debug_func(actor, pdb) debug_func(actor, pdb)
except bdb.BdbQuit: except bdb.BdbQuit:
maybe_release() Lock.maybe_release()
raise raise
# XXX: apparently we can't do this without showing this frame # XXX: apparently we can't do this without showing this frame
@ -607,8 +603,7 @@ def shield_sigint(
''' '''
__tracebackhide__ = True __tracebackhide__ = True
global _local_task_in_debug, _global_actor_in_debug uid_in_debug = Lock.global_actor_in_debug
uid_in_debug = _global_actor_in_debug
actor = tractor.current_actor() actor = tractor.current_actor()
@ -681,7 +676,7 @@ def shield_sigint(
) )
return do_cancel() return do_cancel()
task = _local_task_in_debug task = Lock.local_task_in_debug
if task: if task:
log.pdb( log.pdb(
f"Ignoring SIGINT while task in debug mode: `{task}`" f"Ignoring SIGINT while task in debug mode: `{task}`"
@ -754,8 +749,7 @@ def _set_trace(
pdb, undo_sigint = mk_mpdb() pdb, undo_sigint = mk_mpdb()
# we entered the global ``breakpoint()`` built-in from sync code? # we entered the global ``breakpoint()`` built-in from sync code?
global _local_task_in_debug, _pdb_release_hook Lock.local_task_in_debug = 'sync'
_local_task_in_debug = 'sync'
pdb.set_trace(frame=frame) pdb.set_trace(frame=frame)
@ -830,7 +824,7 @@ async def _maybe_enter_pm(err):
): ):
log.debug("Actor crashed, entering debug mode") log.debug("Actor crashed, entering debug mode")
await post_mortem() await post_mortem()
maybe_release() Lock.maybe_release()
return True return True
else: else:
@ -875,8 +869,6 @@ async def maybe_wait_for_debugger(
if ( if (
is_root_process() is_root_process()
): ):
global _no_remote_has_tty, _global_actor_in_debug, _wait_all_tasks_lock
# If we error in the root but the debugger is # If we error in the root but the debugger is
# engaged we don't want to prematurely kill (and # engaged we don't want to prematurely kill (and
# thus clobber access to) the local tty since it # thus clobber access to) the local tty since it
@ -888,8 +880,8 @@ async def maybe_wait_for_debugger(
for _ in range(poll_steps): for _ in range(poll_steps):
if _global_actor_in_debug: if Lock.global_actor_in_debug:
sub_in_debug = tuple(_global_actor_in_debug) sub_in_debug = tuple(Lock.global_actor_in_debug)
# alive = tractor.current_actor().child_alive(sub_in_debug) # alive = tractor.current_actor().child_alive(sub_in_debug)
# if not alive: # if not alive:
# break # break
@ -905,7 +897,7 @@ async def maybe_wait_for_debugger(
# XXX: doesn't seem to work # XXX: doesn't seem to work
# await trio.testing.wait_all_tasks_blocked(cushion=0) # await trio.testing.wait_all_tasks_blocked(cushion=0)
debug_complete = _no_remote_has_tty debug_complete = Lock.no_remote_has_tty
if ( if (
(debug_complete and (debug_complete and
not debug_complete.is_set()) not debug_complete.is_set())