Simplify to soft and hard reap sequences
This is actually surprisingly easy to grok having gone through a lot of pain understanding edge cases in the zombie lord dev branch. Basically we just need to make sure actors are managed in a 2 step reap sequence. In the "soft" reap phase we wait for the process to terminate on its own concurrently with (maybe) waiting for its portal's final result (if it's a `.run_in_actor()`). If this path is cancelled or errors, then we do a "hard" reap where we timeout and send a signal to the proc to terminate immediately. The only last remaining trick is to tie in the root-is-debugger-aware logic to yet again avoid tty clobbers.immediate_remote_cancels
parent
2df16c1557
commit
77ec29008d
|
@ -8,7 +8,6 @@ from typing import Any, Dict, Optional
|
|||
|
||||
import trio
|
||||
from trio_typing import TaskStatus
|
||||
from async_generator import asynccontextmanager
|
||||
|
||||
try:
|
||||
from multiprocessing import semaphore_tracker # type: ignore
|
||||
|
@ -123,17 +122,15 @@ async def cancel_on_completion(
|
|||
portal: Portal,
|
||||
actor: Actor,
|
||||
errors: Dict[Tuple[str, str], Exception],
|
||||
task_status: TaskStatus[trio.CancelScope] = trio.TASK_STATUS_IGNORED,
|
||||
|
||||
) -> None:
|
||||
"""Cancel actor gracefully once it's "main" portal's
|
||||
"""
|
||||
Cancel actor gracefully once it's "main" portal's
|
||||
result arrives.
|
||||
|
||||
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
|
||||
# in ``errors`` which will be reraised inside
|
||||
# a MultiError and we still send out a cancel request
|
||||
|
@ -191,8 +188,6 @@ async def new_proc(
|
|||
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
|
||||
|
||||
|
@ -258,7 +253,8 @@ async def new_proc(
|
|||
|
||||
async with trio.open_nursery() as nursery:
|
||||
if portal in actor_nursery._cancel_after_result_on_exit:
|
||||
cancel_scope = await nursery.start(
|
||||
# cancel_scope = await nursery.start(
|
||||
nursery.start_soon(
|
||||
cancel_on_completion,
|
||||
portal,
|
||||
subactor,
|
||||
|
@ -271,16 +267,16 @@ async def new_proc(
|
|||
# This is a "soft" (cancellable) join/reap.
|
||||
await proc.wait()
|
||||
|
||||
finally:
|
||||
# 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()
|
||||
nursery.cancel_scope.cancel()
|
||||
|
||||
log.runtime(f"Attempting to kill {proc}")
|
||||
finally:
|
||||
if proc.poll() is None:
|
||||
log.cancel(f"Attempting to hard kill {proc}")
|
||||
|
||||
# The "hard" reap since no actor zombies are allowed!
|
||||
# XXX: do this **after** cancellation/tearfown to avoid
|
||||
|
@ -322,7 +318,7 @@ async def mp_new_proc(
|
|||
task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED
|
||||
|
||||
) -> None:
|
||||
async with trio.open_nursery() as nursery:
|
||||
|
||||
assert _ctx
|
||||
start_method = _ctx.get_start_method()
|
||||
if start_method == 'forkserver':
|
||||
|
@ -386,6 +382,11 @@ async def mp_new_proc(
|
|||
# local actor by the time we get a ref to it
|
||||
event, chan = await actor_nursery._actor.wait_for_peer(
|
||||
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)
|
||||
actor_nursery._children[subactor.uid] = (subactor, proc, portal)
|
||||
|
||||
|
@ -401,37 +402,31 @@ async def mp_new_proc(
|
|||
# while user code is still doing it's thing. Only after the
|
||||
# nursery block closes do we allow subactor results to be
|
||||
# awaited and reported upwards to the supervisor.
|
||||
with trio.CancelScope(shield=True):
|
||||
await actor_nursery._join_procs.wait()
|
||||
|
||||
finally:
|
||||
# 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
|
||||
|
||||
async with trio.open_nursery() as nursery:
|
||||
if portal in actor_nursery._cancel_after_result_on_exit:
|
||||
try:
|
||||
# async with trio.open_nursery() as n:
|
||||
# n.cancel_scope.shield = True
|
||||
cancel_scope = await nursery.start(
|
||||
nursery.start_soon(
|
||||
cancel_on_completion,
|
||||
portal,
|
||||
subactor,
|
||||
errors
|
||||
)
|
||||
except trio.Cancelled as err:
|
||||
cancel_exc = err
|
||||
|
||||
# if the reaping task was cancelled we may have hit
|
||||
# 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
|
||||
await proc_waiter(proc)
|
||||
|
||||
# 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():
|
||||
log.cancel(f"Attempting to hard kill {proc}")
|
||||
with trio.move_on_after(0.1) as cs:
|
||||
cs.shield = True
|
||||
await proc_waiter(proc)
|
||||
|
@ -439,24 +434,8 @@ async def mp_new_proc(
|
|||
if cs.cancelled_caught:
|
||||
proc.terminate()
|
||||
|
||||
if not reaping_cancelled and proc.is_alive():
|
||||
await proc_waiter(proc)
|
||||
|
||||
# TODO: timeout block here?
|
||||
proc.join()
|
||||
|
||||
log.debug(f"Joined {proc}")
|
||||
|
||||
# pop child entry to indicate we are no longer managing subactor
|
||||
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
|
||||
|
|
Loading…
Reference in New Issue