Compare commits

..

10 Commits

Author SHA1 Message Date
Tyler Goodlet 00c4d69bd2 Change trace to transport level 2021-07-05 16:42:40 -04:00
Tyler Goodlet bd3832e70f Flip "trace" level to "transport" level logging 2021-07-05 16:42:40 -04:00
Tyler Goodlet af61d93bad Add fast fail test using the context api 2021-07-05 16:42:40 -04:00
Tyler Goodlet 1bca5c8144 Adjust debug tests to accomodate no more root clobbering
We may get multiple re-entries to debugger by `bp_forever` sub-actor
now since the root will incrementally try to cancel it only when the tty
lock is not held.
2021-07-05 16:42:40 -04:00
Tyler Goodlet c174a59758 Go back to only logging tbs on no debugger 2021-07-05 16:42:40 -04:00
Tyler Goodlet 91c70cad24 Comment hard-kill-sidestep for now since nursery version covers it? 2021-07-05 16:42:40 -04:00
Tyler Goodlet 9b4f7a3076 Go back to only logging crashes if no pdb gets engaged 2021-07-05 16:42:38 -04:00
Tyler Goodlet 3179eded73 Solve the root-cancels-child-in-tty-lock race
Finally this makes a cancelled root actor nursery not clobber child
tasks which request and lock the root's tty for the debugger repl.

Using an edge triggered event which is set after all fifo-lock-queued
tasks are complete, we can be sure that no lingering child tasks are
going to get interrupted during pdb use and tty lock acquisition.
Further, even if new tasks do queue up to get the lock, the root will
incrementally send cancel msgs to each sub-actor only once the tty is
not locked by a (set of) child request task(s). Add shielding around all
the critical sections where the child attempts to allocate the lock from
the root such that it won't be disrupted from cancel messages from the
root after the acquire lock transaction has started.
2021-07-05 16:40:58 -04:00
Tyler Goodlet b603904d3e Distinguish between a local pdb unlock and the tty unlock in root 2021-07-05 16:40:58 -04:00
Tyler Goodlet 1ec1743c48 Fix hard kill in debug mode; only do it when debug lock is empty 2021-07-05 16:40:58 -04:00
9 changed files with 395 additions and 143 deletions

View File

@ -0,0 +1,53 @@
'''
fast fail test with a context.
ensure the partially initialized sub-actor process
doesn't cause a hang on error/cancel of the parent
nrusery.
'''
import trio
import tractor
@tractor.context
async def sleep(
ctx: tractor.Context,
):
await trio.sleep(0.5)
await ctx.started()
await trio.sleep_forever()
async def open_ctx(
n: tractor._trionics.ActorNursery
):
# spawn both actors
portal = await n.start_actor(
name='sleeper',
enable_modules=[__name__],
)
async with portal.open_context(
sleep,
) as (ctx, first):
assert first is None
async def main():
async with tractor.open_nursery(
debug_mode=True,
loglevel='runtime',
) as an:
async with trio.open_nursery() as n:
n.start_soon(open_ctx, an)
await trio.sleep(0.2)
await trio.sleep(0.1)
assert 0
if __name__ == '__main__':
trio.run(main)

View File

@ -309,32 +309,58 @@ def test_multi_daemon_subactors(spawn, loglevel):
next_msg = name_error_msg next_msg = name_error_msg
elif name_error_msg in before: elif name_error_msg in before:
next_msg = None next_msg = bp_forever_msg
else: else:
raise ValueError("Neither log msg was found !?") raise ValueError("Neither log msg was found !?")
child.sendline('c') # NOTE: previously since we did not have clobber prevention
# in the root actor this final resume could result in the debugger
# tearing down since both child actors would be cancelled and it was
# unlikely that `bp_forever` would re-acquire the tty loack again.
# Now, we should have a final resumption in the root plus a possible
# second entry by `bp_forever`.
# first name_error failure child.sendline('c')
child.expect(r"\(Pdb\+\+\)") child.expect(r"\(Pdb\+\+\)")
before = str(child.before.decode()) before = str(child.before.decode())
if next_msg:
assert next_msg in before assert next_msg in before
child.sendline('c') # XXX: hoorayy the root clobering the child here was fixed!
# IMO, this demonstrates the true power of SC system design.
# now the root actor won't clobber the bp_forever child
# during it's first access to the debug lock, but will instead
# wait for the lock to release, by the edge triggered
# ``_debug._no_remote_has_tty`` event before sending cancel messages
# (via portals) to its underlings B)
# at some point here there should have been some warning msg from
# the root announcing it avoided a clobber of the child's lock, but
# it seems unreliable in testing here to gnab it:
# assert "in use by child ('bp_forever'," in before
# wait for final error in root
while True:
child.sendline('c')
child.expect(r"\(Pdb\+\+\)") child.expect(r"\(Pdb\+\+\)")
before = str(child.before.decode()) before = str(child.before.decode())
assert "tractor._exceptions.RemoteActorError: ('name_error'" in before try:
# root error should be packed as remote error
assert "_exceptions.RemoteActorError: ('name_error'" in before
break
except AssertionError:
assert bp_forever_msg in before
try: try:
child.sendline('c') child.sendline('c')
child.expect(pexpect.EOF) child.expect(pexpect.EOF)
except pexpect.exceptions.TIMEOUT: except pexpect.exceptions.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)
@ -389,7 +415,7 @@ def test_multi_nested_subactors_error_through_nurseries(spawn):
child = spawn('multi_nested_subactors_error_up_through_nurseries') child = spawn('multi_nested_subactors_error_up_through_nurseries')
# startup time can be iffy # startup time can be iffy
time.sleep(1) # time.sleep(1)
for i in range(12): for i in range(12):
try: try:
@ -471,3 +497,21 @@ def test_root_nursery_cancels_before_child_releases_tty_lock(
assert "tractor._exceptions.RemoteActorError: ('spawner0'" in before assert "tractor._exceptions.RemoteActorError: ('spawner0'" in before
assert "tractor._exceptions.RemoteActorError: ('name_error'" in before assert "tractor._exceptions.RemoteActorError: ('name_error'" in before
assert "NameError: name 'doggypants' is not defined" in before assert "NameError: name 'doggypants' is not defined" in before
def test_root_cancels_child_context_during_startup(
spawn,
):
'''Verify a fast fail in the root doesn't lock up the child reaping
and all while using the new context api.
'''
child = spawn('fast_error_in_root_after_spawn')
child.expect(r"\(Pdb\+\+\)")
before = str(child.before.decode())
assert "AssertionError" in before
child.sendline('c')
child.expect(pexpect.EOF)

View File

@ -28,6 +28,7 @@ from ._exceptions import (
ModuleNotExposed, ModuleNotExposed,
is_multi_cancelled, is_multi_cancelled,
TransportClosed, TransportClosed,
ContextCancelled,
) )
from . import _debug from . import _debug
from ._discovery import get_arbiter from ._discovery import get_arbiter
@ -125,18 +126,32 @@ async def _invoke(
except (Exception, trio.MultiError) as err: except (Exception, trio.MultiError) as err:
# TODO: maybe we'll want differnet "levels" of debugging if not is_multi_cancelled(err):
log.exception("Actor crashed:")
# TODO: maybe we'll want different "levels" of debugging
# eventualy such as ('app', 'supervisory', 'runtime') ? # eventualy such as ('app', 'supervisory', 'runtime') ?
if not isinstance(err, trio.ClosedResourceError) and (
not is_multi_cancelled(err) # if not isinstance(err, trio.ClosedResourceError) and (
# if not is_multi_cancelled(err) and (
entered_debug: bool = False
if not isinstance(err, ContextCancelled) or (
isinstance(err, ContextCancelled) and ctx._cancel_called
): ):
# XXX: is there any case where we'll want to debug IPC # XXX: is there any case where we'll want to debug IPC
# disconnects? I can't think of a reason that inspecting # disconnects as a default?
#
# I can't think of a reason that inspecting
# this type of failure will be useful for respawns or # this type of failure will be useful for respawns or
# recovery logic - the only case is some kind of strange bug # recovery logic - the only case is some kind of strange bug
# in `trio` itself? # in our transport layer itself? Going to keep this
entered = await _debug._maybe_enter_pm(err) # open ended for now.
if not entered:
entered_debug = await _debug._maybe_enter_pm(err)
if not entered_debug:
log.exception("Actor crashed:") log.exception("Actor crashed:")
# always ship errors back to caller # always ship errors back to caller
@ -369,7 +384,8 @@ class Actor:
log.warning( log.warning(
f"already have channel(s) for {uid}:{chans}?" f"already have channel(s) for {uid}:{chans}?"
) )
log.trace(f"Registered {chan} for {uid}") # type: ignore
log.runtime(f"Registered {chan} for {uid}") # type: ignore
# append new channel # append new channel
self._peers[uid].append(chan) self._peers[uid].append(chan)
@ -502,7 +518,7 @@ class Actor:
f" {chan} from {chan.uid}") f" {chan} from {chan.uid}")
break break
log.trace( # type: ignore log.transport( # type: ignore
f"Received msg {msg} from {chan.uid}") f"Received msg {msg} from {chan.uid}")
cid = msg.get('cid') cid = msg.get('cid')

View File

@ -45,7 +45,8 @@ _global_actor_in_debug: Optional[Tuple[str, str]] = None
# lock in root actor preventing multi-access to local tty # lock in root actor preventing multi-access to local tty
_debug_lock: trio.StrictFIFOLock = trio.StrictFIFOLock() _debug_lock: trio.StrictFIFOLock = trio.StrictFIFOLock()
_pdb_complete: Optional[trio.Event] = 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 # XXX: set by the current task waiting on the root tty lock
# and must be cancelled if this actor is cancelled via message # and must be cancelled if this actor is cancelled via message
@ -109,7 +110,7 @@ class PdbwTeardown(pdbpp.Pdb):
# async with aclosing(async_stdin): # async with aclosing(async_stdin):
# async for msg in async_stdin: # async for msg in async_stdin:
# log.trace(f"Stdin input:\n{msg}") # log.runtime(f"Stdin input:\n{msg}")
# # encode to bytes # # encode to bytes
# bmsg = str.encode(msg) # bmsg = str.encode(msg)
@ -123,24 +124,71 @@ class PdbwTeardown(pdbpp.Pdb):
@asynccontextmanager @asynccontextmanager
async def _acquire_debug_lock(uid: Tuple[str, str]) -> AsyncIterator[None]: async def _acquire_debug_lock(uid: Tuple[str, str]) -> AsyncIterator[None]:
"""Acquire a actor local FIFO lock meant to mutex entry to a local '''Acquire a actor local FIFO lock meant to mutex entry to a local
debugger entry point to avoid tty clobbering by multiple processes. debugger entry point to avoid tty clobbering a global root process.
"""
global _debug_lock, _global_actor_in_debug '''
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.pdb(
f"Attempting to acquire TTY lock, remote task: {task_name}:{uid}"
)
we_acquired = False
if _no_remote_has_tty is None:
# mark the tty lock as being in use so that the runtime
# can try to avoid clobbering any connection from a child
# that's currently relying on it.
_no_remote_has_tty = trio.Event()
try:
log.debug( log.debug(
f"Attempting to acquire TTY lock, remote task: {task_name}:{uid}") f"entering lock checkpoint, remote task: {task_name}:{uid}"
)
we_acquired = True
await _debug_lock.acquire()
async with _debug_lock: # we_acquired = True
# _debug_lock._uid = uid
_global_actor_in_debug = uid _global_actor_in_debug = uid
log.debug(f"TTY lock acquired, remote task: {task_name}:{uid}") log.debug(f"TTY lock acquired, remote task: {task_name}:{uid}")
yield
# NOTE: critical section!
# this yield is unshielded.
# IF we received a cancel during the shielded lock
# entry of some next-in-queue requesting task,
# then the resumption here will result in that
# Cancelled being raised to our caller below!
# in this case the finally below should trigger
# and the surrounding calle side context should cancel
# normally relaying back to the caller.
yield _debug_lock
finally:
# if _global_actor_in_debug == uid:
if we_acquired and _debug_lock.locked():
_debug_lock.release()
# IFF there are no more requesting tasks queued up fire, the
# "tty-unlocked" event thereby alerting any monitors of the lock that
# we are now back in the "tty unlocked" state. This is basically
# and edge triggered signal around an empty queue of sub-actor
# tasks that may have tried to acquire the lock.
stats = _debug_lock.statistics()
if (
not stats.owner
):
log.pdb(f"No more tasks waiting on tty lock! says {uid}")
_no_remote_has_tty.set()
_no_remote_has_tty = None
_global_actor_in_debug = None _global_actor_in_debug = None
log.debug(f"TTY lock released, remote task: {task_name}:{uid}") log.debug(f"TTY lock released, remote task: {task_name}:{uid}")
@ -162,29 +210,30 @@ async def _hijack_stdin_relay_to_child(
subactor_uid: Tuple[str, str] subactor_uid: Tuple[str, str]
) -> str: ) -> str:
'''Hijack the tty in the root process of an actor tree such that
the pdbpp debugger console can be allocated to a sub-actor for repl
bossing.
global _pdb_complete '''
task_name = trio.lowlevel.current_task().name task_name = trio.lowlevel.current_task().name
# TODO: when we get to true remote debugging # TODO: when we get to true remote debugging
# this will deliver stdin data? # this will deliver stdin data?
log.debug( log.debug(
"Attempting to acquire TTY lock, " "Attempting to acquire TTY lock\n"
f"remote task: {task_name}:{subactor_uid}" f"remote task: {task_name}:{subactor_uid}"
) )
log.debug(f"Actor {subactor_uid} is WAITING on stdin hijack lock") log.debug(f"Actor {subactor_uid} is WAITING on stdin hijack lock")
async with _acquire_debug_lock(subactor_uid):
# XXX: only shield the context sync step!
with trio.CancelScope(shield=True): with trio.CancelScope(shield=True):
async with _acquire_debug_lock(subactor_uid):
# 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.runtime( # type: ignore log.pdb( # type: ignore
f"Actor {subactor_uid} ACQUIRED stdin hijack lock") f"Actor {subactor_uid} ACQUIRED stdin hijack lock")
# wait for unlock pdb by child # wait for unlock pdb by child
@ -203,7 +252,6 @@ async def _hijack_stdin_relay_to_child(
log.debug( log.debug(
f"TTY lock released, remote task: {task_name}:{subactor_uid}") f"TTY lock released, remote task: {task_name}:{subactor_uid}")
log.debug(f"Actor {subactor_uid} RELEASED stdin hijack lock")
return "pdb_unlock_complete" return "pdb_unlock_complete"
@ -228,20 +276,24 @@ 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 _pdb_complete, _pdb_release_hook global _local_pdb_complete, _pdb_release_hook
global _local_task_in_debug, _global_actor_in_debug global _local_task_in_debug, _global_actor_in_debug
await trio.lowlevel.checkpoint()
async def wait_for_parent_stdin_hijack( async def wait_for_parent_stdin_hijack(
task_status=trio.TASK_STATUS_IGNORED task_status=trio.TASK_STATUS_IGNORED
): ):
global _debugger_request_cs global _debugger_request_cs
with trio.CancelScope() as cs: with trio.CancelScope(shield=True) as cs:
_debugger_request_cs = cs _debugger_request_cs = cs
try: try:
async with get_root() as portal: async with get_root() as portal:
log.error('got portal')
# this syncs to child's ``Context.started()`` call. # this syncs to child's ``Context.started()`` call.
async with portal.open_context( async with portal.open_context(
@ -250,17 +302,21 @@ async def _breakpoint(
) as (ctx, val): ) as (ctx, val):
log.error('locked context')
assert val == 'Locked' assert val == 'Locked'
async with ctx.open_stream() as stream: async with ctx.open_stream() as stream:
log.error('opened stream')
# unblock local caller # unblock local caller
task_status.started() task_status.started()
# TODO: shielding currently can cause hangs... try:
# with trio.CancelScope(shield=True): await _local_pdb_complete.wait()
await _pdb_complete.wait() finally:
# TODO: shielding currently can cause hangs...
with trio.CancelScope(shield=True):
await stream.send('pdb_unlock') await stream.send('pdb_unlock')
# sync with callee termination # sync with callee termination
@ -275,11 +331,12 @@ async def _breakpoint(
_local_task_in_debug = None _local_task_in_debug = None
log.debug(f"Child {actor} released parent stdio lock") log.debug(f"Child {actor} released parent stdio lock")
if not _pdb_complete or _pdb_complete.is_set(): if not _local_pdb_complete or _local_pdb_complete.is_set():
_pdb_complete = trio.Event() _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 _local_task_in_debug:
if _local_task_in_debug == task_name: if _local_task_in_debug == task_name:
# this task already has the lock and is # this task already has the lock and is
@ -291,7 +348,7 @@ 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 _pdb_complete.wait() await _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
@ -299,11 +356,17 @@ async def _breakpoint(
_local_task_in_debug = task_name _local_task_in_debug = task_name
# assign unlock callback for debugger teardown hooks # assign unlock callback for debugger teardown hooks
_pdb_release_hook = _pdb_complete.set _pdb_release_hook = _local_pdb_complete.set
# 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
# being restricted by the scope of a new task nursery. # being restricted by the scope of a new task nursery.
# NOTE: if we want to debug a trio.Cancelled triggered exception
# we have to figure out how to avoid having the service nursery
# cancel on this task start? I *think* this works below?
# actor._service_n.cancel_scope.shield = shield
with trio.CancelScope(shield=True):
await actor._service_n.start(wait_for_parent_stdin_hijack) await actor._service_n.start(wait_for_parent_stdin_hijack)
elif is_root_process(): elif is_root_process():
@ -321,6 +384,11 @@ async def _breakpoint(
# 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():
log.warning(
'Root actor attempting to acquire active tty lock'
f' owned by {_global_actor_in_debug}')
await _debug_lock.acquire() await _debug_lock.acquire()
_global_actor_in_debug = actor.uid _global_actor_in_debug = actor.uid
@ -328,13 +396,13 @@ async def _breakpoint(
# the lock must be released on pdb completion # the lock must be released on pdb completion
def teardown(): def teardown():
global _pdb_complete, _debug_lock global _local_pdb_complete, _debug_lock
global _global_actor_in_debug, _local_task_in_debug global _global_actor_in_debug, _local_task_in_debug
_debug_lock.release() _debug_lock.release()
_global_actor_in_debug = None _global_actor_in_debug = None
_local_task_in_debug = None _local_task_in_debug = None
_pdb_complete.set() _local_pdb_complete.set()
_pdb_release_hook = teardown _pdb_release_hook = teardown
@ -362,7 +430,7 @@ def _set_trace(actor=None):
pdb = _mk_pdb() pdb = _mk_pdb()
if actor is not None: if actor is not None:
log.runtime(f"\nAttaching pdb to actor: {actor.uid}\n") # type: ignore log.pdb(f"\nAttaching pdb to actor: {actor.uid}\n") # type: ignore
pdb.set_trace( pdb.set_trace(
# start 2 levels up in user code # start 2 levels up in user code
@ -392,7 +460,7 @@ breakpoint = partial(
def _post_mortem(actor): def _post_mortem(actor):
log.runtime(f"\nAttaching to pdb in crashed actor: {actor.uid}\n") log.pdb(f"\nAttaching to pdb in crashed actor: {actor.uid}\n")
pdb = _mk_pdb() pdb = _mk_pdb()
# custom Pdb post-mortem entry # custom Pdb post-mortem entry

View File

@ -1,5 +1,6 @@
""" """
Inter-process comms abstractions Inter-process comms abstractions
""" """
import platform import platform
import typing import typing
@ -61,7 +62,6 @@ class MsgpackTCPStream:
use_list=False, use_list=False,
) )
while True: while True:
try: try:
data = await self.stream.receive_some(2**10) data = await self.stream.receive_some(2**10)
@ -88,7 +88,7 @@ class MsgpackTCPStream:
else: else:
raise raise
log.trace(f"received {data}") # type: ignore log.transport(f"received {data}") # type: ignore
if data == b'': if data == b'':
raise TransportClosed( raise TransportClosed(
@ -169,6 +169,7 @@ class Channel:
return self.msgstream.raddr if self.msgstream else None return self.msgstream.raddr if self.msgstream else None
async def connect( async def connect(
self, self,
destaddr: Tuple[Any, ...] = None, destaddr: Tuple[Any, ...] = None,
**kwargs **kwargs
@ -180,13 +181,21 @@ class Channel:
destaddr = destaddr or self._destaddr destaddr = destaddr or self._destaddr
assert isinstance(destaddr, tuple) assert isinstance(destaddr, tuple)
stream = await trio.open_tcp_stream(*destaddr, **kwargs)
stream = await trio.open_tcp_stream(
*destaddr,
**kwargs
)
self.msgstream = MsgpackTCPStream(stream) self.msgstream = MsgpackTCPStream(stream)
log.transport(
f'Opened channel to peer {self.laddr} -> {self.raddr}'
)
return stream return stream
async def send(self, item: Any) -> None: async def send(self, item: Any) -> None:
log.trace(f"send `{item}`") # type: ignore log.transport(f"send `{item}`") # type: ignore
assert self.msgstream assert self.msgstream
await self.msgstream.send(item) await self.msgstream.send(item)
@ -205,7 +214,8 @@ class Channel:
raise raise
async def aclose(self) -> None: async def aclose(self) -> None:
log.debug(
log.transport(
f'Closing channel to {self.uid} ' f'Closing channel to {self.uid} '
f'{self.laddr} -> {self.raddr}' f'{self.laddr} -> {self.raddr}'
) )
@ -234,11 +244,11 @@ class Channel:
await self.connect() await self.connect()
cancelled = cancel_scope.cancelled_caught cancelled = cancel_scope.cancelled_caught
if cancelled: if cancelled:
log.warning( log.transport(
"Reconnect timed out after 3 seconds, retrying...") "Reconnect timed out after 3 seconds, retrying...")
continue continue
else: else:
log.warning("Stream connection re-established!") log.transport("Stream connection re-established!")
# run any reconnection sequence # run any reconnection sequence
on_recon = self._recon_seq on_recon = self._recon_seq
if on_recon: if on_recon:
@ -247,7 +257,7 @@ class Channel:
except (OSError, ConnectionRefusedError): except (OSError, ConnectionRefusedError):
if not down: if not down:
down = True down = True
log.warning( log.transport(
f"Connection to {self.raddr} went down, waiting" f"Connection to {self.raddr} went down, waiting"
" for re-establishment") " for re-establishment")
await trio.sleep(1) await trio.sleep(1)

View File

@ -171,8 +171,11 @@ async def open_root_actor(
yield actor yield actor
except (Exception, trio.MultiError) as err: except (Exception, trio.MultiError) as err:
logger.exception("Actor crashed:") # with trio.CancelScope(shield=True):
await _debug._maybe_enter_pm(err) entered = await _debug._maybe_enter_pm(err)
if not entered:
logger.exception("Root actor crashed:")
# always re-raise # always re-raise
raise raise

View File

@ -28,6 +28,7 @@ from ._state import (
is_root_process, is_root_process,
_runtime_vars, _runtime_vars,
) )
from ._debug import _global_actor_in_debug
from .log import get_logger from .log import get_logger
from ._portal import Portal from ._portal import Portal
@ -154,6 +155,27 @@ async def cancel_on_completion(
# cancel the process now that we have a final result # cancel the process now that we have a final result
await portal.cancel_actor() await portal.cancel_actor()
async def do_hard_kill(
proc: trio.Process,
) -> None:
# NOTE: this timeout used to do nothing since we were shielding
# the ``.wait()`` inside ``new_proc()`` which will pretty much
# never release until the process exits, now it acts as
# a hard-kill time ultimatum.
with trio.move_on_after(3) as cs:
# NOTE: This ``__aexit__()`` shields internally.
async with proc: # calls ``trio.Process.aclose()``
log.debug(f"Terminating {proc}")
if cs.cancelled_caught:
# XXX: should pretty much never get here unless we have
# to move the bits from ``proc.__aexit__()`` out and
# into here.
log.critical(f"HARD KILLING {proc}")
proc.kill()
@asynccontextmanager @asynccontextmanager
async def spawn_subactor( async def spawn_subactor(
@ -188,46 +210,46 @@ async def spawn_subactor(
yield proc yield proc
finally: finally:
log.debug(f"Attempting to kill {proc}") log.runtime(f"Attempting to kill {proc}")
# XXX: do this **after** cancellation/tearfown # XXX: do this **after** cancellation/tearfown
# to avoid killing the process too early # to avoid killing the process too early
# since trio does this internally on ``__aexit__()`` # since trio does this internally on ``__aexit__()``
if ( # if (
is_root_process() # is_root_process()
# XXX: basically the pre-closing of stdstreams in a # # XXX: basically the pre-closing of stdstreams in a
# root-processe's ``trio.Process.aclose()`` can clobber # # root-processe's ``trio.Process.aclose()`` can clobber
# any existing debugger session so we avoid # # any existing debugger session so we avoid
and _runtime_vars['_debug_mode'] # and _runtime_vars['_debug_mode']
): # and _global_actor_in_debug is not None
# XXX: this is ``trio.Process.aclose()`` minus # ):
# the std-streams pre-closing steps and ``Process.kill()`` # # XXX: this is ``trio.Process.aclose()`` MINUS the
# calls. # # std-streams pre-closing steps inside ``proc.__aexit__()``
try: # # (see below) which incluses a ``Process.kill()`` call
await proc.wait()
finally:
if proc.returncode is None:
# XXX: skip this when in debug and a session might
# still be live
# proc.kill()
with trio.CancelScope(shield=True):
await proc.wait()
else:
# NOTE: this timeout used to do nothing since we were shielding
# the ``.wait()`` inside ``new_proc()`` which will pretty much
# never release until the process exits, now it acts as
# a hard-kill time ultimatum.
with trio.move_on_after(3) as cs:
# NOTE: This ``__aexit__()`` shields internally. # log.error(
async with proc: # calls ``trio.Process.aclose()`` # "Root process tty is locked in debug mode by "
log.debug(f"Terminating {proc}") # f"{_global_actor_in_debug}. If the console is hanging, you "
# "may need to trigger a KBI to kill any "
# "not-fully-initialized" " subprocesses and allow errors "
# "from `trio` to propagate"
# )
# try:
# # one more graceful wait try can can be cancelled by KBI
# # sent by user.
# await proc.wait()
if cs.cancelled_caught: # finally:
log.critical(f"HARD KILLING {proc}") # if proc.returncode is None:
proc.kill() # # with trio.CancelScope(shield=True):
# # await proc.wait()
# await do_hard_kill(proc)
# else:
await do_hard_kill(proc)
async def new_proc( async def new_proc(
@ -304,9 +326,14 @@ async def new_proc(
# reaping more stringently without the shield # reaping more stringently without the shield
# we used to have below... # we used to have below...
# always "hard" join sub procs:
# no actor zombies allowed
# with trio.CancelScope(shield=True): # with trio.CancelScope(shield=True):
# async with proc:
# Always "hard" join sub procs since no actor zombies
# are allowed!
# this is a "light" (cancellable) join, the hard join is
# in the enclosing scope (see above).
await proc.wait() await proc.wait()
log.debug(f"Joined {proc}") log.debug(f"Joined {proc}")

View File

@ -170,16 +170,25 @@ class ActorNursery:
log.warning(f"Cancelling nursery in {self._actor.uid}") log.warning(f"Cancelling nursery in {self._actor.uid}")
with trio.move_on_after(3) as cs: with trio.move_on_after(3) as cs:
async with trio.open_nursery() as nursery: async with trio.open_nursery() as nursery:
for subactor, proc, portal in self._children.values(): for subactor, proc, portal in self._children.values():
# TODO: are we ever even going to use this or
# is the spawning backend responsible for such
# things? I'm thinking latter.
if hard_kill: if hard_kill:
proc.terminate() proc.terminate()
else: else:
if portal is None: # actor hasn't fully spawned yet if portal is None: # actor hasn't fully spawned yet
event = self._actor._peer_connected[subactor.uid] event = self._actor._peer_connected[subactor.uid]
log.warning( log.warning(
f"{subactor.uid} wasn't finished spawning?") f"{subactor.uid} wasn't finished spawning?")
await event.wait() await event.wait()
# channel/portal should now be up # channel/portal should now be up
_, _, portal = self._children[subactor.uid] _, _, portal = self._children[subactor.uid]
@ -239,6 +248,7 @@ async def _open_and_supervise_one_cancels_all_nursery(
# As such if the strategy propagates any error(s) upwards # As such if the strategy propagates any error(s) upwards
# the above "daemon actor" nursery will be notified. # the above "daemon actor" nursery will be notified.
async with trio.open_nursery() as ria_nursery: async with trio.open_nursery() as ria_nursery:
anursery = ActorNursery( anursery = ActorNursery(
actor, actor,
ria_nursery, ria_nursery,
@ -249,35 +259,53 @@ async def _open_and_supervise_one_cancels_all_nursery(
# spawning of actors happens in the caller's scope # spawning of actors happens in the caller's scope
# after we yield upwards # after we yield upwards
yield anursery yield anursery
log.debug(
log.runtime(
f"Waiting on subactors {anursery._children} " f"Waiting on subactors {anursery._children} "
"to complete" "to complete"
) )
# Last bit before first nursery block ends in the case
# where we didn't error in the caller's scope
# signal all process monitor tasks to conduct
# hard join phase.
anursery._join_procs.set()
except BaseException as err: except BaseException as err:
if is_root_process() and ( # If we error in the root but the debugger is
type(err) in {
Exception, trio.MultiError, trio.Cancelled
}
):
# 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 streams. # thus clobber access to) the local tty since it
# instead try to wait for pdb to be released before # will make the pdb repl unusable.
# Instead try to wait for pdb to be released before
# tearing down. # tearing down.
debug_complete = _debug._pdb_complete if is_root_process():
if debug_complete and not debug_complete.is_set(): log.exception(f"we're root with {err}")
log.warning(
"Root has errored but pdb is active..waiting "
"on debug lock")
await _debug._pdb_complete.wait()
# raise # wait to see if a sub-actor task
# will be scheduled and grab the tty
# lock on the next tick
# await trio.testing.wait_all_tasks_blocked()
debug_complete = _debug._no_remote_has_tty
if (
debug_complete and
not debug_complete.is_set()
):
log.warning(
'Root has errored but pdb is in use by '
f'child {_debug._global_actor_in_debug}\n'
'Waiting on tty lock to release..')
with trio.CancelScope(shield=True):
await debug_complete.wait()
# if the caller's scope errored then we activate our # if the caller's scope errored then we activate our
# one-cancels-all supervisor strategy (don't # one-cancels-all supervisor strategy (don't
# worry more are coming). # worry more are coming).
anursery._join_procs.set() anursery._join_procs.set()
try: try:
# XXX: hypothetically an error could be # XXX: hypothetically an error could be
# raised and then a cancel signal shows up # raised and then a cancel signal shows up
@ -313,15 +341,18 @@ async def _open_and_supervise_one_cancels_all_nursery(
else: else:
raise raise
# Last bit before first nursery block ends in the case
# where we didn't error in the caller's scope
log.debug("Waiting on all subactors to complete")
anursery._join_procs.set()
# ria_nursery scope end # ria_nursery scope end
# XXX: do we need a `trio.Cancelled` catch here as well? # XXX: do we need a `trio.Cancelled` catch here as well?
except (Exception, trio.MultiError, trio.Cancelled) as err: # this is the catch around the ``.run_in_actor()`` nursery
except (
Exception,
trio.MultiError,
trio.Cancelled
) as err:
# If actor-local error was raised while waiting on # If actor-local error was raised while waiting on
# ".run_in_actor()" actors then we also want to cancel all # ".run_in_actor()" actors then we also want to cancel all
# remaining sub-actors (due to our lone strategy: # remaining sub-actors (due to our lone strategy:

View File

@ -29,7 +29,7 @@ LOG_FORMAT = (
DATE_FORMAT = '%b %d %H:%M:%S' DATE_FORMAT = '%b %d %H:%M:%S'
LEVELS = { LEVELS = {
'GARBAGE': 1, 'GARBAGE': 1,
'TRACE': 5, 'TRANSPORT': 5,
'RUNTIME': 15, 'RUNTIME': 15,
'PDB': 500, 'PDB': 500,
'QUIET': 1000, 'QUIET': 1000,
@ -42,7 +42,7 @@ STD_PALETTE = {
'INFO': 'green', 'INFO': 'green',
'RUNTIME': 'white', 'RUNTIME': 'white',
'DEBUG': 'white', 'DEBUG': 'white',
'TRACE': 'cyan', 'TRANSPORT': 'cyan',
'GARBAGE': 'blue', 'GARBAGE': 'blue',
} }
BOLD_PALETTE = { BOLD_PALETTE = {
@ -77,7 +77,7 @@ def get_logger(
# additional levels # additional levels
for name, val in LEVELS.items(): for name, val in LEVELS.items():
logging.addLevelName(val, name) logging.addLevelName(val, name)
# ex. create ``logger.trace()`` # ex. create ``logger.runtime()``
setattr(logger, name.lower(), partial(logger.log, val)) setattr(logger, name.lower(), partial(logger.log, val))
return logger return logger