Merge pull request #245 from goodboy/immediate_remote_cancels

Immediate remote cancels
pubsub_startup_response_msg
goodboy 2021-10-17 08:16:50 -04:00 committed by GitHub
commit 828754dbb5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 591 additions and 398 deletions

View File

@ -1,3 +1,8 @@
'''
Test that a nested nursery will avoid clobbering
the debugger latched by a broken child.
'''
import trio import trio
import tractor import tractor
@ -35,6 +40,7 @@ async def main():
""" """
async with tractor.open_nursery( async with tractor.open_nursery(
debug_mode=True, debug_mode=True,
# loglevel='cancel',
) as n: ) as n:
# spawn both actors # spawn both actors

View File

@ -0,0 +1,13 @@
Change the core message loop to handle task and actor-runtime cancel
requests immediately instead of scheduling them as is done for rpc-task
requests.
In order to obtain more reliable teardown mechanics for (complex) actor
trees it's important that we specially treat cancel requests as having
higher priority. Previously, it was possible that task cancel requests
could actually also themselves be cancelled if a "actor-runtime" cancel
request was received (can happen during messy multi actor crashes that
propagate). Instead cancels now block the msg loop until serviced and
a response is relayed back to the requester. This also allows for
improved debugger support since we have determinism guarantees about
which processes must wait before hard killing their children.

View File

@ -4,5 +4,5 @@ now and use the default `fragment set`_.
.. _towncrier docs: https://github.com/twisted/towncrier#quick-start .. _towncrier docs: https://github.com/twisted/towncrier#quick-start
.. _pluggy release readme: https://github.com/twisted/towncrier#quick-start .. _pluggy release readme: https://github.com/pytest-dev/pluggy/blob/main/changelog/README.rst
.. _fragment set: https://github.com/twisted/towncrier#news-fragments .. _fragment set: https://github.com/twisted/towncrier#news-fragments

View File

@ -1,5 +1,6 @@
""" """
Cancellation and error propagation Cancellation and error propagation
""" """
import os import os
import signal import signal
@ -365,7 +366,8 @@ async def test_nested_multierrors(loglevel, start_method):
# to happen before an actor is spawned # to happen before an actor is spawned
if isinstance(subexc, trio.Cancelled): if isinstance(subexc, trio.Cancelled):
continue continue
else:
elif isinstance(subexc, tractor.RemoteActorError):
# on windows it seems we can't exactly be sure wtf # on windows it seems we can't exactly be sure wtf
# will happen.. # will happen..
assert subexc.type in ( assert subexc.type in (
@ -373,6 +375,17 @@ async def test_nested_multierrors(loglevel, start_method):
trio.Cancelled, trio.Cancelled,
trio.MultiError trio.MultiError
) )
elif isinstance(subexc, trio.MultiError):
for subsub in subexc.exceptions:
if subsub in (tractor.RemoteActorError,):
subsub = subsub.type
assert type(subsub) in (
trio.Cancelled,
trio.MultiError,
)
else: else:
assert isinstance(subexc, tractor.RemoteActorError) assert isinstance(subexc, tractor.RemoteActorError)
@ -381,13 +394,14 @@ async def test_nested_multierrors(loglevel, start_method):
# on windows sometimes spawning is just too slow and # on windows sometimes spawning is just too slow and
# we get back the (sent) cancel signal instead # we get back the (sent) cancel signal instead
if platform.system() == 'Windows': if platform.system() == 'Windows':
assert (subexc.type is trio.MultiError) or ( if isinstance(subexc, tractor.RemoteActorError):
subexc.type is tractor.RemoteActorError) assert subexc.type in (trio.MultiError, tractor.RemoteActorError)
else:
assert isinstance(subexc, trio.MultiError)
else: else:
assert subexc.type is trio.MultiError assert subexc.type is trio.MultiError
else: else:
assert (subexc.type is tractor.RemoteActorError) or ( assert subexc.type in (tractor.RemoteActorError, trio.Cancelled)
subexc.type is trio.Cancelled)
@no_windows @no_windows
@ -448,6 +462,7 @@ def test_cancel_via_SIGINT_other_task(
with pytest.raises(KeyboardInterrupt): with pytest.raises(KeyboardInterrupt):
trio.run(main) trio.run(main)
async def spin_for(period=3): async def spin_for(period=3):
"Sync sleep." "Sync sleep."
time.sleep(period) time.sleep(period)

View File

@ -236,7 +236,8 @@ def test_subactor_breakpoint(spawn):
def test_multi_subactors(spawn): def test_multi_subactors(spawn):
"""Multiple subactors, both erroring and breakpointing as well as """
Multiple subactors, both erroring and breakpointing as well as
a nested subactor erroring. a nested subactor erroring.
""" """
child = spawn(r'multi_subactors') child = spawn(r'multi_subactors')
@ -259,6 +260,7 @@ def test_multi_subactors(spawn):
# first name_error failure # first name_error failure
child.expect(r"\(Pdb\+\+\)") child.expect(r"\(Pdb\+\+\)")
before = str(child.before.decode()) before = str(child.before.decode())
assert "Attaching to pdb in crashed actor: ('name_error'" in before
assert "NameError" in before assert "NameError" in before
# continue again # continue again
@ -267,6 +269,7 @@ def test_multi_subactors(spawn):
# 2nd name_error failure # 2nd name_error failure
child.expect(r"\(Pdb\+\+\)") child.expect(r"\(Pdb\+\+\)")
before = str(child.before.decode()) before = str(child.before.decode())
assert "Attaching to pdb in crashed actor: ('name_error_1'" in before
assert "NameError" in before assert "NameError" in before
# breakpoint loop should re-engage # breakpoint loop should re-engage
@ -275,6 +278,19 @@ def test_multi_subactors(spawn):
before = str(child.before.decode()) before = str(child.before.decode())
assert "Attaching pdb to actor: ('breakpoint_forever'" in before assert "Attaching pdb to actor: ('breakpoint_forever'" in before
# wait for spawn error to show up
while 'breakpoint_forever' in before:
child.sendline('c')
child.expect(r"\(Pdb\+\+\)")
before = str(child.before.decode())
# 2nd depth nursery should trigger
# child.sendline('c')
# child.expect(r"\(Pdb\+\+\)")
# before = str(child.before.decode())
assert "Attaching to pdb in crashed actor: ('spawn_error'" in before
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):
child.sendline('c') child.sendline('c')
@ -284,16 +300,24 @@ def test_multi_subactors(spawn):
child.sendline('q') child.sendline('q')
child.expect(r"\(Pdb\+\+\)") child.expect(r"\(Pdb\+\+\)")
before = str(child.before.decode()) before = str(child.before.decode())
# debugger attaches to root
assert "Attaching to pdb in crashed actor: ('root'" in before assert "Attaching to pdb in crashed actor: ('root'" in before
# expect a multierror with exceptions for each sub-actor
assert "RemoteActorError: ('breakpoint_forever'" in before assert "RemoteActorError: ('breakpoint_forever'" in before
assert "RemoteActorError: ('name_error'" in before
assert "RemoteActorError: ('spawn_error'" in before
assert "RemoteActorError: ('name_error_1'" in before
assert 'bdb.BdbQuit' in before assert 'bdb.BdbQuit' in before
# process should exit # process should exit
child.sendline('c') child.sendline('c')
child.expect(pexpect.EOF) child.expect(pexpect.EOF)
# repeat of previous multierror for final output
before = str(child.before.decode()) before = str(child.before.decode())
assert "RemoteActorError: ('breakpoint_forever'" in before assert "RemoteActorError: ('breakpoint_forever'" in before
assert "RemoteActorError: ('name_error'" in before
assert "RemoteActorError: ('spawn_error'" in before
assert "RemoteActorError: ('name_error_1'" in before
assert 'bdb.BdbQuit' in before assert 'bdb.BdbQuit' in before
@ -387,16 +411,29 @@ def test_multi_subactors_root_errors(spawn):
before = str(child.before.decode()) before = str(child.before.decode())
assert "NameError: name 'doggypants' is not defined" in before assert "NameError: name 'doggypants' is not defined" in before
# continue again # continue again to catch 2nd name error from
# actor 'name_error_1' (which is 2nd depth).
child.sendline('c') child.sendline('c')
child.expect(r"\(Pdb\+\+\)") child.expect(r"\(Pdb\+\+\)")
# should now get attached in root with assert error
before = str(child.before.decode()) before = str(child.before.decode())
assert "Attaching to pdb in crashed actor: ('name_error_1'" in before
assert "NameError" in before
# should have come just after priot prompt child.sendline('c')
child.expect(r"\(Pdb\+\+\)")
before = str(child.before.decode())
assert "Attaching to pdb in crashed actor: ('spawn_error'" in before
# boxed error from previous step
assert "RemoteActorError: ('name_error_1'" in before
assert "NameError" in before
child.sendline('c')
child.expect(r"\(Pdb\+\+\)")
before = str(child.before.decode())
assert "Attaching to pdb in crashed actor: ('root'" in before assert "Attaching to pdb in crashed actor: ('root'" in before
assert "AssertionError" in before # boxed error from first level failure
assert "RemoteActorError: ('name_error'" in before
assert "NameError" in before
# 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
@ -406,6 +443,7 @@ def test_multi_subactors_root_errors(spawn):
child.expect(pexpect.EOF) child.expect(pexpect.EOF)
before = str(child.before.decode()) before = str(child.before.decode())
# error from root actor and root task that created top level nursery
assert "AssertionError" in before assert "AssertionError" in before

View File

@ -180,6 +180,7 @@ def test_multi_actor_subs_arbiter_pub(
'streamer', 'streamer',
enable_modules=[__name__], enable_modules=[__name__],
) )
name = 'streamer'
even_portal = await n.run_in_actor( even_portal = await n.run_in_actor(
subs, subs,

View File

@ -49,6 +49,7 @@ async def _invoke(
chan: Channel, chan: Channel,
func: typing.Callable, func: typing.Callable,
kwargs: Dict[str, Any], kwargs: Dict[str, Any],
is_rpc: bool = True,
task_status: TaskStatus[ task_status: TaskStatus[
Union[trio.CancelScope, BaseException] Union[trio.CancelScope, BaseException]
] = trio.TASK_STATUS_IGNORED, ] = trio.TASK_STATUS_IGNORED,
@ -243,6 +244,7 @@ async def _invoke(
scope, func, is_complete = actor._rpc_tasks.pop((chan, cid)) scope, func, is_complete = actor._rpc_tasks.pop((chan, cid))
is_complete.set() is_complete.set()
except KeyError: except KeyError:
if is_rpc:
# If we're cancelled before the task returns then the # If we're cancelled before the task returns then the
# cancel scope will not have been inserted yet # cancel scope will not have been inserted yet
log.warning( log.warning(
@ -503,8 +505,8 @@ class Actor:
log.runtime(f"Peers is {self._peers}") log.runtime(f"Peers is {self._peers}")
if not self._peers: # no more channels connected if not self._peers: # no more channels connected
self._no_more_peers.set()
log.runtime("Signalling no more peer channels") log.runtime("Signalling no more peer channels")
self._no_more_peers.set()
# # XXX: is this necessary (GC should do it?) # # XXX: is this necessary (GC should do it?)
if chan.connected(): if chan.connected():
@ -671,16 +673,39 @@ class Actor:
f"{ns}.{funcname}({kwargs})") f"{ns}.{funcname}({kwargs})")
if ns == 'self': if ns == 'self':
func = getattr(self, funcname) func = getattr(self, funcname)
if funcname == 'cancel':
# don't start entire actor runtime cancellation if this
# actor is in debug mode
pdb_complete = _debug._local_pdb_complete
if pdb_complete:
await pdb_complete.wait()
# we immediately start the runtime machinery shutdown
with trio.CancelScope(shield=True):
# self.cancel() was called so kill this msg loop
# and break out into ``_async_main()``
log.cancel(
f"Actor {self.uid} was remotely cancelled; "
"waiting on cancellation completion..")
await _invoke(self, cid, chan, func, kwargs, is_rpc=False)
# await self._cancel_complete.wait()
loop_cs.cancel()
break
if funcname == '_cancel_task': if funcname == '_cancel_task':
# XXX: a special case is made here for
# remote calls since we don't want the # we immediately start the runtime machinery shutdown
# remote actor have to know which channel with trio.CancelScope(shield=True):
# the task is associated with and we can't # self.cancel() was called so kill this msg loop
# pass non-primitive types between actors. # and break out into ``_async_main()``
# This means you can use:
# Portal.run('self', '_cancel_task, cid=did)
# without passing the `chan` arg.
kwargs['chan'] = chan kwargs['chan'] = chan
log.cancel(
f"Actor {self.uid} was remotely cancelled; "
"waiting on cancellation completion..")
await _invoke(self, cid, chan, func, kwargs, is_rpc=False)
continue
else: else:
# complain to client about restricted modules # complain to client about restricted modules
try: try:
@ -699,14 +724,15 @@ class Actor:
partial(_invoke, self, cid, chan, func, kwargs), partial(_invoke, self, cid, chan, func, kwargs),
name=funcname, name=funcname,
) )
except RuntimeError: except (RuntimeError, trio.MultiError):
# avoid reporting a benign race condition # avoid reporting a benign race condition
# during actor runtime teardown. # during actor runtime teardown.
nursery_cancelled_before_task = True nursery_cancelled_before_task = True
break
# never allow cancelling cancel requests (results in # never allow cancelling cancel requests (results in
# deadlock and other weird behaviour) # deadlock and other weird behaviour)
if func != self.cancel: # if func != self.cancel:
if isinstance(cs, Exception): if isinstance(cs, Exception):
log.warning( log.warning(
f"Task for RPC func {func} failed with" f"Task for RPC func {func} failed with"
@ -719,20 +745,11 @@ class Actor:
# cancelled gracefully if requested # cancelled gracefully if requested
self._rpc_tasks[(chan, cid)] = ( self._rpc_tasks[(chan, cid)] = (
cs, func, trio.Event()) cs, func, trio.Event())
else:
# self.cancel() was called so kill this msg loop
# and break out into ``_async_main()``
log.warning(
f"Actor {self.uid} was remotely cancelled; "
"waiting on cancellation completion..")
await self._cancel_complete.wait()
loop_cs.cancel()
break
log.runtime( log.runtime(
f"Waiting on next msg for {chan} from {chan.uid}") f"Waiting on next msg for {chan} from {chan.uid}")
else:
# channel disconnect # end of async for, channel disconnect vis ``trio.EndOfChannel``
log.runtime( log.runtime(
f"{chan} for {chan.uid} disconnected, cancelling tasks" f"{chan} for {chan.uid} disconnected, cancelling tasks"
) )
@ -947,6 +964,9 @@ class Actor:
# Blocks here as expected until the root nursery is # Blocks here as expected until the root nursery is
# killed (i.e. this actor is cancelled or signalled by the parent) # killed (i.e. this actor is cancelled or signalled by the parent)
except Exception as err: except Exception as err:
log.info("Closing all actor lifetime contexts")
_lifetime_stack.close()
if not registered_with_arbiter: if not registered_with_arbiter:
# TODO: I guess we could try to connect back # TODO: I guess we could try to connect back
# to the parent through a channel and engage a debugger # to the parent through a channel and engage a debugger
@ -976,11 +996,21 @@ class Actor:
raise raise
finally: finally:
log.runtime("Root nursery complete") log.info("Runtime nursery complete")
# tear down all lifetime contexts if not in guest mode # tear down all lifetime contexts if not in guest mode
# XXX: should this just be in the entrypoint? # XXX: should this just be in the entrypoint?
log.cancel("Closing all actor lifetime contexts") log.info("Closing all actor lifetime contexts")
# TODO: we can't actually do this bc the debugger
# uses the _service_n to spawn the lock task, BUT,
# in theory if we had the root nursery surround this finally
# block it might be actually possible to debug THIS
# machinery in the same way as user task code?
# if self.name == 'brokerd.ib':
# with trio.CancelScope(shield=True):
# await _debug.breakpoint()
_lifetime_stack.close() _lifetime_stack.close()
# Unregister actor from the arbiter # Unregister actor from the arbiter
@ -1065,7 +1095,7 @@ class Actor:
self._service_n.start_soon(self.cancel) self._service_n.start_soon(self.cancel)
async def cancel(self) -> bool: async def cancel(self) -> bool:
"""Cancel this actor. """Cancel this actor's runtime.
The "deterministic" teardown sequence in order is: The "deterministic" teardown sequence in order is:
- cancel all ongoing rpc tasks by cancel scope - cancel all ongoing rpc tasks by cancel scope
@ -1099,7 +1129,7 @@ class Actor:
if self._service_n: if self._service_n:
self._service_n.cancel_scope.cancel() self._service_n.cancel_scope.cancel()
log.cancel(f"{self.uid} was sucessfullly cancelled") log.cancel(f"{self.uid} called `Actor.cancel()`")
self._cancel_complete.set() self._cancel_complete.set()
return True return True
@ -1158,13 +1188,15 @@ class Actor:
registered for each. registered for each.
""" """
tasks = self._rpc_tasks tasks = self._rpc_tasks
if tasks:
log.cancel(f"Cancelling all {len(tasks)} rpc tasks:\n{tasks} ") log.cancel(f"Cancelling all {len(tasks)} rpc tasks:\n{tasks} ")
for (chan, cid) in tasks.copy(): for (chan, cid), (scope, func, is_complete) in tasks.copy().items():
if only_chan is not None: if only_chan is not None:
if only_chan != chan: if only_chan != chan:
continue continue
# TODO: this should really done in a nursery batch # TODO: this should really done in a nursery batch
if func != self._cancel_task:
await self._cancel_task(cid, chan) await self._cancel_task(cid, chan)
log.cancel( log.cancel(

View File

@ -5,16 +5,23 @@ Multi-core debugging for da peeps!
import bdb import bdb
import sys import sys
from functools import partial from functools import partial
from contextlib import asynccontextmanager from contextlib import asynccontextmanager as acm
from typing import Tuple, Optional, Callable, AsyncIterator from typing import (
Tuple,
Optional,
Callable,
AsyncIterator,
AsyncGenerator,
)
import tractor import tractor
import trio import trio
from trio_typing import TaskStatus
from .log import get_logger from .log import get_logger
from . import _state from . import _state
from ._discovery import get_root from ._discovery import get_root
from ._state import is_root_process from ._state import is_root_process, debug_mode
from ._exceptions import is_multi_cancelled from ._exceptions import is_multi_cancelled
try: try:
@ -122,7 +129,7 @@ class PdbwTeardown(pdbpp.Pdb):
# break # break
@asynccontextmanager @acm
async def _acquire_debug_lock( async def _acquire_debug_lock(
uid: Tuple[str, str] uid: Tuple[str, str]
@ -139,7 +146,7 @@ async def _acquire_debug_lock(
task_name = trio.lowlevel.current_task().name task_name = trio.lowlevel.current_task().name
log.pdb( log.debug(
f"Attempting to acquire TTY lock, remote task: {task_name}:{uid}" f"Attempting to acquire TTY lock, remote task: {task_name}:{uid}"
) )
@ -187,7 +194,7 @@ async def _acquire_debug_lock(
if ( if (
not stats.owner not stats.owner
): ):
log.pdb(f"No more tasks waiting on tty lock! says {uid}") log.debug(f"No more tasks waiting on tty lock! says {uid}")
_no_remote_has_tty.set() _no_remote_has_tty.set()
_no_remote_has_tty = None _no_remote_has_tty = None
@ -219,7 +226,8 @@ async def _hijack_stdin_for_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 '''
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 the pdbpp debugger console can be allocated to a sub-actor for repl
bossing. bossing.
@ -254,6 +262,8 @@ async def _hijack_stdin_for_child(
# assert await stream.receive() == 'pdb_unlock' # assert await stream.receive() == 'pdb_unlock'
except ( except (
# BaseException,
trio.MultiError,
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
@ -268,12 +278,74 @@ async def _hijack_stdin_for_child(
if isinstance(err, trio.Cancelled): if isinstance(err, trio.Cancelled):
raise raise
finally:
log.debug(f"TTY lock released, remote task: {task_name}:{subactor_uid}") log.debug(
"TTY lock released, remote task:"
f"{task_name}:{subactor_uid}")
return "pdb_unlock_complete" return "pdb_unlock_complete"
async def wait_for_parent_stdin_hijack(
actor_uid: Tuple[str, str],
task_status: TaskStatus[trio.CancelScope] = trio.TASK_STATUS_IGNORED
):
'''
Connect to the root actor via a ctx and invoke a task which locks
a root-local TTY lock.
This function is used by any sub-actor to acquire mutex access to
pdb and the root's TTY for interactive debugging (see below inside
``_breakpoint()``). It can be used to ensure that an intermediate
nursery-owning actor does not clobber its children if they are in
debug (see below inside ``maybe_wait_for_debugger()``).
'''
global _debugger_request_cs
with trio.CancelScope(shield=True) as cs:
_debugger_request_cs = cs
try:
async with get_root() as portal:
# this syncs to child's ``Context.started()`` call.
async with portal.open_context(
tractor._debug._hijack_stdin_for_child,
subactor_uid=actor_uid,
) as (ctx, val):
log.pdb('locked context')
assert val == 'Locked'
async with ctx.open_stream() as stream:
# unblock local caller
task_status.started(cs)
try:
assert _local_pdb_complete
await _local_pdb_complete.wait()
finally:
# TODO: shielding currently can cause hangs...
with trio.CancelScope(shield=True):
await stream.send('pdb_unlock')
# sync with callee termination
assert await ctx.result() == "pdb_unlock_complete"
except tractor.ContextCancelled:
log.warning('Root actor cancelled debug lock')
finally:
log.debug(f"Exiting debugger for actor {actor_uid}")
global _local_task_in_debug
_local_task_in_debug = None
log.debug(f"Child {actor_uid} released parent stdio lock")
async def _breakpoint( async def _breakpoint(
debug_func, debug_func,
@ -300,56 +372,6 @@ async def _breakpoint(
await trio.lowlevel.checkpoint() await trio.lowlevel.checkpoint()
async def wait_for_parent_stdin_hijack(
task_status=trio.TASK_STATUS_IGNORED
):
global _debugger_request_cs
with trio.CancelScope(shield=True) as cs:
_debugger_request_cs = cs
try:
async with get_root() as portal:
log.pdb('got portal')
# this syncs to child's ``Context.started()`` call.
async with portal.open_context(
tractor._debug._hijack_stdin_for_child,
subactor_uid=actor.uid,
) as (ctx, val):
log.pdb('locked context')
assert val == 'Locked'
async with ctx.open_stream() as stream:
log.error('opened stream')
# unblock local caller
task_status.started()
try:
await _local_pdb_complete.wait()
finally:
# TODO: shielding currently can cause hangs...
with trio.CancelScope(shield=True):
await stream.send('pdb_unlock')
# sync with callee termination
assert await ctx.result() == "pdb_unlock_complete"
except tractor.ContextCancelled:
log.warning('Root actor cancelled debug lock')
finally:
log.debug(f"Exiting debugger for actor {actor}")
global _local_task_in_debug
_local_task_in_debug = None
log.debug(f"Child {actor} released parent stdio lock")
if not _local_pdb_complete or _local_pdb_complete.is_set(): if not _local_pdb_complete or _local_pdb_complete.is_set():
_local_pdb_complete = trio.Event() _local_pdb_complete = trio.Event()
@ -386,7 +408,10 @@ async def _breakpoint(
# cancel on this task start? I *think* this works below? # cancel on this task start? I *think* this works below?
# actor._service_n.cancel_scope.shield = shield # actor._service_n.cancel_scope.shield = shield
with trio.CancelScope(shield=True): 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,
actor.uid,
)
elif is_root_process(): elif is_root_process():
@ -407,11 +432,10 @@ async def _breakpoint(
'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 {_global_actor_in_debug}')
with trio.CancelScope(shield=True):
# 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):
await _debug_lock.acquire() await _debug_lock.acquire()
else: else:
# may be cancelled # may be cancelled
await _debug_lock.acquire() await _debug_lock.acquire()
@ -501,7 +525,7 @@ post_mortem = partial(
async def _maybe_enter_pm(err): async def _maybe_enter_pm(err):
if ( if (
_state.debug_mode() debug_mode()
# NOTE: don't enter debug mode recursively after quitting pdb # NOTE: don't enter debug mode recursively after quitting pdb
# Iow, don't re-enter the repl if the `quit` command was issued # Iow, don't re-enter the repl if the `quit` command was issued
@ -524,3 +548,80 @@ async def _maybe_enter_pm(err):
else: else:
return False return False
@acm
async def acquire_debug_lock(
subactor_uid: Tuple[str, str],
) -> AsyncGenerator[None, tuple]:
'''
Grab root's debug lock on entry, release on exit.
'''
async with trio.open_nursery() as n:
cs = await n.start(
wait_for_parent_stdin_hijack,
subactor_uid,
)
yield None
cs.cancel()
async def maybe_wait_for_debugger(
poll_steps: int = 2,
poll_delay: float = 0.1,
) -> None:
if not debug_mode():
return
if (
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
# engaged we don't want to prematurely kill (and
# thus clobber access to) the local tty since it
# will make the pdb repl unusable.
# Instead try to wait for pdb to be released before
# tearing down.
sub_in_debug = None
for _ in range(poll_steps):
if _global_actor_in_debug:
sub_in_debug = tuple(_global_actor_in_debug)
log.warning(
'Root polling for debug')
with trio.CancelScope(shield=True):
await trio.sleep(poll_delay)
# TODO: could this make things more deterministic? wait
# to see if a sub-actor task will be scheduled and grab
# the tty lock on the next tick?
# XXX: doesn't seem to work
# await trio.testing.wait_all_tasks_blocked(cushion=0)
debug_complete = _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 {sub_in_debug}\n'
'Waiting on tty lock to release..')
await debug_complete.wait()
await trio.sleep(poll_delay)
continue
else:
log.warning(
'Root acquired TTY LOCK'
)
return

View File

@ -44,7 +44,7 @@ async def get_arbiter(
@asynccontextmanager @asynccontextmanager
async def get_root( async def get_root(
**kwargs, **kwargs,
) -> typing.AsyncGenerator[Union[Portal, LocalPortal], None]: ) -> typing.AsyncGenerator[Portal, None]:
host, port = _runtime_vars['_root_mailbox'] host, port = _runtime_vars['_root_mailbox']
assert host is not None assert host is not None

View File

@ -1,5 +1,6 @@
""" """
Machinery for actor process spawning using multiple backends. Machinery for actor process spawning using multiple backends.
""" """
import sys import sys
import multiprocessing as mp import multiprocessing as mp
@ -8,7 +9,6 @@ from typing import Any, Dict, Optional
import trio import trio
from trio_typing import TaskStatus from trio_typing import TaskStatus
from async_generator import asynccontextmanager
try: try:
from multiprocessing import semaphore_tracker # type: ignore from multiprocessing import semaphore_tracker # type: ignore
@ -22,9 +22,15 @@ from multiprocessing import forkserver # type: ignore
from typing import Tuple from typing import Tuple
from . import _forkserver_override from . import _forkserver_override
from ._debug import (
maybe_wait_for_debugger,
acquire_debug_lock,
)
from ._state import ( from ._state import (
current_actor, current_actor,
is_main_process, is_main_process,
is_root_process,
debug_mode,
) )
from .log import get_logger from .log import get_logger
@ -123,17 +129,15 @@ async def cancel_on_completion(
portal: Portal, portal: Portal,
actor: Actor, actor: Actor,
errors: Dict[Tuple[str, str], Exception], errors: Dict[Tuple[str, str], Exception],
task_status: TaskStatus[trio.CancelScope] = trio.TASK_STATUS_IGNORED,
) -> None: ) -> None:
"""Cancel actor gracefully once it's "main" portal's """
Cancel actor gracefully once it's "main" portal's
result arrives. result arrives.
Should only be called for actors spawned with `run_in_actor()`. Should only be called for actors spawned with `run_in_actor()`.
""" """
with trio.CancelScope() as cs:
task_status.started(cs)
# if this call errors we store the exception for later # if this call errors we store the exception for later
# in ``errors`` which will be reraised inside # in ``errors`` which will be reraised inside
# a MultiError and we still send out a cancel request # a MultiError and we still send out a cancel request
@ -155,12 +159,13 @@ async def cancel_on_completion(
async def do_hard_kill( async def do_hard_kill(
proc: trio.Process, proc: trio.Process,
terminate_after: int = 3,
) -> None: ) -> None:
# NOTE: this timeout used to do nothing since we were shielding # NOTE: this timeout used to do nothing since we were shielding
# the ``.wait()`` inside ``new_proc()`` which will pretty much # the ``.wait()`` inside ``new_proc()`` which will pretty much
# never release until the process exits, now it acts as # never release until the process exits, now it acts as
# a hard-kill time ultimatum. # a hard-kill time ultimatum.
with trio.move_on_after(3) as cs: with trio.move_on_after(terminate_after) as cs:
# NOTE: This ``__aexit__()`` shields internally. # NOTE: This ``__aexit__()`` shields internally.
async with proc: # calls ``trio.Process.aclose()`` async with proc: # calls ``trio.Process.aclose()``
@ -174,11 +179,33 @@ async def do_hard_kill(
proc.kill() proc.kill()
@asynccontextmanager async def new_proc(
async def spawn_subactor(
subactor: 'Actor', name: str,
actor_nursery: 'ActorNursery', # type: ignore # noqa
subactor: Actor,
errors: Dict[Tuple[str, str], Exception],
# passed through to actor main
bind_addr: Tuple[str, int],
parent_addr: Tuple[str, int], parent_addr: Tuple[str, int],
): _runtime_vars: Dict[str, Any], # serialized and sent to _child
*,
task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED
) -> None:
"""
Create a new ``multiprocessing.Process`` using the
spawn method as configured using ``try_set_start_method()``.
"""
# mark the new actor with the global spawn method
subactor._spawn_method = _spawn_method
uid = subactor.uid
if _spawn_method == 'trio':
spawn_cmd = [ spawn_cmd = [
sys.executable, sys.executable,
"-m", "-m",
@ -202,53 +229,34 @@ async def spawn_subactor(
subactor.loglevel subactor.loglevel
] ]
proc = await trio.open_process(spawn_cmd) cancelled_during_spawn: bool = False
try: try:
yield proc proc = await trio.open_process(spawn_cmd)
finally:
log.runtime(f"Attempting to kill {proc}")
# XXX: do this **after** cancellation/tearfown
# to avoid killing the process too early
# since trio does this internally on ``__aexit__()``
await do_hard_kill(proc)
async def new_proc(
name: str,
actor_nursery: 'ActorNursery', # type: ignore # noqa
subactor: Actor,
errors: Dict[Tuple[str, str], Exception],
# passed through to actor main
bind_addr: Tuple[str, int],
parent_addr: Tuple[str, int],
_runtime_vars: Dict[str, Any], # serialized and sent to _child
*,
task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED
) -> None:
"""Create a new ``multiprocessing.Process`` using the
spawn method as configured using ``try_set_start_method()``.
"""
cancel_scope = None
# mark the new actor with the global spawn method
subactor._spawn_method = _spawn_method
if _spawn_method == 'trio':
async with trio.open_nursery() as nursery:
async with spawn_subactor(
subactor,
parent_addr,
) as proc:
log.runtime(f"Started {proc}") log.runtime(f"Started {proc}")
# wait for actor to spawn and connect back to us # wait for actor to spawn and connect back to us
# channel should have handshake completed by the # channel should have handshake completed by the
# local actor by the time we get a ref to it # local actor by the time we get a ref to it
try:
event, chan = await actor_nursery._actor.wait_for_peer( event, chan = await actor_nursery._actor.wait_for_peer(
subactor.uid) subactor.uid)
except trio.Cancelled:
cancelled_during_spawn = True
# we may cancel before the child connects back in which
# case avoid clobbering the pdb tty.
if debug_mode():
with trio.CancelScope(shield=True):
# don't clobber an ongoing pdb
if is_root_process():
await maybe_wait_for_debugger()
else:
async with acquire_debug_lock(uid):
# soft wait on the proc to terminate
with trio.move_on_after(0.5):
await proc.wait()
raise
portal = Portal(chan) portal = Portal(chan)
actor_nursery._children[subactor.uid] = ( actor_nursery._children[subactor.uid] = (
subactor, proc, portal) subactor, proc, portal)
@ -274,8 +282,9 @@ async def new_proc(
with trio.CancelScope(shield=True): with trio.CancelScope(shield=True):
await actor_nursery._join_procs.wait() await actor_nursery._join_procs.wait()
async with trio.open_nursery() as nursery:
if portal in actor_nursery._cancel_after_result_on_exit: if portal in actor_nursery._cancel_after_result_on_exit:
cancel_scope = await nursery.start( nursery.start_soon(
cancel_on_completion, cancel_on_completion,
portal, portal,
subactor, subactor,
@ -285,32 +294,45 @@ async def new_proc(
# Wait for proc termination but **dont' yet** call # Wait for proc termination but **dont' yet** call
# ``trio.Process.__aexit__()`` (it tears down stdio # ``trio.Process.__aexit__()`` (it tears down stdio
# which will kill any waiting remote pdb trace). # which will kill any waiting remote pdb trace).
# This is a "soft" (cancellable) join/reap.
# TODO: No idea how we can enforce zombie
# reaping more stringently without the shield
# we used to have below...
# 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}")
# pop child entry to indicate we no longer managing this subactor
subactor, proc, portal = actor_nursery._children.pop(subactor.uid)
# cancel result waiter that may have been spawned in # cancel result waiter that may have been spawned in
# tandem if not done already # tandem if not done already
if cancel_scope:
log.warning( log.warning(
"Cancelling existing result waiter task for " "Cancelling existing result waiter task for "
f"{subactor.uid}") f"{subactor.uid}")
cancel_scope.cancel() nursery.cancel_scope.cancel()
finally:
# The "hard" reap since no actor zombies are allowed!
# XXX: do this **after** cancellation/tearfown to avoid
# killing the process too early.
log.cancel(f'Hard reap sequence starting for {uid}')
with trio.CancelScope(shield=True):
# don't clobber an ongoing pdb
if cancelled_during_spawn:
# Try again to avoid TTY clobbering.
async with acquire_debug_lock(uid):
with trio.move_on_after(0.5):
await proc.wait()
if is_root_process():
await maybe_wait_for_debugger()
if proc.poll() is None:
log.cancel(f"Attempting to hard kill {proc}")
await do_hard_kill(proc)
log.debug(f"Joined {proc}")
if not cancelled_during_spawn:
# pop child entry to indicate we no longer managing this
# subactor
actor_nursery._children.pop(subactor.uid)
else: else:
# `multiprocessing` # `multiprocessing`
# async with trio.open_nursery() as nursery: # async with trio.open_nursery() as nursery:
@ -341,7 +363,7 @@ async def mp_new_proc(
task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED
) -> None: ) -> None:
async with trio.open_nursery() as nursery:
assert _ctx assert _ctx
start_method = _ctx.get_start_method() start_method = _ctx.get_start_method()
if start_method == 'forkserver': if start_method == 'forkserver':
@ -405,6 +427,11 @@ async def mp_new_proc(
# local actor by the time we get a ref to it # local actor by the time we get a ref to it
event, chan = await actor_nursery._actor.wait_for_peer( event, chan = await actor_nursery._actor.wait_for_peer(
subactor.uid) subactor.uid)
# except:
# TODO: in the case we were cancelled before the sub-proc
# registered itself back we must be sure to try and clean
# any process we may have started.
portal = Portal(chan) portal = Portal(chan)
actor_nursery._children[subactor.uid] = (subactor, proc, portal) actor_nursery._children[subactor.uid] = (subactor, proc, portal)
@ -420,37 +447,31 @@ async def mp_new_proc(
# while user code is still doing it's thing. Only after the # while user code is still doing it's thing. Only after the
# nursery block closes do we allow subactor results to be # nursery block closes do we allow subactor results to be
# awaited and reported upwards to the supervisor. # awaited and reported upwards to the supervisor.
with trio.CancelScope(shield=True):
await actor_nursery._join_procs.wait() await actor_nursery._join_procs.wait()
finally: async with trio.open_nursery() as nursery:
# XXX: in the case we were cancelled before the sub-proc
# registered itself back we must be sure to try and clean
# any process we may have started.
reaping_cancelled: bool = False
cancel_scope: Optional[trio.CancelScope] = None
cancel_exc: Optional[trio.Cancelled] = None
if portal in actor_nursery._cancel_after_result_on_exit: if portal in actor_nursery._cancel_after_result_on_exit:
try: nursery.start_soon(
# async with trio.open_nursery() as n:
# n.cancel_scope.shield = True
cancel_scope = await nursery.start(
cancel_on_completion, cancel_on_completion,
portal, portal,
subactor, subactor,
errors errors
) )
except trio.Cancelled as err:
cancel_exc = err
# if the reaping task was cancelled we may have hit await proc_waiter(proc)
# a race where the subproc disconnected before we
# could send it a message to cancel (classic 2 generals)
# in that case, wait shortly then kill the process.
reaping_cancelled = True
# cancel result waiter that may have been spawned in
# tandem if not done already
log.warning(
"Cancelling existing result waiter task for "
f"{subactor.uid}")
nursery.cancel_scope.cancel()
finally:
# hard reap sequence
if proc.is_alive(): if proc.is_alive():
log.cancel(f"Attempting to hard kill {proc}")
with trio.move_on_after(0.1) as cs: with trio.move_on_after(0.1) as cs:
cs.shield = True cs.shield = True
await proc_waiter(proc) await proc_waiter(proc)
@ -458,24 +479,8 @@ async def mp_new_proc(
if cs.cancelled_caught: if cs.cancelled_caught:
proc.terminate() proc.terminate()
if not reaping_cancelled and proc.is_alive():
await proc_waiter(proc)
# TODO: timeout block here?
proc.join() proc.join()
log.debug(f"Joined {proc}") log.debug(f"Joined {proc}")
# pop child entry to indicate we are no longer managing subactor # pop child entry to indicate we are no longer managing subactor
subactor, proc, portal = actor_nursery._children.pop(subactor.uid) subactor, proc, portal = actor_nursery._children.pop(subactor.uid)
# cancel result waiter that may have been spawned in
# tandem if not done already
if cancel_scope:
log.warning(
"Cancelling existing result waiter task for "
f"{subactor.uid}")
cancel_scope.cancel()
elif reaping_cancelled: # let the cancellation bubble up
assert cancel_exc
raise cancel_exc

View File

@ -12,6 +12,7 @@ import trio
from async_generator import asynccontextmanager from async_generator import asynccontextmanager
from . import _debug from . import _debug
from ._debug import maybe_wait_for_debugger
from ._state import current_actor, is_main_process, is_root_process from ._state import current_actor, is_main_process, is_root_process
from .log import get_logger, get_loglevel from .log import get_logger, get_loglevel
from ._actor import Actor from ._actor import Actor
@ -280,26 +281,7 @@ async def _open_and_supervise_one_cancels_all_nursery(
# will make the pdb repl unusable. # will make the pdb repl unusable.
# Instead try to wait for pdb to be released before # Instead try to wait for pdb to be released before
# tearing down. # tearing down.
if is_root_process(): await maybe_wait_for_debugger()
# TODO: could this make things more deterministic?
# 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