Compare commits

...

3 Commits

Author SHA1 Message Date
Tyler Goodlet 6a5ccc2425 Allocate bind-addrs in subactors
Previously whenever an `ActorNursery.start_actor()` call did not receive
a `bind_addrs` arg we would allocate the default `(localhost, 0)` pairs
in the parent, for UDS this obviously won't work nor is it ideal bc it's
nicer to have the actor to be a socket server (who calls
`Address.open_listener()`) define the socket-file-name containing their
unique ID info such as pid, actor-uuid etc.

As such this moves "random" generation of server addresses to the
child-side of a subactor's spawn-sequence when it's sin-`bind_addrs`;
i.e. we do the allocation of the `Address.get_random()` addrs inside
`._runtime.async_main()` instead of `Portal.start_actor()` and **only
when** `accept_addrs`/`bind_addrs` was **not provided by the spawning
parent**.

Further this patch get's way more rigorous about the `SpawnSpec`
processing in the child inside `Actor._from_parent()` such that we
handle any invalid msgs **very loudly and pedantically!**

Impl deats,
- do the "random addr generation" in an explicit `for` loop (instead of
  prior comprehension) to allow for more detailed typing of the layered
  calls to the new `._addr` mod.
- use a `match:/case:` for process any invalid `SpawnSpec` payload case
  where we can instead receive a `MsgTypeError` from the `chan.recv()`
  call in `Actor._from_parent()` to raise it immediately instead of
  triggering downstream type-errors XD
  |_ as per the big `#TODO` we prolly want to take from other callers
     of `Channel.recv()` (like in the `._rpc.process_messages()` loop).
  |_ always raise `InternalError` on non-match/fall-through case!
  |_ add a note about not being able to use `breakpoint()` in this
     section due to causality of `SpawnSpec._runtime_vars` not having
     been processed yet..
  |_ always return a third element from `._from_rent()` eventually to be
     the `preferred_transports: list[str]` from the spawning rent.
- use new `._addr.mk_uuid()` and pass to new `Actor.__init__(uuid: str)`
  for all actor creation (including in all the mods tweaked here).
- Move to new type-alias-name `UnwrappedAddress` throughout.
2025-03-30 22:01:43 -04:00
Tyler Goodlet 23acd0f4cb Adjust imports to use new `UnwrappedAddress`
For those mods where it's just a type-alias (name) import change.
2025-03-30 21:24:48 -04:00
Tyler Goodlet 2c11d1d44a Implement peer-info tracking for UDS streams
Such that any UDS socket pair is represented (and with the recent
updates to) a `USDAddress` via a similar pair-`tuple[str, int]` as TCP
sockets, a pair of the `.filepath: Path` & the peer proc's `.pid: int`
which we read from the underlying `socket.socket` using
`.set/getsockopt()` calls

Impl deats,
- using the Linux specific APIs, we add a `get_peer_info()` which reads
  the `(pid, uid, gid)` using the `SOL_SOCKET` and `SOL_PEECRED` opts to
  `sock.getsockopt()`.
  |_ this presumes the client has been correspondingly configured to
     deliver the creds via a `sock.setsockopt(SOL_SOCKET, SO_PASSCRED,
     1)` call - this required us to override `trio.open_unix_socket()`.
- override `trio.open_unix_socket()` as per the above bullet to ensure
  connecting peers always transmit "credentials" options info to the
  listener.
- update `.get_stream_addrs()` to always call `get_peer_info()` and
  extract the peer's pid for the `raddr` and use `os.getpid()` for
  `laddr` (obvi).
  |_ as part of the new impl also `log.info()` the creds-info deats and
    socket-file path.
  |_ handle the oddity where it depends which of `.getpeername()` or
    `.getsockname()` will return the file-path; i think it's to do with
    who is client vs. server?

Related refinements,
- set `.layer_key: int = 4` for the "transport layer" ;)
- tweak some typing and multi-line unpacking in `.ipc/_tcp`.
2025-03-30 21:14:12 -04:00
9 changed files with 303 additions and 139 deletions

View File

@ -31,7 +31,7 @@ from tractor.log import get_logger
from .trionics import gather_contexts
from .ipc import _connect_chan, Channel
from ._addr import (
AddressTypes,
UnwrappedAddress,
Address,
preferred_transport,
wrap_address
@ -54,7 +54,9 @@ log = get_logger(__name__)
@acm
async def get_registry(addr: AddressTypes | None = None) -> AsyncGenerator[
async def get_registry(
addr: UnwrappedAddress|None = None,
) -> AsyncGenerator[
Portal | LocalPortal | None,
None,
]:
@ -71,7 +73,9 @@ async def get_registry(addr: AddressTypes | None = None) -> AsyncGenerator[
# (likely a re-entrant call from the arbiter actor)
yield LocalPortal(
actor,
await Channel.from_addr(addr)
Channel(transport=None)
# ^XXX, we DO NOT actually provide nor connect an
# underlying transport since this is merely an API shim.
)
else:
# TODO: try to look pre-existing connection from
@ -135,10 +139,10 @@ def get_peer_by_name(
@acm
async def query_actor(
name: str,
regaddr: AddressTypes|None = None,
regaddr: UnwrappedAddress|None = None,
) -> AsyncGenerator[
AddressTypes|None,
UnwrappedAddress|None,
None,
]:
'''
@ -168,7 +172,7 @@ async def query_actor(
async with get_registry(regaddr) as reg_portal:
# TODO: return portals to all available actors - for now
# just the last one that registered
addr: AddressTypes = await reg_portal.run_from_ns(
addr: UnwrappedAddress = await reg_portal.run_from_ns(
'self',
'find_actor',
name=name,
@ -178,7 +182,7 @@ async def query_actor(
@acm
async def maybe_open_portal(
addr: AddressTypes,
addr: UnwrappedAddress,
name: str,
):
async with query_actor(
@ -198,7 +202,7 @@ async def maybe_open_portal(
@acm
async def find_actor(
name: str,
registry_addrs: list[AddressTypes]|None = None,
registry_addrs: list[UnwrappedAddress]|None = None,
enable_transports: list[str] = [preferred_transport],
only_first: bool = True,
@ -234,7 +238,7 @@ async def find_actor(
)
maybe_portals: list[
AsyncContextManager[AddressTypes]
AsyncContextManager[UnwrappedAddress]
] = list(
maybe_open_portal(
addr=addr,
@ -276,7 +280,7 @@ async def find_actor(
@acm
async def wait_for_actor(
name: str,
registry_addr: AddressTypes | None = None,
registry_addr: UnwrappedAddress | None = None,
) -> AsyncGenerator[Portal, None]:
'''
@ -293,7 +297,7 @@ async def wait_for_actor(
yield peer_portal
return
regaddr: AddressTypes = (
regaddr: UnwrappedAddress = (
registry_addr
or
actor.reg_addrs[0]
@ -310,7 +314,7 @@ async def wait_for_actor(
# get latest registered addr by default?
# TODO: offer multi-portal yields in multi-homed case?
addr: AddressTypes = addrs[-1]
addr: UnwrappedAddress = addrs[-1]
async with _connect_chan(addr) as chan:
async with open_portal(chan) as portal:

View File

@ -37,7 +37,7 @@ from .log import (
from . import _state
from .devx import _debug
from .to_asyncio import run_as_asyncio_guest
from ._addr import AddressTypes
from ._addr import UnwrappedAddress
from ._runtime import (
async_main,
Actor,
@ -53,10 +53,10 @@ log = get_logger(__name__)
def _mp_main(
actor: Actor,
accept_addrs: list[AddressTypes],
accept_addrs: list[UnwrappedAddress],
forkserver_info: tuple[Any, Any, Any, Any, Any],
start_method: SpawnMethodKey,
parent_addr: AddressTypes | None = None,
parent_addr: UnwrappedAddress | None = None,
infect_asyncio: bool = False,
) -> None:
@ -207,7 +207,7 @@ def nest_from_op(
def _trio_main(
actor: Actor,
*,
parent_addr: AddressTypes | None = None,
parent_addr: UnwrappedAddress|None = None,
infect_asyncio: bool = False,
) -> None:

View File

@ -47,10 +47,11 @@ from .ipc import (
_connect_chan,
)
from ._addr import (
AddressTypes,
wrap_address,
UnwrappedAddress,
default_lo_addrs,
mk_uuid,
preferred_transport,
default_lo_addrs
wrap_address,
)
from ._exceptions import is_multi_cancelled
@ -63,10 +64,10 @@ async def open_root_actor(
*,
# defaults are above
registry_addrs: list[AddressTypes]|None = None,
registry_addrs: list[UnwrappedAddress]|None = None,
# defaults are above
arbiter_addr: tuple[AddressTypes]|None = None,
arbiter_addr: tuple[UnwrappedAddress]|None = None,
enable_transports: list[str] = [preferred_transport],
@ -195,7 +196,9 @@ async def open_root_actor(
registry_addrs = [arbiter_addr]
if not registry_addrs:
registry_addrs: list[AddressTypes] = default_lo_addrs(enable_transports)
registry_addrs: list[UnwrappedAddress] = default_lo_addrs(
enable_transports
)
assert registry_addrs
@ -245,10 +248,10 @@ async def open_root_actor(
enable_stack_on_sig()
# closed into below ping task-func
ponged_addrs: list[AddressTypes] = []
ponged_addrs: list[UnwrappedAddress] = []
async def ping_tpt_socket(
addr: AddressTypes,
addr: UnwrappedAddress,
timeout: float = 1,
) -> None:
'''
@ -284,7 +287,7 @@ async def open_root_actor(
addr,
)
trans_bind_addrs: list[AddressTypes] = []
trans_bind_addrs: list[UnwrappedAddress] = []
# Create a new local root-actor instance which IS NOT THE
# REGISTRAR
@ -302,6 +305,7 @@ async def open_root_actor(
actor = Actor(
name=name or 'anonymous',
uuid=mk_uuid(),
registry_addrs=ponged_addrs,
loglevel=loglevel,
enable_modules=enable_modules,
@ -336,7 +340,8 @@ async def open_root_actor(
# https://github.com/goodboy/tractor/issues/296
actor = Arbiter(
name or 'registrar',
name=name or 'registrar',
uuid=mk_uuid(),
registry_addrs=registry_addrs,
loglevel=loglevel,
enable_modules=enable_modules,
@ -462,7 +467,7 @@ def run_daemon(
# runtime kwargs
name: str | None = 'root',
registry_addrs: list[AddressTypes]|None = None,
registry_addrs: list[UnwrappedAddress]|None = None,
start_method: str | None = None,
debug_mode: bool = False,

View File

@ -52,6 +52,7 @@ import sys
from typing import (
Any,
Callable,
Type,
TYPE_CHECKING,
)
import uuid
@ -75,11 +76,12 @@ from tractor.msg import (
)
from .ipc import Channel
from ._addr import (
AddressTypes,
UnwrappedAddress,
Address,
wrap_address,
default_lo_addrs,
get_address_cls,
preferred_transport,
default_lo_addrs
wrap_address,
)
from ._context import (
mk_context,
@ -182,15 +184,15 @@ class Actor:
def __init__(
self,
name: str,
uuid: str,
*,
enable_modules: list[str] = [],
uid: str|None = None,
loglevel: str|None = None,
registry_addrs: list[AddressTypes]|None = None,
registry_addrs: list[UnwrappedAddress]|None = None,
spawn_method: str|None = None,
# TODO: remove!
arbiter_addr: AddressTypes|None = None,
arbiter_addr: UnwrappedAddress|None = None,
) -> None:
'''
@ -199,10 +201,7 @@ class Actor:
'''
self.name = name
self.uid = (
name,
uid or str(uuid.uuid4())
)
self.uid = (name, uuid)
self._cancel_complete = trio.Event()
self._cancel_called_by_remote: tuple[str, tuple]|None = None
@ -230,7 +229,7 @@ class Actor:
DeprecationWarning,
stacklevel=2,
)
registry_addrs: list[AddressTypes] = [arbiter_addr]
registry_addrs: list[UnwrappedAddress] = [arbiter_addr]
# marked by the process spawning backend at startup
# will be None for the parent most process started manually
@ -277,13 +276,13 @@ class Actor:
# when provided, init the registry addresses property from
# input via the validator.
self._reg_addrs: list[AddressTypes] = []
self._reg_addrs: list[UnwrappedAddress] = []
if registry_addrs:
self.reg_addrs: list[AddressTypes] = registry_addrs
self.reg_addrs: list[UnwrappedAddress] = registry_addrs
_state._runtime_vars['_registry_addrs'] = registry_addrs
@property
def reg_addrs(self) -> list[AddressTypes]:
def reg_addrs(self) -> list[UnwrappedAddress]:
'''
List of (socket) addresses for all known (and contactable)
registry actors.
@ -294,7 +293,7 @@ class Actor:
@reg_addrs.setter
def reg_addrs(
self,
addrs: list[AddressTypes],
addrs: list[UnwrappedAddress],
) -> None:
if not addrs:
log.warning(
@ -1023,11 +1022,12 @@ class Actor:
async def _from_parent(
self,
parent_addr: AddressTypes|None,
parent_addr: UnwrappedAddress|None,
) -> tuple[
Channel,
list[AddressTypes]|None,
list[UnwrappedAddress]|None,
list[str]|None, # preferred tpts
]:
'''
Bootstrap this local actor's runtime config from its parent by
@ -1039,30 +1039,58 @@ class Actor:
# Connect back to the parent actor and conduct initial
# handshake. From this point on if we error, we
# attempt to ship the exception back to the parent.
chan = await Channel.from_addr(wrap_address(parent_addr))
chan = await Channel.from_addr(
addr=wrap_address(parent_addr)
)
assert isinstance(chan, Channel)
# TODO: move this into a `Channel.handshake()`?
# Initial handshake: swap names.
await self._do_handshake(chan)
accept_addrs: list[AddressTypes]|None = None
accept_addrs: list[UnwrappedAddress]|None = None
if self._spawn_method == "trio":
# Receive post-spawn runtime state from our parent.
spawnspec: msgtypes.SpawnSpec = await chan.recv()
self._spawn_spec = spawnspec
match spawnspec:
case MsgTypeError():
raise spawnspec
case msgtypes.SpawnSpec():
self._spawn_spec = spawnspec
log.runtime(
'Received runtime spec from parent:\n\n'
log.runtime(
'Received runtime spec from parent:\n\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'
)
# 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[AddressTypes] = spawnspec.bind_addrs
case _:
raise InternalError(
f'Received invalid non-`SpawnSpec` payload !?\n'
f'{spawnspec}\n'
)
# ^^TODO XXX!! when the `SpawnSpec` fails to decode
# the above will raise a `MsgTypeError` which if we
# do NOT ALSO RAISE it will tried to be pprinted in
# the log.runtime() below..
#
# SO we gotta look at how other `chan.recv()` calls
# are wrapped and do the same for this spec receive!
# -[ ] see `._rpc` likely has the answer?
#
# XXX NOTE, can't be called here in subactor
# bc we haven't yet received the
# `SpawnSpec._runtime_vars: dict` which would
# declare whether `debug_mode` is set!
# breakpoint()
# import pdbp; pdbp.set_trace()
accept_addrs: list[UnwrappedAddress] = spawnspec.bind_addrs
# TODO: another `Struct` for rtvs..
rvs: dict[str, Any] = spawnspec._runtime_vars
@ -1154,6 +1182,9 @@ class Actor:
return (
chan,
accept_addrs,
None,
# ^TODO, preferred tpts list from rent!
# -[ ] need to extend the `SpawnSpec` tho!
)
except OSError: # failed to connect
@ -1169,7 +1200,7 @@ class Actor:
self,
handler_nursery: Nursery,
*,
listen_addrs: list[AddressTypes]|None = None,
listen_addrs: list[UnwrappedAddress]|None = None,
task_status: TaskStatus[Nursery] = trio.TASK_STATUS_IGNORED,
) -> None:
@ -1578,7 +1609,7 @@ class Actor:
return False
@property
def accept_addrs(self) -> list[AddressTypes]:
def accept_addrs(self) -> list[UnwrappedAddress]:
'''
All addresses to which the transport-channel server binds
and listens for new connections.
@ -1587,7 +1618,7 @@ class Actor:
return [a.unwrap() for a in self._listen_addrs]
@property
def accept_addr(self) -> AddressTypes:
def accept_addr(self) -> UnwrappedAddress:
'''
Primary address to which the IPC transport server is
bound and listening for new connections.
@ -1639,8 +1670,6 @@ class Actor:
chan.aid = aid
uid: tuple[str, str] = (
# str(value[0]),
# str(value[1])
aid.name,
aid.uuid,
)
@ -1664,7 +1693,7 @@ class Actor:
async def async_main(
actor: Actor,
accept_addrs: AddressTypes|None = None,
accept_addrs: UnwrappedAddress|None = None,
# XXX: currently ``parent_addr`` is only needed for the
# ``multiprocessing`` backend (which pickles state sent to
@ -1673,7 +1702,7 @@ async def async_main(
# change this to a simple ``is_subactor: bool`` which will
# be False when running as root actor and True when as
# a subactor.
parent_addr: AddressTypes|None = None,
parent_addr: UnwrappedAddress|None = None,
task_status: TaskStatus[None] = trio.TASK_STATUS_IGNORED,
) -> None:
@ -1702,16 +1731,31 @@ async def async_main(
(
actor._parent_chan,
set_accept_addr_says_rent,
maybe_preferred_transports_says_rent,
) = await actor._from_parent(parent_addr)
# either it's passed in because we're not a child or
# because we're running in mp mode
accept_addrs: list[UnwrappedAddress] = []
if (
set_accept_addr_says_rent
and
set_accept_addr_says_rent is not None
):
accept_addrs = set_accept_addr_says_rent
else:
enable_transports: list[str] = (
maybe_preferred_transports_says_rent
or
[preferred_transport]
)
for transport_key in enable_transports:
transport_cls: Type[Address] = get_address_cls(
transport_key
)
addr: Address = transport_cls.get_random()
accept_addrs.append(addr.unwrap())
# The "root" nursery ensures the channel with the immediate
# parent is kept alive as a resilient service until
@ -1779,7 +1823,7 @@ async def async_main(
raise
accept_addrs: list[AddressTypes] = actor.accept_addrs
accept_addrs: list[UnwrappedAddress] = actor.accept_addrs
# NOTE: only set the loopback addr for the
# process-tree-global "root" mailbox since
@ -2028,7 +2072,7 @@ class Arbiter(Actor):
self._registry: dict[
tuple[str, str],
AddressTypes,
UnwrappedAddress,
] = {}
self._waiters: dict[
str,
@ -2044,7 +2088,7 @@ class Arbiter(Actor):
self,
name: str,
) -> AddressTypes|None:
) -> UnwrappedAddress|None:
for uid, addr in self._registry.items():
if name in uid:
@ -2055,7 +2099,7 @@ class Arbiter(Actor):
async def get_registry(
self
) -> dict[str, AddressTypes]:
) -> dict[str, UnwrappedAddress]:
'''
Return current name registry.
@ -2075,7 +2119,7 @@ class Arbiter(Actor):
self,
name: str,
) -> list[AddressTypes]:
) -> list[UnwrappedAddress]:
'''
Wait for a particular actor to register.
@ -2083,8 +2127,8 @@ class Arbiter(Actor):
registered.
'''
addrs: list[AddressTypes] = []
addr: AddressTypes
addrs: list[UnwrappedAddress] = []
addr: UnwrappedAddress
mailbox_info: str = 'Actor registry contact infos:\n'
for uid, addr in self._registry.items():
@ -2110,7 +2154,7 @@ class Arbiter(Actor):
async def register_actor(
self,
uid: tuple[str, str],
addr: AddressTypes
addr: UnwrappedAddress
) -> None:
uid = name, hash = (str(uid[0]), str(uid[1]))
waddr: Address = wrap_address(addr)

View File

@ -46,7 +46,7 @@ from tractor._state import (
_runtime_vars,
)
from tractor.log import get_logger
from tractor._addr import AddressTypes
from tractor._addr import UnwrappedAddress
from tractor._portal import Portal
from tractor._runtime import Actor
from tractor._entry import _mp_main
@ -393,8 +393,8 @@ async def new_proc(
errors: dict[tuple[str, str], Exception],
# passed through to actor main
bind_addrs: list[AddressTypes],
parent_addr: AddressTypes,
bind_addrs: list[UnwrappedAddress],
parent_addr: UnwrappedAddress,
_runtime_vars: dict[str, Any], # serialized and sent to _child
*,
@ -432,8 +432,8 @@ async def trio_proc(
errors: dict[tuple[str, str], Exception],
# passed through to actor main
bind_addrs: list[AddressTypes],
parent_addr: AddressTypes,
bind_addrs: list[UnwrappedAddress],
parent_addr: UnwrappedAddress,
_runtime_vars: dict[str, Any], # serialized and sent to _child
*,
infect_asyncio: bool = False,
@ -639,8 +639,8 @@ async def mp_proc(
subactor: Actor,
errors: dict[tuple[str, str], Exception],
# passed through to actor main
bind_addrs: list[AddressTypes],
parent_addr: AddressTypes,
bind_addrs: list[UnwrappedAddress],
parent_addr: UnwrappedAddress,
_runtime_vars: dict[str, Any], # serialized and sent to _child
*,
infect_asyncio: bool = False,

View File

@ -22,7 +22,9 @@ from contextlib import asynccontextmanager as acm
from functools import partial
import inspect
from pprint import pformat
from typing import TYPE_CHECKING
from typing import (
TYPE_CHECKING,
)
import typing
import warnings
@ -31,9 +33,9 @@ import trio
from .devx._debug import maybe_wait_for_debugger
from ._addr import (
AddressTypes,
UnwrappedAddress,
preferred_transport,
get_address_cls
mk_uuid,
)
from ._state import current_actor, is_main_process
from .log import get_logger, get_loglevel
@ -134,7 +136,7 @@ class ActorNursery:
*,
bind_addrs: list[AddressTypes]|None = None,
bind_addrs: list[UnwrappedAddress]|None = None,
rpc_module_paths: list[str]|None = None,
enable_transports: list[str] = [preferred_transport],
enable_modules: list[str]|None = None,
@ -161,12 +163,6 @@ class ActorNursery:
or get_loglevel()
)
if not bind_addrs:
bind_addrs: list[AddressTypes] = [
get_address_cls(transport).get_random().unwrap()
for transport in enable_transports
]
# configure and pass runtime state
_rtv = _state._runtime_vars.copy()
_rtv['_is_root'] = False
@ -189,7 +185,9 @@ class ActorNursery:
enable_modules.extend(rpc_module_paths)
subactor = Actor(
name,
name=name,
uuid=mk_uuid(),
# modules allowed to invoked funcs from
enable_modules=enable_modules,
loglevel=loglevel,
@ -197,7 +195,7 @@ class ActorNursery:
# verbatim relay this actor's registrar addresses
registry_addrs=current_actor().reg_addrs,
)
parent_addr = self._actor.accept_addr
parent_addr: UnwrappedAddress = self._actor.accept_addr
assert parent_addr
# start a task to spawn a process
@ -235,7 +233,7 @@ class ActorNursery:
*,
name: str | None = None,
bind_addrs: AddressTypes|None = None,
bind_addrs: UnwrappedAddress|None = None,
rpc_module_paths: list[str] | None = None,
enable_modules: list[str] | None = None,
loglevel: str | None = None, # set log level per subactor

View File

@ -42,24 +42,15 @@ class MsgpackTCPStream(MsgpackTransport):
address_type = TCPAddress
layer_key: int = 4
# def __init__(
# self,
# stream: trio.SocketStream,
# prefix_size: int = 4,
# codec: CodecType = None,
# ) -> None:
# super().__init__(
# stream,
# prefix_size=prefix_size,
# codec=codec
# )
@property
def maddr(self) -> str:
host, port = self.raddr.unwrap()
return (
# TODO, use `ipaddress` from stdlib to handle
# first detecting which of `ipv4/6` before
# choosing the routing prefix part.
f'/ipv4/{host}'
f'/{self.address_type.name_key}/{port}'
# f'/{self.chan.uid[0]}'
# f'/{self.cid}'
@ -94,12 +85,15 @@ class MsgpackTCPStream(MsgpackTransport):
cls,
stream: trio.SocketStream
) -> tuple[
tuple[str, int],
tuple[str, int]
TCPAddress,
TCPAddress,
]:
# TODO, what types are these?
lsockname = stream.socket.getsockname()
l_sockaddr: tuple[str, int] = tuple(lsockname[:2])
rsockname = stream.socket.getpeername()
r_sockaddr: tuple[str, int] = tuple(rsockname[:2])
return (
TCPAddress.from_addr(tuple(lsockname[:2])),
TCPAddress.from_addr(tuple(rsockname[:2])),
TCPAddress.from_addr(l_sockaddr),
TCPAddress.from_addr(r_sockaddr),
)

View File

@ -18,8 +18,23 @@ Unix Domain Socket implementation of tractor.ipc._transport.MsgTransport protoco
'''
from __future__ import annotations
from pathlib import Path
import os
from socket import (
# socket,
AF_UNIX,
SOCK_STREAM,
SO_PASSCRED,
SO_PEERCRED,
SOL_SOCKET,
)
import struct
import trio
from trio._highlevel_open_unix_stream import (
close_on_error,
has_unix,
)
from tractor.msg import MsgCodec
from tractor.log import get_logger
@ -30,33 +45,80 @@ from tractor.ipc._transport import MsgpackTransport
log = get_logger(__name__)
async def open_unix_socket_w_passcred(
filename: str|bytes|os.PathLike[str]|os.PathLike[bytes],
) -> trio.SocketStream:
'''
Literally the exact same as `trio.open_unix_socket()` except we set the additiona
`socket.SO_PASSCRED` option to ensure the server side (the process calling `accept()`)
can extract the connecting peer's credentials, namely OS specific process
related IDs.
See this SO for "why" the extra opts,
- https://stackoverflow.com/a/7982749
'''
if not has_unix:
raise RuntimeError("Unix sockets are not supported on this platform")
# much more simplified logic vs tcp sockets - one socket type and only one
# possible location to connect to
sock = trio.socket.socket(AF_UNIX, SOCK_STREAM)
sock.setsockopt(SOL_SOCKET, SO_PASSCRED, 1)
with close_on_error(sock):
await sock.connect(os.fspath(filename))
return trio.SocketStream(sock)
def get_peer_info(sock: trio.socket.socket) -> tuple[
int, # pid
int, # uid
int, # guid
]:
'''
Deliver the connecting peer's "credentials"-info as defined in
a very Linux specific way..
For more deats see,
- `man accept`,
- `man unix`,
this great online guide to all things sockets,
- https://beej.us/guide/bgnet/html/split-wide/man-pages.html#setsockoptman
AND this **wonderful SO answer**
- https://stackoverflow.com/a/7982749
'''
creds: bytes = sock.getsockopt(
SOL_SOCKET,
SO_PEERCRED,
struct.calcsize('3i')
)
# i.e a tuple of the fields,
# pid: int, "process"
# uid: int, "user"
# gid: int, "group"
return struct.unpack('3i', creds)
class MsgpackUDSStream(MsgpackTransport):
'''
A ``trio.SocketStream`` delivering ``msgpack`` formatted data
using the ``msgspec`` codec lib.
A `trio.SocketStream` around a Unix-Domain-Socket transport
delivering `msgpack` encoded msgs using the `msgspec` codec lib.
'''
address_type = UDSAddress
layer_key: int = 7
# def __init__(
# self,
# stream: trio.SocketStream,
# prefix_size: int = 4,
# codec: CodecType = None,
# ) -> None:
# super().__init__(
# stream,
# prefix_size=prefix_size,
# codec=codec
# )
layer_key: int = 4
@property
def maddr(self) -> str:
filepath = self.raddr.unwrap()
if not self.raddr:
return '<unknown-peer>'
filepath: Path = Path(self.raddr.unwrap()[0])
return (
f'/ipv4/localhost'
f'/{self.address_type.name_key}/{filepath}'
# f'/{self.chan.uid[0]}'
# f'/{self.cid}'
@ -76,22 +138,72 @@ class MsgpackUDSStream(MsgpackTransport):
codec: MsgCodec|None = None,
**kwargs
) -> MsgpackUDSStream:
stream = await trio.open_unix_socket(
addr.unwrap(),
filepath: Path
pid: int
(
filepath,
pid,
) = addr.unwrap()
# XXX NOTE, we don't need to provide the `.pid` part from
# the addr since the OS does this implicitly! .. lel
# stream = await trio.open_unix_socket(
stream = await open_unix_socket_w_passcred(
str(filepath),
**kwargs
)
return MsgpackUDSStream(
stream = MsgpackUDSStream(
stream,
prefix_size=prefix_size,
codec=codec
)
stream._raddr = addr
return stream
@classmethod
def get_stream_addrs(
cls,
stream: trio.SocketStream
) -> tuple[UDSAddress, UDSAddress]:
return (
UDSAddress.from_addr(stream.socket.getsockname()),
UDSAddress.from_addr(stream.socket.getsockname()),
) -> tuple[
Path,
int,
]:
sock: trio.socket.socket = stream.socket
# NOTE XXX, it's unclear why one or the other ends up being
# `bytes` versus the socket-file-path, i presume it's
# something to do with who is the server (called `.listen()`)?
# maybe could be better implemented using another info-query
# on the socket like,
# https://beej.us/guide/bgnet/html/split-wide/system-calls-or-bust.html#gethostnamewho-am-i
sockname: str|bytes = sock.getsockname()
# https://beej.us/guide/bgnet/html/split-wide/system-calls-or-bust.html#getpeernamewho-are-you
peername: str|bytes = sock.getpeername()
match (peername, sockname):
case (str(), bytes()):
sock_path: Path = Path(peername)
case (bytes(), str()):
sock_path: Path = Path(sockname)
(
pid,
uid,
gid,
) = get_peer_info(sock)
log.info(
f'UDS connection from process {pid!r}\n'
f'>[\n'
f'|_{sock_path}\n'
f' |_pid: {pid}\n'
f' |_uid: {uid}\n'
f' |_gid: {gid}\n'
)
laddr = UDSAddress.from_addr((
sock_path,
os.getpid(),
))
raddr = UDSAddress.from_addr((
sock_path,
pid
))
return (laddr, raddr)

View File

@ -31,6 +31,7 @@ from typing import (
Type,
TypeVar,
TypeAlias,
# TYPE_CHECKING,
Union,
)
@ -47,7 +48,7 @@ from tractor.msg import (
pretty_struct,
)
from tractor.log import get_logger
from tractor._addr import AddressTypes
from tractor._addr import UnwrappedAddress
log = get_logger('tractor.msgspec')
@ -142,9 +143,15 @@ class Aid(
'''
name: str
uuid: str
# TODO: use built-in support for UUIDs?
# -[ ] `uuid.UUID` which has multi-protocol support
# https://jcristharif.com/msgspec/supported-types.html#uuid
# TODO? can/should we extend this field set?
# -[ ] use built-in support for UUIDs? `uuid.UUID` which has
# multi-protocol support
# https://jcristharif.com/msgspec/supported-types.html#uuid
#
# -[ ] as per the `.ipc._uds` / `._addr` comments, maybe we
# should also include at least `.pid` (equiv to port for tcp)
# and/or host-part always?
class SpawnSpec(
@ -168,8 +175,8 @@ class SpawnSpec(
# TODO: not just sockaddr pairs?
# -[ ] abstract into a `TransportAddr` type?
reg_addrs: list[AddressTypes]
bind_addrs: list[AddressTypes]
reg_addrs: list[UnwrappedAddress]
bind_addrs: list[UnwrappedAddress]|None
# TODO: caps based RPC support in the payload?