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 itself
asyncgen_closing_fix
Tyler Goodlet 2018-07-04 12:51:04 -04:00
parent bf08310224
commit 56d3f6cffb
1 changed files with 293 additions and 191 deletions

View File

@ -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)