Re-org root actor startup into context manager
This begins the move to dropping support for `tractor.run()` which we don't really need since the runtime is started (as it always has been) from a new sub-task / nursery. Instead this introduces starting the actor tree through a `open_root_actor()` async context manager which we'll likely implicitly call (from the root) on the first use of an actor nursery. Drop `_actor._start_actor()` and factor its contents into this new api. Make `run()` and `run_daemon()` use `open_root_actor()` until we decide to remove them. Relates to #168 and #177drop_tractor_run
							parent
							
								
									f427c98cf6
								
							
						
					
					
						commit
						f05534e472
					
				|  | @ -2,181 +2,38 @@ | ||||||
| tractor: An actor model micro-framework built on | tractor: An actor model micro-framework built on | ||||||
|          ``trio`` and ``multiprocessing``. |          ``trio`` and ``multiprocessing``. | ||||||
| """ | """ | ||||||
| import importlib |  | ||||||
| from functools import partial |  | ||||||
| from typing import Tuple, Any, Optional, List |  | ||||||
| import typing |  | ||||||
| 
 |  | ||||||
| import trio  # type: ignore |  | ||||||
| from trio import MultiError | from trio import MultiError | ||||||
| 
 | 
 | ||||||
| from . import log | from ._ipc import Channel | ||||||
| from ._ipc import _connect_chan, Channel |  | ||||||
| from ._streaming import Context, stream | from ._streaming import Context, stream | ||||||
| from ._discovery import get_arbiter, find_actor, wait_for_actor | from ._discovery import get_arbiter, find_actor, wait_for_actor | ||||||
| from ._actor import Actor, _start_actor, Arbiter |  | ||||||
| from ._trionics import open_nursery | from ._trionics import open_nursery | ||||||
| from ._state import current_actor, is_root_process | from ._state import current_actor, is_root_process | ||||||
| from . import _state |  | ||||||
| from ._exceptions import RemoteActorError, ModuleNotExposed | from ._exceptions import RemoteActorError, ModuleNotExposed | ||||||
| from ._debug import breakpoint, post_mortem | from ._debug import breakpoint, post_mortem | ||||||
| from . import _spawn |  | ||||||
| from . import msg | from . import msg | ||||||
|  | from ._root import run, run_daemon, open_root_actor | ||||||
| 
 | 
 | ||||||
| 
 | 
 | ||||||
| __all__ = [ | __all__ = [ | ||||||
|  |     'Channel', | ||||||
|  |     'Context', | ||||||
|  |     'ModuleNotExposed', | ||||||
|  |     'MultiError', | ||||||
|  |     'RemoteActorError', | ||||||
|     'breakpoint', |     'breakpoint', | ||||||
|     'post_mortem', |  | ||||||
|     'current_actor', |     'current_actor', | ||||||
|     'find_actor', |     'find_actor', | ||||||
|     'get_arbiter', |     'get_arbiter', | ||||||
|  |     'is_root_process', | ||||||
|  |     'msg', | ||||||
|     'open_nursery', |     'open_nursery', | ||||||
|     'wait_for_actor', |     'open_root_actor', | ||||||
|     'Channel', |     'post_mortem', | ||||||
|     'Context', |     'run', | ||||||
|  |     'run_daemon', | ||||||
|     'stream', |     'stream', | ||||||
|     'MultiError', |     'wait_for_actor', | ||||||
|     'RemoteActorError', |     'to_asyncio', | ||||||
|     'ModuleNotExposed', |     'wait_for_actor', | ||||||
|     'msg' |  | ||||||
| ] | ] | ||||||
| 
 |  | ||||||
| 
 |  | ||||||
| # set at startup and after forks |  | ||||||
| _default_arbiter_host = '127.0.0.1' |  | ||||||
| _default_arbiter_port = 1616 |  | ||||||
| 
 |  | ||||||
| 
 |  | ||||||
| async def _main( |  | ||||||
|     async_fn: typing.Callable[..., typing.Awaitable], |  | ||||||
|     args: Tuple, |  | ||||||
|     arbiter_addr: Tuple[str, int], |  | ||||||
|     name: Optional[str] = None, |  | ||||||
|     start_method: Optional[str] = None, |  | ||||||
|     debug_mode: bool = False, |  | ||||||
|     **kwargs, |  | ||||||
| ) -> typing.Any: |  | ||||||
|     """Async entry point for ``tractor``. |  | ||||||
|     """ |  | ||||||
|     logger = log.get_logger('tractor') |  | ||||||
| 
 |  | ||||||
|     # mark top most level process as root actor |  | ||||||
|     _state._runtime_vars['_is_root'] = True |  | ||||||
| 
 |  | ||||||
|     if start_method is not None: |  | ||||||
|         _spawn.try_set_start_method(start_method) |  | ||||||
| 
 |  | ||||||
|     if debug_mode and _spawn._spawn_method == 'trio': |  | ||||||
|         _state._runtime_vars['_debug_mode'] = True |  | ||||||
| 
 |  | ||||||
|         # expose internal debug module to every actor allowing |  | ||||||
|         # for use of ``await tractor.breakpoint()`` |  | ||||||
|         kwargs.setdefault('rpc_module_paths', []).append('tractor._debug') |  | ||||||
| 
 |  | ||||||
|     elif debug_mode: |  | ||||||
|         raise RuntimeError( |  | ||||||
|             "Debug mode is only supported for the `trio` backend!" |  | ||||||
|         ) |  | ||||||
| 
 |  | ||||||
|     main = partial(async_fn, *args) |  | ||||||
| 
 |  | ||||||
|     arbiter_addr = (host, port) = arbiter_addr or ( |  | ||||||
|         _default_arbiter_host, |  | ||||||
|         _default_arbiter_port |  | ||||||
|     ) |  | ||||||
| 
 |  | ||||||
|     loglevel = kwargs.get('loglevel', log.get_loglevel()) |  | ||||||
|     if loglevel is not None: |  | ||||||
|         log._default_loglevel = loglevel |  | ||||||
|         log.get_console_log(loglevel) |  | ||||||
| 
 |  | ||||||
|     # make a temporary connection to see if an arbiter exists |  | ||||||
|     arbiter_found = False |  | ||||||
|     try: |  | ||||||
|         async with _connect_chan(host, port): |  | ||||||
|             arbiter_found = True |  | ||||||
|     except OSError: |  | ||||||
|         logger.warning(f"No actor could be found @ {host}:{port}") |  | ||||||
| 
 |  | ||||||
|     # create a local actor and start up its main routine/task |  | ||||||
|     if arbiter_found:  # we were able to connect to an arbiter |  | ||||||
|         logger.info(f"Arbiter seems to exist @ {host}:{port}") |  | ||||||
|         actor = Actor( |  | ||||||
|             name or 'anonymous', |  | ||||||
|             arbiter_addr=arbiter_addr, |  | ||||||
|             **kwargs |  | ||||||
|         ) |  | ||||||
|         host, port = (host, 0) |  | ||||||
|     else: |  | ||||||
|         # start this local actor as the arbiter |  | ||||||
|         actor = Arbiter( |  | ||||||
|             name or 'arbiter', arbiter_addr=arbiter_addr, **kwargs) |  | ||||||
| 
 |  | ||||||
|     # ``Actor._async_main()`` creates an internal nursery if one is not |  | ||||||
|     # provided and thus blocks here until it's main task completes. |  | ||||||
|     # Note that if the current actor is the arbiter it is desirable |  | ||||||
|     # for it to stay up indefinitely until a re-election process has |  | ||||||
|     # taken place - which is not implemented yet FYI). |  | ||||||
| 
 |  | ||||||
|     try: |  | ||||||
|         return await _start_actor( |  | ||||||
|             actor, main, host, port, arbiter_addr=arbiter_addr |  | ||||||
|         ) |  | ||||||
|     finally: |  | ||||||
|         logger.info("Root actor terminated") |  | ||||||
| 
 |  | ||||||
| 
 |  | ||||||
| def run( |  | ||||||
|     async_fn: typing.Callable[..., typing.Awaitable], |  | ||||||
|     *args, |  | ||||||
|     name: Optional[str] = None, |  | ||||||
|     arbiter_addr: Tuple[str, int] = ( |  | ||||||
|         _default_arbiter_host, |  | ||||||
|         _default_arbiter_port, |  | ||||||
|     ), |  | ||||||
|     # either the `multiprocessing` start method: |  | ||||||
|     # https://docs.python.org/3/library/multiprocessing.html#contexts-and-start-methods |  | ||||||
|     # OR `trio` (the new default). |  | ||||||
|     start_method: Optional[str] = None, |  | ||||||
|     debug_mode: bool = False, |  | ||||||
|     **kwargs, |  | ||||||
| ) -> Any: |  | ||||||
|     """Run a trio-actor async function in process. |  | ||||||
| 
 |  | ||||||
|     This is tractor's main entry and the start point for any async actor. |  | ||||||
|     """ |  | ||||||
|     return trio.run( |  | ||||||
|         partial( |  | ||||||
|             # our entry |  | ||||||
|             _main, |  | ||||||
| 
 |  | ||||||
|             # user entry point |  | ||||||
|             async_fn, |  | ||||||
|             args, |  | ||||||
| 
 |  | ||||||
|             # global kwargs |  | ||||||
|             arbiter_addr=arbiter_addr, |  | ||||||
|             name=name, |  | ||||||
|             start_method=start_method, |  | ||||||
|             debug_mode=debug_mode, |  | ||||||
|             **kwargs, |  | ||||||
|         ) |  | ||||||
|     ) |  | ||||||
| 
 |  | ||||||
| 
 |  | ||||||
| def run_daemon( |  | ||||||
|     rpc_module_paths: List[str], |  | ||||||
|     **kwargs |  | ||||||
| ) -> None: |  | ||||||
|     """Spawn daemon actor which will respond to RPC. |  | ||||||
| 
 |  | ||||||
|     This is a convenience wrapper around |  | ||||||
|     ``tractor.run(trio.sleep(float('inf')))`` such that the first actor spawned |  | ||||||
|     is meant to run forever responding to RPC requests. |  | ||||||
|     """ |  | ||||||
|     kwargs['rpc_module_paths'] = list(rpc_module_paths) |  | ||||||
| 
 |  | ||||||
|     for path in rpc_module_paths: |  | ||||||
|         importlib.import_module(path) |  | ||||||
| 
 |  | ||||||
|     return run(partial(trio.sleep, float('inf')), **kwargs) |  | ||||||
|  |  | ||||||
|  | @ -1076,49 +1076,3 @@ class Arbiter(Actor): | ||||||
| 
 | 
 | ||||||
|     def unregister_actor(self, uid: Tuple[str, str]) -> None: |     def unregister_actor(self, uid: Tuple[str, str]) -> None: | ||||||
|         self._registry.pop(uid, None) |         self._registry.pop(uid, None) | ||||||
| 
 |  | ||||||
| 
 |  | ||||||
| async def _start_actor( |  | ||||||
|     actor: Actor, |  | ||||||
|     main: typing.Callable[..., typing.Awaitable], |  | ||||||
|     host: str, |  | ||||||
|     port: int, |  | ||||||
|     arbiter_addr: Tuple[str, int], |  | ||||||
|     nursery: trio.Nursery = None |  | ||||||
| ) -> Any: |  | ||||||
|     """Spawn a local actor by starting a task to execute it's main async |  | ||||||
|     function. |  | ||||||
| 
 |  | ||||||
|     Blocks if no nursery is provided, in which case it is expected the nursery |  | ||||||
|     provider is responsible for waiting on the task to complete. |  | ||||||
|     """ |  | ||||||
|     # assign process-local actor |  | ||||||
|     _state._current_actor = actor |  | ||||||
| 
 |  | ||||||
|     # start local channel-server and fake the portal API |  | ||||||
|     # NOTE: this won't block since we provide the nursery |  | ||||||
|     log.info(f"Starting local {actor} @ {host}:{port}") |  | ||||||
| 
 |  | ||||||
|     async with trio.open_nursery() as nursery: |  | ||||||
|         await nursery.start( |  | ||||||
|             partial( |  | ||||||
|                 actor._async_main, |  | ||||||
|                 accept_addr=(host, port), |  | ||||||
|                 parent_addr=None |  | ||||||
|             ) |  | ||||||
|         ) |  | ||||||
|         try: |  | ||||||
|             result = await main() |  | ||||||
|         except (Exception, trio.MultiError) as err: |  | ||||||
|             log.exception("Actor crashed:") |  | ||||||
|             await _debug._maybe_enter_pm(err) |  | ||||||
| 
 |  | ||||||
|             raise |  | ||||||
|         finally: |  | ||||||
|             await actor.cancel() |  | ||||||
| 
 |  | ||||||
|     # unset module state |  | ||||||
|     _state._current_actor = None |  | ||||||
|     log.info("Completed async main") |  | ||||||
| 
 |  | ||||||
|     return result |  | ||||||
|  |  | ||||||
|  | @ -1,5 +1,5 @@ | ||||||
| """ | """ | ||||||
| Process entry points. | Sub-process entry points. | ||||||
| """ | """ | ||||||
| from functools import partial | from functools import partial | ||||||
| from typing import Tuple, Any | from typing import Tuple, Any | ||||||
|  |  | ||||||
|  | @ -0,0 +1,194 @@ | ||||||
|  | """ | ||||||
|  | Root actor runtime ignition(s). | ||||||
|  | """ | ||||||
|  | from contextlib import asynccontextmanager | ||||||
|  | from functools import partial | ||||||
|  | import importlib | ||||||
|  | from typing import Tuple, Optional, List, Any | ||||||
|  | import typing | ||||||
|  | 
 | ||||||
|  | import trio | ||||||
|  | 
 | ||||||
|  | from ._actor import Actor, Arbiter | ||||||
|  | from . import _debug | ||||||
|  | from . import _spawn | ||||||
|  | from . import _state | ||||||
|  | from . import log | ||||||
|  | from ._ipc import _connect_chan | ||||||
|  | 
 | ||||||
|  | 
 | ||||||
|  | # set at startup and after forks | ||||||
|  | _default_arbiter_host = '127.0.0.1' | ||||||
|  | _default_arbiter_port = 1616 | ||||||
|  | 
 | ||||||
|  | 
 | ||||||
|  | logger = log.get_logger('tractor') | ||||||
|  | 
 | ||||||
|  | 
 | ||||||
|  | @asynccontextmanager | ||||||
|  | async def open_root_actor( | ||||||
|  |     arbiter_addr: Tuple[str, int], | ||||||
|  |     name: Optional[str] = None, | ||||||
|  |     start_method: Optional[str] = None, | ||||||
|  |     debug_mode: bool = False, | ||||||
|  |     **kwargs, | ||||||
|  | ) -> typing.Any: | ||||||
|  |     """Async entry point for ``tractor``. | ||||||
|  | 
 | ||||||
|  |     """ | ||||||
|  |     # mark top most level process as root actor | ||||||
|  |     _state._runtime_vars['_is_root'] = True | ||||||
|  | 
 | ||||||
|  |     if start_method is not None: | ||||||
|  |         _spawn.try_set_start_method(start_method) | ||||||
|  | 
 | ||||||
|  |     if debug_mode and _spawn._spawn_method == 'trio': | ||||||
|  |         _state._runtime_vars['_debug_mode'] = True | ||||||
|  | 
 | ||||||
|  |         # expose internal debug module to every actor allowing | ||||||
|  |         # for use of ``await tractor.breakpoint()`` | ||||||
|  |         kwargs.setdefault('rpc_module_paths', []).append('tractor._debug') | ||||||
|  | 
 | ||||||
|  |     elif debug_mode: | ||||||
|  |         raise RuntimeError( | ||||||
|  |             "Debug mode is only supported for the `trio` backend!" | ||||||
|  |         ) | ||||||
|  | 
 | ||||||
|  |     arbiter_addr = (host, port) = arbiter_addr or ( | ||||||
|  |         _default_arbiter_host, | ||||||
|  |         _default_arbiter_port | ||||||
|  |     ) | ||||||
|  | 
 | ||||||
|  |     loglevel = kwargs.get('loglevel', log.get_loglevel()) | ||||||
|  |     if loglevel is not None: | ||||||
|  |         log._default_loglevel = loglevel | ||||||
|  |         log.get_console_log(loglevel) | ||||||
|  | 
 | ||||||
|  |     # make a temporary connection to see if an arbiter exists | ||||||
|  |     arbiter_found = False | ||||||
|  | 
 | ||||||
|  |     try: | ||||||
|  |         async with _connect_chan(host, port): | ||||||
|  |             arbiter_found = True | ||||||
|  | 
 | ||||||
|  |     except OSError: | ||||||
|  |         logger.warning(f"No actor could be found @ {host}:{port}") | ||||||
|  | 
 | ||||||
|  |     # create a local actor and start up its main routine/task | ||||||
|  |     if arbiter_found:  # we were able to connect to an arbiter | ||||||
|  |         logger.info(f"Arbiter seems to exist @ {host}:{port}") | ||||||
|  | 
 | ||||||
|  |         actor = Actor( | ||||||
|  |             name or 'anonymous', | ||||||
|  |             arbiter_addr=arbiter_addr, | ||||||
|  |             **kwargs | ||||||
|  |         ) | ||||||
|  |         host, port = (host, 0) | ||||||
|  | 
 | ||||||
|  |     else: | ||||||
|  |         # start this local actor as the arbiter | ||||||
|  | 
 | ||||||
|  |         # Note that if the current actor is the arbiter it is desirable | ||||||
|  |         # for it to stay up indefinitely until a re-election process has | ||||||
|  |         # taken place - which is not implemented yet FYI). | ||||||
|  | 
 | ||||||
|  |         actor = Arbiter( | ||||||
|  |             name or 'arbiter', | ||||||
|  |             arbiter_addr=arbiter_addr, | ||||||
|  |             **kwargs | ||||||
|  |         ) | ||||||
|  | 
 | ||||||
|  |     try: | ||||||
|  |         # assign process-local actor | ||||||
|  |         _state._current_actor = actor | ||||||
|  | 
 | ||||||
|  |         # start local channel-server and fake the portal API | ||||||
|  |         # NOTE: this won't block since we provide the nursery | ||||||
|  |         logger.info(f"Starting local {actor} @ {host}:{port}") | ||||||
|  | 
 | ||||||
|  |         # start the actor runtime in a new task | ||||||
|  |         async with trio.open_nursery() as nursery: | ||||||
|  | 
 | ||||||
|  |             # ``Actor._async_main()`` creates an internal nursery and | ||||||
|  |             # thus blocks here until the entire underlying actor tree has | ||||||
|  |             # terminated thereby conducting structured concurrency. | ||||||
|  | 
 | ||||||
|  |             await nursery.start( | ||||||
|  |                 partial( | ||||||
|  |                     actor._async_main, | ||||||
|  |                     accept_addr=(host, port), | ||||||
|  |                     parent_addr=None | ||||||
|  |                 ) | ||||||
|  |             ) | ||||||
|  |             try: | ||||||
|  |                 yield actor | ||||||
|  |                 # result = await main() | ||||||
|  |             except (Exception, trio.MultiError) as err: | ||||||
|  |                 logger.exception("Actor crashed:") | ||||||
|  |                 await _debug._maybe_enter_pm(err) | ||||||
|  | 
 | ||||||
|  |                 raise | ||||||
|  |             finally: | ||||||
|  |                 logger.info("Shutting down root actor") | ||||||
|  |                 await actor.cancel() | ||||||
|  |     finally: | ||||||
|  |         _state._current_actor = None | ||||||
|  |         logger.info("Root actor terminated") | ||||||
|  | 
 | ||||||
|  | 
 | ||||||
|  | def run( | ||||||
|  | 
 | ||||||
|  |     # target | ||||||
|  |     async_fn: typing.Callable[..., typing.Awaitable], | ||||||
|  |     *args, | ||||||
|  | 
 | ||||||
|  |     # runtime kwargs | ||||||
|  |     name: Optional[str] = None, | ||||||
|  |     arbiter_addr: Tuple[str, int] = ( | ||||||
|  |         _default_arbiter_host, | ||||||
|  |         _default_arbiter_port, | ||||||
|  |     ), | ||||||
|  | 
 | ||||||
|  |     # either the `multiprocessing` start method: | ||||||
|  |     # https://docs.python.org/3/library/multiprocessing.html#contexts-and-start-methods | ||||||
|  |     # OR `trio` (the new default). | ||||||
|  |     start_method: Optional[str] = None, | ||||||
|  |     debug_mode: bool = False, | ||||||
|  |     **kwargs, | ||||||
|  | 
 | ||||||
|  | ) -> Any: | ||||||
|  |     """Run a trio-actor async function in process. | ||||||
|  | 
 | ||||||
|  |     This is tractor's main entry and the start point for any async actor. | ||||||
|  |     """ | ||||||
|  |     async def _main(): | ||||||
|  | 
 | ||||||
|  |         async with open_root_actor( | ||||||
|  |             arbiter_addr=arbiter_addr, | ||||||
|  |             name=name, | ||||||
|  |             start_method=start_method, | ||||||
|  |             debug_mode=debug_mode, | ||||||
|  |             **kwargs, | ||||||
|  |         ): | ||||||
|  | 
 | ||||||
|  |             return await async_fn(*args) | ||||||
|  | 
 | ||||||
|  |     return trio.run(_main) | ||||||
|  | 
 | ||||||
|  | 
 | ||||||
|  | def run_daemon( | ||||||
|  |     rpc_module_paths: List[str], | ||||||
|  |     **kwargs | ||||||
|  | ) -> None: | ||||||
|  |     """Spawn daemon actor which will respond to RPC. | ||||||
|  | 
 | ||||||
|  |     This is a convenience wrapper around | ||||||
|  |     ``tractor.run(trio.sleep(float('inf')))`` such that the first actor spawned | ||||||
|  |     is meant to run forever responding to RPC requests. | ||||||
|  |     """ | ||||||
|  |     kwargs['rpc_module_paths'] = list(rpc_module_paths) | ||||||
|  | 
 | ||||||
|  |     for path in rpc_module_paths: | ||||||
|  |         importlib.import_module(path) | ||||||
|  | 
 | ||||||
|  |     return run(partial(trio.sleep, float('inf')), **kwargs) | ||||||
		Loading…
	
		Reference in New Issue