Vastly improve error triggered cancellation

At the expense of a bit more complexity in `ActorNursery.wait()`
(which I commented the heck out of fwiw) this adds far superior and
correct cancellation semantics for when a nursery is cancelled due
to (remote) errors in subactors.

This includes:
- `wait()` will now raise a `trio.MultiError` if multiple subactors
  error with the same semantics as in `trio`.
- in `wait()` portals which are paired with `run_in_actor()`
  spawned subactors (versus `start_actor()`) are waited on separately
  and if the nursery **hasn't** been cancelled but there are errors
  those are raised immediately before waiting on `start_actor()`
  subactors which will block indefinitely if they haven't been
  explicitly cancelled.
- if `wait()` does raise when the nursery hasn't yet been cancelled
  it's expected that it will be called again depending on the actor
  supervision strategy (i.e. right now we operate with a one-cancels-all
  strategy, the same as `trio`, so `ActorNursery.__aexit__() calls
  `cancel()` if any error is raised by `wait()`).

Oh and I added `is_main_process()` helper; can't remember why..
improved_errors
Tyler Goodlet 2018-11-19 04:12:54 -05:00
parent e75b25dc21
commit 835d1fa07a
1 changed files with 121 additions and 61 deletions

View File

@ -32,7 +32,8 @@ class ActorNursery:
Tuple[str, str], Tuple[str, str],
Tuple[Actor, mp.Process, Optional[Portal]] Tuple[Actor, mp.Process, Optional[Portal]]
] = {} ] = {}
# portals spawned with ``run_in_actor()`` # portals spawned with ``run_in_actor()`` are
# cancelled when their "main" result arrives
self._cancel_after_result_on_exit: set = set() self._cancel_after_result_on_exit: set = set()
self.cancelled: bool = False self.cancelled: bool = False
self._forkserver: forkserver.ForkServer = None self._forkserver: forkserver.ForkServer = None
@ -132,6 +133,8 @@ class ActorNursery:
bind_addr=bind_addr, bind_addr=bind_addr,
statespace=statespace, statespace=statespace,
) )
# this marks the actor to be cancelled after its portal result
# is retreived, see ``wait()`` below.
self._cancel_after_result_on_exit.add(portal) self._cancel_after_result_on_exit.add(portal)
await portal._submit_for_result( await portal._submit_for_result(
mod_path, mod_path,
@ -142,29 +145,65 @@ class ActorNursery:
async def wait(self) -> None: async def wait(self) -> None:
"""Wait for all subactors to complete. """Wait for all subactors to complete.
This is probably the most complicated (and confusing, sorry)
function that does all the clever crap to deal with cancellation,
error propagation, and graceful subprocess tear down.
""" """
async def maybe_consume_result(portal, actor): async def exhaust_portal(portal, actor):
if ( """Pull final result from portal (assuming it was one).
portal in self._cancel_after_result_on_exit and
(portal._result is None and portal._exc is None) If the main task is an async generator do our best to consume
): what's left of it.
log.debug(f"Waiting on final result from {subactor.uid}") """
res = await portal.result() try:
# if it's an async-gen then we should alert the user log.debug(f"Waiting on final result from {actor.uid}")
# that we're cancelling it final = res = await portal.result()
# if it's an async-gen then alert that we're cancelling it
if inspect.isasyncgen(res): if inspect.isasyncgen(res):
final = []
log.warning( log.warning(
f"Blindly consuming asyncgen for {actor.uid}") f"Blindly consuming asyncgen for {actor.uid}")
with trio.fail_after(1): with trio.fail_after(1):
async with aclosing(res) as agen: async with aclosing(res) as agen:
async for item in agen: async for item in agen:
log.debug(f"Consuming item {item}") log.debug(f"Consuming item {item}")
final.append(item)
except Exception as err:
# we reraise in the parent task via a ``trio.MultiError``
return err
else:
return final
async def cancel_on_completion(
portal: Portal,
actor: Actor,
task_status=trio.TASK_STATUS_IGNORED,
) -> None:
"""Cancel actor gracefully once it's "main" portal's
result arrives.
Should only be called for actors spawned with `run_in_actor()`.
"""
with trio.open_cancel_scope() as cs:
task_status.started(cs)
# this may error in which case we expect the far end
# actor to have already terminated itself
result = await exhaust_portal(portal, actor)
if isinstance(result, Exception):
errors.append(result)
log.info(f"Cancelling {portal.channel.uid} gracefully")
await portal.cancel_actor()
if cs.cancelled_caught:
log.warning(
"Result waiter was cancelled, process may have died")
async def wait_for_proc( async def wait_for_proc(
proc: mp.Process, proc: mp.Process,
actor: Actor, actor: Actor,
portal: Portal, portal: Portal,
cancel_scope: trio._core._run.CancelScope, cancel_scope: Optional[trio._core._run.CancelScope] = None,
) -> None: ) -> None:
# TODO: timeout block here? # TODO: timeout block here?
if proc.is_alive(): if proc.is_alive():
@ -172,42 +211,57 @@ class ActorNursery:
# please god don't hang # please god don't hang
proc.join() proc.join()
log.debug(f"Joined {proc}") log.debug(f"Joined {proc}")
await maybe_consume_result(portal, actor)
self._children.pop(actor.uid) self._children.pop(actor.uid)
# proc terminated, cancel result waiter
# proc terminated, cancel result waiter that may have
# been spawned in tandem
if cancel_scope: if cancel_scope:
log.warning( log.warning(
f"Cancelling existing result waiter task for {actor.uid}") f"Cancelling existing result waiter task for {actor.uid}")
cancel_scope.cancel() cancel_scope.cancel()
async def wait_for_actor( log.debug(f"Waiting on all subactors to complete")
portal: Portal,
actor: Actor,
task_status=trio.TASK_STATUS_IGNORED,
) -> None:
# cancel the actor gracefully
with trio.open_cancel_scope() as cs:
task_status.started(cs)
await maybe_consume_result(portal, actor)
log.info(f"Cancelling {portal.channel.uid} gracefully")
await portal.cancel_actor()
if cs.cancelled_caught:
log.warning("Result waiter was cancelled")
# unblocks when all waiter tasks have completed
children = self._children.copy() children = self._children.copy()
errors = []
# wait on run_in_actor() tasks, unblocks when all complete
async with trio.open_nursery() as nursery: async with trio.open_nursery() as nursery:
for subactor, proc, portal in children.values(): for subactor, proc, portal in children.values():
cs = None cs = None
# portal from ``run_in_actor()``
if portal in self._cancel_after_result_on_exit: if portal in self._cancel_after_result_on_exit:
cs = await nursery.start(wait_for_actor, portal, subactor) cs = await nursery.start(
cancel_on_completion, portal, subactor)
# TODO: how do we handle remote host spawned actors?
nursery.start_soon(
wait_for_proc, proc, subactor, portal, cs)
if errors:
if not self.cancelled:
# halt here and expect to be called again once the nursery
# has been cancelled externally (ex. from within __aexit__()
# if an error is captured from ``wait()`` then ``cancel()``
# is called immediately after which in turn calls ``wait()``
# again.)
raise trio.MultiError(errors)
# wait on all `start_actor()` subactors to complete
# if errors were captured above and we have not been cancelled
# then these ``start_actor()`` spawned actors will block until
# cancelled externally
children = self._children.copy()
async with trio.open_nursery() as nursery:
for subactor, proc, portal in children.values():
# TODO: how do we handle remote host spawned actors?
nursery.start_soon(wait_for_proc, proc, subactor, portal, cs) nursery.start_soon(wait_for_proc, proc, subactor, portal, cs)
log.debug(f"All subactors for {self} have terminated")
if errors:
# always raise any error if we're also cancelled
raise trio.MultiError(errors)
async def cancel(self, hard_kill: bool = False) -> None: async def cancel(self, hard_kill: bool = False) -> None:
"""Cancel this nursery by instructing each subactor to cancel """Cancel this nursery by instructing each subactor to cancel
iteslf and wait for all subprocesses to terminate. itself and wait for all subactors to terminate.
If ``hard_killl`` is set to ``True`` then kill the processes If ``hard_killl`` is set to ``True`` then kill the processes
directly without any far end graceful ``trio`` cancellation. directly without any far end graceful ``trio`` cancellation.
@ -234,56 +288,57 @@ class ActorNursery:
# channel/portal should now be up # channel/portal should now be up
_, _, portal = self._children[subactor.uid] _, _, portal = self._children[subactor.uid]
if portal is None: if portal is None:
# cancelled while waiting on the event? # cancelled while waiting on the event
# to arrive
chan = self._actor._peers[subactor.uid][-1] chan = self._actor._peers[subactor.uid][-1]
if chan: if chan:
portal = Portal(chan) portal = Portal(chan)
else: # there's no other choice left else: # there's no other choice left
do_hard_kill(proc) do_hard_kill(proc)
# spawn cancel tasks async # spawn cancel tasks
assert portal assert portal
n.start_soon(portal.cancel_actor) n.start_soon(portal.cancel_actor)
log.debug(f"Waiting on all subactors to complete") # mark ourselves as having (tried to have) cancelled all subactors
await self.wait()
self.cancelled = True self.cancelled = True
log.debug(f"All subactors for {self} have terminated") await self.wait()
async def __aexit__(self, etype, value, tb): async def __aexit__(self, etype, value, tb):
"""Wait on all subactor's main routines to complete. """Wait on all subactor's main routines to complete.
""" """
try: if etype is not None:
if etype is not None: try:
# XXX: hypothetically an error could be raised and then # XXX: hypothetically an error could be raised and then
# a cancel signal shows up slightly after in which case the # a cancel signal shows up slightly after in which case
# else block here might not complete? Should both be shielded? # the `else:` block here might not complete?
# For now, shield both.
with trio.open_cancel_scope(shield=True): with trio.open_cancel_scope(shield=True):
if etype is trio.Cancelled: if etype is trio.Cancelled:
log.warning( log.warning(
f"{current_actor().uid} was cancelled with {etype}" f"Nursery for {current_actor().uid} was "
", cancelling actor nursery") f"cancelled with {etype}")
await self.cancel()
else: else:
log.exception( log.exception(
f"{current_actor().uid} errored with {etype}, " f"Nursery for {current_actor().uid} "
"cancelling actor nursery") f"errored with {etype}, ")
await self.cancel()
else:
# XXX: this is effectively the lone cancellation/supervisor
# strategy which exactly mimicks trio's behaviour
log.debug(f"Waiting on subactors {self._children} to complete")
try:
await self.wait()
except Exception as err:
log.warning(f"Nursery caught {err}, cancelling")
await self.cancel() await self.cancel()
raise except trio.MultiError as merr:
log.debug(f"Nursery teardown complete") if value not in merr.exceptions:
except Exception: raise trio.MultiError(merr.exceptions + [value])
log.exception("Error on nursery exit:") raise
await self.wait() else:
raise # XXX: this is effectively the (for now) lone
# cancellation/supervisor strategy which exactly
# mimicks trio's behaviour
log.debug(f"Waiting on subactors {self._children} to complete")
try:
await self.wait()
except Exception as err:
log.warning(f"Nursery caught {err}, cancelling")
await self.cancel()
raise
log.debug(f"Nursery teardown complete")
@asynccontextmanager @asynccontextmanager
@ -297,3 +352,8 @@ async def open_nursery() -> typing.AsyncGenerator[ActorNursery, None]:
# TODO: figure out supervisors from erlang # TODO: figure out supervisors from erlang
async with ActorNursery(current_actor()) as nursery: async with ActorNursery(current_actor()) as nursery:
yield nursery yield nursery
def is_main_process():
"Bool determining if this actor is running in the top-most process."
return mp.current_process().name == 'MainProcess'