Add a working arbiter registry system
Every actor now registers (and unregisters) with the arbiter at startup/teardown. For now the registry is stored in a plain `dict` in the arbiter's memory. This makes it possible to easily coordinate actors started as plain Python processes or via `multiprocessing`. A whole smörgåsbord of changes was required to accomplish this: - factor handshake steps into a func - track *every* channel connected to an actor including multiples to the same remote peer (may want to optimize this later) - handle `trio.ClosedStreamError` gracefully in the message loop - add an `open_portal` asynccontextmanager which handles channel creation, handshaking, and spawning a bg task for msg processing - add a `start_actor()` for starting in-process actors directly - add working `get_arbiter()` and `find_actor()` public routines - `_main` now tries an anonymous channel connect to the stated arbiter sockaddr and uses that to determine whether to crown itselfasyncgen_closing_fix
parent
bf08310224
commit
56d3f6cffb
372
piker/tractor.py
372
piker/tractor.py
|
@ -21,6 +21,8 @@ log = get_logger('tractor')
|
||||||
|
|
||||||
# set at startup and after forks
|
# set at startup and after forks
|
||||||
_current_actor = None
|
_current_actor = None
|
||||||
|
_default_arbiter_host = '127.0.0.1'
|
||||||
|
_default_arbiter_port = 1616
|
||||||
|
|
||||||
|
|
||||||
class ActorFailure(Exception):
|
class ActorFailure(Exception):
|
||||||
|
@ -103,12 +105,25 @@ async def result_from_q(q):
|
||||||
raise ValueError(f"{first_msg} is an invalid response packet?")
|
raise ValueError(f"{first_msg} is an invalid response packet?")
|
||||||
|
|
||||||
|
|
||||||
|
async def _do_handshake(actor, chan):
|
||||||
|
await chan.send(actor.uid)
|
||||||
|
uid = 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 Actor:
|
class Actor:
|
||||||
"""The fundamental concurrency primitive.
|
"""The fundamental concurrency primitive.
|
||||||
|
|
||||||
An actor is the combination of a ``multiprocessing.Process``
|
An *actor* is the combination of a regular Python or
|
||||||
executing a ``trio`` task tree, communicating with other actors
|
``multiprocessing.Process`` executing a ``trio`` task tree, communicating
|
||||||
through "portals" which provide a native async API around "channels".
|
with other actors through "portals" which provide a native async API
|
||||||
|
around "channels".
|
||||||
"""
|
"""
|
||||||
is_arbiter = False
|
is_arbiter = False
|
||||||
|
|
||||||
|
@ -122,6 +137,7 @@ class Actor:
|
||||||
allow_rpc: bool = True,
|
allow_rpc: bool = True,
|
||||||
outlive_main: bool = False,
|
outlive_main: bool = False,
|
||||||
):
|
):
|
||||||
|
self.name = name
|
||||||
self.uid = (name, uid or str(uuid.uuid1()))
|
self.uid = (name, uid or str(uuid.uuid1()))
|
||||||
self.rpc_module_paths = rpc_module_paths
|
self.rpc_module_paths = rpc_module_paths
|
||||||
self._mods = {}
|
self._mods = {}
|
||||||
|
@ -133,7 +149,7 @@ class Actor:
|
||||||
self._outlive_main = outlive_main
|
self._outlive_main = outlive_main
|
||||||
|
|
||||||
# filled in by `_async_main` after fork
|
# filled in by `_async_main` after fork
|
||||||
self._peers = {}
|
self._peers = defaultdict(list)
|
||||||
self._no_more_peers = trio.Event()
|
self._no_more_peers = trio.Event()
|
||||||
self._no_more_peers.set()
|
self._no_more_peers.set()
|
||||||
self._actors2calls = {} # map {uids -> {callids -> waiter queues}}
|
self._actors2calls = {} # map {uids -> {callids -> waiter queues}}
|
||||||
|
@ -148,7 +164,8 @@ class Actor:
|
||||||
log.debug(f"Waiting for peer {uid} to connect")
|
log.debug(f"Waiting for peer {uid} to connect")
|
||||||
event = self._peers.setdefault(uid, trio.Event())
|
event = self._peers.setdefault(uid, trio.Event())
|
||||||
await event.wait()
|
await event.wait()
|
||||||
return event, self._peers[uid]
|
log.debug(f"{uid} successfully connected back to us")
|
||||||
|
return event, self._peers[uid][-1]
|
||||||
|
|
||||||
def load_namespaces(self):
|
def load_namespaces(self):
|
||||||
# We load namespaces after fork since this actor may
|
# We load namespaces after fork since this actor may
|
||||||
|
@ -168,26 +185,33 @@ class Actor:
|
||||||
self._no_more_peers.clear()
|
self._no_more_peers.clear()
|
||||||
chan = Channel(stream=stream)
|
chan = Channel(stream=stream)
|
||||||
log.info(f"New connection to us {chan}")
|
log.info(f"New connection to us {chan}")
|
||||||
|
|
||||||
# send/receive initial handshake response
|
# send/receive initial handshake response
|
||||||
await chan.send(self.uid)
|
try:
|
||||||
uid = await chan.recv()
|
uid = await _do_handshake(self, chan)
|
||||||
chan.uid = uid
|
except StopAsyncIteration:
|
||||||
log.info(f"Handshake with actor {uid}@{chan.raddr} complete")
|
log.warn(f"Channel {chan} failed to handshake")
|
||||||
|
return
|
||||||
|
|
||||||
# channel tracking
|
# channel tracking
|
||||||
event = self._peers.pop(uid, None)
|
event_or_chans = self._peers.pop(uid, None)
|
||||||
chan.event = event
|
if isinstance(event_or_chans, trio.Event):
|
||||||
self._peers[uid] = chan
|
|
||||||
log.debug(f"Registered {chan} for {uid}")
|
|
||||||
|
|
||||||
# Instructing connection: this is likely a new channel to
|
# Instructing connection: this is likely a new channel to
|
||||||
# a recently spawned actor which we'd like to control via
|
# a recently spawned actor which we'd like to control via
|
||||||
# async-rpc calls.
|
# async-rpc calls.
|
||||||
if event and getattr(event, 'set', None):
|
log.debug(f"Waking channel waiters {event_or_chans.statistics()}")
|
||||||
log.debug(f"Waking channel waiters {event.statistics()}")
|
|
||||||
# Alert any task waiting on this connection to come up
|
# Alert any task waiting on this connection to come up
|
||||||
event.set()
|
event_or_chans.set()
|
||||||
event.clear() # now consumer can wait on this channel to close
|
event_or_chans.clear() # consumer can wait on channel to close
|
||||||
|
elif isinstance(event_or_chans, list):
|
||||||
|
log.warn(
|
||||||
|
f"already have channel(s) for {uid}:{event_or_chans}?"
|
||||||
|
)
|
||||||
|
# append new channel
|
||||||
|
self._peers[uid].extend(event_or_chans)
|
||||||
|
|
||||||
|
log.debug(f"Registered {chan} for {uid}")
|
||||||
|
self._peers[uid].append(chan)
|
||||||
|
|
||||||
# Begin channel management - respond to remote requests and
|
# Begin channel management - respond to remote requests and
|
||||||
# process received reponses.
|
# process received reponses.
|
||||||
|
@ -197,23 +221,24 @@ class Actor:
|
||||||
# Drop ref to channel so it can be gc-ed and disconnected
|
# Drop ref to channel so it can be gc-ed and disconnected
|
||||||
if chan is not self._parent_chan:
|
if chan is not self._parent_chan:
|
||||||
log.debug(f"Releasing channel {chan}")
|
log.debug(f"Releasing channel {chan}")
|
||||||
|
chans = self._peers.get(chan.uid)
|
||||||
|
chans.remove(chan)
|
||||||
|
if not chans:
|
||||||
|
log.debug(f"No more channels for {chan.uid}")
|
||||||
self._peers.pop(chan.uid, None)
|
self._peers.pop(chan.uid, None)
|
||||||
chan.event.set() # signal teardown/disconnection
|
|
||||||
if not self._peers: # no more channels connected
|
if not self._peers: # no more channels connected
|
||||||
self._no_more_peers.set()
|
self._no_more_peers.set()
|
||||||
log.debug(f"No more peer channels")
|
log.debug(f"No more peer channels")
|
||||||
|
|
||||||
def _push_result(self, actorid, cid, msg):
|
def _push_result(self, actorid, cid, msg):
|
||||||
|
assert actorid, f"`actorid` can't be {actorid}"
|
||||||
q = self.get_waitq(actorid, cid)
|
q = self.get_waitq(actorid, cid)
|
||||||
log.debug(f"Delivering {msg} from {actorid} to caller {cid}")
|
log.debug(f"Delivering {msg} from {actorid} to caller {cid}")
|
||||||
q.put_nowait(msg)
|
q.put_nowait(msg)
|
||||||
|
|
||||||
def get_waitq(self, actorid, cid):
|
def get_waitq(self, actorid, cid):
|
||||||
if actorid not in self._actors2calls:
|
log.debug(f"Registering for callid {cid} queue results from {actorid}")
|
||||||
log.debug(f"Registering for results from {actorid}")
|
|
||||||
cids2qs = self._actors2calls.setdefault(actorid, {})
|
cids2qs = self._actors2calls.setdefault(actorid, {})
|
||||||
if cid not in cids2qs:
|
|
||||||
log.debug(f"Registering for result from call id {cid}")
|
|
||||||
return cids2qs.setdefault(cid, trio.Queue(1000))
|
return cids2qs.setdefault(cid, trio.Queue(1000))
|
||||||
|
|
||||||
async def send_cmd(self, chan, ns, func, kwargs):
|
async def send_cmd(self, chan, ns, func, kwargs):
|
||||||
|
@ -232,8 +257,11 @@ class Actor:
|
||||||
|
|
||||||
Process rpc requests and deliver retrieved responses from channels.
|
Process rpc requests and deliver retrieved responses from channels.
|
||||||
"""
|
"""
|
||||||
log.debug(f"Entering async-rpc loop for {chan}")
|
# TODO: once https://github.com/python-trio/trio/issues/467 gets
|
||||||
|
# worked out we'll likely want to use that!
|
||||||
|
log.debug(f"Entering msg loop for {chan}")
|
||||||
async with trio.open_nursery() as nursery:
|
async with trio.open_nursery() as nursery:
|
||||||
|
try:
|
||||||
async for msg in chan.aiter_recv():
|
async for msg in chan.aiter_recv():
|
||||||
if msg is None: # terminate sentinel
|
if msg is None: # terminate sentinel
|
||||||
log.debug(f"Terminating msg loop for {chan}")
|
log.debug(f"Terminating msg loop for {chan}")
|
||||||
|
@ -245,6 +273,7 @@ class Actor:
|
||||||
if 'error' in msg:
|
if 'error' in msg:
|
||||||
# TODO: need something better then this slop
|
# TODO: need something better then this slop
|
||||||
raise RemoteActorError(msg['error'])
|
raise RemoteActorError(msg['error'])
|
||||||
|
log.debug(f"Waiting on next msg for {chan}")
|
||||||
continue
|
continue
|
||||||
else:
|
else:
|
||||||
ns, funcname, kwargs, actorid, cid = msg['cmd']
|
ns, funcname, kwargs, actorid, cid = msg['cmd']
|
||||||
|
@ -252,7 +281,6 @@ class Actor:
|
||||||
log.debug(
|
log.debug(
|
||||||
f"Processing request from {actorid}\n"
|
f"Processing request from {actorid}\n"
|
||||||
f"{ns}.{funcname}({kwargs})")
|
f"{ns}.{funcname}({kwargs})")
|
||||||
# TODO: accept a sentinel which cancels this task tree?
|
|
||||||
if ns == 'self':
|
if ns == 'self':
|
||||||
func = getattr(self, funcname)
|
func = getattr(self, funcname)
|
||||||
else:
|
else:
|
||||||
|
@ -273,17 +301,22 @@ class Actor:
|
||||||
# signature will be treated as one.
|
# signature will be treated as one.
|
||||||
treat_as_gen = True
|
treat_as_gen = True
|
||||||
|
|
||||||
|
log.debug(f"Spawning task for {func}")
|
||||||
nursery.start_soon(
|
nursery.start_soon(
|
||||||
_invoke, cid, chan, func, kwargs, treat_as_gen,
|
_invoke, cid, chan, func, kwargs, treat_as_gen,
|
||||||
name=funcname
|
name=funcname
|
||||||
)
|
)
|
||||||
|
log.debug(f"Waiting on next msg for {chan}")
|
||||||
else: # channel disconnect
|
else: # channel disconnect
|
||||||
log.warn(f"Cancelling all tasks for {chan}")
|
log.debug(f"{chan} disconnected")
|
||||||
nursery.cancel_scope.cancel()
|
except trio.ClosedStreamError:
|
||||||
|
log.error(f"{chan} broke")
|
||||||
|
|
||||||
|
log.debug(f"Cancelling all tasks for {chan}")
|
||||||
|
nursery.cancel_scope.cancel()
|
||||||
log.debug(f"Exiting msg loop for {chan}")
|
log.debug(f"Exiting msg loop for {chan}")
|
||||||
|
|
||||||
def _fork_main(self, accept_addr, parent_addr=None, loglevel='debug'):
|
def _fork_main(self, accept_addr, parent_addr=None, loglevel=None):
|
||||||
# after fork routine which invokes a fresh ``trio.run``
|
# after fork routine which invokes a fresh ``trio.run``
|
||||||
log.info(
|
log.info(
|
||||||
f"Started new {ctx.current_process()} for actor {self.uid}")
|
f"Started new {ctx.current_process()} for actor {self.uid}")
|
||||||
|
@ -299,7 +332,13 @@ class Actor:
|
||||||
pass # handle it the same way trio does?
|
pass # handle it the same way trio does?
|
||||||
log.debug(f"Actor {self.uid} terminated")
|
log.debug(f"Actor {self.uid} terminated")
|
||||||
|
|
||||||
async def _async_main(self, accept_addr, parent_addr=None, nursery=None):
|
async def _async_main(
|
||||||
|
self,
|
||||||
|
accept_addr,
|
||||||
|
arbiter_addr=(_default_arbiter_host, _default_arbiter_port),
|
||||||
|
parent_addr=None,
|
||||||
|
nursery=None
|
||||||
|
):
|
||||||
"""Start the channel server and main task.
|
"""Start the channel server and main task.
|
||||||
|
|
||||||
A "root-most" (or "top-level") nursery for this actor is opened here
|
A "root-most" (or "top-level") nursery for this actor is opened here
|
||||||
|
@ -325,20 +364,23 @@ class Actor:
|
||||||
on_reconnect=self.main
|
on_reconnect=self.main
|
||||||
)
|
)
|
||||||
await chan.connect()
|
await chan.connect()
|
||||||
|
|
||||||
# initial handshake, report who we are, who they are
|
# initial handshake, report who we are, who they are
|
||||||
await chan.send(self.uid)
|
await _do_handshake(self, chan)
|
||||||
uid = await chan.recv()
|
|
||||||
if uid in self._peers:
|
|
||||||
log.warn(
|
|
||||||
f"already have channel for {uid} registered?"
|
|
||||||
)
|
|
||||||
|
|
||||||
# handle new connection back to parent optionally
|
# handle new connection back to parent optionally
|
||||||
# begin responding to RPC
|
# begin responding to RPC
|
||||||
if self._allow_rpc:
|
if self._allow_rpc:
|
||||||
self.load_namespaces()
|
self.load_namespaces()
|
||||||
nursery.start_soon(self._process_messages, chan)
|
if self._parent_chan:
|
||||||
|
nursery.start_soon(
|
||||||
|
self._process_messages, self._parent_chan)
|
||||||
|
|
||||||
|
# register with the arbiter if we're told its addr
|
||||||
|
log.debug(f"Registering {self} for role `{self.name}`")
|
||||||
|
async with get_arbiter(*arbiter_addr) as arb_portal:
|
||||||
|
await arb_portal.run(
|
||||||
|
'self', 'register_actor',
|
||||||
|
name=self.name, sockaddr=self.accept_addr)
|
||||||
|
|
||||||
if self.main:
|
if self.main:
|
||||||
if self._parent_chan:
|
if self._parent_chan:
|
||||||
|
@ -374,6 +416,16 @@ class Actor:
|
||||||
{'error': traceback.format_exc(), 'cid': 'main'})
|
{'error': traceback.format_exc(), 'cid': 'main'})
|
||||||
else:
|
else:
|
||||||
raise
|
raise
|
||||||
|
finally:
|
||||||
|
# UNregister actor from the arbiter
|
||||||
|
try:
|
||||||
|
if arbiter_addr is not None:
|
||||||
|
async with get_arbiter(*arbiter_addr) as arb_portal:
|
||||||
|
await arb_portal.run(
|
||||||
|
'self', 'register_actor',
|
||||||
|
name=self.name, sockaddr=self.accept_addr)
|
||||||
|
except OSError:
|
||||||
|
log.warn(f"Unable to unregister {self.name} from arbiter")
|
||||||
|
|
||||||
return result
|
return result
|
||||||
|
|
||||||
|
@ -389,7 +441,6 @@ class Actor:
|
||||||
listening for new messages.
|
listening for new messages.
|
||||||
|
|
||||||
"""
|
"""
|
||||||
log.debug(f"Starting tcp server on {accept_host}:{accept_port}")
|
|
||||||
async with trio.open_nursery() as nursery:
|
async with trio.open_nursery() as nursery:
|
||||||
self._server_nursery = nursery
|
self._server_nursery = nursery
|
||||||
# TODO: might want to consider having a separate nursery
|
# TODO: might want to consider having a separate nursery
|
||||||
|
@ -403,8 +454,9 @@ class Actor:
|
||||||
port=accept_port, host=accept_host,
|
port=accept_port, host=accept_host,
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
log.debug(
|
||||||
|
f"Started tcp server(s) on {[l.socket for l in listeners]}")
|
||||||
self._listeners.extend(listeners)
|
self._listeners.extend(listeners)
|
||||||
log.debug(f"Spawned {listeners}")
|
|
||||||
task_status.started()
|
task_status.started()
|
||||||
|
|
||||||
def cancel(self):
|
def cancel(self):
|
||||||
|
@ -431,8 +483,8 @@ class Actor:
|
||||||
def get_parent(self):
|
def get_parent(self):
|
||||||
return Portal(self._parent_chan)
|
return Portal(self._parent_chan)
|
||||||
|
|
||||||
def get_chan(self, actorid):
|
def get_chans(self, actorid):
|
||||||
return self._peers.get(actorid)
|
return self._peers[actorid]
|
||||||
|
|
||||||
|
|
||||||
class Arbiter(Actor):
|
class Arbiter(Actor):
|
||||||
|
@ -453,6 +505,14 @@ class Arbiter(Actor):
|
||||||
def register_actor(self, name, sockaddr):
|
def register_actor(self, name, sockaddr):
|
||||||
self._registry[name].append(sockaddr)
|
self._registry[name].append(sockaddr)
|
||||||
|
|
||||||
|
def unregister_actor(self, name, sockaddr):
|
||||||
|
sockaddrs = self._registry.get(name)
|
||||||
|
if sockaddrs:
|
||||||
|
try:
|
||||||
|
sockaddrs.remove(sockaddr)
|
||||||
|
except ValueError:
|
||||||
|
pass
|
||||||
|
|
||||||
|
|
||||||
class Portal:
|
class Portal:
|
||||||
"""A 'portal' to a(n) (remote) ``Actor``.
|
"""A 'portal' to a(n) (remote) ``Actor``.
|
||||||
|
@ -463,28 +523,14 @@ class Portal:
|
||||||
|
|
||||||
Think of this like an native async IPC API.
|
Think of this like an native async IPC API.
|
||||||
"""
|
"""
|
||||||
def __init__(self, channel, event=None):
|
def __init__(self, channel):
|
||||||
self.channel = channel
|
self.channel = channel
|
||||||
self._uid = channel.uid
|
|
||||||
self._event = event
|
|
||||||
|
|
||||||
async def __aenter__(self):
|
|
||||||
await self.channel.connect()
|
|
||||||
# do the handshake
|
|
||||||
await self.channel.send(_current_actor.uid)
|
|
||||||
self._uid = uid = await self.channel.recv()
|
|
||||||
_current_actor._peers[uid] = self.channel
|
|
||||||
return self
|
|
||||||
|
|
||||||
async def aclose(self):
|
async def aclose(self):
|
||||||
|
log.debug(f"Closing {self}")
|
||||||
|
# XXX: won't work until https://github.com/python-trio/trio/pull/460
|
||||||
|
# gets in!
|
||||||
await self.channel.aclose()
|
await self.channel.aclose()
|
||||||
if self._event:
|
|
||||||
# alert the _stream_handler task that we are done with the channel
|
|
||||||
# so it can terminate / move on
|
|
||||||
self._event.set()
|
|
||||||
|
|
||||||
async def __aexit__(self, etype, value, tb):
|
|
||||||
await self.aclose()
|
|
||||||
|
|
||||||
async def run(self, ns, func, **kwargs):
|
async def run(self, ns, func, **kwargs):
|
||||||
"""Submit a function to be scheduled and run by actor, return its
|
"""Submit a function to be scheduled and run by actor, return its
|
||||||
|
@ -512,7 +558,7 @@ class Portal:
|
||||||
except KeyError:
|
except KeyError:
|
||||||
raise RemoteActorError(msg['error'])
|
raise RemoteActorError(msg['error'])
|
||||||
except GeneratorExit:
|
except GeneratorExit:
|
||||||
log.warn(f"Cancelling async gen call {cid} to {chan.uid}")
|
log.debug(f"Cancelling async gen call {cid} to {chan.uid}")
|
||||||
|
|
||||||
return yield_from_q()
|
return yield_from_q()
|
||||||
|
|
||||||
|
@ -522,6 +568,39 @@ class Portal:
|
||||||
raise ValueError(f"Unknown msg response type: {first_msg}")
|
raise ValueError(f"Unknown msg response type: {first_msg}")
|
||||||
|
|
||||||
|
|
||||||
|
@asynccontextmanager
|
||||||
|
async def open_portal(channel, nursery=None):
|
||||||
|
"""Open a ``Portal`` through the provided ``channel``.
|
||||||
|
|
||||||
|
Spawns a background task to handle rpc message processing.
|
||||||
|
"""
|
||||||
|
actor = current_actor()
|
||||||
|
assert actor
|
||||||
|
was_connected = False
|
||||||
|
|
||||||
|
async with maybe_open_nursery(nursery) as nursery:
|
||||||
|
|
||||||
|
if not channel.connected():
|
||||||
|
await channel.connect()
|
||||||
|
was_connected = True
|
||||||
|
|
||||||
|
if channel.uid is None:
|
||||||
|
await _do_handshake(actor, channel)
|
||||||
|
|
||||||
|
if not actor.get_chans(channel.uid):
|
||||||
|
# actor is not currently managing this channel
|
||||||
|
actor._peers[channel.uid].append(channel)
|
||||||
|
|
||||||
|
nursery.start_soon(actor._process_messages, channel)
|
||||||
|
yield Portal(channel)
|
||||||
|
|
||||||
|
# cancel background msg loop task
|
||||||
|
nursery.cancel_scope.cancel()
|
||||||
|
if was_connected:
|
||||||
|
actor._peers[channel.uid].remove(channel)
|
||||||
|
await channel.aclose()
|
||||||
|
|
||||||
|
|
||||||
class LocalPortal:
|
class LocalPortal:
|
||||||
"""A 'portal' to a local ``Actor``.
|
"""A 'portal' to a local ``Actor``.
|
||||||
|
|
||||||
|
@ -590,7 +669,7 @@ class ActorNursery:
|
||||||
main_q = self._actor.get_waitq(actor.uid, 'main')
|
main_q = self._actor.get_waitq(actor.uid, 'main')
|
||||||
self._children[(name, proc.pid)] = (actor, proc, main_q)
|
self._children[(name, proc.pid)] = (actor, proc, main_q)
|
||||||
|
|
||||||
return Portal(chan, event=event)
|
return Portal(chan)
|
||||||
|
|
||||||
async def wait(self):
|
async def wait(self):
|
||||||
|
|
||||||
|
@ -623,8 +702,9 @@ class ActorNursery:
|
||||||
else:
|
else:
|
||||||
# send cancel cmd - likely no response from subactor
|
# send cancel cmd - likely no response from subactor
|
||||||
actor = self._actor
|
actor = self._actor
|
||||||
chan = actor.get_chan(subactor.uid)
|
chans = actor.get_chans(subactor.uid)
|
||||||
if chan:
|
if chans:
|
||||||
|
for chan in chans:
|
||||||
await actor.send_cmd(chan, 'self', 'cancel', {})
|
await actor.send_cmd(chan, 'self', 'cancel', {})
|
||||||
else:
|
else:
|
||||||
log.warn(
|
log.warn(
|
||||||
|
@ -642,8 +722,10 @@ class ActorNursery:
|
||||||
log.info(f"{actor.uid} main task completed with {msg}")
|
log.info(f"{actor.uid} main task completed with {msg}")
|
||||||
if not actor._outlive_main:
|
if not actor._outlive_main:
|
||||||
# trigger msg loop to break
|
# trigger msg loop to break
|
||||||
|
chans = self._actor.get_chans(actor.uid)
|
||||||
|
for chan in chans:
|
||||||
log.info(f"Signalling msg loop exit for {actor.uid}")
|
log.info(f"Signalling msg loop exit for {actor.uid}")
|
||||||
await self._actor.get_chan(actor.uid).send(None)
|
await chan.send(None)
|
||||||
|
|
||||||
if etype is not None:
|
if etype is not None:
|
||||||
log.warn(f"{current_actor().uid} errored with {etype}, "
|
log.warn(f"{current_actor().uid} errored with {etype}, "
|
||||||
|
@ -655,6 +737,7 @@ class ActorNursery:
|
||||||
for subactor, proc, main_q in self._children.values():
|
for subactor, proc, main_q in self._children.values():
|
||||||
nursery.start_soon(wait_for_actor, subactor, proc, main_q)
|
nursery.start_soon(wait_for_actor, subactor, proc, main_q)
|
||||||
|
|
||||||
|
await self.wait()
|
||||||
log.debug(f"Nursery teardown complete")
|
log.debug(f"Nursery teardown complete")
|
||||||
|
|
||||||
|
|
||||||
|
@ -677,117 +760,136 @@ async def open_nursery(supervisor=None, loglevel='WARNING'):
|
||||||
yield nursery
|
yield nursery
|
||||||
|
|
||||||
|
|
||||||
async def serve_local_actor(actor, nursery=None, accept_addr=(None, 0)):
|
class NoArbiterFound(Exception):
|
||||||
|
"Couldn't find the arbiter?"
|
||||||
|
|
||||||
|
|
||||||
|
async def start_actor(actor, host, port, arbiter_addr, nursery=None):
|
||||||
"""Spawn a local actor by starting a task to execute it's main
|
"""Spawn a local actor by starting a task to execute it's main
|
||||||
async function.
|
async function.
|
||||||
|
|
||||||
Blocks if no nursery is provided, in which case it is expected the nursery
|
Blocks if no nursery is provided, in which case it is expected the nursery
|
||||||
provider is responsible for waiting on the task to complete.
|
provider is responsible for waiting on the task to complete.
|
||||||
"""
|
"""
|
||||||
|
# assign process-local actor
|
||||||
|
global _current_actor
|
||||||
|
_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}")
|
||||||
|
|
||||||
await actor._async_main(
|
await actor._async_main(
|
||||||
accept_addr=accept_addr,
|
accept_addr=(host, port),
|
||||||
parent_addr=None,
|
parent_addr=None,
|
||||||
|
arbiter_addr=arbiter_addr,
|
||||||
nursery=nursery,
|
nursery=nursery,
|
||||||
)
|
)
|
||||||
return actor
|
# XXX: If spawned locally, the actor is cancelled when this
|
||||||
|
# context is complete given that there are no more active
|
||||||
|
# peer channels connected to it.
|
||||||
|
if not actor._outlive_main:
|
||||||
|
actor.cancel_server()
|
||||||
|
|
||||||
|
# unset module state
|
||||||
class NoArbiterFound:
|
_current_actor = None
|
||||||
"Couldn't find the arbiter?"
|
log.info("Completed async main")
|
||||||
|
|
||||||
|
|
||||||
@asynccontextmanager
|
@asynccontextmanager
|
||||||
async def get_arbiter(host='127.0.0.1', port=1617, main=None, **kwargs):
|
async def _connect_chan(host, port):
|
||||||
|
"""Attempt to connect to an arbiter's channel server.
|
||||||
|
Return the channel on success or None on failure.
|
||||||
|
"""
|
||||||
|
chan = Channel((host, port))
|
||||||
|
await chan.connect()
|
||||||
|
yield chan
|
||||||
|
await chan.aclose()
|
||||||
|
|
||||||
|
|
||||||
|
@asynccontextmanager
|
||||||
|
async def get_arbiter(host, port):
|
||||||
|
"""Return a portal instance connected to a local or remote
|
||||||
|
arbiter.
|
||||||
|
"""
|
||||||
actor = current_actor()
|
actor = current_actor()
|
||||||
if actor and not actor.is_arbiter:
|
if not actor:
|
||||||
try:
|
raise RuntimeError("No actor instance has been defined yet?")
|
||||||
# If an arbiter is already running on this host connect to it
|
|
||||||
async with Portal(Channel((host, port))) as portal:
|
if actor.is_arbiter:
|
||||||
yield portal
|
|
||||||
except OSError as err:
|
|
||||||
raise NoArbiterFound(err)
|
|
||||||
else:
|
|
||||||
if actor and actor.is_arbiter:
|
|
||||||
# we're already the arbiter
|
# we're already the arbiter
|
||||||
# (likely a re-entrant call from the arbiter actor)
|
# (likely a re-entrant call from the arbiter actor)
|
||||||
yield LocalPortal(actor)
|
yield LocalPortal(actor)
|
||||||
else:
|
else:
|
||||||
arbiter = Arbiter('arbiter', main=main, **kwargs)
|
async with _connect_chan(host, port) as chan:
|
||||||
# assign process-local actor
|
async with open_portal(chan) as arb_portal:
|
||||||
global _current_actor
|
yield arb_portal
|
||||||
_current_actor = arbiter
|
|
||||||
|
|
||||||
# start the arbiter in process in a new task
|
|
||||||
async with trio.open_nursery() as nursery:
|
|
||||||
|
|
||||||
# start local channel-server and fake the portal API
|
|
||||||
# NOTE: this won't block since we provide the nursery
|
|
||||||
await serve_local_actor(
|
|
||||||
arbiter, nursery=nursery, accept_addr=(host, port))
|
|
||||||
|
|
||||||
yield LocalPortal(arbiter)
|
|
||||||
|
|
||||||
# XXX: If spawned locally, the arbiter is cancelled when this
|
|
||||||
# context is complete given that there are no more active
|
|
||||||
# peer channels connected to it.
|
|
||||||
if not arbiter._outlive_main:
|
|
||||||
arbiter.cancel_server()
|
|
||||||
|
|
||||||
|
|
||||||
@asynccontextmanager
|
@asynccontextmanager
|
||||||
async def find_actor(name):
|
async def find_actor(
|
||||||
|
name,
|
||||||
|
arbiter_sockaddr=(_default_arbiter_host, _default_arbiter_port)
|
||||||
|
):
|
||||||
"""Ask the arbiter to find actor(s) by name.
|
"""Ask the arbiter to find actor(s) by name.
|
||||||
|
|
||||||
Returns a sequence of unconnected portals for each matching actor
|
Returns a sequence of unconnected portals for each matching actor
|
||||||
known to the arbiter (client code is expected to connect the portals).
|
known to the arbiter (client code is expected to connect the portals).
|
||||||
"""
|
"""
|
||||||
async with get_arbiter() as portal:
|
actor = current_actor()
|
||||||
sockaddrs = await portal.run('self', 'find_actor', name=name)
|
if not actor:
|
||||||
portals = []
|
raise RuntimeError("No actor instance has been defined yet?")
|
||||||
|
|
||||||
|
async with get_arbiter(*arbiter_sockaddr) as arb_portal:
|
||||||
|
sockaddrs = await arb_portal.run('self', 'find_actor', name=name)
|
||||||
|
# TODO: return portals to all available actors - for now just
|
||||||
|
# the first one we find
|
||||||
if sockaddrs:
|
if sockaddrs:
|
||||||
for sockaddr in sockaddrs:
|
sockaddr = sockaddrs[-1]
|
||||||
portals.append(Portal(Channel(sockaddr)))
|
async with _connect_chan(*sockaddr) as chan:
|
||||||
|
async with open_portal(chan) as portal:
|
||||||
yield portals # XXX: these are "unconnected" portals
|
yield portal
|
||||||
|
else:
|
||||||
|
yield
|
||||||
|
|
||||||
|
|
||||||
async def _main(async_fn, args, kwargs, name):
|
async def _main(async_fn, args, kwargs, name, arbiter_addr):
|
||||||
|
"""Async entry point for ``tractor``.
|
||||||
|
"""
|
||||||
main = partial(async_fn, *args) if async_fn else None
|
main = partial(async_fn, *args) if async_fn else None
|
||||||
# Creates an internal nursery which shouldn't be cancelled even if
|
arbiter_addr = (host, port) = arbiter_addr or (
|
||||||
# the one opened below is (this is desirable because the arbitter should
|
_default_arbiter_host, _default_arbiter_port)
|
||||||
# stay up until a re-election process has taken place - which is not
|
# make a temporary connection to see if an arbiter exists
|
||||||
# implemented yet FYI).
|
arbiter_found = False
|
||||||
async with get_arbiter(
|
try:
|
||||||
host=kwargs.pop('arbiter_host', '127.0.0.1'),
|
async with _connect_chan(host, port):
|
||||||
port=kwargs.pop('arbiter_port', 1617),
|
arbiter_found = True
|
||||||
main=main,
|
except OSError:
|
||||||
**kwargs,
|
log.warn(f"No actor could be found @ {host}:{port}")
|
||||||
):
|
|
||||||
if not current_actor().is_arbiter:
|
if arbiter_found: # we were able to connect to an arbiter
|
||||||
# create a local actor and start it up its main routine
|
log.info(f"Arbiter seems to exist @ {host}:{port}")
|
||||||
|
# create a local actor and start up its main routine/task
|
||||||
actor = Actor(
|
actor = Actor(
|
||||||
name or 'anonymous',
|
name or 'anonymous',
|
||||||
main=main, # main coroutine to be invoked
|
main=main,
|
||||||
**kwargs
|
**kwargs
|
||||||
)
|
)
|
||||||
# this will block and yield control to the `trio` run loop
|
host, port = (_default_arbiter_host, 0)
|
||||||
await serve_local_actor(
|
|
||||||
actor, accept_addr=kwargs.pop('accept_addr', (None, 0)))
|
|
||||||
log.info("Completed async main")
|
|
||||||
# TODO: when the local actor's main has completed we cancel?
|
|
||||||
# actor.cancel()
|
|
||||||
else:
|
else:
|
||||||
# block waiting for the arbiter main task to complete
|
# start this local actor as the arbiter
|
||||||
pass
|
actor = Arbiter(name or 'arbiter', main=main, **kwargs)
|
||||||
|
|
||||||
# unset module state
|
await start_actor(actor, host, port, arbiter_addr=arbiter_addr)
|
||||||
global _current_actor
|
# Creates an internal nursery which shouldn't be cancelled even if
|
||||||
_current_actor = None
|
# the one opened below is (this is desirable because the arbiter should
|
||||||
|
# stay up until a re-election process has taken place - which is not
|
||||||
|
# implemented yet FYI).
|
||||||
|
|
||||||
|
|
||||||
def run(async_fn, *args, arbiter_host=None, name='anonymous', **kwargs):
|
def run(async_fn, *args, name=None, arbiter_addr=None, **kwargs):
|
||||||
"""Run a trio-actor async function in process.
|
"""Run a trio-actor async function in process.
|
||||||
|
|
||||||
This is tractor's main entry and the start point for any async actor.
|
This is tractor's main entry and the start point for any async actor.
|
||||||
"""
|
"""
|
||||||
return trio.run(_main, async_fn, args, kwargs, name)
|
return trio.run(_main, async_fn, args, kwargs, name, arbiter_addr)
|
||||||
|
|
Loading…
Reference in New Issue