Even more `.ipc.*` repr refinements

Mostly adjusting indentation, noise level, and clarity via `.pformat()`
tweaks more general use of `.devx.pformat.nest_from_op()`.

Specific impl deats,
- use `pformat.ppfmt()/`nest_from_op()` more seriously throughout
  `._server`.
- add a `._server.Endpoint.pformat()`.
- add `._server.Server.len_peers()` and `.repr_state()`.
- polish `Server.pformat()`.
- drop some redundant `log.runtime()`s from `._serve_ipc_eps()` instead
  leaving-them-only/putting-them in the caller pub meth.
- `._tcp.start_listener()` log the bound addr, not the input (which may
  be the 0-port.
enable_tpts
Tyler Goodlet 2025-07-03 23:33:02 -04:00
parent 3201437f4e
commit 065104401c
3 changed files with 191 additions and 90 deletions

View File

@ -462,8 +462,8 @@ class Channel:
await self.send(aid) await self.send(aid)
peer_aid: Aid = await self.recv() peer_aid: Aid = await self.recv()
log.runtime( log.runtime(
f'Received hanshake with peer ' f'Received hanshake with peer\n'
f'{peer_aid.reprol(sin_uuid=False)}\n' f'<= {peer_aid.reprol(sin_uuid=False)}\n'
) )
# NOTE, we always are referencing the remote peer! # NOTE, we always are referencing the remote peer!
self.aid = peer_aid self.aid = peer_aid

View File

@ -26,7 +26,7 @@ from contextlib import (
from functools import partial from functools import partial
from itertools import chain from itertools import chain
import inspect import inspect
from pprint import pformat import textwrap
from types import ( from types import (
ModuleType, ModuleType,
) )
@ -43,7 +43,10 @@ from trio import (
SocketListener, SocketListener,
) )
# from ..devx import debug from ..devx.pformat import (
ppfmt,
nest_from_op,
)
from .._exceptions import ( from .._exceptions import (
TransportClosed, TransportClosed,
) )
@ -141,9 +144,8 @@ async def maybe_wait_on_canced_subs(
): ):
log.cancel( log.cancel(
'Waiting on cancel request to peer..\n' 'Waiting on cancel request to peer\n'
f'c)=>\n' f'c)=> {chan.aid.reprol()}@[{chan.maddr}]\n'
f' |_{chan.aid}\n'
) )
# XXX: this is a soft wait on the channel (and its # XXX: this is a soft wait on the channel (and its
@ -179,7 +181,7 @@ async def maybe_wait_on_canced_subs(
log.warning( log.warning(
'Draining msg from disconnected peer\n' 'Draining msg from disconnected peer\n'
f'{chan_info}' f'{chan_info}'
f'{pformat(msg)}\n' f'{ppfmt(msg)}\n'
) )
# cid: str|None = msg.get('cid') # cid: str|None = msg.get('cid')
cid: str|None = msg.cid cid: str|None = msg.cid
@ -248,7 +250,7 @@ async def maybe_wait_on_canced_subs(
if children := local_nursery._children: if children := local_nursery._children:
# indent from above local-nurse repr # indent from above local-nurse repr
report += ( report += (
f' |_{pformat(children)}\n' f' |_{ppfmt(children)}\n'
) )
log.warning(report) log.warning(report)
@ -279,8 +281,9 @@ async def maybe_wait_on_canced_subs(
log.runtime( 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.aid}@{chan.raddr}\n' f'<=x {chan.aid.reprol()}@[{chan.maddr}]\n'
f' |_{proc}\n' f'\n'
f'{proc}\n'
) )
return local_nursery return local_nursery
@ -324,9 +327,10 @@ async def handle_stream_from_peer(
chan = Channel.from_stream(stream) chan = Channel.from_stream(stream)
con_status: str = ( con_status: str = (
'New inbound IPC connection <=\n' f'New inbound IPC transport connection\n'
f'|_{chan}\n' f'<=( {stream!r}\n'
) )
con_status_steps: str = ''
# initial handshake with peer phase # initial handshake with peer phase
try: try:
@ -372,7 +376,7 @@ async def handle_stream_from_peer(
if _pre_chan := server._peers.get(uid): if _pre_chan := server._peers.get(uid):
familiar: str = 'pre-existing-peer' familiar: str = 'pre-existing-peer'
uid_short: str = f'{uid[0]}[{uid[1][-6:]}]' uid_short: str = f'{uid[0]}[{uid[1][-6:]}]'
con_status += ( con_status_steps += (
f' -> Handshake with {familiar} `{uid_short}` complete\n' f' -> Handshake with {familiar} `{uid_short}` complete\n'
) )
@ -397,7 +401,7 @@ async def handle_stream_from_peer(
None, None,
) )
if event: if event:
con_status += ( con_status_steps += (
' -> 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'
@ -408,7 +412,7 @@ async def handle_stream_from_peer(
event.set() event.set()
else: else:
con_status += ( con_status_steps += (
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
@ -422,8 +426,15 @@ async def handle_stream_from_peer(
# TODO: can we just use list-ref directly? # TODO: can we just use list-ref directly?
chans.append(chan) chans.append(chan)
con_status += ' -> Entering RPC msg loop..\n' con_status_steps += ' -> Entering RPC msg loop..\n'
log.runtime(con_status) log.runtime(
con_status
+
textwrap.indent(
con_status_steps,
prefix=' '*3, # align to first-ln
)
)
# Begin channel management - respond to remote requests and # Begin channel management - respond to remote requests and
# process received reponses. # process received reponses.
@ -456,41 +467,67 @@ async def handle_stream_from_peer(
disconnected=disconnected, disconnected=disconnected,
) )
# ``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
con_teardown_status: str = ( #
f'IPC channel disconnected:\n' # -[x]TODO mk this be like
f'<=x uid: {chan.aid}\n' # <=x Channel(
f' |_{pformat(chan)}\n\n' # |_field: blah
# )>
op_repr: str = '<=x '
chan_repr: str = nest_from_op(
input_op=op_repr,
op_suffix='',
nest_prefix='',
text=chan.pformat(),
nest_indent=len(op_repr)-1,
rm_from_first_ln='<',
) )
con_teardown_status: str = (
f'IPC channel disconnect\n'
f'\n'
f'{chan_repr}\n'
f'\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:
con_teardown_status += ( con_teardown_status += (
f'-> No more channels with {chan.aid}' f'-> No more channels with {chan.aid.reprol()!r}\n'
) )
server._peers.pop(uid, None) server._peers.pop(uid, None)
peers_str: str = '' if peers := list(server._peers.values()):
for uid, chans in server._peers.items(): peer_cnt: int = len(peers)
peers_str += ( if (
f'uid: {uid}\n' (first := peers[0][0]) is not chan
) and
for i, chan in enumerate(chans): not disconnected
peers_str += ( and
f' |_[{i}] {pformat(chan)}\n' peer_cnt > 1
):
con_teardown_status += (
f'-> Remaining IPC {peer_cnt-1!r} peers:\n'
) )
for chans in server._peers.values():
con_teardown_status += ( first: Channel = chans[0]
f'-> Remaining IPC {len(server._peers)} peers: {peers_str}\n' if not (
) first is chan
and
disconnected
):
con_teardown_status += (
f' |_{first.aid.reprol()!r} -> {len(chans)!r} chans\n'
)
# No more channels to other actors (at all) registered # No more channels to other actors (at all) registered
# as connected. # as connected.
if not server._peers: if not server._peers:
con_teardown_status += ( con_teardown_status += (
'Signalling no more peer channel connections' '-> Signalling no more peer connections!\n'
) )
server._no_more_peers.set() server._no_more_peers.set()
@ -579,10 +616,10 @@ async def handle_stream_from_peer(
class Endpoint(Struct): class Endpoint(Struct):
''' '''
An instance of an IPC "bound" address where the lifetime of the An instance of an IPC "bound" address where the lifetime of an
"ability to accept connections" (from clients) and then handle "ability to accept connections" and handle the subsequent
those inbound sessions or sequences-of-packets is determined by sequence-of-packets (maybe oriented as sessions) is determined by
a (maybe pair of) nurser(y/ies). the underlying nursery scope(s).
''' '''
addr: Address addr: Address
@ -600,6 +637,24 @@ class Endpoint(Struct):
MsgTransport, # handle to encoded-msg transport stream MsgTransport, # handle to encoded-msg transport stream
] = {} ] = {}
def pformat(
self,
indent: int = 0,
privates: bool = False,
) -> str:
type_repr: str = type(self).__name__
fmtstr: str = (
# !TODO, always be ns aware!
# f'|_netns: {netns}\n'
f' |.addr: {self.addr!r}\n'
f' |_peers: {len(self.peer_tpts)}\n'
)
return (
f'<{type_repr}(\n'
f'{fmtstr}'
f')>'
)
async def start_listener(self) -> SocketListener: async def start_listener(self) -> SocketListener:
tpt_mod: ModuleType = inspect.getmodule(self.addr) tpt_mod: ModuleType = inspect.getmodule(self.addr)
lstnr: SocketListener = await tpt_mod.start_listener( lstnr: SocketListener = await tpt_mod.start_listener(
@ -639,11 +694,13 @@ class Endpoint(Struct):
class Server(Struct): class Server(Struct):
_parent_tn: Nursery _parent_tn: Nursery
_stream_handler_tn: Nursery _stream_handler_tn: Nursery
# level-triggered sig for whether "no peers are currently # level-triggered sig for whether "no peers are currently
# connected"; field is **always** set to an instance but # connected"; field is **always** set to an instance but
# initialized with `.is_set() == True`. # initialized with `.is_set() == True`.
_no_more_peers: trio.Event _no_more_peers: trio.Event
# active eps as allocated by `.listen_on()`
_endpoints: list[Endpoint] = [] _endpoints: list[Endpoint] = []
# connection tracking & mgmt # connection tracking & mgmt
@ -651,12 +708,19 @@ class Server(Struct):
str, # uaid str, # uaid
list[Channel], # IPC conns from peer list[Channel], # IPC conns from peer
] = defaultdict(list) ] = defaultdict(list)
# events-table with entries registered unset while the local
# actor is waiting on a new actor to inbound connect, often
# a parent waiting on its child just after spawn.
_peer_connected: dict[ _peer_connected: dict[
tuple[str, str], tuple[str, str],
trio.Event, trio.Event,
] = {} ] = {}
# syncs for setup/teardown sequences # syncs for setup/teardown sequences
# - null when not yet booted,
# - unset when active,
# - set when fully shutdown with 0 eps active.
_shutdown: trio.Event|None = None _shutdown: trio.Event|None = None
# TODO, maybe just make `._endpoints: list[Endpoint]` and # TODO, maybe just make `._endpoints: list[Endpoint]` and
@ -664,7 +728,6 @@ class Server(Struct):
# @property # @property
# def addrs2eps(self) -> dict[Address, Endpoint]: # def addrs2eps(self) -> dict[Address, Endpoint]:
# ... # ...
@property @property
def proto_keys(self) -> list[str]: def proto_keys(self) -> list[str]:
return [ return [
@ -690,7 +753,7 @@ class Server(Struct):
# TODO: obvi a different server type when we eventually # TODO: obvi a different server type when we eventually
# support some others XD # support some others XD
log.runtime( log.runtime(
f'Cancelling server(s) for\n' f'Cancelling server(s) for tpt-protos\n'
f'{self.proto_keys!r}\n' f'{self.proto_keys!r}\n'
) )
self._parent_tn.cancel_scope.cancel() self._parent_tn.cancel_scope.cancel()
@ -717,6 +780,14 @@ class Server(Struct):
f'protos: {tpt_protos!r}\n' f'protos: {tpt_protos!r}\n'
) )
def len_peers(
self,
) -> int:
return len([
chan.connected()
for chan in chain(*self._peers.values())
])
def has_peers( def has_peers(
self, self,
check_chans: bool = False, check_chans: bool = False,
@ -730,13 +801,11 @@ class Server(Struct):
has_peers has_peers
and and
check_chans check_chans
and
(peer_cnt := self.len_peers())
): ):
has_peers: bool = ( has_peers: bool = (
any(chan.connected() peer_cnt > 0
for chan in chain(
*self._peers.values()
)
)
and and
has_peers has_peers
) )
@ -803,30 +872,66 @@ class Server(Struct):
return ev.is_set() return ev.is_set()
def pformat(self) -> str: @property
def repr_state(self) -> str:
'''
A `str`-status describing the current state of this
IPC server in terms of the current operating "phase".
'''
status = 'server is active'
if self.has_peers():
peer_cnt: int = self.len_peers()
status: str = (
f'{peer_cnt!r} peer chans'
)
else:
status: str = 'No peer chans'
if self.is_shutdown():
status: str = 'server-shutdown'
return status
def pformat(
self,
privates: bool = False,
) -> str:
eps: list[Endpoint] = self._endpoints eps: list[Endpoint] = self._endpoints
state_repr: str = ( # state_repr: str = (
f'{len(eps)!r} IPC-endpoints active' # f'{len(eps)!r} endpoints active'
) # )
fmtstr = ( fmtstr = (
f' |_state: {state_repr}\n' f' |_state: {self.repr_state!r}\n'
f' no_more_peers: {self.has_peers()}\n'
) )
if self._shutdown is not None: if privates:
shutdown_stats: EventStatistics = self._shutdown.statistics() fmtstr += f' no_more_peers: {self.has_peers()}\n'
if self._shutdown is not None:
shutdown_stats: EventStatistics = self._shutdown.statistics()
fmtstr += (
f' task_waiting_on_shutdown: {shutdown_stats}\n'
)
if eps := self._endpoints:
addrs: list[tuple] = [
ep.addr for ep in eps
]
repr_eps: str = ppfmt(addrs)
fmtstr += ( fmtstr += (
f' task_waiting_on_shutdown: {shutdown_stats}\n' f' |_endpoints: {repr_eps}\n'
# ^TODO? how to indent closing ']'..
) )
fmtstr += ( if peers := self._peers:
# TODO, use the `ppfmt()` helper from `modden`! fmtstr += (
f' |_endpoints: {pformat(self._endpoints)}\n' f' |_peers: {len(peers)} connected\n'
f' |_peers: {len(self._peers)} connected\n' )
)
return ( return (
f'<IPCServer(\n' f'<Server(\n'
f'{fmtstr}' f'{fmtstr}'
f')>\n' f')>\n'
) )
@ -885,8 +990,8 @@ class Server(Struct):
) )
log.runtime( log.runtime(
f'Binding to endpoints for,\n' f'Binding endpoints\n'
f'{accept_addrs}\n' f'{ppfmt(accept_addrs)}\n'
) )
eps: list[Endpoint] = await self._parent_tn.start( eps: list[Endpoint] = await self._parent_tn.start(
partial( partial(
@ -896,13 +1001,19 @@ class Server(Struct):
listen_addrs=accept_addrs, listen_addrs=accept_addrs,
) )
) )
self._endpoints.extend(eps)
serv_repr: str = nest_from_op(
input_op='(>',
text=self.pformat(),
nest_indent=1,
)
log.runtime( log.runtime(
f'Started IPC endpoints\n' f'Started IPC server\n'
f'{eps}\n' f'{serv_repr}'
) )
self._endpoints.extend(eps) # XXX, a little sanity on new ep allocations
# XXX, just a little bit of sanity
group_tn: Nursery|None = None group_tn: Nursery|None = None
ep: Endpoint ep: Endpoint
for ep in eps: for ep in eps:
@ -956,9 +1067,13 @@ async def _serve_ipc_eps(
stream_handler_tn=stream_handler_tn, stream_handler_tn=stream_handler_tn,
) )
try: try:
ep_sclang: str = nest_from_op(
input_op='>[',
text=f'{ep.pformat()}',
)
log.runtime( log.runtime(
f'Starting new endpoint listener\n' f'Starting new endpoint listener\n'
f'{ep}\n' f'{ep_sclang}\n'
) )
listener: trio.abc.Listener = await ep.start_listener() listener: trio.abc.Listener = await ep.start_listener()
assert listener is ep._listener assert listener is ep._listener
@ -996,17 +1111,6 @@ async def _serve_ipc_eps(
handler_nursery=stream_handler_tn handler_nursery=stream_handler_tn
) )
) )
# TODO, wow make this message better! XD
log.runtime(
'Started server(s)\n'
+
'\n'.join([f'|_{addr}' for addr in listen_addrs])
)
log.runtime(
f'Started IPC endpoints\n'
f'{eps}\n'
)
task_status.started( task_status.started(
eps, eps,
) )
@ -1049,8 +1153,7 @@ async def open_ipc_server(
try: try:
yield ipc_server yield ipc_server
log.runtime( log.runtime(
f'Waiting on server to shutdown or be cancelled..\n' 'Server-tn running until terminated\n'
f'{ipc_server}'
) )
# TODO? when if ever would we want/need this? # TODO? when if ever would we want/need this?
# with trio.CancelScope(shield=True): # with trio.CancelScope(shield=True):

View File

@ -127,10 +127,9 @@ async def start_listener(
Start a TCP socket listener on the given `TCPAddress`. Start a TCP socket listener on the given `TCPAddress`.
''' '''
log.info( log.runtime(
f'Attempting to bind TCP socket\n' f'Trying socket bind\n'
f'>[\n' f'>[ {addr}\n'
f'|_{addr}\n'
) )
# ?TODO, maybe we should just change the lower-level call this is # ?TODO, maybe we should just change the lower-level call this is
# using internall per-listener? # using internall per-listener?
@ -145,11 +144,10 @@ async def start_listener(
assert len(listeners) == 1 assert len(listeners) == 1
listener = listeners[0] listener = listeners[0]
host, port = listener.socket.getsockname()[:2] host, port = listener.socket.getsockname()[:2]
bound_addr: TCPAddress = type(addr).from_addr((host, port))
log.info( log.info(
f'Listening on TCP socket\n' f'Listening on TCP socket\n'
f'[>\n' f'[> {bound_addr}\n'
f' |_{addr}\n'
) )
return listener return listener