Run first soft wait inside a task

Theoretically we can actually concurrently soft wait the process and the
nursery exit event, the only problem at the moment is some pubsub tests
puking. You're right in guessing this isn't really changing anything but
it is meant to be a reminder. If we can add this the spawn task can
report when a process dies earlier then expected and in the longer term
once we remove the `ActorNursery.run_in_actor()` API, we can probably do
away with both the nursery exit event and the portal result fetching.
sigint_ignore_in_pdb_repl
Tyler Goodlet 2022-02-14 10:25:42 -05:00
parent fc90e1f171
commit f8494de478
1 changed files with 74 additions and 49 deletions

View File

@ -18,17 +18,30 @@
Machinery for actor process spawning using multiple backends. Machinery for actor process spawning using multiple backends.
""" """
from __future__ import annotations
import sys import sys
import multiprocessing as mp
import platform import platform
from typing import ( from typing import (
Any, Optional, Callable, TypeVar, TYPE_CHECKING Any, Dict, Optional, Callable,
TypeVar,
) )
from collections.abc import Awaitable from collections.abc import Awaitable
import trio import trio
from trio_typing import TaskStatus from trio_typing import TaskStatus
try:
from multiprocessing import semaphore_tracker # type: ignore
resource_tracker = semaphore_tracker
resource_tracker._resource_tracker = resource_tracker._semaphore_tracker
except ImportError:
# 3.8 introduces a more general version that also tracks shared mems
from multiprocessing import resource_tracker # type: ignore
from multiprocessing import forkserver # type: ignore
from typing import Tuple
from . import _forkserver_override
from ._debug import ( from ._debug import (
maybe_wait_for_debugger, maybe_wait_for_debugger,
acquire_debug_lock, acquire_debug_lock,
@ -47,11 +60,8 @@ from ._entry import _mp_main
from ._exceptions import ActorFailure from ._exceptions import ActorFailure
if TYPE_CHECKING:
import multiprocessing as mp
ProcessType = TypeVar('ProcessType', mp.Process, trio.Process)
log = get_logger('tractor') log = get_logger('tractor')
ProcessType = TypeVar('ProcessType', mp.Process, trio.Process)
# placeholder for an mp start context if so using that backend # placeholder for an mp start context if so using that backend
_ctx: Optional[mp.context.BaseContext] = None _ctx: Optional[mp.context.BaseContext] = None
@ -60,7 +70,6 @@ _spawn_method: str = "trio"
if platform.system() == 'Windows': if platform.system() == 'Windows':
import multiprocessing as mp
_ctx = mp.get_context("spawn") _ctx = mp.get_context("spawn")
async def proc_waiter(proc: mp.Process) -> None: async def proc_waiter(proc: mp.Process) -> None:
@ -83,7 +92,6 @@ def try_set_start_method(name: str) -> Optional[mp.context.BaseContext]:
``subprocess.Popen``. ``subprocess.Popen``.
''' '''
import multiprocessing as mp
global _ctx global _ctx
global _spawn_method global _spawn_method
@ -100,7 +108,6 @@ def try_set_start_method(name: str) -> Optional[mp.context.BaseContext]:
f"Spawn method `{name}` is invalid please choose one of {methods}" f"Spawn method `{name}` is invalid please choose one of {methods}"
) )
elif name == 'forkserver': elif name == 'forkserver':
from . import _forkserver_override
_forkserver_override.override_stdlib() _forkserver_override.override_stdlib()
_ctx = mp.get_context(name) _ctx = mp.get_context(name)
elif name == 'trio': elif name == 'trio':
@ -148,7 +155,7 @@ 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],
) -> None: ) -> None:
''' '''
@ -251,12 +258,12 @@ async def new_proc(
name: str, name: str,
actor_nursery: 'ActorNursery', # type: ignore # noqa actor_nursery: 'ActorNursery', # type: ignore # noqa
subactor: Actor, subactor: Actor,
errors: dict[tuple[str, str], Exception], errors: Dict[Tuple[str, str], Exception],
# passed through to actor main # passed through to actor main
bind_addr: tuple[str, int], 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 _runtime_vars: Dict[str, Any], # serialized and sent to _child
*, *,
@ -288,7 +295,7 @@ async def new_proc(
# the OS; it otherwise can be passed via the parent channel if # the OS; it otherwise can be passed via the parent channel if
# we prefer in the future (for privacy). # we prefer in the future (for privacy).
"--uid", "--uid",
str(subactor.uid), str(uid),
# Address the child must connect to on startup # Address the child must connect to on startup
"--parent_addr", "--parent_addr",
str(parent_addr) str(parent_addr)
@ -314,8 +321,7 @@ async def new_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
event, chan = await actor_nursery._actor.wait_for_peer( event, chan = await actor_nursery._actor.wait_for_peer(uid)
subactor.uid)
except trio.Cancelled: except trio.Cancelled:
cancelled_during_spawn = True cancelled_during_spawn = True
@ -356,18 +362,13 @@ async def new_proc(
task_status.started(portal) task_status.started(portal)
# wait for ActorNursery.wait() to be called # wait for ActorNursery.wait() to be called
n_exited = actor_nursery._join_procs
with trio.CancelScope(shield=True): with trio.CancelScope(shield=True):
await actor_nursery._join_procs.wait() await n_exited.wait()
async with trio.open_nursery() as nursery: async with trio.open_nursery() as nursery:
if portal in actor_nursery._cancel_after_result_on_exit:
nursery.start_soon(
cancel_on_completion,
portal,
subactor,
errors
)
async def soft_wait_and_maybe_cancel_ria_task():
# This is a "soft" (cancellable) join/reap which # This is a "soft" (cancellable) join/reap which
# will remote cancel the actor on a ``trio.Cancelled`` # will remote cancel the actor on a ``trio.Cancelled``
# condition. # condition.
@ -377,13 +378,46 @@ async def new_proc(
portal portal
) )
if n_exited.is_set():
# 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
log.warning( log.warning(
"Cancelling existing result waiter task for " "Cancelling existing result waiter task for "
f"{subactor.uid}") f"{subactor.uid}"
)
nursery.cancel_scope.cancel() nursery.cancel_scope.cancel()
else:
log.warning(
f'Process for actor {uid} terminated before'
'nursery exit. ' 'This may mean an IPC'
'connection failed!'
)
nursery.start_soon(soft_wait_and_maybe_cancel_ria_task)
# TODO: when we finally remove the `.run_in_actor()` api
# we should be able to entirely drop these 2 blocking calls:
# - we don't need to wait on nursery exit to capture
# process-spawn-machinery level errors (and propagate them).
# - we don't need to wait on final results from ria portals
# since this will be done in some higher level wrapper API.
# XXX: interestingly we can't put this here bc it causes
# the pub-sub tests to fail? wth.. should probably drop
# those XD
# wait for ActorNursery.wait() to be called
# with trio.CancelScope(shield=True):
# await n_exited.wait()
if portal in actor_nursery._cancel_after_result_on_exit:
nursery.start_soon(
cancel_on_completion,
portal,
subactor,
errors
)
finally: finally:
# The "hard" reap since no actor zombies are allowed! # The "hard" reap since no actor zombies are allowed!
# XXX: do this **after** cancellation/tearfown to avoid # XXX: do this **after** cancellation/tearfown to avoid
@ -400,9 +434,10 @@ async def new_proc(
await proc.wait() await proc.wait()
if is_root_process(): if is_root_process():
await maybe_wait_for_debugger( await maybe_wait_for_debugger(
child_in_debug=_runtime_vars.get( child_in_debug=_runtime_vars.get(
'_debug_mode', False), '_debug_mode', False)
) )
if proc.poll() is None: if proc.poll() is None:
@ -441,30 +476,20 @@ async def mp_new_proc(
name: str, name: str,
actor_nursery: 'ActorNursery', # type: ignore # noqa actor_nursery: 'ActorNursery', # type: ignore # noqa
subactor: Actor, subactor: Actor,
errors: dict[tuple[str, str], Exception], errors: Dict[Tuple[str, str], Exception],
# passed through to actor main # passed through to actor main
bind_addr: tuple[str, int], 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 _runtime_vars: Dict[str, Any], # serialized and sent to _child
*, *,
infect_asyncio: bool = False, infect_asyncio: bool = False,
task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED
) -> None: ) -> None:
# uggh zone
try:
from multiprocessing import semaphore_tracker # type: ignore
resource_tracker = semaphore_tracker
resource_tracker._resource_tracker = resource_tracker._semaphore_tracker # noqa
except ImportError:
# 3.8 introduces a more general version that also tracks shared mems
from multiprocessing import resource_tracker # type: ignore
assert _ctx assert _ctx
start_method = _ctx.get_start_method() start_method = _ctx.get_start_method()
if start_method == 'forkserver': if start_method == 'forkserver':
from multiprocessing import forkserver # type: ignore
# XXX do our hackery on the stdlib to avoid multiple # XXX do our hackery on the stdlib to avoid multiple
# forkservers (one at each subproc layer). # forkservers (one at each subproc layer).
fs = forkserver._forkserver fs = forkserver._forkserver