forked from goodboy/tractor
Move all module vars into a `Lock` type
parent
4366873582
commit
e6ad7a117b
|
@ -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()
|
||||||
|
|
|
@ -33,6 +33,7 @@ from typing import (
|
||||||
)
|
)
|
||||||
from types import FrameType
|
from types import FrameType
|
||||||
|
|
||||||
|
from msgspec import Struct
|
||||||
import tractor
|
import tractor
|
||||||
import trio
|
import trio
|
||||||
from trio_typing import TaskStatus
|
from trio_typing import TaskStatus
|
||||||
|
@ -60,26 +61,38 @@ 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
|
||||||
|
|
||||||
|
@classmethod
|
||||||
|
def maybe_release(cls):
|
||||||
|
cls.local_task_in_debug = None
|
||||||
|
if cls.pdb_release_hook:
|
||||||
|
cls.pdb_release_hook()
|
||||||
|
|
||||||
|
|
||||||
class TractorConfig(pdbpp.DefaultConfig):
|
class TractorConfig(pdbpp.DefaultConfig):
|
||||||
|
@ -108,13 +121,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 +166,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 +180,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 +193,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 +214,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}"
|
||||||
|
@ -348,10 +351,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 +372,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 +391,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")
|
||||||
|
|
||||||
|
|
||||||
|
@ -440,9 +440,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 +447,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 +464,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 +483,8 @@ 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
|
||||||
|
# _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 +501,63 @@ async def _breakpoint(
|
||||||
actor.uid,
|
actor.uid,
|
||||||
)
|
)
|
||||||
except RuntimeError:
|
except RuntimeError:
|
||||||
_pdb_release_hook()
|
Lock.pdb_release_hook()
|
||||||
|
# _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():
|
def root_release():
|
||||||
global _local_pdb_complete, _debug_lock
|
|
||||||
global _global_actor_in_debug, _local_task_in_debug
|
|
||||||
|
|
||||||
try:
|
try:
|
||||||
_debug_lock.release()
|
Lock._debug_lock.release()
|
||||||
except RuntimeError:
|
except RuntimeError:
|
||||||
# uhhh makes no sense but been seeing the non-owner
|
# uhhh makes no sense but been seeing the non-owner
|
||||||
# release error even though this is definitely the task
|
# release error even though this is definitely the task
|
||||||
# that locked?
|
# that locked?
|
||||||
owner = _debug_lock.statistics().owner
|
owner = Lock._debug_lock.statistics().owner
|
||||||
if owner:
|
if owner:
|
||||||
raise
|
raise
|
||||||
|
|
||||||
_global_actor_in_debug = None
|
Lock.global_actor_in_debug = None
|
||||||
_local_task_in_debug = None
|
Lock.local_task_in_debug = None
|
||||||
|
|
||||||
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()
|
||||||
|
|
||||||
_pdb_release_hook = root_release
|
# _pdb_release_hook = root_release
|
||||||
|
Lock.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 +566,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 +602,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 +675,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 +748,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 +823,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 +868,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 +879,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 +896,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())
|
||||||
|
|
Loading…
Reference in New Issue