forked from goodboy/tractor
commit
e0f4894071
277
README.rst
277
README.rst
|
@ -280,8 +280,60 @@ to all others with ease over standard network protocols).
|
|||
.. _Executor: https://docs.python.org/3/library/concurrent.futures.html#concurrent.futures.Executor
|
||||
|
||||
|
||||
Async IPC using *portals*
|
||||
*************************
|
||||
Cancellation
|
||||
************
|
||||
``tractor`` supports ``trio``'s cancellation_ system verbatim.
|
||||
Cancelling a nursery block cancels all actors spawned by it.
|
||||
Eventually ``tractor`` plans to support different `supervision strategies`_ like ``erlang``.
|
||||
|
||||
.. _supervision strategies: http://erlang.org/doc/man/supervisor.html#sup_flags
|
||||
|
||||
|
||||
Remote error propagation
|
||||
************************
|
||||
Any task invoked in a remote actor should ship any error(s) back to the calling
|
||||
actor where it is raised and expected to be dealt with. This way remote actors
|
||||
are never cancelled unless explicitly asked or there's a bug in ``tractor`` itself.
|
||||
|
||||
.. code:: python
|
||||
|
||||
async def assert_err():
|
||||
assert 0
|
||||
|
||||
|
||||
async def main():
|
||||
async with tractor.open_nursery() as n:
|
||||
real_actors = []
|
||||
for i in range(3):
|
||||
real_actors.append(await n.start_actor(
|
||||
f'actor_{i}',
|
||||
rpc_module_paths=[__name__],
|
||||
))
|
||||
|
||||
# start one actor that will fail immediately
|
||||
await n.run_in_actor('extra', assert_err)
|
||||
|
||||
# should error here with a ``RemoteActorError`` containing
|
||||
# an ``AssertionError`` and all the other actors have been cancelled
|
||||
|
||||
try:
|
||||
# also raises
|
||||
tractor.run(main)
|
||||
except tractor.RemoteActorError:
|
||||
print("Look Maa that actor failed hard, hehhh!")
|
||||
|
||||
|
||||
You'll notice the nursery cancellation conducts a *one-cancels-all*
|
||||
supervisory strategy `exactly like trio`_. The plan is to add more
|
||||
`erlang strategies`_ in the near future by allowing nurseries to accept
|
||||
a ``Supervisor`` type.
|
||||
|
||||
.. _exactly like trio: https://trio.readthedocs.io/en/latest/reference-core.html#cancellation-semantics
|
||||
.. _erlang strategies: http://learnyousomeerlang.com/supervisors
|
||||
|
||||
|
||||
IPC using *portals*
|
||||
*******************
|
||||
``tractor`` introduces the concept of a *portal* which is an API
|
||||
borrowed_ from ``trio``. A portal may seem similar to the idea of
|
||||
a RPC future_ except a *portal* allows invoking remote *async* functions and
|
||||
|
@ -305,10 +357,26 @@ channels_ system or shipping code over the network.
|
|||
|
||||
This *portal* approach turns out to be paricularly exciting with the
|
||||
introduction of `asynchronous generators`_ in Python 3.6! It means that
|
||||
actors can compose nicely in a data processing pipeline.
|
||||
actors can compose nicely in a data streaming pipeline.
|
||||
|
||||
As an example here's an actor that streams for 1 second from a remote async
|
||||
generator function running in a separate actor:
|
||||
|
||||
Streaming
|
||||
*********
|
||||
By now you've figured out that ``tractor`` lets you spawn process based
|
||||
*actors* that can invoke cross-process (async) functions and all with
|
||||
structured concurrency built in. But the **real cool stuff** is the
|
||||
native support for cross-process *streaming*.
|
||||
|
||||
|
||||
Asynchronous generators
|
||||
+++++++++++++++++++++++
|
||||
The default streaming function is simply an async generator definition.
|
||||
Every value *yielded* from the generator is delivered to the calling
|
||||
portal exactly like if you had invoked the function in-process meaning
|
||||
you can ``async for`` to receive each value on the calling side.
|
||||
|
||||
As an example here's a parent actor that streams for 1 second from a
|
||||
spawned subactor:
|
||||
|
||||
.. code:: python
|
||||
|
||||
|
@ -346,10 +414,87 @@ generator function running in a separate actor:
|
|||
|
||||
tractor.run(main)
|
||||
|
||||
By default async generator functions are treated as inter-actor
|
||||
*streams* when invoked via a portal (how else could you really interface
|
||||
with them anyway) so no special syntax to denote the streaming *service*
|
||||
is necessary.
|
||||
|
||||
|
||||
Channels and Contexts
|
||||
+++++++++++++++++++++
|
||||
If you aren't fond of having to write an async generator to stream data
|
||||
between actors (or need something more flexible) you can instead use
|
||||
a ``Context``. A context wraps an actor-local spawned task and
|
||||
a ``Channel`` so that tasks executing across multiple processes can
|
||||
stream data to one another using a low level, request oriented API.
|
||||
|
||||
A ``Channel`` wraps an underlying *transport* and *interchange* format
|
||||
to enable *inter-actor-communication*. In its present state ``tractor``
|
||||
uses TCP and msgpack_.
|
||||
|
||||
As an example if you wanted to create a streaming server without writing
|
||||
an async generator that *yields* values you instead define a decorated
|
||||
async function:
|
||||
|
||||
.. code:: python
|
||||
|
||||
@tractor.stream
|
||||
async def streamer(ctx: tractor.Context, rate: int = 2) -> None:
|
||||
"""A simple web response streaming server.
|
||||
"""
|
||||
while True:
|
||||
val = await web_request('http://data.feed.com')
|
||||
|
||||
# this is the same as ``yield`` in the async gen case
|
||||
await ctx.send_yield(val)
|
||||
|
||||
await trio.sleep(1 / rate)
|
||||
|
||||
|
||||
You must decorate the function with ``@tractor.stream`` and declare
|
||||
a ``ctx`` argument as the first in your function signature and then
|
||||
``tractor`` will treat the async function like an async generator - as
|
||||
a stream from the calling/client side.
|
||||
|
||||
This turns out to be handy particularly if you have multiple tasks
|
||||
pushing responses concurrently:
|
||||
|
||||
.. code:: python
|
||||
|
||||
async def streamer(
|
||||
ctx: tractor.Context,
|
||||
rate: int = 2
|
||||
) -> None:
|
||||
"""A simple web response streaming server.
|
||||
"""
|
||||
while True:
|
||||
val = await web_request(url)
|
||||
|
||||
# this is the same as ``yield`` in the async gen case
|
||||
await ctx.send_yield(val)
|
||||
|
||||
await trio.sleep(1 / rate)
|
||||
|
||||
|
||||
@tractor.stream
|
||||
async def stream_multiple_sources(
|
||||
ctx: tractor.Context,
|
||||
sources: List[str]
|
||||
) -> None:
|
||||
async with trio.open_nursery() as n:
|
||||
for url in sources:
|
||||
n.start_soon(streamer, ctx, url)
|
||||
|
||||
|
||||
The context notion comes from the context_ in nanomsg_.
|
||||
|
||||
.. _context: https://nanomsg.github.io/nng/man/tip/nng_ctx.5
|
||||
.. _msgpack: https://en.wikipedia.org/wiki/MessagePack
|
||||
|
||||
|
||||
|
||||
A full fledged streaming service
|
||||
********************************
|
||||
++++++++++++++++++++++++++++++++
|
||||
Alright, let's get fancy.
|
||||
|
||||
Say you wanted to spawn two actors which each pull data feeds from
|
||||
|
@ -471,58 +616,6 @@ as ``multiprocessing`` calls it) which is running ``main()``.
|
|||
.. _remote function execution: https://codespeak.net/execnet/example/test_info.html#remote-exec-a-function-avoiding-inlined-source-part-i
|
||||
|
||||
|
||||
Cancellation
|
||||
************
|
||||
``tractor`` supports ``trio``'s cancellation_ system verbatim.
|
||||
Cancelling a nursery block cancels all actors spawned by it.
|
||||
Eventually ``tractor`` plans to support different `supervision strategies`_ like ``erlang``.
|
||||
|
||||
.. _supervision strategies: http://erlang.org/doc/man/supervisor.html#sup_flags
|
||||
|
||||
|
||||
Remote error propagation
|
||||
************************
|
||||
Any task invoked in a remote actor should ship any error(s) back to the calling
|
||||
actor where it is raised and expected to be dealt with. This way remote actors
|
||||
are never cancelled unless explicitly asked or there's a bug in ``tractor`` itself.
|
||||
|
||||
.. code:: python
|
||||
|
||||
async def assert_err():
|
||||
assert 0
|
||||
|
||||
|
||||
async def main():
|
||||
async with tractor.open_nursery() as n:
|
||||
real_actors = []
|
||||
for i in range(3):
|
||||
real_actors.append(await n.start_actor(
|
||||
f'actor_{i}',
|
||||
rpc_module_paths=[__name__],
|
||||
))
|
||||
|
||||
# start one actor that will fail immediately
|
||||
await n.run_in_actor('extra', assert_err)
|
||||
|
||||
# should error here with a ``RemoteActorError`` containing
|
||||
# an ``AssertionError`` and all the other actors have been cancelled
|
||||
|
||||
try:
|
||||
# also raises
|
||||
tractor.run(main)
|
||||
except tractor.RemoteActorError:
|
||||
print("Look Maa that actor failed hard, hehhh!")
|
||||
|
||||
|
||||
You'll notice the nursery cancellation conducts a *one-cancels-all*
|
||||
supervisory strategy `exactly like trio`_. The plan is to add more
|
||||
`erlang strategies`_ in the near future by allowing nurseries to accept
|
||||
a ``Supervisor`` type.
|
||||
|
||||
.. _exactly like trio: https://trio.readthedocs.io/en/latest/reference-core.html#cancellation-semantics
|
||||
.. _erlang strategies: http://learnyousomeerlang.com/supervisors
|
||||
|
||||
|
||||
Actor local variables
|
||||
*********************
|
||||
Although ``tractor`` uses a *shared-nothing* architecture between processes
|
||||
|
@ -556,8 +649,8 @@ a convenience for passing simple data to newly spawned actors); building
|
|||
out a state sharing system per-actor is totally up to you.
|
||||
|
||||
|
||||
How do actors find each other (a poor man's *service discovery*)?
|
||||
*****************************************************************
|
||||
Service Discovery
|
||||
*****************
|
||||
Though it will be built out much more in the near future, ``tractor``
|
||||
currently keeps track of actors by ``(name: str, id: str)`` using a
|
||||
special actor called the *arbiter*. Currently the *arbiter* must exist
|
||||
|
@ -590,70 +683,6 @@ The ``name`` value you should pass to ``find_actor()`` is the one you passed as
|
|||
*first* argument to either ``tractor.run()`` or ``ActorNursery.start_actor()``.
|
||||
|
||||
|
||||
Streaming using channels and contexts
|
||||
*************************************
|
||||
``Channel`` is the API which wraps an underlying *transport* and *interchange*
|
||||
format to enable *inter-actor-communication*. In its present state ``tractor``
|
||||
uses TCP and msgpack_.
|
||||
|
||||
If you aren't fond of having to write an async generator to stream data
|
||||
between actors (or need something more flexible) you can instead use a
|
||||
``Context``. A context wraps an actor-local spawned task and a ``Channel``
|
||||
so that tasks executing across multiple processes can stream data
|
||||
to one another using a low level, request oriented API.
|
||||
|
||||
As an example if you wanted to create a streaming server without writing
|
||||
an async generator that *yields* values you instead define an async
|
||||
function:
|
||||
|
||||
.. code:: python
|
||||
|
||||
async def streamer(ctx: tractor.Context, rate: int = 2) -> None:
|
||||
"""A simple web response streaming server.
|
||||
"""
|
||||
while True:
|
||||
val = await web_request('http://data.feed.com')
|
||||
|
||||
# this is the same as ``yield`` in the async gen case
|
||||
await ctx.send_yield(val)
|
||||
|
||||
await trio.sleep(1 / rate)
|
||||
|
||||
|
||||
All that's required is declaring a ``ctx`` argument name somewhere in
|
||||
your function signature and ``tractor`` will treat the async function
|
||||
like an async generator - as a streaming function from the client side.
|
||||
This turns out to be handy particularly if you have
|
||||
multiple tasks streaming responses concurrently:
|
||||
|
||||
.. code:: python
|
||||
|
||||
async def streamer(ctx: tractor.Context, rate: int = 2) -> None:
|
||||
"""A simple web response streaming server.
|
||||
"""
|
||||
while True:
|
||||
val = await web_request(url)
|
||||
|
||||
# this is the same as ``yield`` in the async gen case
|
||||
await ctx.send_yield(val)
|
||||
|
||||
await trio.sleep(1 / rate)
|
||||
|
||||
|
||||
async def stream_multiple_sources(
|
||||
ctx: tractor.Context, sources: List[str]
|
||||
) -> None:
|
||||
async with trio.open_nursery() as n:
|
||||
for url in sources:
|
||||
n.start_soon(streamer, ctx, url)
|
||||
|
||||
|
||||
The context notion comes from the context_ in nanomsg_.
|
||||
|
||||
.. _context: https://nanomsg.github.io/nng/man/tip/nng_ctx.5
|
||||
.. _msgpack: https://en.wikipedia.org/wiki/MessagePack
|
||||
|
||||
|
||||
Running actors standalone
|
||||
*************************
|
||||
You don't have to spawn any actors using ``open_nursery()`` if you just
|
||||
|
|
|
@ -2,13 +2,29 @@
|
|||
Streaming via async gen api
|
||||
"""
|
||||
import time
|
||||
from functools import partial
|
||||
|
||||
import trio
|
||||
import tractor
|
||||
import pytest
|
||||
|
||||
|
||||
async def stream_seq(sequence):
|
||||
|
||||
def test_must_define_ctx():
|
||||
|
||||
with pytest.raises(TypeError) as err:
|
||||
@tractor.stream
|
||||
async def no_ctx():
|
||||
pass
|
||||
|
||||
assert "no_ctx must be `ctx: tractor.Context" in str(err.value)
|
||||
|
||||
@tractor.stream
|
||||
async def no_ctx(ctx):
|
||||
pass
|
||||
|
||||
|
||||
async def async_gen_stream(sequence):
|
||||
for i in sequence:
|
||||
yield i
|
||||
await trio.sleep(0.1)
|
||||
|
@ -20,10 +36,23 @@ async def stream_seq(sequence):
|
|||
assert cs.cancelled_caught
|
||||
|
||||
|
||||
async def stream_from_single_subactor():
|
||||
@tractor.stream
|
||||
async def context_stream(ctx, sequence):
|
||||
for i in sequence:
|
||||
await ctx.send_yield(i)
|
||||
await trio.sleep(0.1)
|
||||
|
||||
# block indefinitely waiting to be cancelled by ``aclose()`` call
|
||||
with trio.CancelScope() as cs:
|
||||
await trio.sleep(float('inf'))
|
||||
assert 0
|
||||
assert cs.cancelled_caught
|
||||
|
||||
|
||||
async def stream_from_single_subactor(stream_func_name):
|
||||
"""Verify we can spawn a daemon actor and retrieve streamed data.
|
||||
"""
|
||||
async with tractor.find_actor('brokerd') as portals:
|
||||
async with tractor.find_actor('streamerd') as portals:
|
||||
if not portals:
|
||||
# only one per host address, spawns an actor if None
|
||||
async with tractor.open_nursery() as nursery:
|
||||
|
@ -36,37 +65,43 @@ async def stream_from_single_subactor():
|
|||
|
||||
seq = range(10)
|
||||
|
||||
agen = await portal.run(
|
||||
stream = await portal.run(
|
||||
__name__,
|
||||
'stream_seq', # the func above
|
||||
stream_func_name, # one of the funcs above
|
||||
sequence=list(seq), # has to be msgpack serializable
|
||||
)
|
||||
# it'd sure be nice to have an asyncitertools here...
|
||||
iseq = iter(seq)
|
||||
ival = next(iseq)
|
||||
async for val in agen:
|
||||
async for val in stream:
|
||||
assert val == ival
|
||||
try:
|
||||
ival = next(iseq)
|
||||
except StopIteration:
|
||||
# should cancel far end task which will be
|
||||
# caught and no error is raised
|
||||
await agen.aclose()
|
||||
await stream.aclose()
|
||||
|
||||
await trio.sleep(0.3)
|
||||
try:
|
||||
await agen.__anext__()
|
||||
await stream.__anext__()
|
||||
except StopAsyncIteration:
|
||||
# stop all spawned subactors
|
||||
await portal.cancel_actor()
|
||||
# await nursery.cancel()
|
||||
|
||||
|
||||
def test_stream_from_single_subactor(arb_addr, start_method):
|
||||
@pytest.mark.parametrize(
|
||||
'stream_func', ['async_gen_stream', 'context_stream']
|
||||
)
|
||||
def test_stream_from_single_subactor(arb_addr, start_method, stream_func):
|
||||
"""Verify streaming from a spawned async generator.
|
||||
"""
|
||||
tractor.run(
|
||||
stream_from_single_subactor,
|
||||
partial(
|
||||
stream_from_single_subactor,
|
||||
stream_func_name=stream_func,
|
||||
),
|
||||
arbiter_addr=arb_addr,
|
||||
start_method=start_method,
|
||||
)
|
||||
|
|
|
@ -11,10 +11,10 @@ import trio # type: ignore
|
|||
from trio import MultiError
|
||||
|
||||
from . import log
|
||||
from ._ipc import _connect_chan, Channel, Context
|
||||
from ._actor import (
|
||||
Actor, _start_actor, Arbiter, get_arbiter, find_actor, wait_for_actor
|
||||
)
|
||||
from ._ipc import _connect_chan, Channel
|
||||
from ._streaming import Context, stream
|
||||
from ._discovery import get_arbiter, find_actor, wait_for_actor
|
||||
from ._actor import Actor, _start_actor, Arbiter
|
||||
from ._trionics import open_nursery
|
||||
from ._state import current_actor
|
||||
from ._exceptions import RemoteActorError, ModuleNotExposed
|
||||
|
@ -30,6 +30,7 @@ __all__ = [
|
|||
'wait_for_actor',
|
||||
'Channel',
|
||||
'Context',
|
||||
'stream',
|
||||
'MultiError',
|
||||
'RemoteActorError',
|
||||
'ModuleNotExposed',
|
||||
|
|
|
@ -8,26 +8,22 @@ import importlib
|
|||
import inspect
|
||||
import uuid
|
||||
import typing
|
||||
from typing import Dict, List, Tuple, Any, Optional, Union
|
||||
from typing import Dict, List, Tuple, Any, Optional
|
||||
|
||||
import trio # type: ignore
|
||||
from async_generator import asynccontextmanager, aclosing
|
||||
from async_generator import aclosing
|
||||
|
||||
from ._ipc import Channel, _connect_chan, Context
|
||||
from ._ipc import Channel
|
||||
from ._streaming import Context, _context
|
||||
from .log import get_console_log, get_logger
|
||||
from ._exceptions import (
|
||||
pack_error,
|
||||
unpack_error,
|
||||
ModuleNotExposed
|
||||
)
|
||||
from ._portal import (
|
||||
Portal,
|
||||
open_portal,
|
||||
_do_handshake,
|
||||
LocalPortal,
|
||||
)
|
||||
from ._discovery import get_arbiter
|
||||
from ._portal import Portal
|
||||
from . import _state
|
||||
from ._state import current_actor
|
||||
|
||||
|
||||
log = get_logger('tractor')
|
||||
|
@ -45,19 +41,16 @@ async def _invoke(
|
|||
kwargs: Dict[str, Any],
|
||||
task_status=trio.TASK_STATUS_IGNORED
|
||||
):
|
||||
"""Invoke local func and return results over provided channel.
|
||||
"""Invoke local func and deliver result(s) over provided channel.
|
||||
"""
|
||||
sig = inspect.signature(func)
|
||||
treat_as_gen = False
|
||||
cs = None
|
||||
ctx = Context(chan, cid)
|
||||
if 'ctx' in sig.parameters:
|
||||
cancel_scope = trio.CancelScope()
|
||||
ctx = Context(chan, cid, cancel_scope)
|
||||
_context.set(ctx)
|
||||
if getattr(func, '_tractor_stream_function', False):
|
||||
# handle decorated ``@tractor.stream`` async functions
|
||||
kwargs['ctx'] = ctx
|
||||
# TODO: eventually we want to be more stringent
|
||||
# about what is considered a far-end async-generator.
|
||||
# Right now both actual async gens and any async
|
||||
# function which declares a `ctx` kwarg in its
|
||||
# signature will be treated as one.
|
||||
treat_as_gen = True
|
||||
try:
|
||||
is_async_partial = False
|
||||
|
@ -73,7 +66,7 @@ async def _invoke(
|
|||
not is_async_gen_partial
|
||||
):
|
||||
await chan.send({'functype': 'function', 'cid': cid})
|
||||
with trio.CancelScope() as cs:
|
||||
with cancel_scope as cs:
|
||||
task_status.started(cs)
|
||||
await chan.send({'return': func(**kwargs), 'cid': cid})
|
||||
else:
|
||||
|
@ -88,7 +81,7 @@ async def _invoke(
|
|||
# have to properly handle the closing (aclosing)
|
||||
# of the async gen in order to be sure the cancel
|
||||
# is propagated!
|
||||
with trio.CancelScope() as cs:
|
||||
with cancel_scope as cs:
|
||||
task_status.started(cs)
|
||||
async with aclosing(coro) as agen:
|
||||
async for item in agen:
|
||||
|
@ -113,7 +106,7 @@ async def _invoke(
|
|||
# back values like an async-generator would but must
|
||||
# manualy construct the response dict-packet-responses as
|
||||
# above
|
||||
with trio.CancelScope() as cs:
|
||||
with cancel_scope as cs:
|
||||
task_status.started(cs)
|
||||
await coro
|
||||
if not cs.cancelled_caught:
|
||||
|
@ -122,7 +115,7 @@ async def _invoke(
|
|||
await chan.send({'stop': True, 'cid': cid})
|
||||
else:
|
||||
await chan.send({'functype': 'asyncfunction', 'cid': cid})
|
||||
with trio.CancelScope() as cs:
|
||||
with cancel_scope as cs:
|
||||
task_status.started(cs)
|
||||
await chan.send({'return': await coro, 'cid': cid})
|
||||
except Exception as err:
|
||||
|
@ -174,7 +167,7 @@ class Actor:
|
|||
arbiter_addr: Optional[Tuple[str, int]] = None,
|
||||
) -> None:
|
||||
self.name = name
|
||||
self.uid = (name, uid or str(uuid.uuid1()))
|
||||
self.uid = (name, uid or str(uuid.uuid4()))
|
||||
self.rpc_module_paths = rpc_module_paths
|
||||
self._mods: dict = {}
|
||||
# TODO: consider making this a dynamically defined
|
||||
|
@ -247,7 +240,7 @@ class Actor:
|
|||
|
||||
# send/receive initial handshake response
|
||||
try:
|
||||
uid = await _do_handshake(self, chan)
|
||||
uid = await self._do_handshake(chan)
|
||||
except StopAsyncIteration:
|
||||
log.warning(f"Channel {chan} failed to handshake")
|
||||
return
|
||||
|
@ -351,7 +344,7 @@ class Actor:
|
|||
caller id and a ``trio.Queue`` that can be used to wait for
|
||||
responses delivered by the local message processing loop.
|
||||
"""
|
||||
cid = str(uuid.uuid1())
|
||||
cid = str(uuid.uuid4())
|
||||
assert chan.uid
|
||||
recv_chan = self.get_memchans(chan.uid, cid)
|
||||
log.debug(f"Sending cmd to {chan.uid}: {ns}.{func}({kwargs})")
|
||||
|
@ -373,11 +366,12 @@ class Actor:
|
|||
msg = None
|
||||
log.debug(f"Entering msg loop for {chan} from {chan.uid}")
|
||||
try:
|
||||
# internal scope allows for keeping this message
|
||||
# loop running despite the current task having been
|
||||
# cancelled (eg. `open_portal()` may call this method from
|
||||
# a locally spawned task)
|
||||
with trio.CancelScope(shield=shield) as cs:
|
||||
# this internal scope allows for keeping this message
|
||||
# loop running despite the current task having been
|
||||
# cancelled (eg. `open_portal()` may call this method from
|
||||
# a locally spawned task) and recieve this scope using
|
||||
# ``scope = Nursery.start()``
|
||||
task_status.started(cs)
|
||||
async for msg in chan:
|
||||
if msg is None: # loop terminate sentinel
|
||||
|
@ -385,7 +379,7 @@ class Actor:
|
|||
f"Cancelling all tasks for {chan} from {chan.uid}")
|
||||
for (channel, cid) in self._rpc_tasks:
|
||||
if channel is chan:
|
||||
self.cancel_task(cid, Context(channel, cid))
|
||||
self._cancel_task(cid, channel)
|
||||
log.debug(
|
||||
f"Msg loop signalled to terminate for"
|
||||
f" {chan} from {chan.uid}")
|
||||
|
@ -419,6 +413,16 @@ class Actor:
|
|||
f"{ns}.{funcname}({kwargs})")
|
||||
if ns == 'self':
|
||||
func = getattr(self, funcname)
|
||||
if funcname == '_cancel_task':
|
||||
# XXX: a special case is made here for
|
||||
# remote calls since we don't want the
|
||||
# remote actor have to know which channel
|
||||
# the task is associated with and we can't
|
||||
# pass non-primitive types between actors.
|
||||
# This means you can use:
|
||||
# Portal.run('self', '_cancel_task, cid=did)
|
||||
# without passing the `chan` arg.
|
||||
kwargs['chan'] = chan
|
||||
else:
|
||||
# complain to client about restricted modules
|
||||
try:
|
||||
|
@ -537,7 +541,7 @@ class Actor:
|
|||
)
|
||||
await chan.connect()
|
||||
# initial handshake, report who we are, who they are
|
||||
await _do_handshake(self, chan)
|
||||
await self._do_handshake(chan)
|
||||
except OSError: # failed to connect
|
||||
log.warning(
|
||||
f"Failed to connect to parent @ {parent_addr},"
|
||||
|
@ -661,21 +665,20 @@ class Actor:
|
|||
self.cancel_server()
|
||||
self._root_nursery.cancel_scope.cancel()
|
||||
|
||||
async def cancel_task(self, cid, ctx):
|
||||
"""Cancel a local task.
|
||||
async def _cancel_task(self, cid, chan):
|
||||
"""Cancel a local task by call-id / channel.
|
||||
|
||||
Note this method will be treated as a streaming funciton
|
||||
Note this method will be treated as a streaming function
|
||||
by remote actor-callers due to the declaration of ``ctx``
|
||||
in the signature (for now).
|
||||
"""
|
||||
# right now this is only implicitly called by
|
||||
# streaming IPC but it should be called
|
||||
# to cancel any remotely spawned task
|
||||
chan = ctx.chan
|
||||
try:
|
||||
# this ctx based lookup ensures the requested task to
|
||||
# be cancelled was indeed spawned by a request from this channel
|
||||
scope, func, is_complete = self._rpc_tasks[(ctx.chan, cid)]
|
||||
scope, func, is_complete = self._rpc_tasks[(chan, cid)]
|
||||
except KeyError:
|
||||
log.warning(f"{cid} has already completed/terminated?")
|
||||
return
|
||||
|
@ -686,7 +689,7 @@ class Actor:
|
|||
|
||||
# don't allow cancelling this function mid-execution
|
||||
# (is this necessary?)
|
||||
if func is self.cancel_task:
|
||||
if func is self._cancel_task:
|
||||
return
|
||||
|
||||
scope.cancel()
|
||||
|
@ -704,7 +707,7 @@ class Actor:
|
|||
log.info(f"Cancelling all {len(tasks)} rpc tasks:\n{tasks} ")
|
||||
for (chan, cid) in tasks.copy():
|
||||
# TODO: this should really done in a nursery batch
|
||||
await self.cancel_task(cid, Context(chan, cid))
|
||||
await self._cancel_task(cid, chan)
|
||||
# if tasks:
|
||||
log.info(
|
||||
f"Waiting for remaining rpc tasks to complete {tasks}")
|
||||
|
@ -735,6 +738,25 @@ class Actor:
|
|||
"""Return all channels to the actor with provided uid."""
|
||||
return self._peers[uid]
|
||||
|
||||
async def _do_handshake(
|
||||
self,
|
||||
chan: Channel
|
||||
) -> Tuple[str, str]:
|
||||
"""Exchange (name, UUIDs) identifiers as the first communication step.
|
||||
|
||||
These are essentially the "mailbox addresses" found in actor model
|
||||
parlance.
|
||||
"""
|
||||
await chan.send(self.uid)
|
||||
uid: Tuple[str, str] = await chan.recv()
|
||||
|
||||
if not isinstance(uid, tuple):
|
||||
raise ValueError(f"{uid} is not a valid uid?!")
|
||||
|
||||
chan.uid = uid
|
||||
log.info(f"Handshake with actor {uid}@{chan.raddr} complete")
|
||||
return uid
|
||||
|
||||
|
||||
class Arbiter(Actor):
|
||||
"""A special actor who knows all the other actors and always has
|
||||
|
@ -840,66 +862,3 @@ async def _start_actor(
|
|||
log.info("Completed async main")
|
||||
|
||||
return result
|
||||
|
||||
|
||||
@asynccontextmanager
|
||||
async def get_arbiter(
|
||||
host: str, port: int
|
||||
) -> typing.AsyncGenerator[Union[Portal, LocalPortal], None]:
|
||||
"""Return a portal instance connected to a local or remote
|
||||
arbiter.
|
||||
"""
|
||||
actor = current_actor()
|
||||
if not actor:
|
||||
raise RuntimeError("No actor instance has been defined yet?")
|
||||
|
||||
if actor.is_arbiter:
|
||||
# we're already the arbiter
|
||||
# (likely a re-entrant call from the arbiter actor)
|
||||
yield LocalPortal(actor)
|
||||
else:
|
||||
async with _connect_chan(host, port) as chan:
|
||||
async with open_portal(chan) as arb_portal:
|
||||
yield arb_portal
|
||||
|
||||
|
||||
@asynccontextmanager
|
||||
async def find_actor(
|
||||
name: str, arbiter_sockaddr: Tuple[str, int] = None
|
||||
) -> typing.AsyncGenerator[Optional[Portal], None]:
|
||||
"""Ask the arbiter to find actor(s) by name.
|
||||
|
||||
Returns a connected portal to the last registered matching actor
|
||||
known to the arbiter.
|
||||
"""
|
||||
actor = current_actor()
|
||||
async with get_arbiter(*arbiter_sockaddr or actor._arb_addr) as arb_portal:
|
||||
sockaddr = await arb_portal.run('self', 'find_actor', name=name)
|
||||
# TODO: return portals to all available actors - for now just
|
||||
# the last one that registered
|
||||
if name == 'arbiter' and actor.is_arbiter:
|
||||
raise RuntimeError("The current actor is the arbiter")
|
||||
elif sockaddr:
|
||||
async with _connect_chan(*sockaddr) as chan:
|
||||
async with open_portal(chan) as portal:
|
||||
yield portal
|
||||
else:
|
||||
yield None
|
||||
|
||||
|
||||
@asynccontextmanager
|
||||
async def wait_for_actor(
|
||||
name: str,
|
||||
arbiter_sockaddr: Tuple[str, int] = None
|
||||
) -> typing.AsyncGenerator[Portal, None]:
|
||||
"""Wait on an actor to register with the arbiter.
|
||||
|
||||
A portal to the first registered actor is returned.
|
||||
"""
|
||||
actor = current_actor()
|
||||
async with get_arbiter(*arbiter_sockaddr or actor._arb_addr) as arb_portal:
|
||||
sockaddrs = await arb_portal.run('self', 'wait_for_actor', name=name)
|
||||
sockaddr = sockaddrs[-1]
|
||||
async with _connect_chan(*sockaddr) as chan:
|
||||
async with open_portal(chan) as portal:
|
||||
yield portal
|
||||
|
|
|
@ -0,0 +1,77 @@
|
|||
"""
|
||||
Actor discovery API.
|
||||
"""
|
||||
import typing
|
||||
from typing import Tuple, Optional, Union
|
||||
from async_generator import asynccontextmanager
|
||||
|
||||
from ._ipc import _connect_chan
|
||||
from ._portal import (
|
||||
Portal,
|
||||
open_portal,
|
||||
LocalPortal,
|
||||
)
|
||||
from ._state import current_actor
|
||||
|
||||
|
||||
@asynccontextmanager
|
||||
async def get_arbiter(
|
||||
host: str, port: int
|
||||
) -> typing.AsyncGenerator[Union[Portal, LocalPortal], None]:
|
||||
"""Return a portal instance connected to a local or remote
|
||||
arbiter.
|
||||
"""
|
||||
actor = current_actor()
|
||||
if not actor:
|
||||
raise RuntimeError("No actor instance has been defined yet?")
|
||||
|
||||
if actor.is_arbiter:
|
||||
# we're already the arbiter
|
||||
# (likely a re-entrant call from the arbiter actor)
|
||||
yield LocalPortal(actor)
|
||||
else:
|
||||
async with _connect_chan(host, port) as chan:
|
||||
async with open_portal(chan) as arb_portal:
|
||||
yield arb_portal
|
||||
|
||||
|
||||
@asynccontextmanager
|
||||
async def find_actor(
|
||||
name: str, arbiter_sockaddr: Tuple[str, int] = None
|
||||
) -> typing.AsyncGenerator[Optional[Portal], None]:
|
||||
"""Ask the arbiter to find actor(s) by name.
|
||||
|
||||
Returns a connected portal to the last registered matching actor
|
||||
known to the arbiter.
|
||||
"""
|
||||
actor = current_actor()
|
||||
async with get_arbiter(*arbiter_sockaddr or actor._arb_addr) as arb_portal:
|
||||
sockaddr = await arb_portal.run('self', 'find_actor', name=name)
|
||||
# TODO: return portals to all available actors - for now just
|
||||
# the last one that registered
|
||||
if name == 'arbiter' and actor.is_arbiter:
|
||||
raise RuntimeError("The current actor is the arbiter")
|
||||
elif sockaddr:
|
||||
async with _connect_chan(*sockaddr) as chan:
|
||||
async with open_portal(chan) as portal:
|
||||
yield portal
|
||||
else:
|
||||
yield None
|
||||
|
||||
|
||||
@asynccontextmanager
|
||||
async def wait_for_actor(
|
||||
name: str,
|
||||
arbiter_sockaddr: Tuple[str, int] = None
|
||||
) -> typing.AsyncGenerator[Portal, None]:
|
||||
"""Wait on an actor to register with the arbiter.
|
||||
|
||||
A portal to the first registered actor is returned.
|
||||
"""
|
||||
actor = current_actor()
|
||||
async with get_arbiter(*arbiter_sockaddr or actor._arb_addr) as arb_portal:
|
||||
sockaddrs = await arb_portal.run('self', 'wait_for_actor', name=name)
|
||||
sockaddr = sockaddrs[-1]
|
||||
async with _connect_chan(*sockaddr) as chan:
|
||||
async with open_portal(chan) as portal:
|
||||
yield portal
|
|
@ -1,7 +1,6 @@
|
|||
"""
|
||||
Inter-process comms abstractions
|
||||
"""
|
||||
from dataclasses import dataclass
|
||||
import typing
|
||||
from typing import Any, Tuple, Optional
|
||||
|
||||
|
@ -205,28 +204,6 @@ class Channel:
|
|||
return self.msgstream.connected() if self.msgstream else False
|
||||
|
||||
|
||||
@dataclass(frozen=True)
|
||||
class Context:
|
||||
"""An IAC (inter-actor communication) context.
|
||||
|
||||
Allows maintaining task or protocol specific state between communicating
|
||||
actors. A unique context is created on the receiving end for every request
|
||||
to a remote actor.
|
||||
"""
|
||||
chan: Channel
|
||||
cid: str
|
||||
|
||||
# TODO: we should probably attach the actor-task
|
||||
# cancel scope here now that trio is exposing it
|
||||
# as a public object
|
||||
|
||||
async def send_yield(self, data: Any) -> None:
|
||||
await self.chan.send({'yield': data, 'cid': self.cid})
|
||||
|
||||
async def send_stop(self) -> None:
|
||||
await self.chan.send({'stop': True, 'cid': self.cid})
|
||||
|
||||
|
||||
@asynccontextmanager
|
||||
async def _connect_chan(
|
||||
host: str, port: int
|
||||
|
|
|
@ -33,21 +33,6 @@ async def maybe_open_nursery(nursery: trio._core._run.Nursery = None):
|
|||
yield nursery
|
||||
|
||||
|
||||
async def _do_handshake(
|
||||
actor: 'Actor', # type: ignore
|
||||
chan: Channel
|
||||
) -> Any:
|
||||
await chan.send(actor.uid)
|
||||
uid: Tuple[str, str] = await chan.recv()
|
||||
|
||||
if not isinstance(uid, tuple):
|
||||
raise ValueError(f"{uid} is not a valid uid?!")
|
||||
|
||||
chan.uid = uid
|
||||
log.info(f"Handshake with actor {uid}@{chan.raddr} complete")
|
||||
return uid
|
||||
|
||||
|
||||
class StreamReceiveChannel(trio.abc.ReceiveChannel):
|
||||
"""A wrapper around a ``trio._channel.MemoryReceiveChannel`` with
|
||||
special behaviour for signalling stream termination across an
|
||||
|
@ -95,8 +80,8 @@ class StreamReceiveChannel(trio.abc.ReceiveChannel):
|
|||
raise unpack_error(msg, self._portal.channel)
|
||||
|
||||
async def aclose(self):
|
||||
"""Cancel associate remote actor task on close
|
||||
as well as the local memory channel.
|
||||
"""Cancel associated remote actor task and local memory channel
|
||||
on close.
|
||||
"""
|
||||
if self._rx_chan._closed:
|
||||
log.warning(f"{self} is already closed")
|
||||
|
@ -107,15 +92,10 @@ class StreamReceiveChannel(trio.abc.ReceiveChannel):
|
|||
log.warning(
|
||||
f"Cancelling stream {cid} to "
|
||||
f"{self._portal.channel.uid}")
|
||||
# TODO: yeah.. it'd be nice if this was just an
|
||||
# async func on the far end. Gotta figure out a
|
||||
# better way then implicitly feeding the ctx
|
||||
# to declaring functions; likely a decorator
|
||||
# system.
|
||||
rchan = await self._portal.run(
|
||||
'self', 'cancel_task', cid=cid)
|
||||
async for _ in rchan:
|
||||
pass
|
||||
# NOTE: we're telling the far end actor to cancel a task
|
||||
# corresponding to *this actor*. The far end local channel
|
||||
# instance is passed to `Actor._cancel_task()` implicitly.
|
||||
await self._portal.run('self', '_cancel_task', cid=cid)
|
||||
|
||||
if cs.cancelled_caught:
|
||||
# XXX: there's no way to know if the remote task was indeed
|
||||
|
@ -153,6 +133,7 @@ class Portal:
|
|||
Tuple[str, Any, str, Dict[str, Any]]
|
||||
] = None
|
||||
self._streams: Set[StreamReceiveChannel] = set()
|
||||
self.actor = current_actor()
|
||||
|
||||
async def _submit(
|
||||
self,
|
||||
|
@ -167,7 +148,7 @@ class Portal:
|
|||
This is an async call.
|
||||
"""
|
||||
# ship a function call request to the remote actor
|
||||
cid, recv_chan = await current_actor().send_cmd(
|
||||
cid, recv_chan = await self.actor.send_cmd(
|
||||
self.channel, ns, func, kwargs)
|
||||
|
||||
# wait on first response msg and handle (this should be
|
||||
|
@ -345,7 +326,7 @@ async def open_portal(
|
|||
was_connected = True
|
||||
|
||||
if channel.uid is None:
|
||||
await _do_handshake(actor, channel)
|
||||
await actor._do_handshake(channel)
|
||||
|
||||
msg_loop_cs = await nursery.start(
|
||||
partial(
|
||||
|
|
|
@ -0,0 +1,49 @@
|
|||
import inspect
|
||||
from contextvars import ContextVar
|
||||
from dataclasses import dataclass
|
||||
from typing import Any
|
||||
|
||||
import trio
|
||||
|
||||
from ._ipc import Channel
|
||||
|
||||
|
||||
_context: ContextVar['Context'] = ContextVar('context')
|
||||
|
||||
|
||||
@dataclass(frozen=True)
|
||||
class Context:
|
||||
"""An IAC (inter-actor communication) context.
|
||||
|
||||
Allows maintaining task or protocol specific state between communicating
|
||||
actors. A unique context is created on the receiving end for every request
|
||||
to a remote actor.
|
||||
"""
|
||||
chan: Channel
|
||||
cid: str
|
||||
cancel_scope: trio.CancelScope
|
||||
|
||||
async def send_yield(self, data: Any) -> None:
|
||||
await self.chan.send({'yield': data, 'cid': self.cid})
|
||||
|
||||
async def send_stop(self) -> None:
|
||||
await self.chan.send({'stop': True, 'cid': self.cid})
|
||||
|
||||
|
||||
def current_context():
|
||||
"""Get the current task's context instance.
|
||||
"""
|
||||
return _context.get()
|
||||
|
||||
|
||||
def stream(func):
|
||||
"""Mark an async function as a streaming routine.
|
||||
"""
|
||||
func._tractor_stream_function = True
|
||||
sig = inspect.signature(func)
|
||||
if 'ctx' not in sig.parameters:
|
||||
raise TypeError(
|
||||
"The first argument to the stream function "
|
||||
f"{func.__name__} must be `ctx: tractor.Context`"
|
||||
)
|
||||
return func
|
|
@ -12,7 +12,7 @@ import wrapt
|
|||
|
||||
from .log import get_logger
|
||||
from . import current_actor
|
||||
from ._ipc import Context
|
||||
from ._streaming import Context
|
||||
|
||||
__all__ = ['pub']
|
||||
|
||||
|
@ -97,29 +97,32 @@ def pub(
|
|||
):
|
||||
"""Publisher async generator decorator.
|
||||
|
||||
A publisher can be called multiple times from different actors
|
||||
but will only spawn a finite set of internal tasks to stream values
|
||||
to each caller. The ``tasks` argument to the decorator (``Set[str]``)
|
||||
specifies the names of the mutex set of publisher tasks.
|
||||
When the publisher function is called, an argument ``task_name`` must be
|
||||
passed to specify which task (of the set named in ``tasks``) should be
|
||||
used. This allows for using the same publisher with different input
|
||||
(arguments) without allowing more concurrent tasks then necessary.
|
||||
A publisher can be called multiple times from different actors but
|
||||
will only spawn a finite set of internal tasks to stream values to
|
||||
each caller. The ``tasks: Set[str]`` argument to the decorator
|
||||
specifies the names of the mutex set of publisher tasks. When the
|
||||
publisher function is called, an argument ``task_name`` must be
|
||||
passed to specify which task (of the set named in ``tasks``) should
|
||||
be used. This allows for using the same publisher with different
|
||||
input (arguments) without allowing more concurrent tasks then
|
||||
necessary.
|
||||
|
||||
Values yielded from the decorated async generator
|
||||
must be ``Dict[str, Dict[str, Any]]`` where the fist level key is the
|
||||
topic string an determines which subscription the packet will be delivered
|
||||
to and the value is a packet ``Dict[str, Any]`` by default of the form:
|
||||
Values yielded from the decorated async generator must be
|
||||
``Dict[str, Dict[str, Any]]`` where the fist level key is the topic
|
||||
string and determines which subscription the packet will be
|
||||
delivered to and the value is a packet ``Dict[str, Any]`` by default
|
||||
of the form:
|
||||
|
||||
.. ::python
|
||||
|
||||
{topic: value}
|
||||
{topic: str: value: Any}
|
||||
|
||||
The caller can instead opt to pass a ``packetizer`` callback who's return
|
||||
value will be delivered as the published response.
|
||||
The caller can instead opt to pass a ``packetizer`` callback who's
|
||||
return value will be delivered as the published response.
|
||||
|
||||
The decorated function must *accept* an argument :func:`get_topics` which
|
||||
dynamically returns the tuple of current subscriber topics:
|
||||
The decorated async generator function must accept an argument
|
||||
:func:`get_topics` which dynamically returns the tuple of current
|
||||
subscriber topics:
|
||||
|
||||
.. code:: python
|
||||
|
||||
|
@ -162,15 +165,15 @@ def pub(
|
|||
print(f"Subscriber received {value}")
|
||||
|
||||
|
||||
Here, you don't need to provide the ``ctx`` argument since the remote actor
|
||||
provides it automatically to the spawned task. If you were to call
|
||||
``pub_service()`` directly from a wrapping function you would need to
|
||||
provide this explicitly.
|
||||
Here, you don't need to provide the ``ctx`` argument since the
|
||||
remote actor provides it automatically to the spawned task. If you
|
||||
were to call ``pub_service()`` directly from a wrapping function you
|
||||
would need to provide this explicitly.
|
||||
|
||||
Remember you only need this if you need *a finite set of tasks* running in
|
||||
a single actor to stream data to an arbitrary number of subscribers. If you
|
||||
are ok to have a new task running for every call to ``pub_service()`` then
|
||||
probably don't need this.
|
||||
Remember you only need this if you need *a finite set of tasks*
|
||||
running in a single actor to stream data to an arbitrary number of
|
||||
subscribers. If you are ok to have a new task running for every call
|
||||
to ``pub_service()`` then probably don't need this.
|
||||
"""
|
||||
# handle the decorator not called with () case
|
||||
if wrapped is None:
|
||||
|
@ -181,10 +184,7 @@ def pub(
|
|||
for name in tasks:
|
||||
task2lock[name] = trio.StrictFIFOLock()
|
||||
|
||||
async def takes_ctx(get_topics, ctx=None):
|
||||
pass
|
||||
|
||||
@wrapt.decorator(adapter=takes_ctx)
|
||||
@wrapt.decorator
|
||||
async def wrapper(agen, instance, args, kwargs):
|
||||
# this is used to extract arguments properly as per
|
||||
# the `wrapt` docs
|
||||
|
@ -249,7 +249,6 @@ def pub(
|
|||
# invoke it
|
||||
await _execute(*args, **kwargs)
|
||||
|
||||
|
||||
funcname = wrapped.__name__
|
||||
if not inspect.isasyncgenfunction(wrapped):
|
||||
raise TypeError(
|
||||
|
@ -261,4 +260,8 @@ def pub(
|
|||
"`get_topics` argument"
|
||||
)
|
||||
|
||||
# XXX: manually monkey the wrapped function since
|
||||
# ``wrapt.decorator`` doesn't seem to want to play nice with its
|
||||
# whole "adapter" thing...
|
||||
wrapped._tractor_stream_function = True # type: ignore
|
||||
return wrapper(wrapped)
|
||||
|
|
Loading…
Reference in New Issue