forked from goodboy/tractor
1
0
Fork 0

Merge pull request #392 from goodboy/introspect_ipc

Introspect-ipc: some `.ipc` subpkg iface refinements for reading cancel statuses and `Address.__repr__()`
Bd 2025-08-18 22:15:40 -04:00 committed by GitHub
commit a9aa5ec04e
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
9 changed files with 392 additions and 129 deletions

View File

@ -0,0 +1,114 @@
'''
Unit-ish tests for specific IPC transport protocol backends.
'''
from __future__ import annotations
from pathlib import Path
import pytest
import trio
import tractor
from tractor import (
Actor,
_state,
_addr,
)
@pytest.fixture
def bindspace_dir_str() -> str:
rt_dir: Path = tractor._state.get_rt_dir()
bs_dir: Path = rt_dir / 'doggy'
bs_dir_str: str = str(bs_dir)
assert not bs_dir.is_dir()
yield bs_dir_str
# delete it on suite teardown.
# ?TODO? should we support this internally
# or is leaking it ok?
if bs_dir.is_dir():
bs_dir.rmdir()
def test_uds_bindspace_created_implicitly(
debug_mode: bool,
bindspace_dir_str: str,
):
registry_addr: tuple = (
f'{bindspace_dir_str}',
'registry@doggy.sock',
)
bs_dir_str: str = registry_addr[0]
# XXX, ensure bindspace-dir DNE beforehand!
assert not Path(bs_dir_str).is_dir()
async def main():
async with tractor.open_nursery(
enable_transports=['uds'],
registry_addrs=[registry_addr],
debug_mode=debug_mode,
) as _an:
# XXX MUST be created implicitly by
# `.ipc._uds.start_listener()`!
assert Path(bs_dir_str).is_dir()
root: Actor = tractor.current_actor()
assert root.is_registrar
assert registry_addr in root.reg_addrs
assert (
registry_addr
in
_state._runtime_vars['_registry_addrs']
)
assert (
_addr.wrap_address(registry_addr)
in
root.registry_addrs
)
trio.run(main)
def test_uds_double_listen_raises_connerr(
debug_mode: bool,
bindspace_dir_str: str,
):
registry_addr: tuple = (
f'{bindspace_dir_str}',
'registry@doggy.sock',
)
async def main():
async with tractor.open_nursery(
enable_transports=['uds'],
registry_addrs=[registry_addr],
debug_mode=debug_mode,
) as _an:
# runtime up
root: Actor = tractor.current_actor()
from tractor.ipc._uds import (
start_listener,
UDSAddress,
)
ya_bound_addr: UDSAddress = root.registry_addrs[0]
try:
await start_listener(
addr=ya_bound_addr,
)
except ConnectionError as connerr:
assert type(src_exc := connerr.__context__) is OSError
assert 'Address already in use' in src_exc.args
# complete, exit test.
else:
pytest.fail('It dint raise a connerr !?')
trio.run(main)

View File

@ -154,7 +154,7 @@ class Context:
2 cancel-scope-linked, communicating and parallel executing
`Task`s. Contexts are allocated on each side of any task
RPC-linked msg dialog, i.e. for every request to a remote
actor from a `Portal`. On the "callee" side a context is
actor from a `Portal`. On the "child" side a context is
always allocated inside `._rpc._invoke()`.
TODO: more detailed writeup on cancellation, error and
@ -222,8 +222,8 @@ class Context:
# `._runtime.invoke()`.
_remote_func_type: str | None = None
# NOTE: (for now) only set (a portal) on the caller side since
# the callee doesn't generally need a ref to one and should
# NOTE: (for now) only set (a portal) on the parent side since
# the child doesn't generally need a ref to one and should
# normally need to explicitly ask for handle to its peer if
# more the the `Context` is needed?
_portal: Portal | None = None
@ -252,12 +252,12 @@ class Context:
_outcome_msg: Return|Error|ContextCancelled = Unresolved
# on a clean exit there should be a final value
# delivered from the far end "callee" task, so
# delivered from the far end "child" task, so
# this value is only set on one side.
# _result: Any | int = None
_result: PayloadT|Unresolved = Unresolved
# if the local "caller" task errors this value is always set
# if the local "parent" task errors this value is always set
# to the error that was captured in the
# `Portal.open_context().__aexit__()` teardown block OR, in
# 2 special cases when an (maybe) expected remote error
@ -293,7 +293,7 @@ class Context:
# a `ContextCancelled` due to a call to `.cancel()` triggering
# "graceful closure" on either side:
# - `._runtime._invoke()` will check this flag before engaging
# the crash handler REPL in such cases where the "callee"
# the crash handler REPL in such cases where the "child"
# raises the cancellation,
# - `.devx.debug.lock_stdio_for_peer()` will set it to `False` if
# the global tty-lock has been configured to filter out some
@ -307,8 +307,8 @@ class Context:
_stream_opened: bool = False
_stream: MsgStream|None = None
# caller of `Portal.open_context()` for
# logging purposes mostly
# the parent-task's calling-fn's frame-info, the frame above
# `Portal.open_context()`, for introspection/logging.
_caller_info: CallerInfo|None = None
# overrun handling machinery
@ -529,11 +529,11 @@ class Context:
'''
Exactly the value of `self._scope.cancelled_caught`
(delegation) and should only be (able to be read as)
`True` for a `.side == "caller"` ctx wherein the
`True` for a `.side == "parent"` ctx wherein the
`Portal.open_context()` block was exited due to a call to
`._scope.cancel()` - which should only ocurr in 2 cases:
- a caller side calls `.cancel()`, the far side cancels
- a parent side calls `.cancel()`, the far side cancels
and delivers back a `ContextCancelled` (making
`.cancel_acked == True`) and `._scope.cancel()` is
called by `._maybe_cancel_and_set_remote_error()` which
@ -542,20 +542,20 @@ class Context:
=> `._scope.cancelled_caught == True` by normal `trio`
cs semantics.
- a caller side is delivered a `._remote_error:
- a parent side is delivered a `._remote_error:
RemoteActorError` via `._deliver_msg()` and a transitive
call to `_maybe_cancel_and_set_remote_error()` calls
`._scope.cancel()` and that cancellation eventually
results in `trio.Cancelled`(s) caught in the
`.open_context()` handling around the @acm's `yield`.
Only as an FYI, in the "callee" side case it can also be
Only as an FYI, in the "child" side case it can also be
set but never is readable by any task outside the RPC
machinery in `._invoke()` since,:
- when a callee side calls `.cancel()`, `._scope.cancel()`
- when a child side calls `.cancel()`, `._scope.cancel()`
is called immediately and handled specially inside
`._invoke()` to raise a `ContextCancelled` which is then
sent to the caller side.
sent to the parent side.
However, `._scope.cancelled_caught` can NEVER be
accessed/read as `True` by any RPC invoked task since it
@ -666,7 +666,7 @@ class Context:
when called/closed by actor local task(s).
NOTEs:
- It is expected that the caller has previously unwrapped
- It is expected that the parent has previously unwrapped
the remote error using a call to `unpack_error()` and
provides that output exception value as the input
`error` argument *here*.
@ -676,7 +676,7 @@ class Context:
`Portal.open_context()` (ideally) we want to interrupt
any ongoing local tasks operating within that
`Context`'s cancel-scope so as to be notified ASAP of
the remote error and engage any caller handling (eg.
the remote error and engage any parent handling (eg.
for cross-process task supervision).
- In some cases we may want to raise the remote error
@ -886,6 +886,11 @@ class Context:
@property
def repr_caller(self) -> str:
'''
Render a "namespace-path" style representation of the calling
task-fn.
'''
ci: CallerInfo|None = self._caller_info
if ci:
return (
@ -899,7 +904,7 @@ class Context:
def repr_api(self) -> str:
return 'Portal.open_context()'
# TODO: use `.dev._frame_stack` scanning to find caller!
# TODO: use `.dev._frame_stack` scanning to find caller fn!
# ci: CallerInfo|None = self._caller_info
# if ci:
# return (
@ -934,7 +939,7 @@ class Context:
=> That is, an IPC `Context` (this) **does not**
have the same semantics as a `trio.CancelScope`.
If the caller (who entered the `Portal.open_context()`)
If the parent (who entered the `Portal.open_context()`)
desires that the internal block's cancel-scope be
cancelled it should open its own `trio.CancelScope` and
manage it as needed.
@ -1006,7 +1011,6 @@ class Context:
else:
log.cancel(
f'Timed out on cancel request of remote task?\n'
f'\n'
f'{reminfo}'
)
@ -1017,7 +1021,7 @@ class Context:
# `_invoke()` RPC task.
#
# NOTE: on this side we ALWAYS cancel the local scope
# since the caller expects a `ContextCancelled` to be sent
# since the parent expects a `ContextCancelled` to be sent
# from `._runtime._invoke()` back to the other side. The
# logic for catching the result of the below
# `._scope.cancel()` is inside the `._runtime._invoke()`
@ -1190,8 +1194,8 @@ class Context:
) -> Any|Exception:
'''
From some (caller) side task, wait for and return the final
result from the remote (callee) side's task.
From some (parent) side task, wait for and return the final
result from the remote (child) side's task.
This provides a mechanism for one task running in some actor to wait
on another task at the other side, in some other actor, to terminate.
@ -1487,6 +1491,12 @@ class Context:
):
status = 'peer-cancelled'
case (
Unresolved,
trio.Cancelled(), # any error-type
) if self.canceller:
status = 'actor-cancelled'
# (remote) error condition
case (
Unresolved,
@ -1600,7 +1610,7 @@ class Context:
raise err
# TODO: maybe a flag to by-pass encode op if already done
# here in caller?
# here in parent?
await self.chan.send(started_msg)
# set msg-related internal runtime-state
@ -1676,7 +1686,7 @@ class Context:
XXX RULES XXX
------ - ------
- NEVER raise remote errors from this method; a runtime task caller.
- NEVER raise remote errors from this method; a calling runtime-task.
An error "delivered" to a ctx should always be raised by
the corresponding local task operating on the
`Portal`/`Context` APIs.
@ -1752,7 +1762,7 @@ class Context:
else:
report = (
'Queueing OVERRUN msg on caller task:\n\n'
'Queueing OVERRUN msg on parent task:\n\n'
+ report
)
log.debug(report)
@ -1948,12 +1958,12 @@ async def open_context_from_portal(
IPC protocol.
The yielded `tuple` is a pair delivering a `tractor.Context`
and any first value "sent" by the "callee" task via a call
and any first value "sent" by the "child" task via a call
to `Context.started(<value: Any>)`; this side of the
context does not unblock until the "callee" task calls
context does not unblock until the "child" task calls
`.started()` in similar style to `trio.Nursery.start()`.
When the "callee" (side that is "called"/started by a call
to *this* method) returns, the caller side (this) unblocks
When the "child" (side that is "called"/started by a call
to *this* method) returns, the parent side (this) unblocks
and any final value delivered from the other end can be
retrieved using the `Contex.wait_for_result()` api.
@ -1966,7 +1976,7 @@ async def open_context_from_portal(
__tracebackhide__: bool = hide_tb
# denote this frame as a "runtime frame" for stack
# introspection where we report the caller code in logging
# introspection where we report the parent code in logging
# and error message content.
# NOTE: 2 bc of the wrapping `@acm`
__runtimeframe__: int = 2 # noqa
@ -2025,7 +2035,7 @@ async def open_context_from_portal(
# placeholder for any exception raised in the runtime
# or by user tasks which cause this context's closure.
scope_err: BaseException|None = None
ctxc_from_callee: ContextCancelled|None = None
ctxc_from_child: ContextCancelled|None = None
try:
async with (
collapse_eg(),
@ -2104,7 +2114,7 @@ async def open_context_from_portal(
# that we can re-use it around the `yield` ^ here
# or vice versa?
#
# maybe TODO NOTE: between the caller exiting and
# maybe TODO NOTE: between the parent exiting and
# arriving here the far end may have sent a ctxc-msg or
# other error, so the quetion is whether we should check
# for it here immediately and maybe raise so as to engage
@ -2170,16 +2180,16 @@ async def open_context_from_portal(
# request in which case we DO let the error bubble to the
# opener.
#
# 2-THIS "caller" task somewhere invoked `Context.cancel()`
# and received a `ContextCanclled` from the "callee"
# 2-THIS "parent" task somewhere invoked `Context.cancel()`
# and received a `ContextCanclled` from the "child"
# task, in which case we mask the `ContextCancelled` from
# bubbling to this "caller" (much like how `trio.Nursery`
# bubbling to this "parent" (much like how `trio.Nursery`
# swallows any `trio.Cancelled` bubbled by a call to
# `Nursery.cancel_scope.cancel()`)
except ContextCancelled as ctxc:
scope_err = ctxc
ctx._local_error: BaseException = scope_err
ctxc_from_callee = ctxc
ctxc_from_child = ctxc
# XXX TODO XXX: FIX THIS debug_mode BUGGGG!!!
# using this code and then resuming the REPL will
@ -2216,11 +2226,11 @@ async def open_context_from_portal(
# the above `._scope` can be cancelled due to:
# 1. an explicit self cancel via `Context.cancel()` or
# `Actor.cancel()`,
# 2. any "callee"-side remote error, possibly also a cancellation
# 2. any "child"-side remote error, possibly also a cancellation
# request by some peer,
# 3. any "caller" (aka THIS scope's) local error raised in the above `yield`
# 3. any "parent" (aka THIS scope's) local error raised in the above `yield`
except (
# CASE 3: standard local error in this caller/yieldee
# CASE 3: standard local error in this parent/yieldee
Exception,
# CASES 1 & 2: can manifest as a `ctx._scope_nursery`
@ -2234,9 +2244,9 @@ async def open_context_from_portal(
# any `Context._maybe_raise_remote_err()` call.
#
# 2.-`BaseExceptionGroup[ContextCancelled | RemoteActorError]`
# from any error delivered from the "callee" side
# from any error delivered from the "child" side
# AND a group-exc is only raised if there was > 1
# tasks started *here* in the "caller" / opener
# tasks started *here* in the "parent" / opener
# block. If any one of those tasks calls
# `.wait_for_result()` or `MsgStream.receive()`
# `._maybe_raise_remote_err()` will be transitively
@ -2249,8 +2259,8 @@ async def open_context_from_portal(
trio.Cancelled, # NOTE: NOT from inside the ctx._scope
KeyboardInterrupt,
) as caller_err:
scope_err = caller_err
) as rent_err:
scope_err = rent_err
ctx._local_error: BaseException = scope_err
# XXX: ALWAYS request the context to CANCEL ON any ERROR.
@ -2268,7 +2278,7 @@ async def open_context_from_portal(
logmeth = log.exception
logmeth(
f'ctx {ctx.side!r}-side exited with {ctx.repr_outcome()}\n'
f'ctx {ctx.side!r}-side exited with {ctx.repr_outcome()!r}\n'
)
if debug_mode():
@ -2289,9 +2299,9 @@ async def open_context_from_portal(
'Calling `ctx.cancel()`!\n'
)
# we don't need to cancel the callee if it already
# we don't need to cancel the child if it already
# told us it's cancelled ;p
if ctxc_from_callee is None:
if ctxc_from_child is None:
try:
await ctx.cancel()
except (
@ -2322,8 +2332,8 @@ async def open_context_from_portal(
# via a call to
# `Context._maybe_cancel_and_set_remote_error()`.
# As per `Context._deliver_msg()`, that error IS
# ALWAYS SET any time "callee" side fails and causes "caller
# side" cancellation via a `ContextCancelled` here.
# ALWAYS SET any time "child" side fails and causes
# "parent side" cancellation via a `ContextCancelled` here.
try:
result_or_err: Exception|Any = await ctx.wait_for_result()
except BaseException as berr:
@ -2359,7 +2369,7 @@ async def open_context_from_portal(
)
case (None, _):
log.runtime(
'Context returned final result from callee task:\n'
'Context returned final result from child task:\n'
f'<= peer: {uid}\n'
f' |_ {nsf}()\n\n'
@ -2454,7 +2464,7 @@ async def open_context_from_portal(
)
# TODO: should we add a `._cancel_req_received`
# flag to determine if the callee manually called
# flag to determine if the child manually called
# `ctx.cancel()`?
# -[ ] going to need a cid check no?
@ -2510,7 +2520,7 @@ def mk_context(
recv_chan: trio.MemoryReceiveChannel
send_chan, recv_chan = trio.open_memory_channel(msg_buffer_size)
# TODO: only scan caller-info if log level so high!
# TODO: only scan parent-info if log level so high!
from .devx._frame_stack import find_caller_info
caller_info: CallerInfo|None = find_caller_info()

View File

@ -300,7 +300,7 @@ class Portal:
)
# XXX the one spot we set it?
self.channel._cancel_called: bool = True
chan._cancel_called: bool = True
try:
# send cancel cmd - might not get response
# XXX: sure would be nice to make this work with

View File

@ -642,7 +642,7 @@ async def _invoke(
tn: Nursery
rpc_ctx_cs: CancelScope
async with (
collapse_eg(),
collapse_eg(hide_tb=False),
trio.open_nursery() as tn,
msgops.maybe_limit_plds(
ctx=ctx,
@ -823,24 +823,44 @@ async def _invoke(
f'after having {ctx.repr_state!r}\n'
)
if merr:
logmeth: Callable = log.error
if isinstance(merr, ContextCancelled):
logmeth: Callable = log.runtime
if (
# ctxc: by `Context.cancel()`
isinstance(merr, ContextCancelled)
if not isinstance(merr, RemoteActorError):
tb_str: str = ''.join(traceback.format_exception(merr))
# out-of-layer cancellation, one of:
# - actorc: by `Portal.cancel_actor()`
# - OSc: by SIGINT or `Process.signal()`
or (
isinstance(merr, trio.Cancelled)
and
ctx.canceller
)
):
logmeth: Callable = log.cancel
descr_str += (
f' with {merr!r}\n'
)
elif (
not isinstance(merr, RemoteActorError)
):
tb_str: str = ''.join(
traceback.format_exception(merr)
)
descr_str += (
f'\n{merr!r}\n' # needed?
f'{tb_str}\n'
f'\n'
f'scope_error:\n'
f'{scope_err!r}\n'
)
else:
descr_str += f'\n{merr!r}\n'
descr_str += (
f'{merr!r}\n'
)
else:
descr_str += f'\nwith final result {ctx.outcome!r}\n'
descr_str += (
f'\n'
f'with final result {ctx.outcome!r}\n'
)
logmeth(
f'{message}\n'

View File

@ -101,11 +101,27 @@ class Channel:
# ^XXX! ONLY set if a remote actor sends an `Error`-msg
self._closed: bool = False
# flag set by ``Portal.cancel_actor()`` indicating remote
# (possibly peer) cancellation of the far end actor
# runtime.
# flag set by `Portal.cancel_actor()` indicating remote
# (possibly peer) cancellation of the far end actor runtime.
self._cancel_called: bool = False
@property
def closed(self) -> bool:
'''
Was `.aclose()` successfully called?
'''
return self._closed
@property
def cancel_called(self) -> bool:
'''
Set when `Portal.cancel_actor()` is called on a portal which
wraps this IPC channel.
'''
return self._cancel_called
@property
def uid(self) -> tuple[str, str]:
'''
@ -169,7 +185,9 @@ class Channel:
addr,
**kwargs,
)
assert transport.raddr == addr
# XXX, for UDS *no!* since we recv the peer-pid and build out
# a new addr..
# assert transport.raddr == addr
chan = Channel(transport=transport)
# ?TODO, compact this into adapter level-methods?
@ -285,7 +303,7 @@ class Channel:
self,
payload: Any,
hide_tb: bool = True,
hide_tb: bool = False,
) -> None:
'''

View File

@ -17,13 +17,38 @@
Utils to tame mp non-SC madeness
'''
import platform
def disable_mantracker():
'''
Disable all ``multiprocessing``` "resource tracking" machinery since
Disable all `multiprocessing` "resource tracking" machinery since
it's an absolute multi-threaded mess of non-SC madness.
'''
from multiprocessing import resource_tracker as mantracker
from multiprocessing.shared_memory import SharedMemory
# 3.13+ only.. can pass `track=False` to disable
# all the resource tracker bs.
# https://docs.python.org/3/library/multiprocessing.shared_memory.html
if (_py_313 := (
platform.python_version_tuple()[:-1]
>=
('3', '13')
)
):
from functools import partial
return partial(
SharedMemory,
track=False,
)
# !TODO, once we drop 3.12- we can obvi remove all this!
else:
from multiprocessing import (
resource_tracker as mantracker,
)
# Tell the "resource tracker" thing to fuck off.
class ManTracker(mantracker.ResourceTracker):
@ -43,3 +68,8 @@ def disable_mantracker():
mantracker.ensure_running = mantracker._resource_tracker.ensure_running
mantracker.unregister = mantracker._resource_tracker.unregister
mantracker.getfd = mantracker._resource_tracker.getfd
# use std type verbatim
shmT = SharedMemory
return shmT

View File

@ -23,14 +23,15 @@ considered optional within the context of this runtime-library.
"""
from __future__ import annotations
from multiprocessing import shared_memory as shm
from multiprocessing.shared_memory import (
# SharedMemory,
ShareableList,
)
import platform
from sys import byteorder
import time
from typing import Optional
from multiprocessing import shared_memory as shm
from multiprocessing.shared_memory import (
SharedMemory,
ShareableList,
)
from msgspec import (
Struct,
@ -61,7 +62,7 @@ except ImportError:
log = get_logger(__name__)
disable_mantracker()
SharedMemory = disable_mantracker()
class SharedInt:
@ -789,10 +790,22 @@ def open_shm_list(
readonly=readonly,
)
# TODO, factor into a @actor_fixture acm-API?
# -[ ] also `@maybe_actor_fixture()` which inludes
# the .current_actor() convenience check?
# |_ orr can that just be in the sin-maybe-version?
#
# "close" attached shm on actor teardown
try:
actor = tractor.current_actor()
actor.lifetime_stack.callback(shml.shm.close)
# XXX on 3.13+ we don't need to call this?
# -> bc we pass `track=False` for `SharedMemeory` orr?
if (
platform.python_version_tuple()[:-1] < ('3', '13')
):
actor.lifetime_stack.callback(shml.shm.unlink)
except RuntimeError:
log.warning('tractor runtime not active, skipping teardown steps')

View File

@ -430,20 +430,25 @@ class MsgpackTransport(MsgTransport):
return await self.stream.send_all(size + bytes_data)
except (
trio.BrokenResourceError,
) as bre:
trans_err = bre
trio.ClosedResourceError,
) as _re:
trans_err = _re
tpt_name: str = f'{type(self).__name__!r}'
match trans_err:
case trio.BrokenResourceError() if (
'[Errno 32] Broken pipe' in trans_err.args[0]
# ^XXX, specifc to UDS transport and its,
# XXX, specifc to UDS transport and its,
# well, "speediness".. XD
# |_ likely todo with races related to how fast
# the socket is setup/torn-down on linux
# as it pertains to rando pings from the
# `.discovery` subsys and protos.
case trio.BrokenResourceError() if (
'[Errno 32] Broken pipe'
in
trans_err.args[0]
):
raise TransportClosed.from_src_exc(
tpt_closed = TransportClosed.from_src_exc(
message=(
f'{tpt_name} already closed by peer\n'
),
@ -451,14 +456,31 @@ class MsgpackTransport(MsgTransport):
src_exc=trans_err,
raise_on_report=True,
loglevel='transport',
) from bre
)
raise tpt_closed from trans_err
# case trio.ClosedResourceError() if (
# 'this socket was already closed'
# in
# trans_err.args[0]
# ):
# tpt_closed = TransportClosed.from_src_exc(
# message=(
# f'{tpt_name} already closed by peer\n'
# ),
# body=f'{self}\n',
# src_exc=trans_err,
# raise_on_report=True,
# loglevel='transport',
# )
# raise tpt_closed from trans_err
# unless the disconnect condition falls under "a
# normal operation breakage" we usualy console warn
# about it.
case _:
log.exception(
'{tpt_name} layer failed pre-send ??\n'
f'{tpt_name} layer failed pre-send ??\n'
)
raise trans_err
@ -503,7 +525,7 @@ class MsgpackTransport(MsgTransport):
def pformat(self) -> str:
return (
f'<{type(self).__name__}(\n'
f' |_peers: 2\n'
f' |_peers: 1\n'
f' laddr: {self._laddr}\n'
f' raddr: {self._raddr}\n'
# f'\n'

View File

@ -18,6 +18,9 @@ Unix Domain Socket implementation of tractor.ipc._transport.MsgTransport protoco
'''
from __future__ import annotations
from contextlib import (
contextmanager as cm,
)
from pathlib import Path
import os
from socket import (
@ -29,6 +32,7 @@ from socket import (
)
import struct
from typing import (
Type,
TYPE_CHECKING,
ClassVar,
)
@ -99,8 +103,6 @@ class UDSAddress(
self.filedir
or
self.def_bindspace
# or
# get_rt_dir()
)
@property
@ -205,12 +207,35 @@ class UDSAddress(
f']'
)
@cm
def _reraise_as_connerr(
src_excs: tuple[Type[Exception]],
addr: UDSAddress,
):
try:
yield
except src_excs as src_exc:
raise ConnectionError(
f'Bad UDS socket-filepath-as-address ??\n'
f'{addr}\n'
f' |_sockpath: {addr.sockpath}\n'
f'\n'
f'from src: {src_exc!r}\n'
) from src_exc
async def start_listener(
addr: UDSAddress,
**kwargs,
) -> SocketListener:
# sock = addr._sock = socket.socket(
'''
Start listening for inbound connections via
a `trio.SocketListener` (task) which `socket.bind()`s on `addr`.
Note, if the `UDSAddress.bindspace: Path` directory dne it is
implicitly created.
'''
sock = socket.socket(
socket.AF_UNIX,
socket.SOCK_STREAM
@ -221,17 +246,25 @@ async def start_listener(
f'|_{addr}\n'
)
# ?TODO? should we use the `actor.lifetime_stack`
# to rm on shutdown?
bindpath: Path = addr.sockpath
try:
await sock.bind(str(bindpath))
except (
if not (bs := addr.bindspace).is_dir():
log.info(
'Creating bindspace dir in file-sys\n'
f'>{{\n'
f'|_{bs!r}\n'
)
bs.mkdir()
with _reraise_as_connerr(
src_excs=(
FileNotFoundError,
) as fdne:
raise ConnectionError(
f'Bad UDS socket-filepath-as-address ??\n'
f'{addr}\n'
f' |_sockpath: {addr.sockpath}\n'
) from fdne
OSError,
),
addr=addr
):
await sock.bind(str(bindpath))
sock.listen(1)
log.info(
@ -356,27 +389,30 @@ class MsgpackUDSStream(MsgpackTransport):
# `.setsockopt()` call tells the OS provide it; the client
# pid can then be read on server/listen() side via
# `get_peer_info()` above.
try:
with _reraise_as_connerr(
src_excs=(
FileNotFoundError,
),
addr=addr
):
stream = await open_unix_socket_w_passcred(
str(sockpath),
**kwargs
)
except (
FileNotFoundError,
) as fdne:
raise ConnectionError(
f'Bad UDS socket-filepath-as-address ??\n'
f'{addr}\n'
f' |_sockpath: {sockpath}\n'
) from fdne
stream = MsgpackUDSStream(
tpt_stream = MsgpackUDSStream(
stream,
prefix_size=prefix_size,
codec=codec
)
stream._raddr = addr
return stream
# XXX assign from new addrs after peer-PID extract!
(
tpt_stream._laddr,
tpt_stream._raddr,
) = cls.get_stream_addrs(stream)
return tpt_stream
@classmethod
def get_stream_addrs(