WIP trying out trio_run_in_process

try_trip^2
Tyler Goodlet 2019-11-26 09:23:37 -05:00
parent 7c0efce84b
commit 1b7cdfe512
3 changed files with 113 additions and 53 deletions

View File

@ -225,10 +225,20 @@ class Actor:
the original nursery we need to try and load the local module the original nursery we need to try and load the local module
code (if it exists). code (if it exists).
""" """
try:
for path in self.rpc_module_paths: for path in self.rpc_module_paths:
log.debug(f"Attempting to import {path}") log.debug(f"Attempting to import {path}")
self._mods[path] = importlib.import_module(path) self._mods[path] = importlib.import_module(path)
# if self.name != 'arbiter':
# importlib.import_module('doggy')
# from celery.contrib import rdb; rdb.set_trace()
except ModuleNotFoundError:
# it is expected the corresponding `ModuleNotExposed` error
# will be raised later
log.error(f"Failed to import {path} in {self.name}")
raise
def _get_rpc_func(self, ns, funcname): def _get_rpc_func(self, ns, funcname):
try: try:
return getattr(self._mods[ns], funcname) return getattr(self._mods[ns], funcname)
@ -488,7 +498,7 @@ class Actor:
f"Exiting msg loop for {chan} from {chan.uid} " f"Exiting msg loop for {chan} from {chan.uid} "
f"with last msg:\n{msg}") f"with last msg:\n{msg}")
def _fork_main( def _mp_main(
self, self,
accept_addr: Tuple[str, int], accept_addr: Tuple[str, int],
forkserver_info: Tuple[Any, Any, Any, Any, Any], forkserver_info: Tuple[Any, Any, Any, Any, Any],
@ -500,13 +510,17 @@ class Actor:
self._forkserver_info = forkserver_info self._forkserver_info = forkserver_info
from ._spawn import try_set_start_method from ._spawn import try_set_start_method
spawn_ctx = try_set_start_method(start_method) spawn_ctx = try_set_start_method(start_method)
if self.loglevel is not None: if self.loglevel is not None:
log.info( log.info(
f"Setting loglevel for {self.uid} to {self.loglevel}") f"Setting loglevel for {self.uid} to {self.loglevel}")
get_console_log(self.loglevel) get_console_log(self.loglevel)
log.info( log.info(
f"Started new {spawn_ctx.current_process()} for {self.uid}") f"Started new {spawn_ctx.current_process()} for {self.uid}")
_state._current_actor = self _state._current_actor = self
log.debug(f"parent_addr is {parent_addr}") log.debug(f"parent_addr is {parent_addr}")
try: try:
trio.run(partial( trio.run(partial(
@ -515,6 +529,21 @@ class Actor:
pass # handle it the same way trio does? pass # handle it the same way trio does?
log.info(f"Actor {self.uid} terminated") log.info(f"Actor {self.uid} terminated")
async def _trip_main(
self,
accept_addr: Tuple[str, int],
parent_addr: Tuple[str, int] = None
) -> None:
if self.loglevel is not None:
log.info(
f"Setting loglevel for {self.uid} to {self.loglevel}")
get_console_log(self.loglevel)
log.info(f"Started new TRIP process for {self.uid}")
_state._current_actor = self
await self._async_main(accept_addr, parent_addr=parent_addr)
log.info(f"Actor {self.uid} terminated")
async def _async_main( async def _async_main(
self, self,
accept_addr: Tuple[str, int], accept_addr: Tuple[str, int],
@ -584,6 +613,8 @@ class Actor:
# blocks here as expected until the channel server is # blocks here as expected until the channel server 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:
# if self.name == 'arbiter':
# import pdb; pdb.set_trace()
if not registered_with_arbiter: if not registered_with_arbiter:
log.exception( log.exception(
f"Actor errored and failed to register with arbiter " f"Actor errored and failed to register with arbiter "
@ -598,12 +629,18 @@ class Actor:
f"Failed to ship error to parent " f"Failed to ship error to parent "
f"{self._parent_chan.uid}, channel was closed") f"{self._parent_chan.uid}, channel was closed")
log.exception("Actor errored:") log.exception("Actor errored:")
if isinstance(err, ModuleNotFoundError):
raise
else: else:
# XXX wait, why? # XXX wait, why?
# causes a hang if I always raise.. # causes a hang if I always raise..
# A parent process does something weird here?
raise raise
finally: finally:
# if self.name == 'arbiter':
# import pdb; pdb.set_trace()
if registered_with_arbiter: if registered_with_arbiter:
await self._do_unreg(arbiter_addr) await self._do_unreg(arbiter_addr)
# terminate actor once all it's peers (actors that connected # terminate actor once all it's peers (actors that connected

View File

@ -5,6 +5,8 @@ Mostly just wrapping around ``multiprocessing``.
""" """
import multiprocessing as mp import multiprocessing as mp
import trio_run_in_process
try: try:
from multiprocessing import semaphore_tracker # type: ignore from multiprocessing import semaphore_tracker # type: ignore
resource_tracker = semaphore_tracker resource_tracker = semaphore_tracker
@ -55,16 +57,28 @@ def is_main_process() -> bool:
return mp.current_process().name == 'MainProcess' return mp.current_process().name == 'MainProcess'
def new_proc( async def new_proc(
name: str, name: str,
actor: Actor, actor: Actor,
# 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],
use_trip: bool = True,
) -> mp.Process: ) -> mp.Process:
"""Create a new ``multiprocessing.Process`` using the """Create a new ``multiprocessing.Process`` using the
spawn method as configured using ``try_set_start_method()``. spawn method as configured using ``try_set_start_method()``.
""" """
if use_trip: # trio_run_in_process
mng = trio_run_in_process.open_in_process(
actor._trip_main,
bind_addr,
parent_addr
)
proc = await mng.__aenter__()
proc.mng = mng
return proc
else:
# use multiprocessing
start_method = _ctx.get_start_method() start_method = _ctx.get_start_method()
if start_method == 'forkserver': if start_method == 'forkserver':
# XXX do our hackery on the stdlib to avoid multiple # XXX do our hackery on the stdlib to avoid multiple
@ -95,8 +109,8 @@ def new_proc(
else: else:
fs_info = (None, None, None, None, None) fs_info = (None, None, None, None, None)
return _ctx.Process( # type: ignore return _ctx.Process(
target=actor._fork_main, target=actor._mp_main,
args=( args=(
bind_addr, bind_addr,
fs_info, fs_info,

View File

@ -9,6 +9,7 @@ import typing
import trio import trio
from async_generator import asynccontextmanager, aclosing from async_generator import asynccontextmanager, aclosing
import trio_run_in_process
from ._state import current_actor from ._state import current_actor
from .log import get_logger, get_loglevel from .log import get_logger, get_loglevel
@ -64,17 +65,20 @@ class ActorNursery:
arbiter_addr=current_actor()._arb_addr, arbiter_addr=current_actor()._arb_addr,
) )
parent_addr = self._actor.accept_addr parent_addr = self._actor.accept_addr
proc = _spawn.new_proc( assert parent_addr
proc = await _spawn.new_proc(
name, name,
actor, actor,
bind_addr, bind_addr,
parent_addr, parent_addr,
) )
# `multiprocessing` only (since no async interface):
# register the process before start in case we get a cancel # register the process before start in case we get a cancel
# request before the actor has fully spawned - then we can wait # request before the actor has fully spawned - then we can wait
# for it to fully come up before sending a cancel request # for it to fully come up before sending a cancel request
self._children[actor.uid] = (actor, proc, None) self._children[actor.uid] = (actor, proc, None)
if not isinstance(proc, trio_run_in_process.process.Process):
proc.start() proc.start()
if not proc.is_alive(): if not proc.is_alive():
raise ActorFailure("Couldn't start sub-actor?") raise ActorFailure("Couldn't start sub-actor?")
@ -193,12 +197,17 @@ class ActorNursery:
actor: Actor, actor: Actor,
cancel_scope: Optional[trio.CancelScope] = None, cancel_scope: Optional[trio.CancelScope] = None,
) -> None: ) -> None:
# please god don't hang
if not isinstance(proc, trio_run_in_process.process.Process):
# TODO: timeout block here? # TODO: timeout block here?
if proc.is_alive(): if proc.is_alive():
await proc_waiter(proc) await proc_waiter(proc)
# please god don't hang
proc.join() proc.join()
else:
# trio_run_in_process blocking wait
await proc.mng.__aexit__(None, None, None)
# proc.nursery.cancel_scope.cancel()
log.debug(f"Joined {proc}") log.debug(f"Joined {proc}")
# indicate we are no longer managing this subactor # indicate we are no longer managing this subactor
self._children.pop(actor.uid) self._children.pop(actor.uid)