forked from goodboy/tractor
1
0
Fork 0

Mk `process_messages()` return last msg; summary logging

Not sure it's **that** useful (yet) but in theory would allow avoiding
certain log level usage around transient RPC requests for discovery methods
(like `.register_actor()` and friends); can't hurt to be able to
introspect that last message for other future cases I'd imagine as well.
Adjust the calling code in `._runtime` to match; other spots are using
the `trio.Nursery.start()` schedule style and are fine as is.

Improve a bunch more log messages throughout a few mods mostly by going
to a "summary" single-emission style where possible/appropriate:
- in `._runtime` more "single summary" status style log emissions:
 |_mk `Actor.load_modules()` render a single mod loaded summary.
 |_use a summary `con_status: str` for `Actor._stream_handler()` conn
   setup and an equiv (`con_teardown_status`) for connection teardowns.
 |_similar thing in `Actor.wait_for_actor()`.
- generally more usage of `.msg.pretty_struct` apis throughout `._runtime`.
runtime_to_msgspec
Tyler Goodlet 2024-04-30 12:15:46 -04:00
parent f139adddca
commit 40c972f0ec
4 changed files with 147 additions and 128 deletions

View File

@ -146,7 +146,7 @@ def _trio_main(
finally: finally:
log.info( log.info(
'Actor terminated\n' 'Subactor terminated\n'
+ +
actor_info actor_info
) )

View File

@ -435,7 +435,6 @@ class Portal:
yield stream yield stream
finally: finally:
# cancel the far end task on consumer close # cancel the far end task on consumer close
# NOTE: this is a special case since we assume that if using # NOTE: this is a special case since we assume that if using
# this ``.open_fream_from()`` api, the stream is one a one # this ``.open_fream_from()`` api, the stream is one a one
@ -496,7 +495,7 @@ class LocalPortal:
async def open_portal( async def open_portal(
channel: Channel, channel: Channel,
nursery: trio.Nursery|None = None, tn: trio.Nursery|None = None,
start_msg_loop: bool = True, start_msg_loop: bool = True,
shield: bool = False, shield: bool = False,
@ -504,15 +503,19 @@ async def open_portal(
''' '''
Open a ``Portal`` through the provided ``channel``. Open a ``Portal`` through the provided ``channel``.
Spawns a background task to handle message processing (normally Spawns a background task to handle RPC processing, normally
done by the actor-runtime implicitly). done by the actor-runtime implicitly via a call to
`._rpc.process_messages()`. just after connection establishment.
''' '''
actor = current_actor() actor = current_actor()
assert actor assert actor
was_connected: bool = False was_connected: bool = False
async with maybe_open_nursery(nursery, shield=shield) as nursery: async with maybe_open_nursery(
tn,
shield=shield,
) as tn:
if not channel.connected(): if not channel.connected():
await channel.connect() await channel.connect()
@ -524,7 +527,7 @@ async def open_portal(
msg_loop_cs: trio.CancelScope|None = None msg_loop_cs: trio.CancelScope|None = None
if start_msg_loop: if start_msg_loop:
from ._runtime import process_messages from ._runtime import process_messages
msg_loop_cs = await nursery.start( msg_loop_cs = await tn.start(
partial( partial(
process_messages, process_messages,
actor, actor,
@ -544,7 +547,7 @@ async def open_portal(
await channel.aclose() await channel.aclose()
# cancel background msg loop task # cancel background msg loop task
if msg_loop_cs: if msg_loop_cs is not None:
msg_loop_cs.cancel() msg_loop_cs.cancel()
nursery.cancel_scope.cancel() tn.cancel_scope.cancel()

View File

@ -64,11 +64,13 @@ from .msg import (
current_codec, current_codec,
MsgCodec, MsgCodec,
NamespacePath, NamespacePath,
pretty_struct,
) )
from tractor.msg.types import ( from tractor.msg.types import (
CancelAck, CancelAck,
Error, Error,
Msg, Msg,
MsgType,
Return, Return,
Start, Start,
StartAck, StartAck,
@ -774,7 +776,10 @@ async def process_messages(
shield: bool = False, shield: bool = False,
task_status: TaskStatus[CancelScope] = trio.TASK_STATUS_IGNORED, task_status: TaskStatus[CancelScope] = trio.TASK_STATUS_IGNORED,
) -> bool: ) -> (
bool, # chan diconnected
MsgType, # last msg
):
''' '''
This is the low-level, per-IPC-channel, RPC task scheduler loop. This is the low-level, per-IPC-channel, RPC task scheduler loop.
@ -816,11 +821,6 @@ async def process_messages(
# |_ for ex, from `aioquic` which exposed "stream ids": # |_ for ex, from `aioquic` which exposed "stream ids":
# - https://github.com/aiortc/aioquic/blob/main/src/aioquic/quic/connection.py#L1175 # - https://github.com/aiortc/aioquic/blob/main/src/aioquic/quic/connection.py#L1175
# - https://github.com/aiortc/aioquic/blob/main/src/aioquic/quic/connection.py#L659 # - https://github.com/aiortc/aioquic/blob/main/src/aioquic/quic/connection.py#L659
log.runtime(
'Entering RPC msg loop:\n'
f'peer: {chan.uid}\n'
f'|_{chan}\n'
)
nursery_cancelled_before_task: bool = False nursery_cancelled_before_task: bool = False
msg: Msg|None = None msg: Msg|None = None
try: try:
@ -834,12 +834,15 @@ async def process_messages(
async for msg in chan: async for msg in chan:
log.transport( # type: ignore log.transport( # type: ignore
f'<= IPC msg from peer: {chan.uid}\n\n' f'IPC msg from peer\n'
f'<= {chan.uid}\n\n'
# TODO: avoid fmting depending on loglevel for perf? # TODO: avoid fmting depending on loglevel for perf?
# -[ ] specifically `pformat()` sub-call..? # -[ ] specifically `pretty_struct.pformat()` sub-call..?
# - how to only log-level-aware actually call this?
# -[ ] use `.msg.pretty_struct` here now instead! # -[ ] use `.msg.pretty_struct` here now instead!
f'{pformat(msg)}\n' # f'{pretty_struct.pformat(msg)}\n'
f'{msg}\n'
) )
match msg: match msg:
@ -953,10 +956,11 @@ async def process_messages(
uid=actorid, uid=actorid,
): ):
log.runtime( log.runtime(
'Handling RPC `Start` request from\n' 'Handling RPC `Start` request\n'
f'peer: {actorid}\n' f'<= peer: {actorid}\n'
'\n' f' |_{ns}.{funcname}({kwargs})\n\n'
f'=> {ns}.{funcname}({kwargs})\n'
f'{pretty_struct.pformat(msg)}\n'
) )
# runtime-internal endpoint: `Actor.<funcname>` # runtime-internal endpoint: `Actor.<funcname>`
@ -1097,25 +1101,24 @@ async def process_messages(
parent_chan=chan, parent_chan=chan,
) )
except ( except TransportClosed:
TransportClosed,
):
# channels "breaking" (for TCP streams by EOF or 104 # channels "breaking" (for TCP streams by EOF or 104
# connection-reset) is ok since we don't have a teardown # connection-reset) is ok since we don't have a teardown
# handshake for them (yet) and instead we simply bail out of # handshake for them (yet) and instead we simply bail out of
# the message loop and expect the teardown sequence to clean # the message loop and expect the teardown sequence to clean
# up.. # up..
# TODO: add a teardown handshake? and, #
# TODO: maybe add a teardown handshake? and,
# -[ ] don't show this msg if it's an ephemeral discovery ep call? # -[ ] don't show this msg if it's an ephemeral discovery ep call?
# -[ ] figure out how this will break with other transports? # -[ ] figure out how this will break with other transports?
log.runtime( log.runtime(
f'channel closed abruptly with\n' f'IPC channel closed abruptly\n'
f'peer: {chan.uid}\n' f'<=x peer: {chan.uid}\n'
f'|_{chan.raddr}\n' f' |_{chan.raddr}\n'
) )
# transport **WAS** disconnected # transport **WAS** disconnected
return True return (True, msg)
except ( except (
Exception, Exception,
@ -1155,9 +1158,9 @@ async def process_messages(
log.runtime( log.runtime(
'Exiting IPC msg loop with final msg\n\n' 'Exiting IPC msg loop with final msg\n\n'
f'<= peer: {chan.uid}\n' f'<= peer: {chan.uid}\n'
f'|_{chan}\n\n' f' |_{chan}\n\n'
f'{pformat(msg)}\n\n' f'{pretty_struct.pformat(msg)}'
) )
# transport **WAS NOT** disconnected # transport **WAS NOT** disconnected
return False return (False, msg)

View File

@ -49,6 +49,7 @@ from pprint import pformat
import signal import signal
import sys import sys
from typing import ( from typing import (
Any,
Callable, Callable,
TYPE_CHECKING, TYPE_CHECKING,
) )
@ -68,7 +69,7 @@ from tractor.msg import (
pretty_struct, pretty_struct,
NamespacePath, NamespacePath,
types as msgtypes, types as msgtypes,
Msg, MsgType,
) )
from ._ipc import Channel from ._ipc import Channel
from ._context import ( from ._context import (
@ -96,19 +97,6 @@ from ._rpc import (
process_messages, process_messages,
try_ship_error_to_remote, try_ship_error_to_remote,
) )
# from tractor.msg.types import (
# Aid,
# SpawnSpec,
# Start,
# StartAck,
# Started,
# Yield,
# Stop,
# Return,
# Error,
# )
if TYPE_CHECKING: if TYPE_CHECKING:
@ -315,29 +303,32 @@ class Actor:
self._reg_addrs = addrs self._reg_addrs = addrs
async def wait_for_peer( async def wait_for_peer(
self, uid: tuple[str, str] self,
uid: tuple[str, str],
) -> tuple[trio.Event, Channel]: ) -> tuple[trio.Event, Channel]:
''' '''
Wait for a connection back from a spawned actor with a `uid` Wait for a connection back from a (spawned sub-)actor with
using a `trio.Event` for sync. a `uid` using a `trio.Event` for sync.
''' '''
log.runtime(f"Waiting for peer {uid} to connect") log.debug(f'Waiting for peer {uid!r} to connect')
event = self._peer_connected.setdefault(uid, trio.Event()) event = self._peer_connected.setdefault(uid, trio.Event())
await event.wait() await event.wait()
log.runtime(f"{uid} successfully connected back to us") log.debug(f'{uid!r} successfully connected back to us')
return event, self._peers[uid][-1] return event, self._peers[uid][-1]
def load_modules( def load_modules(
self, self,
debug_mode: bool = False, # debug_mode: bool = False,
) -> None: ) -> None:
''' '''
Load enabled RPC py-modules locally (after process fork/spawn). Load explicitly enabled python modules from local fs after
process spawn.
Since this actor may be spawned on a different machine from Since this actor may be spawned on a different machine from
the original nursery we need to try and load the local module the original nursery we need to try and load the local module
code (presuming it exists). code manually (presuming it exists).
''' '''
try: try:
@ -350,16 +341,21 @@ class Actor:
_mp_fixup_main._fixup_main_from_path( _mp_fixup_main._fixup_main_from_path(
parent_data['init_main_from_path']) parent_data['init_main_from_path'])
status: str = 'Attempting to import enabled modules:\n'
for modpath, filepath in self.enable_modules.items(): for modpath, filepath in self.enable_modules.items():
# XXX append the allowed module to the python path which # XXX append the allowed module to the python path which
# should allow for relative (at least downward) imports. # should allow for relative (at least downward) imports.
sys.path.append(os.path.dirname(filepath)) sys.path.append(os.path.dirname(filepath))
log.runtime(f"Attempting to import {modpath}@{filepath}") status += (
mod = importlib.import_module(modpath) f'|_{modpath!r} -> {filepath!r}\n'
)
mod: ModuleType = importlib.import_module(modpath)
self._mods[modpath] = mod self._mods[modpath] = mod
if modpath == '__main__': if modpath == '__main__':
self._mods['__mp_main__'] = mod self._mods['__mp_main__'] = mod
log.runtime(status)
except ModuleNotFoundError: except ModuleNotFoundError:
# it is expected the corresponding `ModuleNotExposed` error # it is expected the corresponding `ModuleNotExposed` error
# will be raised later # will be raised later
@ -413,21 +409,23 @@ class Actor:
chan = Channel.from_stream(stream) chan = Channel.from_stream(stream)
their_uid: tuple[str, str]|None = chan.uid their_uid: tuple[str, str]|None = chan.uid
con_msg: str = '' con_status: str = ''
# TODO: remove this branch since can never happen?
# NOTE: `.uid` is only set after first contact
if their_uid: if their_uid:
# NOTE: `.uid` is only set after first contact con_status = (
con_msg = ( 'IPC Re-connection from already known peer?\n'
'IPC Re-connection from already known peer? '
) )
else: else:
con_msg = ( con_status = (
'New IPC connection to us ' 'New inbound IPC connection <=\n'
) )
con_msg += ( con_status += (
f'<= @{chan.raddr}\n'
f'|_{chan}\n' f'|_{chan}\n'
# f' |_@{chan.raddr}\n\n' # f' |_@{chan.raddr}\n\n'
# ^-TODO-^ remove since alfready in chan.__repr__()?
) )
# send/receive initial handshake response # send/receive initial handshake response
try: try:
@ -447,13 +445,13 @@ class Actor:
# a bound listener on the "arbiter" addr. the reset will be # a bound listener on the "arbiter" addr. the reset will be
# because the handshake was never meant took place. # because the handshake was never meant took place.
log.warning( log.warning(
con_msg con_status
+ +
' -> But failed to handshake? Ignoring..\n' ' -> But failed to handshake? Ignoring..\n'
) )
return return
con_msg += ( con_status += (
f' -> Handshake with actor `{uid[0]}[{uid[1][-6:]}]` complete\n' f' -> Handshake with actor `{uid[0]}[{uid[1][-6:]}]` complete\n'
) )
# IPC connection tracking for both peers and new children: # IPC connection tracking for both peers and new children:
@ -466,7 +464,7 @@ class Actor:
None, None,
) )
if event: if event:
con_msg += ( con_status += (
' -> Waking subactor spawn waiters: ' ' -> Waking subactor spawn waiters: '
f'{event.statistics().tasks_waiting}\n' f'{event.statistics().tasks_waiting}\n'
f' -> Registered IPC chan for child actor {uid}@{chan.raddr}\n' f' -> Registered IPC chan for child actor {uid}@{chan.raddr}\n'
@ -477,7 +475,7 @@ class Actor:
event.set() event.set()
else: else:
con_msg += ( con_status += (
f' -> Registered IPC chan for peer actor {uid}@{chan.raddr}\n' f' -> Registered IPC chan for peer actor {uid}@{chan.raddr}\n'
) # type: ignore ) # type: ignore
@ -491,13 +489,18 @@ class Actor:
# TODO: can we just use list-ref directly? # TODO: can we just use list-ref directly?
chans.append(chan) chans.append(chan)
log.runtime(con_msg) con_status += ' -> Entering RPC msg loop..\n'
log.runtime(con_status)
# Begin channel management - respond to remote requests and # Begin channel management - respond to remote requests and
# process received reponses. # process received reponses.
disconnected: bool = False disconnected: bool = False
last_msg: MsgType
try: try:
disconnected: bool = await process_messages( (
disconnected,
last_msg,
) = await process_messages(
self, self,
chan, chan,
) )
@ -598,16 +601,24 @@ class Actor:
# that the IPC layer may have failed # that the IPC layer may have failed
# unexpectedly since it may be the cause of # unexpectedly since it may be the cause of
# other downstream errors. # other downstream errors.
entry = local_nursery._children.get(uid) entry: tuple|None = local_nursery._children.get(uid)
if entry: if entry:
proc: trio.Process proc: trio.Process
_, proc, _ = entry _, proc, _ = entry
if ( if (
(poll := getattr(proc, 'poll', None)) (poll := getattr(proc, 'poll', None))
and poll() is None and
poll() is None # proc still alive
): ):
log.cancel( # TODO: change log level based on
# detecting whether chan was created for
# ephemeral `.register_actor()` request!
# -[ ] also, that should be avoidable by
# re-using any existing chan from the
# `._discovery.get_registry()` call as
# well..
log.runtime(
f'Peer IPC broke but subproc is alive?\n\n' f'Peer IPC broke but subproc is alive?\n\n'
f'<=x {chan.uid}@{chan.raddr}\n' f'<=x {chan.uid}@{chan.raddr}\n'
@ -616,17 +627,17 @@ class Actor:
# ``Channel`` teardown and closure sequence # ``Channel`` teardown and closure sequence
# drop ref to channel so it can be gc-ed and disconnected # drop ref to channel so it can be gc-ed and disconnected
log.runtime( con_teardown_status: str = (
f'Disconnected IPC channel:\n' f'IPC channel disconnected:\n'
f'uid: {chan.uid}\n' f'<=x uid: {chan.uid}\n'
f'|_{pformat(chan)}\n' f' |_{pformat(chan)}\n\n'
) )
chans.remove(chan) chans.remove(chan)
# TODO: do we need to be this pedantic? # TODO: do we need to be this pedantic?
if not chans: if not chans:
log.runtime( con_teardown_status += (
f'No more channels with {chan.uid}' f'-> No more channels with {chan.uid}'
) )
self._peers.pop(uid, None) self._peers.pop(uid, None)
@ -640,15 +651,16 @@ class Actor:
f' |_[{i}] {pformat(chan)}\n' f' |_[{i}] {pformat(chan)}\n'
) )
log.runtime( con_teardown_status += (
f'Remaining IPC {len(self._peers)} peers:\n' f'-> Remaining IPC {len(self._peers)} peers: {peers_str}\n'
+ peers_str
) )
# No more channels to other actors (at all) registered # No more channels to other actors (at all) registered
# as connected. # as connected.
if not self._peers: if not self._peers:
log.runtime("Signalling no more peer channel connections") con_teardown_status += (
'Signalling no more peer channel connections'
)
self._no_more_peers.set() self._no_more_peers.set()
# NOTE: block this actor from acquiring the # NOTE: block this actor from acquiring the
@ -723,13 +735,16 @@ class Actor:
# TODO: figure out why this breaks tests.. # TODO: figure out why this breaks tests..
db_cs.cancel() db_cs.cancel()
log.runtime(con_teardown_status)
# finally block closure
# TODO: rename to `._deliver_payload()` since this handles # TODO: rename to `._deliver_payload()` since this handles
# more then just `result` msgs now obvi XD # more then just `result` msgs now obvi XD
async def _deliver_ctx_payload( async def _deliver_ctx_payload(
self, self,
chan: Channel, chan: Channel,
cid: str, cid: str,
msg: Msg|MsgTypeError, msg: MsgType|MsgTypeError,
) -> None|bool: ) -> None|bool:
''' '''
@ -754,7 +769,7 @@ class Actor:
# XXX don't need right since it's always in msg? # XXX don't need right since it's always in msg?
# f'=> cid: {cid}\n\n' # f'=> cid: {cid}\n\n'
f'{pretty_struct.Struct.pformat(msg)}\n' f'{pretty_struct.pformat(msg)}\n'
) )
return return
@ -896,9 +911,11 @@ class Actor:
cid=cid, cid=cid,
) )
log.runtime( log.runtime(
'Sending RPC start msg\n\n' 'Sending RPC `Start`\n\n'
f'=> peer: {chan.uid}\n' f'=> peer: {chan.uid}\n'
f' |_ {ns}.{func}({kwargs})\n' f' |_ {ns}.{func}({kwargs})\n\n'
f'{pretty_struct.pformat(msg)}'
) )
await chan.send(msg) await chan.send(msg)
@ -955,31 +972,29 @@ class Actor:
if self._spawn_method == "trio": if self._spawn_method == "trio":
# Receive runtime state from our parent # Receive post-spawn runtime state from our parent.
# parent_data: dict[str, Any]
# parent_data = await chan.recv()
# TODO: maybe we should just wrap this directly
# in a `Actor.spawn_info: SpawnInfo` struct?
spawnspec: msgtypes.SpawnSpec = await chan.recv() spawnspec: msgtypes.SpawnSpec = await chan.recv()
self._spawn_spec = spawnspec self._spawn_spec = spawnspec
# TODO: eventually all these msgs as
# `msgspec.Struct` with a special mode that
# pformats them in multi-line mode, BUT only
# if "trace"/"util" mode is enabled?
log.runtime( log.runtime(
'Received runtime spec from parent:\n\n' 'Received runtime spec from parent:\n\n'
f'{pformat(spawnspec)}\n'
# TODO: eventually all these msgs as
# `msgspec.Struct` with a special mode that
# pformats them in multi-line mode, BUT only
# if "trace"/"util" mode is enabled?
f'{pretty_struct.pformat(spawnspec)}\n'
) )
# accept_addrs: list[tuple[str, int]] = parent_data.pop('bind_addrs')
accept_addrs: list[tuple[str, int]] = spawnspec.bind_addrs accept_addrs: list[tuple[str, int]] = spawnspec.bind_addrs
# rvs = parent_data.pop('_runtime_vars') # TODO: another `Struct` for rtvs..
rvs = spawnspec._runtime_vars rvs: dict[str, Any] = spawnspec._runtime_vars
if rvs['_debug_mode']: if rvs['_debug_mode']:
try: try:
log.info( # TODO: maybe return some status msgs upward
# to that we can emit them in `con_status`
# instead?
log.devx(
'Enabling `stackscope` traces on SIGUSR1' 'Enabling `stackscope` traces on SIGUSR1'
) )
from .devx import enable_stack_on_sig from .devx import enable_stack_on_sig
@ -989,7 +1004,6 @@ class Actor:
'`stackscope` not installed for use in debug mode!' '`stackscope` not installed for use in debug mode!'
) )
log.runtime(f'Runtime vars are: {rvs}')
rvs['_is_root'] = False rvs['_is_root'] = False
_state._runtime_vars.update(rvs) _state._runtime_vars.update(rvs)
@ -1006,18 +1020,12 @@ class Actor:
for val in spawnspec.reg_addrs for val in spawnspec.reg_addrs
] ]
# for attr, value in parent_data.items(): # TODO: better then monkey patching..
# -[ ] maybe read the actual f#$-in `._spawn_spec` XD
for _, attr, value in pretty_struct.iter_fields( for _, attr, value in pretty_struct.iter_fields(
spawnspec, spawnspec,
): ):
setattr(self, attr, value) setattr(self, attr, value)
# if (
# attr == 'reg_addrs'
# and value
# ):
# self.reg_addrs = [tuple(val) for val in value]
# else:
# setattr(self, attr, value)
return ( return (
chan, chan,
@ -1026,12 +1034,11 @@ class Actor:
except OSError: # failed to connect except OSError: # failed to connect
log.warning( log.warning(
f'Failed to connect to parent!?\n\n' f'Failed to connect to spawning parent actor!?\n'
'Closing IPC [TCP] transport server to\n' f'x=> {parent_addr}\n'
f'{parent_addr}\n'
f'|_{self}\n\n' f'|_{self}\n\n'
) )
await self.cancel(chan=None) # self cancel await self.cancel(req_chan=None) # self cancel
raise raise
async def _serve_forever( async def _serve_forever(
@ -1109,8 +1116,7 @@ class Actor:
# chan whose lifetime limits the lifetime of its remotely # chan whose lifetime limits the lifetime of its remotely
# requested and locally spawned RPC tasks - similar to the # requested and locally spawned RPC tasks - similar to the
# supervision semantics of a nursery wherein the actual # supervision semantics of a nursery wherein the actual
# implementation does start all such tasks in # implementation does start all such tasks in a sub-nursery.
# a sub-nursery.
req_chan: Channel|None, req_chan: Channel|None,
) -> bool: ) -> bool:
@ -1151,7 +1157,7 @@ class Actor:
# other) repr fields instead of doing this all manual.. # other) repr fields instead of doing this all manual..
msg: str = ( msg: str = (
f'Runtime cancel request from {requester_type}:\n\n' f'Runtime cancel request from {requester_type}:\n\n'
f'<= .cancel(): {requesting_uid}\n' f'<= .cancel(): {requesting_uid}\n\n'
) )
# TODO: what happens here when we self-cancel tho? # TODO: what happens here when we self-cancel tho?
@ -1166,8 +1172,8 @@ class Actor:
dbcs = _debug.DebugStatus.req_cs dbcs = _debug.DebugStatus.req_cs
if dbcs is not None: if dbcs is not None:
msg += ( msg += (
'>> Cancelling active debugger request..\n' '-> Cancelling active debugger request..\n'
f'|_{_debug.Lock}\n' f'|_{_debug.Lock.pformat()}'
) )
dbcs.cancel() dbcs.cancel()
@ -1418,7 +1424,12 @@ class Actor:
''' '''
if self._server_n: if self._server_n:
log.runtime("Shutting down channel server") # TODO: obvi a different server type when we eventually
# support some others XD
server_prot: str = 'TCP'
log.runtime(
f'Cancelling {server_prot} server'
)
self._server_n.cancel_scope.cancel() self._server_n.cancel_scope.cancel()
return True return True
@ -1602,6 +1613,7 @@ async def async_main(
assert accept_addrs assert accept_addrs
try: try:
# TODO: why is this not with the root nursery?
actor._server_n = await service_nursery.start( actor._server_n = await service_nursery.start(
partial( partial(
actor._serve_forever, actor._serve_forever,
@ -1886,13 +1898,13 @@ class Arbiter(Actor):
sockaddrs: list[tuple[str, int]] = [] sockaddrs: list[tuple[str, int]] = []
sockaddr: tuple[str, int] sockaddr: tuple[str, int]
for (aname, _), sockaddr in self._registry.items(): mailbox_info: str = 'Actor registry contact infos:\n'
log.runtime( for uid, sockaddr in self._registry.items():
f'Actor mailbox info:\n' mailbox_info += (
f'aname: {aname}\n' f'|_uid: {uid}\n'
f'sockaddr: {sockaddr}\n' f'|_sockaddr: {sockaddr}\n\n'
) )
if name == aname: if name == uid[0]:
sockaddrs.append(sockaddr) sockaddrs.append(sockaddr)
if not sockaddrs: if not sockaddrs:
@ -1904,6 +1916,7 @@ class Arbiter(Actor):
if not isinstance(uid, trio.Event): if not isinstance(uid, trio.Event):
sockaddrs.append(self._registry[uid]) sockaddrs.append(self._registry[uid])
log.runtime(mailbox_info)
return sockaddrs return sockaddrs
async def register_actor( async def register_actor(