From 414a8c5b7562d7827822e59709e9b2fec55bbf11 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Wed, 12 Mar 2025 16:13:40 -0300 Subject: [PATCH 01/74] IPC ring bug impl with async read --- default.nix | 18 +++ pyproject.toml | 1 + tests/test_shm.py | 80 +++++++++++ tractor/_shm.py | 299 +++++++++++++++++++++++++++++++++++++++++- tractor/_spawn.py | 12 +- tractor/_supervise.py | 4 + uv.lock | 30 +++++ 7 files changed, 438 insertions(+), 6 deletions(-) create mode 100644 default.nix diff --git a/default.nix b/default.nix new file mode 100644 index 00000000..5a936971 --- /dev/null +++ b/default.nix @@ -0,0 +1,18 @@ +{ pkgs ? import {} }: +let + nativeBuildInputs = with pkgs; [ + stdenv.cc.cc.lib + uv + ]; + +in +pkgs.mkShell { + inherit nativeBuildInputs; + + LD_LIBRARY_PATH = pkgs.lib.makeLibraryPath nativeBuildInputs; + + shellHook = '' + set -e + uv venv .venv --python=3.12 + ''; +} diff --git a/pyproject.toml b/pyproject.toml index b3e9e100..fd67bff2 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -45,6 +45,7 @@ dependencies = [ "pdbp>=1.6,<2", # windows only (from `pdbp`) # typed IPC msging "msgspec>=0.19.0", + "cffi>=1.17.1", ] # ------ project ------ diff --git a/tests/test_shm.py b/tests/test_shm.py index 2b7a382f..db0b1818 100644 --- a/tests/test_shm.py +++ b/tests/test_shm.py @@ -2,7 +2,10 @@ Shared mem primitives and APIs. """ +import time import uuid +import string +import random # import numpy import pytest @@ -11,6 +14,7 @@ import tractor from tractor._shm import ( open_shm_list, attach_shm_list, + EventFD, open_ringbuffer_sender, open_ringbuffer_receiver, ) @@ -165,3 +169,79 @@ def test_parent_writer_child_reader( await portal.cancel_actor() trio.run(main) + + +def random_string(size=256): + return ''.join(random.choice(string.ascii_lowercase) for i in range(size)) + + +async def child_read_shm( + msg_amount: int, + key: str, + write_event_fd: int, + wrap_event_fd: int, + max_bytes: int, +) -> None: + log = tractor.log.get_console_log(level='info') + recvd_msgs = 0 + start_ts = time.time() + async with open_ringbuffer_receiver( + write_event_fd, + wrap_event_fd, + key, + max_bytes=max_bytes + ) as receiver: + while recvd_msgs < msg_amount: + msg = await receiver.receive_some() + msgs = bytes(msg).split(b'\n') + first = msgs[0] + last = msgs[-2] + log.info((receiver.ptr - len(msg), receiver.ptr, first[:10], last[:10])) + recvd_msgs += len(msgs) + + end_ts = time.time() + elapsed = end_ts - start_ts + elapsed_ms = int(elapsed * 1000) + log.info(f'elapsed ms: {elapsed_ms}') + log.info(f'msg/sec: {int(msg_amount / elapsed):,}') + log.info(f'bytes/sec: {int(max_bytes / elapsed):,}') + +def test_ring_buff(): + log = tractor.log.get_console_log(level='info') + msg_amount = 100_000 + log.info(f'generating {msg_amount} messages...') + msgs = [ + f'[{i:08}]: {random_string()}\n'.encode('utf-8') + for i in range(msg_amount) + ] + buf_size = sum((len(m) for m in msgs)) + log.info(f'done! buffer size: {buf_size}') + async def main(): + with ( + EventFD(initval=0) as write_event, + EventFD(initval=0) as wrap_event, + ): + async with ( + tractor.open_nursery() as an, + open_ringbuffer_sender( + write_event.fd, + wrap_event.fd, + max_bytes=buf_size + ) as sender + ): + await an.run_in_actor( + child_read_shm, + msg_amount=msg_amount, + key=sender.key, + write_event_fd=write_event.fd, + wrap_event_fd=wrap_event.fd, + max_bytes=buf_size, + proc_kwargs={ + 'pass_fds': (write_event.fd, wrap_event.fd) + } + ) + for msg in msgs: + await sender.send_all(msg) + + + trio.run(main) diff --git a/tractor/_shm.py b/tractor/_shm.py index f8295105..7c177bc5 100644 --- a/tractor/_shm.py +++ b/tractor/_shm.py @@ -25,6 +25,7 @@ considered optional within the context of this runtime-library. from __future__ import annotations from sys import byteorder import time +import platform from typing import Optional from multiprocessing import shared_memory as shm from multiprocessing.shared_memory import ( @@ -32,7 +33,7 @@ from multiprocessing.shared_memory import ( ShareableList, ) -from msgspec import Struct +from msgspec import Struct, to_builtins import tractor from .log import get_logger @@ -142,7 +143,7 @@ class NDToken(Struct, frozen=True): ).descr def as_msg(self): - return self.to_dict() + return to_builtins(self) @classmethod def from_msg(cls, msg: dict) -> NDToken: @@ -831,3 +832,297 @@ def attach_shm_list( name=key, readonly=readonly, ) + + +if platform.system() == 'Linux': + import os + import errno + import string + import random + from contextlib import asynccontextmanager as acm + + import cffi + import trio + + ffi = cffi.FFI() + + # Declare the C functions and types we plan to use. + # - eventfd: for creating the event file descriptor + # - write: for writing to the file descriptor + # - read: for reading from the file descriptor + # - close: for closing the file descriptor + ffi.cdef( + ''' + int eventfd(unsigned int initval, int flags); + + ssize_t write(int fd, const void *buf, size_t count); + ssize_t read(int fd, void *buf, size_t count); + + int close(int fd); + ''' + ) + + # Open the default dynamic library (essentially 'libc' in most cases) + C = ffi.dlopen(None) + + # Constants from , if needed. + EFD_SEMAPHORE = 1 << 0 # 0x1 + EFD_CLOEXEC = 1 << 1 # 0x2 + EFD_NONBLOCK = 1 << 2 # 0x4 + + + def open_eventfd(initval: int = 0, flags: int = 0) -> int: + ''' + Open an eventfd with the given initial value and flags. + Returns the file descriptor on success, otherwise raises OSError. + ''' + fd = C.eventfd(initval, flags) + if fd < 0: + raise OSError(errno.errorcode[ffi.errno], 'eventfd failed') + return fd + + def write_eventfd(fd: int, value: int) -> int: + ''' + Write a 64-bit integer (uint64_t) to the eventfd's counter. + ''' + # Create a uint64_t* in C, store `value` + data_ptr = ffi.new('uint64_t *', value) + + # Call write(fd, data_ptr, 8) + # We expect to write exactly 8 bytes (sizeof(uint64_t)) + ret = C.write(fd, data_ptr, 8) + if ret < 0: + raise OSError(errno.errorcode[ffi.errno], 'write to eventfd failed') + return ret + + def read_eventfd(fd: int) -> int: + ''' + Read a 64-bit integer (uint64_t) from the eventfd, returning the value. + Reading resets the counter to 0 (unless using EFD_SEMAPHORE). + ''' + # Allocate an 8-byte buffer in C for reading + buf = ffi.new('char[]', 8) + + ret = C.read(fd, buf, 8) + if ret < 0: + raise OSError(errno.errorcode[ffi.errno], 'read from eventfd failed') + # Convert the 8 bytes we read into a Python integer + data_bytes = ffi.unpack(buf, 8) # returns a Python bytes object of length 8 + value = int.from_bytes(data_bytes, byteorder='little', signed=False) + return value + + def close_eventfd(fd: int) -> int: + ''' + Close the eventfd. + ''' + ret = C.close(fd) + if ret < 0: + raise OSError(errno.errorcode[ffi.errno], 'close failed') + + + class EventFD: + + def __init__( + self, + initval: int = 0, + flags: int = 0, + fd: int | None = None, + omode: str = 'r' + ): + self._initval: int = initval + self._flags: int = flags + self._fd: int | None = fd + self._omode: str = omode + self._fobj = None + + @property + def fd(self) -> int | None: + return self._fd + + def write(self, value: int) -> int: + return write_eventfd(self._fd, value) + + async def read(self) -> int: + return await trio.to_thread.run_sync(read_eventfd, self._fd) + + def open(self): + if not self._fd: + self._fd = open_eventfd( + initval=self._initval, flags=self._flags) + + else: + self._fobj = os.fdopen(self._fd, self._omode) + + def close(self): + if self._fobj: + self._fobj.close() + return + + if self._fd: + close_eventfd(self._fd) + + def __enter__(self): + self.open() + return self + + def __exit__(self, exc_type, exc_value, traceback): + self.close() + + + class RingBuffSender(trio.abc.SendStream): + + def __init__( + self, + shm: SharedMemory, + write_event: EventFD, + wrap_event: EventFD, + start_ptr: int = 0 + ): + self._shm: SharedMemory = shm + self._write_event = write_event + self._wrap_event = wrap_event + self._ptr = start_ptr + + @property + def key(self) -> str: + return self._shm.name + + @property + def size(self) -> int: + return self._shm.size + + @property + def ptr(self) -> int: + return self._ptr + + @property + def write_fd(self) -> int: + return self._write_event.fd + + @property + def wrap_fd(self) -> int: + return self._wrap_event.fd + + async def send_all(self, data: bytes | bytearray | memoryview): + target_ptr = self.ptr + len(data) + if target_ptr > self.size: + remaining = self.size - self.ptr + self._shm.buf[self.ptr:] = data[:remaining] + self._write_event.write(remaining) + await self._wrap_event.read() + self._ptr = 0 + data = data[remaining:] + target_ptr = self._ptr + len(data) + + self._shm.buf[self.ptr:target_ptr] = data + self._write_event.write(len(data)) + self._ptr = target_ptr + + async def wait_send_all_might_not_block(self): + ... + + async def aclose(self): + ... + + async def __aenter__(self): + self._write_event.open() + self._wrap_event.open() + return self + + async def __aexit__(self, exc_type, exc_value, traceback): + await self.aclose() + + + class RingBuffReceiver(trio.abc.ReceiveStream): + + def __init__( + self, + shm: SharedMemory, + write_event: EventFD, + wrap_event: EventFD, + start_ptr: int = 0 + ): + self._shm: SharedMemory = shm + self._write_event = write_event + self._wrap_event = wrap_event + self._ptr = start_ptr + + @property + def key(self) -> str: + return self._shm.name + + @property + def size(self) -> int: + return self._shm.size + + @property + def ptr(self) -> int: + return self._ptr + + @property + def write_fd(self) -> int: + return self._write_event.fd + + @property + def wrap_fd(self) -> int: + return self._wrap_event.fd + + async def receive_some(self, max_bytes: int | None = None) -> bytes: + delta = await self._write_event.read() + next_ptr = self._ptr + delta + segment = bytes(self._shm.buf[self._ptr:next_ptr]) + self._ptr = next_ptr + if self.ptr == self.size: + self._ptr = 0 + self._wrap_event.write(1) + return segment + + async def aclose(self): + ... + + async def __aenter__(self): + self._write_event.open() + self._wrap_event.open() + return self + + async def __aexit__(self, exc_type, exc_value, traceback): + await self.aclose() + + @acm + async def open_ringbuffer_sender( + write_event_fd: int, + wrap_event_fd: int, + key: str | None = None, + max_bytes: int = 10 * 1024, + start_ptr: int = 0, + ) -> RingBuffSender: + if not key: + key: str = ''.join(random.choice(string.ascii_lowercase) for i in range(32)) + + shm = SharedMemory( + name=key, + size=max_bytes, + create=True + ) + async with RingBuffSender( + shm, EventFD(fd=write_event_fd, omode='w'), EventFD(fd=wrap_event_fd), start_ptr=start_ptr + ) as s: + yield s + + @acm + async def open_ringbuffer_receiver( + write_event_fd: int, + wrap_event_fd: int, + key: str, + max_bytes: int = 10 * 1024, + start_ptr: int = 0, + ) -> RingBuffSender: + shm = SharedMemory( + name=key, + size=max_bytes, + create=False + ) + async with RingBuffReceiver( + shm, EventFD(fd=write_event_fd), EventFD(fd=wrap_event_fd, omode='w'), start_ptr=start_ptr + ) as r: + yield r diff --git a/tractor/_spawn.py b/tractor/_spawn.py index 3159508d..dc2429d9 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -399,7 +399,8 @@ async def new_proc( *, infect_asyncio: bool = False, - task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED + task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED, + proc_kwargs: dict[str, any] = {} ) -> None: @@ -419,6 +420,7 @@ async def new_proc( _runtime_vars, # run time vars infect_asyncio=infect_asyncio, task_status=task_status, + proc_kwargs=proc_kwargs ) @@ -434,7 +436,8 @@ async def trio_proc( _runtime_vars: dict[str, Any], # serialized and sent to _child *, infect_asyncio: bool = False, - task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED + task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED, + proc_kwargs: dict[str, any] = {} ) -> None: ''' @@ -475,7 +478,7 @@ async def trio_proc( proc: trio.Process|None = None try: try: - proc: trio.Process = await trio.lowlevel.open_process(spawn_cmd) + proc: trio.Process = await trio.lowlevel.open_process(spawn_cmd, **proc_kwargs) log.runtime( 'Started new child\n' f'|_{proc}\n' @@ -640,7 +643,8 @@ async def mp_proc( _runtime_vars: dict[str, Any], # serialized and sent to _child *, infect_asyncio: bool = False, - task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED + task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED, + proc_kwargs: dict[str, any] = {} ) -> None: diff --git a/tractor/_supervise.py b/tractor/_supervise.py index bc6bc983..052a5f4c 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -141,6 +141,7 @@ class ActorNursery: # a `._ria_nursery` since the dependent APIs have been # removed! nursery: trio.Nursery|None = None, + proc_kwargs: dict[str, any] = {} ) -> Portal: ''' @@ -204,6 +205,7 @@ class ActorNursery: parent_addr, _rtv, # run time vars infect_asyncio=infect_asyncio, + proc_kwargs=proc_kwargs ) ) @@ -227,6 +229,7 @@ class ActorNursery: enable_modules: list[str] | None = None, loglevel: str | None = None, # set log level per subactor infect_asyncio: bool = False, + proc_kwargs: dict[str, any] = {}, **kwargs, # explicit args to ``fn`` @@ -257,6 +260,7 @@ class ActorNursery: # use the run_in_actor nursery nursery=self._ria_nursery, infect_asyncio=infect_asyncio, + proc_kwargs=proc_kwargs ) # XXX: don't allow stream funcs diff --git a/uv.lock b/uv.lock index e1c409f5..76b22243 100644 --- a/uv.lock +++ b/uv.lock @@ -20,10 +20,38 @@ dependencies = [ ] sdist = { url = "https://files.pythonhosted.org/packages/fc/97/c783634659c2920c3fc70419e3af40972dbaf758daa229a7d6ea6135c90d/cffi-1.17.1.tar.gz", hash = "sha256:1c39c6016c32bc48dd54561950ebd6836e1670f2ae46128f67cf49e789c52824", size = 516621 } wheels = [ + { url = "https://files.pythonhosted.org/packages/6b/f4/927e3a8899e52a27fa57a48607ff7dc91a9ebe97399b357b85a0c7892e00/cffi-1.17.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:a45e3c6913c5b87b3ff120dcdc03f6131fa0065027d0ed7ee6190736a74cd401", size = 182264 }, + { url = "https://files.pythonhosted.org/packages/6c/f5/6c3a8efe5f503175aaddcbea6ad0d2c96dad6f5abb205750d1b3df44ef29/cffi-1.17.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:30c5e0cb5ae493c04c8b42916e52ca38079f1b235c2f8ae5f4527b963c401caf", size = 178651 }, + { url = "https://files.pythonhosted.org/packages/94/dd/a3f0118e688d1b1a57553da23b16bdade96d2f9bcda4d32e7d2838047ff7/cffi-1.17.1-cp311-cp311-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f75c7ab1f9e4aca5414ed4d8e5c0e303a34f4421f8a0d47a4d019ceff0ab6af4", size = 445259 }, + { url = "https://files.pythonhosted.org/packages/2e/ea/70ce63780f096e16ce8588efe039d3c4f91deb1dc01e9c73a287939c79a6/cffi-1.17.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a1ed2dd2972641495a3ec98445e09766f077aee98a1c896dcb4ad0d303628e41", size = 469200 }, + { url = "https://files.pythonhosted.org/packages/1c/a0/a4fa9f4f781bda074c3ddd57a572b060fa0df7655d2a4247bbe277200146/cffi-1.17.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:46bf43160c1a35f7ec506d254e5c890f3c03648a4dbac12d624e4490a7046cd1", size = 477235 }, + { url = "https://files.pythonhosted.org/packages/62/12/ce8710b5b8affbcdd5c6e367217c242524ad17a02fe5beec3ee339f69f85/cffi-1.17.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a24ed04c8ffd54b0729c07cee15a81d964e6fee0e3d4d342a27b020d22959dc6", size = 459721 }, + { url = "https://files.pythonhosted.org/packages/ff/6b/d45873c5e0242196f042d555526f92aa9e0c32355a1be1ff8c27f077fd37/cffi-1.17.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:610faea79c43e44c71e1ec53a554553fa22321b65fae24889706c0a84d4ad86d", size = 467242 }, + { url = "https://files.pythonhosted.org/packages/1a/52/d9a0e523a572fbccf2955f5abe883cfa8bcc570d7faeee06336fbd50c9fc/cffi-1.17.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:a9b15d491f3ad5d692e11f6b71f7857e7835eb677955c00cc0aefcd0669adaf6", size = 477999 }, + { url = "https://files.pythonhosted.org/packages/44/74/f2a2460684a1a2d00ca799ad880d54652841a780c4c97b87754f660c7603/cffi-1.17.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:de2ea4b5833625383e464549fec1bc395c1bdeeb5f25c4a3a82b5a8c756ec22f", size = 454242 }, + { url = "https://files.pythonhosted.org/packages/f8/4a/34599cac7dfcd888ff54e801afe06a19c17787dfd94495ab0c8d35fe99fb/cffi-1.17.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:fc48c783f9c87e60831201f2cce7f3b2e4846bf4d8728eabe54d60700b318a0b", size = 478604 }, { url = "https://files.pythonhosted.org/packages/34/33/e1b8a1ba29025adbdcda5fb3a36f94c03d771c1b7b12f726ff7fef2ebe36/cffi-1.17.1-cp311-cp311-win32.whl", hash = "sha256:85a950a4ac9c359340d5963966e3e0a94a676bd6245a4b55bc43949eee26a655", size = 171727 }, { url = "https://files.pythonhosted.org/packages/3d/97/50228be003bb2802627d28ec0627837ac0bf35c90cf769812056f235b2d1/cffi-1.17.1-cp311-cp311-win_amd64.whl", hash = "sha256:caaf0640ef5f5517f49bc275eca1406b0ffa6aa184892812030f04c2abf589a0", size = 181400 }, + { url = "https://files.pythonhosted.org/packages/5a/84/e94227139ee5fb4d600a7a4927f322e1d4aea6fdc50bd3fca8493caba23f/cffi-1.17.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:805b4371bf7197c329fcb3ead37e710d1bca9da5d583f5073b799d5c5bd1eee4", size = 183178 }, + { url = "https://files.pythonhosted.org/packages/da/ee/fb72c2b48656111c4ef27f0f91da355e130a923473bf5ee75c5643d00cca/cffi-1.17.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:733e99bc2df47476e3848417c5a4540522f234dfd4ef3ab7fafdf555b082ec0c", size = 178840 }, + { url = "https://files.pythonhosted.org/packages/cc/b6/db007700f67d151abadf508cbfd6a1884f57eab90b1bb985c4c8c02b0f28/cffi-1.17.1-cp312-cp312-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1257bdabf294dceb59f5e70c64a3e2f462c30c7ad68092d01bbbfb1c16b1ba36", size = 454803 }, + { url = "https://files.pythonhosted.org/packages/1a/df/f8d151540d8c200eb1c6fba8cd0dfd40904f1b0682ea705c36e6c2e97ab3/cffi-1.17.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:da95af8214998d77a98cc14e3a3bd00aa191526343078b530ceb0bd710fb48a5", size = 478850 }, + { url = "https://files.pythonhosted.org/packages/28/c0/b31116332a547fd2677ae5b78a2ef662dfc8023d67f41b2a83f7c2aa78b1/cffi-1.17.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d63afe322132c194cf832bfec0dc69a99fb9bb6bbd550f161a49e9e855cc78ff", size = 485729 }, + { url = "https://files.pythonhosted.org/packages/91/2b/9a1ddfa5c7f13cab007a2c9cc295b70fbbda7cb10a286aa6810338e60ea1/cffi-1.17.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f79fc4fc25f1c8698ff97788206bb3c2598949bfe0fef03d299eb1b5356ada99", size = 471256 }, + { url = "https://files.pythonhosted.org/packages/b2/d5/da47df7004cb17e4955df6a43d14b3b4ae77737dff8bf7f8f333196717bf/cffi-1.17.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b62ce867176a75d03a665bad002af8e6d54644fad99a3c70905c543130e39d93", size = 479424 }, + { url = "https://files.pythonhosted.org/packages/0b/ac/2a28bcf513e93a219c8a4e8e125534f4f6db03e3179ba1c45e949b76212c/cffi-1.17.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:386c8bf53c502fff58903061338ce4f4950cbdcb23e2902d86c0f722b786bbe3", size = 484568 }, + { url = "https://files.pythonhosted.org/packages/d4/38/ca8a4f639065f14ae0f1d9751e70447a261f1a30fa7547a828ae08142465/cffi-1.17.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:4ceb10419a9adf4460ea14cfd6bc43d08701f0835e979bf821052f1805850fe8", size = 488736 }, { url = "https://files.pythonhosted.org/packages/86/c5/28b2d6f799ec0bdecf44dced2ec5ed43e0eb63097b0f58c293583b406582/cffi-1.17.1-cp312-cp312-win32.whl", hash = "sha256:a08d7e755f8ed21095a310a693525137cfe756ce62d066e53f502a83dc550f65", size = 172448 }, { url = "https://files.pythonhosted.org/packages/50/b9/db34c4755a7bd1cb2d1603ac3863f22bcecbd1ba29e5ee841a4bc510b294/cffi-1.17.1-cp312-cp312-win_amd64.whl", hash = "sha256:51392eae71afec0d0c8fb1a53b204dbb3bcabcb3c9b807eedf3e1e6ccf2de903", size = 181976 }, + { url = "https://files.pythonhosted.org/packages/8d/f8/dd6c246b148639254dad4d6803eb6a54e8c85c6e11ec9df2cffa87571dbe/cffi-1.17.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:f3a2b4222ce6b60e2e8b337bb9596923045681d71e5a082783484d845390938e", size = 182989 }, + { url = "https://files.pythonhosted.org/packages/8b/f1/672d303ddf17c24fc83afd712316fda78dc6fce1cd53011b839483e1ecc8/cffi-1.17.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:0984a4925a435b1da406122d4d7968dd861c1385afe3b45ba82b750f229811e2", size = 178802 }, + { url = "https://files.pythonhosted.org/packages/0e/2d/eab2e858a91fdff70533cab61dcff4a1f55ec60425832ddfdc9cd36bc8af/cffi-1.17.1-cp313-cp313-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d01b12eeeb4427d3110de311e1774046ad344f5b1a7403101878976ecd7a10f3", size = 454792 }, + { url = "https://files.pythonhosted.org/packages/75/b2/fbaec7c4455c604e29388d55599b99ebcc250a60050610fadde58932b7ee/cffi-1.17.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:706510fe141c86a69c8ddc029c7910003a17353970cff3b904ff0686a5927683", size = 478893 }, + { url = "https://files.pythonhosted.org/packages/4f/b7/6e4a2162178bf1935c336d4da8a9352cccab4d3a5d7914065490f08c0690/cffi-1.17.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:de55b766c7aa2e2a3092c51e0483d700341182f08e67c63630d5b6f200bb28e5", size = 485810 }, + { url = "https://files.pythonhosted.org/packages/c7/8a/1d0e4a9c26e54746dc08c2c6c037889124d4f59dffd853a659fa545f1b40/cffi-1.17.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c59d6e989d07460165cc5ad3c61f9fd8f1b4796eacbd81cee78957842b834af4", size = 471200 }, + { url = "https://files.pythonhosted.org/packages/26/9f/1aab65a6c0db35f43c4d1b4f580e8df53914310afc10ae0397d29d697af4/cffi-1.17.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dd398dbc6773384a17fe0d3e7eeb8d1a21c2200473ee6806bb5e6a8e62bb73dd", size = 479447 }, + { url = "https://files.pythonhosted.org/packages/5f/e4/fb8b3dd8dc0e98edf1135ff067ae070bb32ef9d509d6cb0f538cd6f7483f/cffi-1.17.1-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:3edc8d958eb099c634dace3c7e16560ae474aa3803a5df240542b305d14e14ed", size = 484358 }, + { url = "https://files.pythonhosted.org/packages/f1/47/d7145bf2dc04684935d57d67dff9d6d795b2ba2796806bb109864be3a151/cffi-1.17.1-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:72e72408cad3d5419375fc87d289076ee319835bdfa2caad331e377589aebba9", size = 488469 }, { url = "https://files.pythonhosted.org/packages/bf/ee/f94057fa6426481d663b88637a9a10e859e492c73d0384514a17d78ee205/cffi-1.17.1-cp313-cp313-win32.whl", hash = "sha256:e03eab0a8677fa80d646b5ddece1cbeaf556c313dcfac435ba11f107ba117b5d", size = 172475 }, { url = "https://files.pythonhosted.org/packages/7c/fc/6a8cb64e5f0324877d503c854da15d76c1e50eb722e320b15345c4d0c6de/cffi-1.17.1-cp313-cp313-win_amd64.whl", hash = "sha256:f6a16c31041f09ead72d69f583767292f750d24913dadacf5756b966aacb3f1a", size = 182009 }, ] @@ -321,6 +349,7 @@ name = "tractor" version = "0.1.0a6.dev0" source = { editable = "." } dependencies = [ + { name = "cffi" }, { name = "colorlog" }, { name = "msgspec" }, { name = "pdbp" }, @@ -342,6 +371,7 @@ dev = [ [package.metadata] requires-dist = [ + { name = "cffi", specifier = ">=1.17.1" }, { name = "colorlog", specifier = ">=6.8.2,<7" }, { name = "msgspec", specifier = ">=0.19.0" }, { name = "pdbp", specifier = ">=1.6,<2" }, From 9921ea3cae667c21527ee587c1fc3e1e7f0ea127 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 13 Mar 2025 20:17:04 -0300 Subject: [PATCH 02/74] General improvements EventFD class now expects the fd to already be init with open_eventfd RingBuff Sender and Receiver fully manage SharedMemory and EventFD lifecycles, no aditional ctx mngrs needed Separate ring buf tests into its own test bed Add parametrization to test and cancellation Add docstrings Add simple testing data gen module .samples --- tests/test_ringbuf.py | 212 ++++++++++++++++++++++++++++++++++++ tests/test_shm.py | 80 -------------- tractor/_shm.py | 195 +++++++++++++++++++-------------- tractor/_testing/samples.py | 35 ++++++ 4 files changed, 360 insertions(+), 162 deletions(-) create mode 100644 tests/test_ringbuf.py create mode 100644 tractor/_testing/samples.py diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py new file mode 100644 index 00000000..b81ea5f9 --- /dev/null +++ b/tests/test_ringbuf.py @@ -0,0 +1,212 @@ +import time + +import trio +import pytest +import tractor +from tractor._shm import ( + EFD_NONBLOCK, + open_eventfd, + RingBuffSender, + RingBuffReceiver +) +from tractor._testing.samples import generate_sample_messages + + +@tractor.context +async def child_read_shm( + ctx: tractor.Context, + msg_amount: int, + shm_key: str, + write_eventfd: int, + wrap_eventfd: int, + buf_size: int, + total_bytes: int, + flags: int = 0, +) -> None: + recvd_bytes = 0 + await ctx.started() + start_ts = time.time() + async with RingBuffReceiver( + shm_key, + write_eventfd, + wrap_eventfd, + buf_size=buf_size, + flags=flags + ) as receiver: + while recvd_bytes < total_bytes: + msg = await receiver.receive_some() + recvd_bytes += len(msg) + + # make sure we dont hold any memoryviews + # before the ctx manager aclose() + msg = None + + end_ts = time.time() + elapsed = end_ts - start_ts + elapsed_ms = int(elapsed * 1000) + + print(f'\n\telapsed ms: {elapsed_ms}') + print(f'\tmsg/sec: {int(msg_amount / elapsed):,}') + print(f'\tbytes/sec: {int(recvd_bytes / elapsed):,}') + + +@tractor.context +async def child_write_shm( + ctx: tractor.Context, + msg_amount: int, + rand_min: int, + rand_max: int, + shm_key: str, + write_eventfd: int, + wrap_eventfd: int, + buf_size: int, +) -> None: + msgs, total_bytes = generate_sample_messages( + msg_amount, + rand_min=rand_min, + rand_max=rand_max, + ) + await ctx.started(total_bytes) + async with RingBuffSender( + shm_key, + write_eventfd, + wrap_eventfd, + buf_size=buf_size + ) as sender: + for msg in msgs: + await sender.send_all(msg) + + +@pytest.mark.parametrize( + 'msg_amount,rand_min,rand_max,buf_size', + [ + # simple case, fixed payloads, large buffer + (100_000, 0, 0, 10 * 1024), + + # guaranteed wrap around on every write + (100, 10 * 1024, 20 * 1024, 10 * 1024), + + # large payload size, but large buffer + (10_000, 256 * 1024, 512 * 1024, 10 * 1024 * 1024) + ], + ids=[ + 'fixed_payloads_large_buffer', + 'wrap_around_every_write', + 'large_payloads_large_buffer', + ] +) +def test_ring_buff( + msg_amount: int, + rand_min: int, + rand_max: int, + buf_size: int +): + write_eventfd = open_eventfd() + wrap_eventfd = open_eventfd() + + proc_kwargs = { + 'pass_fds': (write_eventfd, wrap_eventfd) + } + + shm_key = 'test_ring_buff' + + common_kwargs = { + 'msg_amount': msg_amount, + 'shm_key': shm_key, + 'write_eventfd': write_eventfd, + 'wrap_eventfd': wrap_eventfd, + 'buf_size': buf_size + } + + async def main(): + async with tractor.open_nursery() as an: + send_p = await an.start_actor( + 'ring_sender', + enable_modules=[__name__], + proc_kwargs=proc_kwargs + ) + recv_p = await an.start_actor( + 'ring_receiver', + enable_modules=[__name__], + proc_kwargs=proc_kwargs + ) + async with ( + send_p.open_context( + child_write_shm, + rand_min=rand_min, + rand_max=rand_max, + **common_kwargs + ) as (sctx, total_bytes), + recv_p.open_context( + child_read_shm, + **common_kwargs, + total_bytes=total_bytes, + ) as (sctx, _sent), + ): + await recv_p.result() + + await send_p.cancel_actor() + await recv_p.cancel_actor() + + + trio.run(main) + + +@tractor.context +async def child_blocked_receiver( + ctx: tractor.Context, + shm_key: str, + write_eventfd: int, + wrap_eventfd: int, + flags: int = 0 +): + async with RingBuffReceiver( + shm_key, + write_eventfd, + wrap_eventfd, + flags=flags + ) as receiver: + await ctx.started() + await receiver.receive_some() + + +def test_ring_reader_cancel(): + flags = EFD_NONBLOCK + write_eventfd = open_eventfd(flags=flags) + wrap_eventfd = open_eventfd() + + proc_kwargs = { + 'pass_fds': (write_eventfd, wrap_eventfd) + } + + shm_key = 'test_ring_cancel' + + async def main(): + async with ( + tractor.open_nursery() as an, + RingBuffSender( + shm_key, + write_eventfd, + wrap_eventfd, + ) as _sender, + ): + recv_p = await an.start_actor( + 'ring_blocked_receiver', + enable_modules=[__name__], + proc_kwargs=proc_kwargs + ) + async with ( + recv_p.open_context( + child_blocked_receiver, + write_eventfd=write_eventfd, + wrap_eventfd=wrap_eventfd, + shm_key=shm_key, + flags=flags + ) as (sctx, _sent), + ): + await trio.sleep(1) + await an.cancel() + + + with pytest.raises(tractor._exceptions.ContextCancelled): + trio.run(main) diff --git a/tests/test_shm.py b/tests/test_shm.py index db0b1818..2b7a382f 100644 --- a/tests/test_shm.py +++ b/tests/test_shm.py @@ -2,10 +2,7 @@ Shared mem primitives and APIs. """ -import time import uuid -import string -import random # import numpy import pytest @@ -14,7 +11,6 @@ import tractor from tractor._shm import ( open_shm_list, attach_shm_list, - EventFD, open_ringbuffer_sender, open_ringbuffer_receiver, ) @@ -169,79 +165,3 @@ def test_parent_writer_child_reader( await portal.cancel_actor() trio.run(main) - - -def random_string(size=256): - return ''.join(random.choice(string.ascii_lowercase) for i in range(size)) - - -async def child_read_shm( - msg_amount: int, - key: str, - write_event_fd: int, - wrap_event_fd: int, - max_bytes: int, -) -> None: - log = tractor.log.get_console_log(level='info') - recvd_msgs = 0 - start_ts = time.time() - async with open_ringbuffer_receiver( - write_event_fd, - wrap_event_fd, - key, - max_bytes=max_bytes - ) as receiver: - while recvd_msgs < msg_amount: - msg = await receiver.receive_some() - msgs = bytes(msg).split(b'\n') - first = msgs[0] - last = msgs[-2] - log.info((receiver.ptr - len(msg), receiver.ptr, first[:10], last[:10])) - recvd_msgs += len(msgs) - - end_ts = time.time() - elapsed = end_ts - start_ts - elapsed_ms = int(elapsed * 1000) - log.info(f'elapsed ms: {elapsed_ms}') - log.info(f'msg/sec: {int(msg_amount / elapsed):,}') - log.info(f'bytes/sec: {int(max_bytes / elapsed):,}') - -def test_ring_buff(): - log = tractor.log.get_console_log(level='info') - msg_amount = 100_000 - log.info(f'generating {msg_amount} messages...') - msgs = [ - f'[{i:08}]: {random_string()}\n'.encode('utf-8') - for i in range(msg_amount) - ] - buf_size = sum((len(m) for m in msgs)) - log.info(f'done! buffer size: {buf_size}') - async def main(): - with ( - EventFD(initval=0) as write_event, - EventFD(initval=0) as wrap_event, - ): - async with ( - tractor.open_nursery() as an, - open_ringbuffer_sender( - write_event.fd, - wrap_event.fd, - max_bytes=buf_size - ) as sender - ): - await an.run_in_actor( - child_read_shm, - msg_amount=msg_amount, - key=sender.key, - write_event_fd=write_event.fd, - wrap_event_fd=wrap_event.fd, - max_bytes=buf_size, - proc_kwargs={ - 'pass_fds': (write_event.fd, wrap_event.fd) - } - ) - for msg in msgs: - await sender.send_all(msg) - - - trio.run(main) diff --git a/tractor/_shm.py b/tractor/_shm.py index 7c177bc5..5038e77a 100644 --- a/tractor/_shm.py +++ b/tractor/_shm.py @@ -837,8 +837,6 @@ def attach_shm_list( if platform.system() == 'Linux': import os import errno - import string - import random from contextlib import asynccontextmanager as acm import cffi @@ -862,19 +860,21 @@ if platform.system() == 'Linux': ''' ) + # Open the default dynamic library (essentially 'libc' in most cases) C = ffi.dlopen(None) # Constants from , if needed. - EFD_SEMAPHORE = 1 << 0 # 0x1 - EFD_CLOEXEC = 1 << 1 # 0x2 - EFD_NONBLOCK = 1 << 2 # 0x4 + EFD_SEMAPHORE = 1 + EFD_CLOEXEC = 0o2000000 + EFD_NONBLOCK = 0o4000 def open_eventfd(initval: int = 0, flags: int = 0) -> int: ''' Open an eventfd with the given initial value and flags. Returns the file descriptor on success, otherwise raises OSError. + ''' fd = C.eventfd(initval, flags) if fd < 0: @@ -884,6 +884,7 @@ if platform.system() == 'Linux': def write_eventfd(fd: int, value: int) -> int: ''' Write a 64-bit integer (uint64_t) to the eventfd's counter. + ''' # Create a uint64_t* in C, store `value` data_ptr = ffi.new('uint64_t *', value) @@ -899,6 +900,7 @@ if platform.system() == 'Linux': ''' Read a 64-bit integer (uint64_t) from the eventfd, returning the value. Reading resets the counter to 0 (unless using EFD_SEMAPHORE). + ''' # Allocate an 8-byte buffer in C for reading buf = ffi.new('char[]', 8) @@ -914,6 +916,7 @@ if platform.system() == 'Linux': def close_eventfd(fd: int) -> int: ''' Close the eventfd. + ''' ret = C.close(fd) if ret < 0: @@ -921,17 +924,19 @@ if platform.system() == 'Linux': class EventFD: + ''' + Use a previously opened eventfd(2), meant to be used in + sub-actors after root actor opens the eventfds then passes + them through pass_fds + + ''' def __init__( self, - initval: int = 0, - flags: int = 0, - fd: int | None = None, - omode: str = 'r' + fd: int, + omode: str ): - self._initval: int = initval - self._flags: int = flags - self._fd: int | None = fd + self._fd: int = fd self._omode: str = omode self._fobj = None @@ -943,23 +948,15 @@ if platform.system() == 'Linux': return write_eventfd(self._fd, value) async def read(self) -> int: + #TODO: how to handle signals? return await trio.to_thread.run_sync(read_eventfd, self._fd) def open(self): - if not self._fd: - self._fd = open_eventfd( - initval=self._initval, flags=self._flags) - - else: - self._fobj = os.fdopen(self._fd, self._omode) + self._fobj = os.fdopen(self._fd, self._omode) def close(self): if self._fobj: self._fobj.close() - return - - if self._fd: - close_eventfd(self._fd) def __enter__(self): self.open() @@ -970,18 +967,34 @@ if platform.system() == 'Linux': class RingBuffSender(trio.abc.SendStream): + ''' + IPC Reliable Ring Buffer sender side implementation + + `eventfd(2)` is used for wrap around sync, and also to signal + writes to the reader. + + TODO: if blocked on wrap around event wait it will not respond + to signals, fix soon TM + ''' def __init__( self, - shm: SharedMemory, - write_event: EventFD, - wrap_event: EventFD, - start_ptr: int = 0 + shm_key: str, + write_eventfd: int, + wrap_eventfd: int, + start_ptr: int = 0, + buf_size: int = 10 * 1024, + clean_shm_on_exit: bool = True ): - self._shm: SharedMemory = shm - self._write_event = write_event - self._wrap_event = wrap_event + self._shm = SharedMemory( + name=shm_key, + size=buf_size, + create=True + ) + self._write_event = EventFD(write_eventfd, 'w') + self._wrap_event = EventFD(wrap_eventfd, 'r') self._ptr = start_ptr + self.clean_shm_on_exit = clean_shm_on_exit @property def key(self) -> str: @@ -1004,25 +1017,37 @@ if platform.system() == 'Linux': return self._wrap_event.fd async def send_all(self, data: bytes | bytearray | memoryview): + # while data is larger than the remaining buf target_ptr = self.ptr + len(data) - if target_ptr > self.size: + while target_ptr > self.size: + # write all bytes that fit remaining = self.size - self.ptr self._shm.buf[self.ptr:] = data[:remaining] + # signal write and wait for reader wrap around self._write_event.write(remaining) await self._wrap_event.read() + + # wrap around and trim already written bytes self._ptr = 0 data = data[remaining:] target_ptr = self._ptr + len(data) + # remaining data fits on buffer self._shm.buf[self.ptr:target_ptr] = data self._write_event.write(len(data)) self._ptr = target_ptr async def wait_send_all_might_not_block(self): - ... + raise NotImplementedError async def aclose(self): - ... + self._write_event.close() + self._wrap_event.close() + if self.clean_shm_on_exit: + self._shm.unlink() + + else: + self._shm.close() async def __aenter__(self): self._write_event.open() @@ -1034,18 +1059,37 @@ if platform.system() == 'Linux': class RingBuffReceiver(trio.abc.ReceiveStream): + ''' + IPC Reliable Ring Buffer receiver side implementation + + `eventfd(2)` is used for wrap around sync, and also to signal + writes to the reader. + + Unless eventfd(2) object is opened with EFD_NONBLOCK flag, + calls to `receive_some` will block the signal handling, + on the main thread, for now solution is using polling, + working on a way to unblock GIL during read(2) to allow + signal processing on the main thread. + ''' def __init__( self, - shm: SharedMemory, - write_event: EventFD, - wrap_event: EventFD, - start_ptr: int = 0 + shm_key: str, + write_eventfd: int, + wrap_eventfd: int, + start_ptr: int = 0, + buf_size: int = 10 * 1024, + flags: int = 0 ): - self._shm: SharedMemory = shm - self._write_event = write_event - self._wrap_event = wrap_event + self._shm = SharedMemory( + name=shm_key, + size=buf_size, + create=False + ) + self._write_event = EventFD(write_eventfd, 'w') + self._wrap_event = EventFD(wrap_eventfd, 'r') self._ptr = start_ptr + self._flags = flags @property def key(self) -> str: @@ -1067,18 +1111,44 @@ if platform.system() == 'Linux': def wrap_fd(self) -> int: return self._wrap_event.fd - async def receive_some(self, max_bytes: int | None = None) -> bytes: - delta = await self._write_event.read() + async def receive_some( + self, + max_bytes: int | None = None, + nb_timeout: float = 0.1 + ) -> memoryview: + # if non blocking eventfd enabled, do polling + # until next write, this allows signal handling + if self._flags | EFD_NONBLOCK: + delta = None + while delta is None: + try: + delta = await self._write_event.read() + + except OSError as e: + if e.errno == 'EAGAIN': + continue + + raise e + + else: + delta = await self._write_event.read() + + # fetch next segment and advance ptr next_ptr = self._ptr + delta - segment = bytes(self._shm.buf[self._ptr:next_ptr]) + segment = self._shm.buf[self._ptr:next_ptr] self._ptr = next_ptr + if self.ptr == self.size: + # reached the end, signal wrap around self._ptr = 0 self._wrap_event.write(1) + return segment async def aclose(self): - ... + self._write_event.close() + self._wrap_event.close() + self._shm.close() async def __aenter__(self): self._write_event.open() @@ -1087,42 +1157,3 @@ if platform.system() == 'Linux': async def __aexit__(self, exc_type, exc_value, traceback): await self.aclose() - - @acm - async def open_ringbuffer_sender( - write_event_fd: int, - wrap_event_fd: int, - key: str | None = None, - max_bytes: int = 10 * 1024, - start_ptr: int = 0, - ) -> RingBuffSender: - if not key: - key: str = ''.join(random.choice(string.ascii_lowercase) for i in range(32)) - - shm = SharedMemory( - name=key, - size=max_bytes, - create=True - ) - async with RingBuffSender( - shm, EventFD(fd=write_event_fd, omode='w'), EventFD(fd=wrap_event_fd), start_ptr=start_ptr - ) as s: - yield s - - @acm - async def open_ringbuffer_receiver( - write_event_fd: int, - wrap_event_fd: int, - key: str, - max_bytes: int = 10 * 1024, - start_ptr: int = 0, - ) -> RingBuffSender: - shm = SharedMemory( - name=key, - size=max_bytes, - create=False - ) - async with RingBuffReceiver( - shm, EventFD(fd=write_event_fd), EventFD(fd=wrap_event_fd, omode='w'), start_ptr=start_ptr - ) as r: - yield r diff --git a/tractor/_testing/samples.py b/tractor/_testing/samples.py new file mode 100644 index 00000000..a87a22c4 --- /dev/null +++ b/tractor/_testing/samples.py @@ -0,0 +1,35 @@ +import os +import random + + +def generate_sample_messages( + amount: int, + rand_min: int = 0, + rand_max: int = 0, + silent: bool = False +) -> tuple[list[bytes], int]: + + msgs = [] + size = 0 + + if not silent: + print(f'\ngenerating {amount} messages...') + + for i in range(amount): + msg = f'[{i:08}]'.encode('utf-8') + + if rand_max > 0: + msg += os.urandom( + random.randint(rand_min, rand_max)) + + size += len(msg) + + msgs.append(msg) + + if not silent and i and i % 10_000 == 0: + print(f'{i} generated') + + if not silent: + print(f'done, {size:,} bytes in total') + + return msgs, size From eceb29241599e941b232350c6f07747acea29a7b Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 13 Mar 2025 20:41:30 -0300 Subject: [PATCH 03/74] move tractor._ipc.py into tractor.ipc._chan.py --- tractor/__init__.py | 2 +- tractor/_context.py | 4 ++-- tractor/_discovery.py | 2 +- tractor/_exceptions.py | 2 +- tractor/_portal.py | 2 +- tractor/_root.py | 2 +- tractor/_rpc.py | 2 +- tractor/_runtime.py | 2 +- tractor/_streaming.py | 2 +- tractor/devx/_debug.py | 2 +- tractor/ipc/__init__.py | 11 +++++++++++ tractor/{_ipc.py => ipc/_chan.py} | 0 tractor/log.py | 4 ++-- 13 files changed, 24 insertions(+), 13 deletions(-) create mode 100644 tractor/ipc/__init__.py rename tractor/{_ipc.py => ipc/_chan.py} (100%) diff --git a/tractor/__init__.py b/tractor/__init__.py index 0c011a22..6fac747f 100644 --- a/tractor/__init__.py +++ b/tractor/__init__.py @@ -64,7 +64,7 @@ from ._root import ( run_daemon as run_daemon, open_root_actor as open_root_actor, ) -from ._ipc import Channel as Channel +from .ipc import Channel as Channel from ._portal import Portal as Portal from ._runtime import Actor as Actor # from . import hilevel as hilevel diff --git a/tractor/_context.py b/tractor/_context.py index 201e920a..d93d7759 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -89,7 +89,7 @@ from .msg import ( pretty_struct, _ops as msgops, ) -from ._ipc import ( +from .ipc import ( Channel, ) from ._streaming import ( @@ -105,7 +105,7 @@ from ._state import ( if TYPE_CHECKING: from ._portal import Portal from ._runtime import Actor - from ._ipc import MsgTransport + from .ipc import MsgTransport from .devx._frame_stack import ( CallerInfo, ) diff --git a/tractor/_discovery.py b/tractor/_discovery.py index 442dd77a..7dcfeaaa 100644 --- a/tractor/_discovery.py +++ b/tractor/_discovery.py @@ -29,7 +29,7 @@ from contextlib import asynccontextmanager as acm from tractor.log import get_logger from .trionics import gather_contexts -from ._ipc import _connect_chan, Channel +from .ipc import _connect_chan, Channel from ._portal import ( Portal, open_portal, diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index f9e18e18..8442ecfd 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -65,7 +65,7 @@ if TYPE_CHECKING: from ._context import Context from .log import StackLevelAdapter from ._stream import MsgStream - from ._ipc import Channel + from .ipc import Channel log = get_logger('tractor') diff --git a/tractor/_portal.py b/tractor/_portal.py index cee10c47..c8a781a7 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -43,7 +43,7 @@ from .trionics import maybe_open_nursery from ._state import ( current_actor, ) -from ._ipc import Channel +from .ipc import Channel from .log import get_logger from .msg import ( # Error, diff --git a/tractor/_root.py b/tractor/_root.py index 2a9beaa3..35639c15 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -43,7 +43,7 @@ from .devx import _debug from . import _spawn from . import _state from . import log -from ._ipc import _connect_chan +from .ipc import _connect_chan from ._exceptions import is_multi_cancelled diff --git a/tractor/_rpc.py b/tractor/_rpc.py index c5daed9e..6dfecd38 100644 --- a/tractor/_rpc.py +++ b/tractor/_rpc.py @@ -42,7 +42,7 @@ from trio import ( TaskStatus, ) -from ._ipc import Channel +from .ipc import Channel from ._context import ( Context, ) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 890a690a..2c8dbbd9 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -73,7 +73,7 @@ from tractor.msg import ( pretty_struct, types as msgtypes, ) -from ._ipc import Channel +from .ipc import Channel from ._context import ( mk_context, Context, diff --git a/tractor/_streaming.py b/tractor/_streaming.py index 2ff2d41c..21e59214 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -56,7 +56,7 @@ from tractor.msg import ( if TYPE_CHECKING: from ._runtime import Actor from ._context import Context - from ._ipc import Channel + from .ipc import Channel log = get_logger(__name__) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index c6ca1d89..b95640dc 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -91,7 +91,7 @@ from tractor._state import ( if TYPE_CHECKING: from trio.lowlevel import Task from threading import Thread - from tractor._ipc import Channel + from tractor.ipc import Channel from tractor._runtime import ( Actor, ) diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py new file mode 100644 index 00000000..d056acaa --- /dev/null +++ b/tractor/ipc/__init__.py @@ -0,0 +1,11 @@ +from ._chan import ( + _connect_chan, + MsgTransport, + Channel +) + +__all__ = [ + '_connect_chan', + 'MsgTransport', + 'Channel' +] diff --git a/tractor/_ipc.py b/tractor/ipc/_chan.py similarity index 100% rename from tractor/_ipc.py rename to tractor/ipc/_chan.py diff --git a/tractor/log.py b/tractor/log.py index 74e0321b..48b5cbd4 100644 --- a/tractor/log.py +++ b/tractor/log.py @@ -92,7 +92,7 @@ class StackLevelAdapter(LoggerAdapter): ) -> None: ''' IPC transport level msg IO; generally anything below - `._ipc.Channel` and friends. + `.ipc.Channel` and friends. ''' return self.log(5, msg) @@ -285,7 +285,7 @@ def get_logger( # NOTE: for handling for modules that use ``get_logger(__name__)`` # we make the following stylistic choice: # - always avoid duplicate project-package token - # in msg output: i.e. tractor.tractor _ipc.py in header + # in msg output: i.e. tractor.tractor.ipc._chan.py in header # looks ridiculous XD # - never show the leaf module name in the {name} part # since in python the {filename} is always this same From 8e3f581d3f10cb1e38f28556c555a0d9f18945bd Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 13 Mar 2025 20:59:14 -0300 Subject: [PATCH 04/74] Move tractor._shm to tractor.ipc._shm --- tests/test_ringbuf.py | 2 +- tests/test_shm.py | 2 +- tractor/ipc/__init__.py | 2 +- tractor/{ => ipc}/_shm.py | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) rename tractor/{ => ipc}/_shm.py (99%) diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index b81ea5f9..e4011768 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -3,7 +3,7 @@ import time import trio import pytest import tractor -from tractor._shm import ( +from tractor.ipc._shm import ( EFD_NONBLOCK, open_eventfd, RingBuffSender, diff --git a/tests/test_shm.py b/tests/test_shm.py index 2b7a382f..ddeb67aa 100644 --- a/tests/test_shm.py +++ b/tests/test_shm.py @@ -8,7 +8,7 @@ import uuid import pytest import trio import tractor -from tractor._shm import ( +from tractor.ipc._shm import ( open_shm_list, attach_shm_list, ) diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index d056acaa..2a401bf6 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -7,5 +7,5 @@ from ._chan import ( __all__ = [ '_connect_chan', 'MsgTransport', - 'Channel' + 'Channel', ] diff --git a/tractor/_shm.py b/tractor/ipc/_shm.py similarity index 99% rename from tractor/_shm.py rename to tractor/ipc/_shm.py index 5038e77a..752f81ff 100644 --- a/tractor/_shm.py +++ b/tractor/ipc/_shm.py @@ -36,7 +36,7 @@ from multiprocessing.shared_memory import ( from msgspec import Struct, to_builtins import tractor -from .log import get_logger +from tractor.log import get_logger _USE_POSIX = getattr(shm, '_USE_POSIX', False) From af69272d160b40b7632861f6a3ee594a8ac6a5df Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 13 Mar 2025 21:10:23 -0300 Subject: [PATCH 05/74] Move linux specifics from tractor.ipc._shm into tractor.ipc._linux --- tests/test_ringbuf.py | 2 +- tractor/ipc/__init__.py | 26 +++- tractor/ipc/_linux.py | 324 +++++++++++++++++++++++++++++++++++++++ tractor/ipc/_shm.py | 326 ---------------------------------------- 4 files changed, 343 insertions(+), 335 deletions(-) create mode 100644 tractor/ipc/_linux.py diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index e4011768..1c4e88f9 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -3,7 +3,7 @@ import time import trio import pytest import tractor -from tractor.ipc._shm import ( +from tractor.ipc import ( EFD_NONBLOCK, open_eventfd, RingBuffSender, diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index 2a401bf6..1b548b65 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -1,11 +1,21 @@ +import platform + from ._chan import ( - _connect_chan, - MsgTransport, - Channel + _connect_chan as _connect_chan, + MsgTransport as MsgTransport, + Channel as Channel ) -__all__ = [ - '_connect_chan', - 'MsgTransport', - 'Channel', -] +if platform.system() == 'Linux': + from ._linux import ( + EFD_SEMAPHORE as EFD_SEMAPHORE, + EFD_CLOEXEC as EFD_CLOEXEC, + EFD_NONBLOCK as EFD_NONBLOCK, + open_eventfd as open_eventfd, + write_eventfd as write_eventfd, + read_eventfd as read_eventfd, + close_eventfd as close_eventfd, + EventFD as EventFD, + RingBuffSender as RingBuffSender, + RingBuffReceiver as RingBuffReceiver + ) diff --git a/tractor/ipc/_linux.py b/tractor/ipc/_linux.py new file mode 100644 index 00000000..2a9eabc1 --- /dev/null +++ b/tractor/ipc/_linux.py @@ -0,0 +1,324 @@ + +import os +import errno + +from multiprocessing.shared_memory import SharedMemory + +import cffi +import trio + +ffi = cffi.FFI() + +# Declare the C functions and types we plan to use. +# - eventfd: for creating the event file descriptor +# - write: for writing to the file descriptor +# - read: for reading from the file descriptor +# - close: for closing the file descriptor +ffi.cdef( + ''' + int eventfd(unsigned int initval, int flags); + + ssize_t write(int fd, const void *buf, size_t count); + ssize_t read(int fd, void *buf, size_t count); + + int close(int fd); + ''' +) + + +# Open the default dynamic library (essentially 'libc' in most cases) +C = ffi.dlopen(None) + +# Constants from , if needed. +EFD_SEMAPHORE = 1 +EFD_CLOEXEC = 0o2000000 +EFD_NONBLOCK = 0o4000 + + +def open_eventfd(initval: int = 0, flags: int = 0) -> int: + ''' + Open an eventfd with the given initial value and flags. + Returns the file descriptor on success, otherwise raises OSError. + + ''' + fd = C.eventfd(initval, flags) + if fd < 0: + raise OSError(errno.errorcode[ffi.errno], 'eventfd failed') + return fd + +def write_eventfd(fd: int, value: int) -> int: + ''' + Write a 64-bit integer (uint64_t) to the eventfd's counter. + + ''' + # Create a uint64_t* in C, store `value` + data_ptr = ffi.new('uint64_t *', value) + + # Call write(fd, data_ptr, 8) + # We expect to write exactly 8 bytes (sizeof(uint64_t)) + ret = C.write(fd, data_ptr, 8) + if ret < 0: + raise OSError(errno.errorcode[ffi.errno], 'write to eventfd failed') + return ret + +def read_eventfd(fd: int) -> int: + ''' + Read a 64-bit integer (uint64_t) from the eventfd, returning the value. + Reading resets the counter to 0 (unless using EFD_SEMAPHORE). + + ''' + # Allocate an 8-byte buffer in C for reading + buf = ffi.new('char[]', 8) + + ret = C.read(fd, buf, 8) + if ret < 0: + raise OSError(errno.errorcode[ffi.errno], 'read from eventfd failed') + # Convert the 8 bytes we read into a Python integer + data_bytes = ffi.unpack(buf, 8) # returns a Python bytes object of length 8 + value = int.from_bytes(data_bytes, byteorder='little', signed=False) + return value + +def close_eventfd(fd: int) -> int: + ''' + Close the eventfd. + + ''' + ret = C.close(fd) + if ret < 0: + raise OSError(errno.errorcode[ffi.errno], 'close failed') + + +class EventFD: + ''' + Use a previously opened eventfd(2), meant to be used in + sub-actors after root actor opens the eventfds then passes + them through pass_fds + + ''' + + def __init__( + self, + fd: int, + omode: str + ): + self._fd: int = fd + self._omode: str = omode + self._fobj = None + + @property + def fd(self) -> int | None: + return self._fd + + def write(self, value: int) -> int: + return write_eventfd(self._fd, value) + + async def read(self) -> int: + #TODO: how to handle signals? + return await trio.to_thread.run_sync(read_eventfd, self._fd) + + def open(self): + self._fobj = os.fdopen(self._fd, self._omode) + + def close(self): + if self._fobj: + self._fobj.close() + + def __enter__(self): + self.open() + return self + + def __exit__(self, exc_type, exc_value, traceback): + self.close() + + +class RingBuffSender(trio.abc.SendStream): + ''' + IPC Reliable Ring Buffer sender side implementation + + `eventfd(2)` is used for wrap around sync, and also to signal + writes to the reader. + + TODO: if blocked on wrap around event wait it will not respond + to signals, fix soon TM + ''' + + def __init__( + self, + shm_key: str, + write_eventfd: int, + wrap_eventfd: int, + start_ptr: int = 0, + buf_size: int = 10 * 1024, + clean_shm_on_exit: bool = True + ): + self._shm = SharedMemory( + name=shm_key, + size=buf_size, + create=True + ) + self._write_event = EventFD(write_eventfd, 'w') + self._wrap_event = EventFD(wrap_eventfd, 'r') + self._ptr = start_ptr + self.clean_shm_on_exit = clean_shm_on_exit + + @property + def key(self) -> str: + return self._shm.name + + @property + def size(self) -> int: + return self._shm.size + + @property + def ptr(self) -> int: + return self._ptr + + @property + def write_fd(self) -> int: + return self._write_event.fd + + @property + def wrap_fd(self) -> int: + return self._wrap_event.fd + + async def send_all(self, data: bytes | bytearray | memoryview): + # while data is larger than the remaining buf + target_ptr = self.ptr + len(data) + while target_ptr > self.size: + # write all bytes that fit + remaining = self.size - self.ptr + self._shm.buf[self.ptr:] = data[:remaining] + # signal write and wait for reader wrap around + self._write_event.write(remaining) + await self._wrap_event.read() + + # wrap around and trim already written bytes + self._ptr = 0 + data = data[remaining:] + target_ptr = self._ptr + len(data) + + # remaining data fits on buffer + self._shm.buf[self.ptr:target_ptr] = data + self._write_event.write(len(data)) + self._ptr = target_ptr + + async def wait_send_all_might_not_block(self): + raise NotImplementedError + + async def aclose(self): + self._write_event.close() + self._wrap_event.close() + if self.clean_shm_on_exit: + self._shm.unlink() + + else: + self._shm.close() + + async def __aenter__(self): + self._write_event.open() + self._wrap_event.open() + return self + + async def __aexit__(self, exc_type, exc_value, traceback): + await self.aclose() + + +class RingBuffReceiver(trio.abc.ReceiveStream): + ''' + IPC Reliable Ring Buffer receiver side implementation + + `eventfd(2)` is used for wrap around sync, and also to signal + writes to the reader. + + Unless eventfd(2) object is opened with EFD_NONBLOCK flag, + calls to `receive_some` will block the signal handling, + on the main thread, for now solution is using polling, + working on a way to unblock GIL during read(2) to allow + signal processing on the main thread. + ''' + + def __init__( + self, + shm_key: str, + write_eventfd: int, + wrap_eventfd: int, + start_ptr: int = 0, + buf_size: int = 10 * 1024, + flags: int = 0 + ): + self._shm = SharedMemory( + name=shm_key, + size=buf_size, + create=False + ) + self._write_event = EventFD(write_eventfd, 'w') + self._wrap_event = EventFD(wrap_eventfd, 'r') + self._ptr = start_ptr + self._flags = flags + + @property + def key(self) -> str: + return self._shm.name + + @property + def size(self) -> int: + return self._shm.size + + @property + def ptr(self) -> int: + return self._ptr + + @property + def write_fd(self) -> int: + return self._write_event.fd + + @property + def wrap_fd(self) -> int: + return self._wrap_event.fd + + async def receive_some( + self, + max_bytes: int | None = None, + nb_timeout: float = 0.1 + ) -> memoryview: + # if non blocking eventfd enabled, do polling + # until next write, this allows signal handling + if self._flags | EFD_NONBLOCK: + delta = None + while delta is None: + try: + delta = await self._write_event.read() + + except OSError as e: + if e.errno == 'EAGAIN': + continue + + raise e + + else: + delta = await self._write_event.read() + + # fetch next segment and advance ptr + next_ptr = self._ptr + delta + segment = self._shm.buf[self._ptr:next_ptr] + self._ptr = next_ptr + + if self.ptr == self.size: + # reached the end, signal wrap around + self._ptr = 0 + self._wrap_event.write(1) + + return segment + + async def aclose(self): + self._write_event.close() + self._wrap_event.close() + self._shm.close() + + async def __aenter__(self): + self._write_event.open() + self._wrap_event.open() + return self + + async def __aexit__(self, exc_type, exc_value, traceback): + await self.aclose() diff --git a/tractor/ipc/_shm.py b/tractor/ipc/_shm.py index 752f81ff..0ee8bf23 100644 --- a/tractor/ipc/_shm.py +++ b/tractor/ipc/_shm.py @@ -25,7 +25,6 @@ considered optional within the context of this runtime-library. from __future__ import annotations from sys import byteorder import time -import platform from typing import Optional from multiprocessing import shared_memory as shm from multiprocessing.shared_memory import ( @@ -832,328 +831,3 @@ def attach_shm_list( name=key, readonly=readonly, ) - - -if platform.system() == 'Linux': - import os - import errno - from contextlib import asynccontextmanager as acm - - import cffi - import trio - - ffi = cffi.FFI() - - # Declare the C functions and types we plan to use. - # - eventfd: for creating the event file descriptor - # - write: for writing to the file descriptor - # - read: for reading from the file descriptor - # - close: for closing the file descriptor - ffi.cdef( - ''' - int eventfd(unsigned int initval, int flags); - - ssize_t write(int fd, const void *buf, size_t count); - ssize_t read(int fd, void *buf, size_t count); - - int close(int fd); - ''' - ) - - - # Open the default dynamic library (essentially 'libc' in most cases) - C = ffi.dlopen(None) - - # Constants from , if needed. - EFD_SEMAPHORE = 1 - EFD_CLOEXEC = 0o2000000 - EFD_NONBLOCK = 0o4000 - - - def open_eventfd(initval: int = 0, flags: int = 0) -> int: - ''' - Open an eventfd with the given initial value and flags. - Returns the file descriptor on success, otherwise raises OSError. - - ''' - fd = C.eventfd(initval, flags) - if fd < 0: - raise OSError(errno.errorcode[ffi.errno], 'eventfd failed') - return fd - - def write_eventfd(fd: int, value: int) -> int: - ''' - Write a 64-bit integer (uint64_t) to the eventfd's counter. - - ''' - # Create a uint64_t* in C, store `value` - data_ptr = ffi.new('uint64_t *', value) - - # Call write(fd, data_ptr, 8) - # We expect to write exactly 8 bytes (sizeof(uint64_t)) - ret = C.write(fd, data_ptr, 8) - if ret < 0: - raise OSError(errno.errorcode[ffi.errno], 'write to eventfd failed') - return ret - - def read_eventfd(fd: int) -> int: - ''' - Read a 64-bit integer (uint64_t) from the eventfd, returning the value. - Reading resets the counter to 0 (unless using EFD_SEMAPHORE). - - ''' - # Allocate an 8-byte buffer in C for reading - buf = ffi.new('char[]', 8) - - ret = C.read(fd, buf, 8) - if ret < 0: - raise OSError(errno.errorcode[ffi.errno], 'read from eventfd failed') - # Convert the 8 bytes we read into a Python integer - data_bytes = ffi.unpack(buf, 8) # returns a Python bytes object of length 8 - value = int.from_bytes(data_bytes, byteorder='little', signed=False) - return value - - def close_eventfd(fd: int) -> int: - ''' - Close the eventfd. - - ''' - ret = C.close(fd) - if ret < 0: - raise OSError(errno.errorcode[ffi.errno], 'close failed') - - - class EventFD: - ''' - Use a previously opened eventfd(2), meant to be used in - sub-actors after root actor opens the eventfds then passes - them through pass_fds - - ''' - - def __init__( - self, - fd: int, - omode: str - ): - self._fd: int = fd - self._omode: str = omode - self._fobj = None - - @property - def fd(self) -> int | None: - return self._fd - - def write(self, value: int) -> int: - return write_eventfd(self._fd, value) - - async def read(self) -> int: - #TODO: how to handle signals? - return await trio.to_thread.run_sync(read_eventfd, self._fd) - - def open(self): - self._fobj = os.fdopen(self._fd, self._omode) - - def close(self): - if self._fobj: - self._fobj.close() - - def __enter__(self): - self.open() - return self - - def __exit__(self, exc_type, exc_value, traceback): - self.close() - - - class RingBuffSender(trio.abc.SendStream): - ''' - IPC Reliable Ring Buffer sender side implementation - - `eventfd(2)` is used for wrap around sync, and also to signal - writes to the reader. - - TODO: if blocked on wrap around event wait it will not respond - to signals, fix soon TM - ''' - - def __init__( - self, - shm_key: str, - write_eventfd: int, - wrap_eventfd: int, - start_ptr: int = 0, - buf_size: int = 10 * 1024, - clean_shm_on_exit: bool = True - ): - self._shm = SharedMemory( - name=shm_key, - size=buf_size, - create=True - ) - self._write_event = EventFD(write_eventfd, 'w') - self._wrap_event = EventFD(wrap_eventfd, 'r') - self._ptr = start_ptr - self.clean_shm_on_exit = clean_shm_on_exit - - @property - def key(self) -> str: - return self._shm.name - - @property - def size(self) -> int: - return self._shm.size - - @property - def ptr(self) -> int: - return self._ptr - - @property - def write_fd(self) -> int: - return self._write_event.fd - - @property - def wrap_fd(self) -> int: - return self._wrap_event.fd - - async def send_all(self, data: bytes | bytearray | memoryview): - # while data is larger than the remaining buf - target_ptr = self.ptr + len(data) - while target_ptr > self.size: - # write all bytes that fit - remaining = self.size - self.ptr - self._shm.buf[self.ptr:] = data[:remaining] - # signal write and wait for reader wrap around - self._write_event.write(remaining) - await self._wrap_event.read() - - # wrap around and trim already written bytes - self._ptr = 0 - data = data[remaining:] - target_ptr = self._ptr + len(data) - - # remaining data fits on buffer - self._shm.buf[self.ptr:target_ptr] = data - self._write_event.write(len(data)) - self._ptr = target_ptr - - async def wait_send_all_might_not_block(self): - raise NotImplementedError - - async def aclose(self): - self._write_event.close() - self._wrap_event.close() - if self.clean_shm_on_exit: - self._shm.unlink() - - else: - self._shm.close() - - async def __aenter__(self): - self._write_event.open() - self._wrap_event.open() - return self - - async def __aexit__(self, exc_type, exc_value, traceback): - await self.aclose() - - - class RingBuffReceiver(trio.abc.ReceiveStream): - ''' - IPC Reliable Ring Buffer receiver side implementation - - `eventfd(2)` is used for wrap around sync, and also to signal - writes to the reader. - - Unless eventfd(2) object is opened with EFD_NONBLOCK flag, - calls to `receive_some` will block the signal handling, - on the main thread, for now solution is using polling, - working on a way to unblock GIL during read(2) to allow - signal processing on the main thread. - ''' - - def __init__( - self, - shm_key: str, - write_eventfd: int, - wrap_eventfd: int, - start_ptr: int = 0, - buf_size: int = 10 * 1024, - flags: int = 0 - ): - self._shm = SharedMemory( - name=shm_key, - size=buf_size, - create=False - ) - self._write_event = EventFD(write_eventfd, 'w') - self._wrap_event = EventFD(wrap_eventfd, 'r') - self._ptr = start_ptr - self._flags = flags - - @property - def key(self) -> str: - return self._shm.name - - @property - def size(self) -> int: - return self._shm.size - - @property - def ptr(self) -> int: - return self._ptr - - @property - def write_fd(self) -> int: - return self._write_event.fd - - @property - def wrap_fd(self) -> int: - return self._wrap_event.fd - - async def receive_some( - self, - max_bytes: int | None = None, - nb_timeout: float = 0.1 - ) -> memoryview: - # if non blocking eventfd enabled, do polling - # until next write, this allows signal handling - if self._flags | EFD_NONBLOCK: - delta = None - while delta is None: - try: - delta = await self._write_event.read() - - except OSError as e: - if e.errno == 'EAGAIN': - continue - - raise e - - else: - delta = await self._write_event.read() - - # fetch next segment and advance ptr - next_ptr = self._ptr + delta - segment = self._shm.buf[self._ptr:next_ptr] - self._ptr = next_ptr - - if self.ptr == self.size: - # reached the end, signal wrap around - self._ptr = 0 - self._wrap_event.write(1) - - return segment - - async def aclose(self): - self._write_event.close() - self._wrap_event.close() - self._shm.close() - - async def __aenter__(self): - self._write_event.open() - self._wrap_event.open() - return self - - async def __aexit__(self, exc_type, exc_value, traceback): - await self.aclose() From 566a11c00dde379234262125004e30227180b0e5 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 13 Mar 2025 21:15:16 -0300 Subject: [PATCH 06/74] Move RingBuffSender|Receiver to its own tractor.ipc._ringbuf module --- tractor/ipc/__init__.py | 3 + tractor/ipc/_linux.py | 195 -------------------------------------- tractor/ipc/_ringbuf.py | 201 ++++++++++++++++++++++++++++++++++++++++ 3 files changed, 204 insertions(+), 195 deletions(-) create mode 100644 tractor/ipc/_ringbuf.py diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index 1b548b65..c0b58951 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -16,6 +16,9 @@ if platform.system() == 'Linux': read_eventfd as read_eventfd, close_eventfd as close_eventfd, EventFD as EventFD, + ) + + from ._ringbuf import ( RingBuffSender as RingBuffSender, RingBuffReceiver as RingBuffReceiver ) diff --git a/tractor/ipc/_linux.py b/tractor/ipc/_linux.py index 2a9eabc1..53feac6a 100644 --- a/tractor/ipc/_linux.py +++ b/tractor/ipc/_linux.py @@ -2,8 +2,6 @@ import os import errno -from multiprocessing.shared_memory import SharedMemory - import cffi import trio @@ -129,196 +127,3 @@ class EventFD: def __exit__(self, exc_type, exc_value, traceback): self.close() - - -class RingBuffSender(trio.abc.SendStream): - ''' - IPC Reliable Ring Buffer sender side implementation - - `eventfd(2)` is used for wrap around sync, and also to signal - writes to the reader. - - TODO: if blocked on wrap around event wait it will not respond - to signals, fix soon TM - ''' - - def __init__( - self, - shm_key: str, - write_eventfd: int, - wrap_eventfd: int, - start_ptr: int = 0, - buf_size: int = 10 * 1024, - clean_shm_on_exit: bool = True - ): - self._shm = SharedMemory( - name=shm_key, - size=buf_size, - create=True - ) - self._write_event = EventFD(write_eventfd, 'w') - self._wrap_event = EventFD(wrap_eventfd, 'r') - self._ptr = start_ptr - self.clean_shm_on_exit = clean_shm_on_exit - - @property - def key(self) -> str: - return self._shm.name - - @property - def size(self) -> int: - return self._shm.size - - @property - def ptr(self) -> int: - return self._ptr - - @property - def write_fd(self) -> int: - return self._write_event.fd - - @property - def wrap_fd(self) -> int: - return self._wrap_event.fd - - async def send_all(self, data: bytes | bytearray | memoryview): - # while data is larger than the remaining buf - target_ptr = self.ptr + len(data) - while target_ptr > self.size: - # write all bytes that fit - remaining = self.size - self.ptr - self._shm.buf[self.ptr:] = data[:remaining] - # signal write and wait for reader wrap around - self._write_event.write(remaining) - await self._wrap_event.read() - - # wrap around and trim already written bytes - self._ptr = 0 - data = data[remaining:] - target_ptr = self._ptr + len(data) - - # remaining data fits on buffer - self._shm.buf[self.ptr:target_ptr] = data - self._write_event.write(len(data)) - self._ptr = target_ptr - - async def wait_send_all_might_not_block(self): - raise NotImplementedError - - async def aclose(self): - self._write_event.close() - self._wrap_event.close() - if self.clean_shm_on_exit: - self._shm.unlink() - - else: - self._shm.close() - - async def __aenter__(self): - self._write_event.open() - self._wrap_event.open() - return self - - async def __aexit__(self, exc_type, exc_value, traceback): - await self.aclose() - - -class RingBuffReceiver(trio.abc.ReceiveStream): - ''' - IPC Reliable Ring Buffer receiver side implementation - - `eventfd(2)` is used for wrap around sync, and also to signal - writes to the reader. - - Unless eventfd(2) object is opened with EFD_NONBLOCK flag, - calls to `receive_some` will block the signal handling, - on the main thread, for now solution is using polling, - working on a way to unblock GIL during read(2) to allow - signal processing on the main thread. - ''' - - def __init__( - self, - shm_key: str, - write_eventfd: int, - wrap_eventfd: int, - start_ptr: int = 0, - buf_size: int = 10 * 1024, - flags: int = 0 - ): - self._shm = SharedMemory( - name=shm_key, - size=buf_size, - create=False - ) - self._write_event = EventFD(write_eventfd, 'w') - self._wrap_event = EventFD(wrap_eventfd, 'r') - self._ptr = start_ptr - self._flags = flags - - @property - def key(self) -> str: - return self._shm.name - - @property - def size(self) -> int: - return self._shm.size - - @property - def ptr(self) -> int: - return self._ptr - - @property - def write_fd(self) -> int: - return self._write_event.fd - - @property - def wrap_fd(self) -> int: - return self._wrap_event.fd - - async def receive_some( - self, - max_bytes: int | None = None, - nb_timeout: float = 0.1 - ) -> memoryview: - # if non blocking eventfd enabled, do polling - # until next write, this allows signal handling - if self._flags | EFD_NONBLOCK: - delta = None - while delta is None: - try: - delta = await self._write_event.read() - - except OSError as e: - if e.errno == 'EAGAIN': - continue - - raise e - - else: - delta = await self._write_event.read() - - # fetch next segment and advance ptr - next_ptr = self._ptr + delta - segment = self._shm.buf[self._ptr:next_ptr] - self._ptr = next_ptr - - if self.ptr == self.size: - # reached the end, signal wrap around - self._ptr = 0 - self._wrap_event.write(1) - - return segment - - async def aclose(self): - self._write_event.close() - self._wrap_event.close() - self._shm.close() - - async def __aenter__(self): - self._write_event.open() - self._wrap_event.open() - return self - - async def __aexit__(self, exc_type, exc_value, traceback): - await self.aclose() diff --git a/tractor/ipc/_ringbuf.py b/tractor/ipc/_ringbuf.py new file mode 100644 index 00000000..0895381f --- /dev/null +++ b/tractor/ipc/_ringbuf.py @@ -0,0 +1,201 @@ +from multiprocessing.shared_memory import SharedMemory + +import trio + +from ._linux import ( + EFD_NONBLOCK, + EventFD +) + + +class RingBuffSender(trio.abc.SendStream): + ''' + IPC Reliable Ring Buffer sender side implementation + + `eventfd(2)` is used for wrap around sync, and also to signal + writes to the reader. + + TODO: if blocked on wrap around event wait it will not respond + to signals, fix soon TM + ''' + + def __init__( + self, + shm_key: str, + write_eventfd: int, + wrap_eventfd: int, + start_ptr: int = 0, + buf_size: int = 10 * 1024, + clean_shm_on_exit: bool = True + ): + self._shm = SharedMemory( + name=shm_key, + size=buf_size, + create=True + ) + self._write_event = EventFD(write_eventfd, 'w') + self._wrap_event = EventFD(wrap_eventfd, 'r') + self._ptr = start_ptr + self.clean_shm_on_exit = clean_shm_on_exit + + @property + def key(self) -> str: + return self._shm.name + + @property + def size(self) -> int: + return self._shm.size + + @property + def ptr(self) -> int: + return self._ptr + + @property + def write_fd(self) -> int: + return self._write_event.fd + + @property + def wrap_fd(self) -> int: + return self._wrap_event.fd + + async def send_all(self, data: bytes | bytearray | memoryview): + # while data is larger than the remaining buf + target_ptr = self.ptr + len(data) + while target_ptr > self.size: + # write all bytes that fit + remaining = self.size - self.ptr + self._shm.buf[self.ptr:] = data[:remaining] + # signal write and wait for reader wrap around + self._write_event.write(remaining) + await self._wrap_event.read() + + # wrap around and trim already written bytes + self._ptr = 0 + data = data[remaining:] + target_ptr = self._ptr + len(data) + + # remaining data fits on buffer + self._shm.buf[self.ptr:target_ptr] = data + self._write_event.write(len(data)) + self._ptr = target_ptr + + async def wait_send_all_might_not_block(self): + raise NotImplementedError + + async def aclose(self): + self._write_event.close() + self._wrap_event.close() + if self.clean_shm_on_exit: + self._shm.unlink() + + else: + self._shm.close() + + async def __aenter__(self): + self._write_event.open() + self._wrap_event.open() + return self + + async def __aexit__(self, exc_type, exc_value, traceback): + await self.aclose() + + +class RingBuffReceiver(trio.abc.ReceiveStream): + ''' + IPC Reliable Ring Buffer receiver side implementation + + `eventfd(2)` is used for wrap around sync, and also to signal + writes to the reader. + + Unless eventfd(2) object is opened with EFD_NONBLOCK flag, + calls to `receive_some` will block the signal handling, + on the main thread, for now solution is using polling, + working on a way to unblock GIL during read(2) to allow + signal processing on the main thread. + ''' + + def __init__( + self, + shm_key: str, + write_eventfd: int, + wrap_eventfd: int, + start_ptr: int = 0, + buf_size: int = 10 * 1024, + flags: int = 0 + ): + self._shm = SharedMemory( + name=shm_key, + size=buf_size, + create=False + ) + self._write_event = EventFD(write_eventfd, 'w') + self._wrap_event = EventFD(wrap_eventfd, 'r') + self._ptr = start_ptr + self._flags = flags + + @property + def key(self) -> str: + return self._shm.name + + @property + def size(self) -> int: + return self._shm.size + + @property + def ptr(self) -> int: + return self._ptr + + @property + def write_fd(self) -> int: + return self._write_event.fd + + @property + def wrap_fd(self) -> int: + return self._wrap_event.fd + + async def receive_some( + self, + max_bytes: int | None = None, + nb_timeout: float = 0.1 + ) -> memoryview: + # if non blocking eventfd enabled, do polling + # until next write, this allows signal handling + if self._flags | EFD_NONBLOCK: + delta = None + while delta is None: + try: + delta = await self._write_event.read() + + except OSError as e: + if e.errno == 'EAGAIN': + continue + + raise e + + else: + delta = await self._write_event.read() + + # fetch next segment and advance ptr + next_ptr = self._ptr + delta + segment = self._shm.buf[self._ptr:next_ptr] + self._ptr = next_ptr + + if self.ptr == self.size: + # reached the end, signal wrap around + self._ptr = 0 + self._wrap_event.write(1) + + return segment + + async def aclose(self): + self._write_event.close() + self._wrap_event.close() + self._shm.close() + + async def __aenter__(self): + self._write_event.open() + self._wrap_event.open() + return self + + async def __aexit__(self, exc_type, exc_value, traceback): + await self.aclose() From 881813e61e9b12a96b6e659bdbd3f5264bfab1a2 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 13 Mar 2025 21:25:50 -0300 Subject: [PATCH 07/74] Add module headers and fix spacing on tractor._ipc._linux --- tractor/ipc/__init__.py | 17 +++++++++++++++++ tractor/ipc/_linux.py | 22 ++++++++++++++++++++++ tractor/ipc/_ringbuf.py | 19 +++++++++++++++++++ 3 files changed, 58 insertions(+) diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index c0b58951..59fc1e16 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -1,3 +1,20 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + + import platform from ._chan import ( diff --git a/tractor/ipc/_linux.py b/tractor/ipc/_linux.py index 53feac6a..12b00260 100644 --- a/tractor/ipc/_linux.py +++ b/tractor/ipc/_linux.py @@ -1,4 +1,22 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . +''' +Linux specifics, for now we are only exposing EventFD + +''' import os import errno @@ -27,6 +45,7 @@ ffi.cdef( # Open the default dynamic library (essentially 'libc' in most cases) C = ffi.dlopen(None) + # Constants from , if needed. EFD_SEMAPHORE = 1 EFD_CLOEXEC = 0o2000000 @@ -44,6 +63,7 @@ def open_eventfd(initval: int = 0, flags: int = 0) -> int: raise OSError(errno.errorcode[ffi.errno], 'eventfd failed') return fd + def write_eventfd(fd: int, value: int) -> int: ''' Write a 64-bit integer (uint64_t) to the eventfd's counter. @@ -59,6 +79,7 @@ def write_eventfd(fd: int, value: int) -> int: raise OSError(errno.errorcode[ffi.errno], 'write to eventfd failed') return ret + def read_eventfd(fd: int) -> int: ''' Read a 64-bit integer (uint64_t) from the eventfd, returning the value. @@ -76,6 +97,7 @@ def read_eventfd(fd: int) -> int: value = int.from_bytes(data_bytes, byteorder='little', signed=False) return value + def close_eventfd(fd: int) -> int: ''' Close the eventfd. diff --git a/tractor/ipc/_ringbuf.py b/tractor/ipc/_ringbuf.py index 0895381f..50a9eff1 100644 --- a/tractor/ipc/_ringbuf.py +++ b/tractor/ipc/_ringbuf.py @@ -1,3 +1,22 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . +''' +IPC Reliable RingBuffer implementation + +''' from multiprocessing.shared_memory import SharedMemory import trio From 2dd3a682c82413601eec09a27a3f9ad8d7ab8cc8 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 13 Mar 2025 22:43:02 -0300 Subject: [PATCH 08/74] Handle cancelation on EventFD.read --- tests/test_ringbuf.py | 7 ++----- tractor/ipc/_linux.py | 6 ++++-- tractor/ipc/_ringbuf.py | 6 +++--- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index 1c4e88f9..9e457b2a 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -4,7 +4,6 @@ import trio import pytest import tractor from tractor.ipc import ( - EFD_NONBLOCK, open_eventfd, RingBuffSender, RingBuffReceiver @@ -95,7 +94,7 @@ async def child_write_shm( 'large_payloads_large_buffer', ] ) -def test_ring_buff( +def test_ringbuf( msg_amount: int, rand_min: int, rand_max: int, @@ -171,8 +170,7 @@ async def child_blocked_receiver( def test_ring_reader_cancel(): - flags = EFD_NONBLOCK - write_eventfd = open_eventfd(flags=flags) + write_eventfd = open_eventfd() wrap_eventfd = open_eventfd() proc_kwargs = { @@ -201,7 +199,6 @@ def test_ring_reader_cancel(): write_eventfd=write_eventfd, wrap_eventfd=wrap_eventfd, shm_key=shm_key, - flags=flags ) as (sctx, _sent), ): await trio.sleep(1) diff --git a/tractor/ipc/_linux.py b/tractor/ipc/_linux.py index 12b00260..88d80d1c 100644 --- a/tractor/ipc/_linux.py +++ b/tractor/ipc/_linux.py @@ -133,8 +133,10 @@ class EventFD: return write_eventfd(self._fd, value) async def read(self) -> int: - #TODO: how to handle signals? - return await trio.to_thread.run_sync(read_eventfd, self._fd) + return await trio.to_thread.run_sync( + read_eventfd, self._fd, + abandon_on_cancel=True + ) def open(self): self._fobj = os.fdopen(self._fd, self._omode) diff --git a/tractor/ipc/_ringbuf.py b/tractor/ipc/_ringbuf.py index 50a9eff1..77a30ab8 100644 --- a/tractor/ipc/_ringbuf.py +++ b/tractor/ipc/_ringbuf.py @@ -45,7 +45,7 @@ class RingBuffSender(trio.abc.SendStream): wrap_eventfd: int, start_ptr: int = 0, buf_size: int = 10 * 1024, - clean_shm_on_exit: bool = True + unlink_on_exit: bool = True ): self._shm = SharedMemory( name=shm_key, @@ -55,7 +55,7 @@ class RingBuffSender(trio.abc.SendStream): self._write_event = EventFD(write_eventfd, 'w') self._wrap_event = EventFD(wrap_eventfd, 'r') self._ptr = start_ptr - self.clean_shm_on_exit = clean_shm_on_exit + self.unlink_on_exit = unlink_on_exit @property def key(self) -> str: @@ -104,7 +104,7 @@ class RingBuffSender(trio.abc.SendStream): async def aclose(self): self._write_event.close() self._wrap_event.close() - if self.clean_shm_on_exit: + if self.unlink_on_exit: self._shm.unlink() else: From 1ed0c861b5bc452203f85e0593959da3abdbe5b8 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 13 Mar 2025 22:47:45 -0300 Subject: [PATCH 09/74] Address some of fomo\'s comments --- tractor/ipc/_ringbuf.py | 6 ------ tractor/ipc/_shm.py | 5 ++++- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/tractor/ipc/_ringbuf.py b/tractor/ipc/_ringbuf.py index 77a30ab8..0a4f3819 100644 --- a/tractor/ipc/_ringbuf.py +++ b/tractor/ipc/_ringbuf.py @@ -115,9 +115,6 @@ class RingBuffSender(trio.abc.SendStream): self._wrap_event.open() return self - async def __aexit__(self, exc_type, exc_value, traceback): - await self.aclose() - class RingBuffReceiver(trio.abc.ReceiveStream): ''' @@ -215,6 +212,3 @@ class RingBuffReceiver(trio.abc.ReceiveStream): self._write_event.open() self._wrap_event.open() return self - - async def __aexit__(self, exc_type, exc_value, traceback): - await self.aclose() diff --git a/tractor/ipc/_shm.py b/tractor/ipc/_shm.py index 0ee8bf23..c101af30 100644 --- a/tractor/ipc/_shm.py +++ b/tractor/ipc/_shm.py @@ -32,7 +32,10 @@ from multiprocessing.shared_memory import ( ShareableList, ) -from msgspec import Struct, to_builtins +from msgspec import ( + Struct, + to_builtins +) import tractor from tractor.log import get_logger From 84d25b5727d0bce85fb2883ca7194fd357922aec Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 13 Mar 2025 23:12:20 -0300 Subject: [PATCH 10/74] Make ring buf api use pickle-able RBToken --- tests/test_ringbuf.py | 165 ++++++++++++++++------------------------ tractor/ipc/__init__.py | 4 +- tractor/ipc/_ringbuf.py | 90 +++++++++++++++------- 3 files changed, 130 insertions(+), 129 deletions(-) diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index 9e457b2a..64fb37e9 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -4,7 +4,8 @@ import trio import pytest import tractor from tractor.ipc import ( - open_eventfd, + open_ringbuf, + RBToken, RingBuffSender, RingBuffReceiver ) @@ -15,22 +16,16 @@ from tractor._testing.samples import generate_sample_messages async def child_read_shm( ctx: tractor.Context, msg_amount: int, - shm_key: str, - write_eventfd: int, - wrap_eventfd: int, + token: RBToken, buf_size: int, total_bytes: int, - flags: int = 0, ) -> None: recvd_bytes = 0 await ctx.started() start_ts = time.time() async with RingBuffReceiver( - shm_key, - write_eventfd, - wrap_eventfd, + token, buf_size=buf_size, - flags=flags ) as receiver: while recvd_bytes < total_bytes: msg = await receiver.receive_some() @@ -55,9 +50,7 @@ async def child_write_shm( msg_amount: int, rand_min: int, rand_max: int, - shm_key: str, - write_eventfd: int, - wrap_eventfd: int, + token: RBToken, buf_size: int, ) -> None: msgs, total_bytes = generate_sample_messages( @@ -67,9 +60,7 @@ async def child_write_shm( ) await ctx.started(total_bytes) async with RingBuffSender( - shm_key, - write_eventfd, - wrap_eventfd, + token, buf_size=buf_size ) as sender: for msg in msgs: @@ -100,52 +91,48 @@ def test_ringbuf( rand_max: int, buf_size: int ): - write_eventfd = open_eventfd() - wrap_eventfd = open_eventfd() - - proc_kwargs = { - 'pass_fds': (write_eventfd, wrap_eventfd) - } - - shm_key = 'test_ring_buff' - - common_kwargs = { - 'msg_amount': msg_amount, - 'shm_key': shm_key, - 'write_eventfd': write_eventfd, - 'wrap_eventfd': wrap_eventfd, - 'buf_size': buf_size - } - async def main(): - async with tractor.open_nursery() as an: - send_p = await an.start_actor( - 'ring_sender', - enable_modules=[__name__], - proc_kwargs=proc_kwargs - ) - recv_p = await an.start_actor( - 'ring_receiver', - enable_modules=[__name__], - proc_kwargs=proc_kwargs - ) - async with ( - send_p.open_context( - child_write_shm, - rand_min=rand_min, - rand_max=rand_max, - **common_kwargs - ) as (sctx, total_bytes), - recv_p.open_context( - child_read_shm, - **common_kwargs, - total_bytes=total_bytes, - ) as (sctx, _sent), - ): - await recv_p.result() + with open_ringbuf( + 'test_ringbuf', + buf_size=buf_size + ) as token: + proc_kwargs = { + 'pass_fds': (token.write_eventfd, token.wrap_eventfd) + } - await send_p.cancel_actor() - await recv_p.cancel_actor() + common_kwargs = { + 'msg_amount': msg_amount, + 'token': token, + 'buf_size': buf_size + } + async with tractor.open_nursery() as an: + send_p = await an.start_actor( + 'ring_sender', + enable_modules=[__name__], + proc_kwargs=proc_kwargs + ) + recv_p = await an.start_actor( + 'ring_receiver', + enable_modules=[__name__], + proc_kwargs=proc_kwargs + ) + async with ( + send_p.open_context( + child_write_shm, + rand_min=rand_min, + rand_max=rand_max, + **common_kwargs + ) as (sctx, total_bytes), + recv_p.open_context( + child_read_shm, + **common_kwargs, + total_bytes=total_bytes, + ) as (sctx, _sent), + ): + await recv_p.result() + + await send_p.cancel_actor() + await recv_p.cancel_actor() trio.run(main) @@ -154,55 +141,35 @@ def test_ringbuf( @tractor.context async def child_blocked_receiver( ctx: tractor.Context, - shm_key: str, - write_eventfd: int, - wrap_eventfd: int, - flags: int = 0 + token: RBToken ): - async with RingBuffReceiver( - shm_key, - write_eventfd, - wrap_eventfd, - flags=flags - ) as receiver: + async with RingBuffReceiver(token) as receiver: await ctx.started() await receiver.receive_some() def test_ring_reader_cancel(): - write_eventfd = open_eventfd() - wrap_eventfd = open_eventfd() - - proc_kwargs = { - 'pass_fds': (write_eventfd, wrap_eventfd) - } - - shm_key = 'test_ring_cancel' - async def main(): - async with ( - tractor.open_nursery() as an, - RingBuffSender( - shm_key, - write_eventfd, - wrap_eventfd, - ) as _sender, - ): - recv_p = await an.start_actor( - 'ring_blocked_receiver', - enable_modules=[__name__], - proc_kwargs=proc_kwargs - ) + with open_ringbuf('test_ring_cancel') as token: async with ( - recv_p.open_context( - child_blocked_receiver, - write_eventfd=write_eventfd, - wrap_eventfd=wrap_eventfd, - shm_key=shm_key, - ) as (sctx, _sent), + tractor.open_nursery() as an, + RingBuffSender(token) as _sender, ): - await trio.sleep(1) - await an.cancel() + recv_p = await an.start_actor( + 'ring_blocked_receiver', + enable_modules=[__name__], + proc_kwargs={ + 'pass_fds': (token.write_eventfd, token.wrap_eventfd) + } + ) + async with ( + recv_p.open_context( + child_blocked_receiver, + token=token + ) as (sctx, _sent), + ): + await trio.sleep(1) + await an.cancel() with pytest.raises(tractor._exceptions.ContextCancelled): diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index 59fc1e16..ec6217a1 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -36,6 +36,8 @@ if platform.system() == 'Linux': ) from ._ringbuf import ( + RBToken as RBToken, RingBuffSender as RingBuffSender, - RingBuffReceiver as RingBuffReceiver + RingBuffReceiver as RingBuffReceiver, + open_ringbuf ) diff --git a/tractor/ipc/_ringbuf.py b/tractor/ipc/_ringbuf.py index 0a4f3819..c590e8e2 100644 --- a/tractor/ipc/_ringbuf.py +++ b/tractor/ipc/_ringbuf.py @@ -17,16 +17,65 @@ IPC Reliable RingBuffer implementation ''' +from __future__ import annotations +from contextlib import contextmanager as cm from multiprocessing.shared_memory import SharedMemory import trio +from msgspec import ( + Struct, + to_builtins +) from ._linux import ( EFD_NONBLOCK, + open_eventfd, EventFD ) +class RBToken(Struct, frozen=True): + ''' + RingBuffer token contains necesary info to open the two + eventfds and the shared memory + + ''' + shm_name: str + write_eventfd: int + wrap_eventfd: int + + def as_msg(self): + return to_builtins(self) + + @classmethod + def from_msg(cls, msg: dict) -> RBToken: + if isinstance(msg, RBToken): + return msg + + return RBToken(**msg) + + +@cm +def open_ringbuf( + shm_name: str, + buf_size: int = 10 * 1024, + write_efd_flags: int = 0, + wrap_efd_flags: int = 0 +) -> RBToken: + shm = SharedMemory( + name=shm_name, + size=buf_size, + create=True + ) + token = RBToken( + shm_name=shm_name, + write_eventfd=open_eventfd(flags=write_efd_flags), + wrap_eventfd=open_eventfd(flags=wrap_efd_flags) + ) + yield token + shm.close() + + class RingBuffSender(trio.abc.SendStream): ''' IPC Reliable Ring Buffer sender side implementation @@ -34,28 +83,22 @@ class RingBuffSender(trio.abc.SendStream): `eventfd(2)` is used for wrap around sync, and also to signal writes to the reader. - TODO: if blocked on wrap around event wait it will not respond - to signals, fix soon TM ''' - def __init__( self, - shm_key: str, - write_eventfd: int, - wrap_eventfd: int, + token: RBToken, start_ptr: int = 0, buf_size: int = 10 * 1024, - unlink_on_exit: bool = True ): + token = RBToken.from_msg(token) self._shm = SharedMemory( - name=shm_key, + name=token.shm_name, size=buf_size, - create=True + create=False ) - self._write_event = EventFD(write_eventfd, 'w') - self._wrap_event = EventFD(wrap_eventfd, 'r') + self._write_event = EventFD(token.write_eventfd, 'w') + self._wrap_event = EventFD(token.wrap_eventfd, 'r') self._ptr = start_ptr - self.unlink_on_exit = unlink_on_exit @property def key(self) -> str: @@ -104,11 +147,7 @@ class RingBuffSender(trio.abc.SendStream): async def aclose(self): self._write_event.close() self._wrap_event.close() - if self.unlink_on_exit: - self._shm.unlink() - - else: - self._shm.close() + self._shm.close() async def __aenter__(self): self._write_event.open() @@ -123,29 +162,22 @@ class RingBuffReceiver(trio.abc.ReceiveStream): `eventfd(2)` is used for wrap around sync, and also to signal writes to the reader. - Unless eventfd(2) object is opened with EFD_NONBLOCK flag, - calls to `receive_some` will block the signal handling, - on the main thread, for now solution is using polling, - working on a way to unblock GIL during read(2) to allow - signal processing on the main thread. ''' - def __init__( self, - shm_key: str, - write_eventfd: int, - wrap_eventfd: int, + token: RBToken, start_ptr: int = 0, buf_size: int = 10 * 1024, flags: int = 0 ): + token = RBToken.from_msg(token) self._shm = SharedMemory( - name=shm_key, + name=token.shm_name, size=buf_size, create=False ) - self._write_event = EventFD(write_eventfd, 'w') - self._wrap_event = EventFD(wrap_eventfd, 'r') + self._write_event = EventFD(token.write_eventfd, 'w') + self._wrap_event = EventFD(token.wrap_eventfd, 'r') self._ptr = start_ptr self._flags = flags From 26fef82d33cc126521c18ee26c1d3e0cc350508a Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Fri, 14 Mar 2025 00:25:10 -0300 Subject: [PATCH 11/74] Add buf_size to RBToken and add sender cancel test, move disable_mantracker to its own _mp_bs module --- tests/test_ringbuf.py | 53 +++++++++++++++++++++++++++++++---------- tractor/ipc/_mp_bs.py | 45 ++++++++++++++++++++++++++++++++++ tractor/ipc/_ringbuf.py | 29 +++++++++++++--------- tractor/ipc/_shm.py | 29 +--------------------- 4 files changed, 105 insertions(+), 51 deletions(-) create mode 100644 tractor/ipc/_mp_bs.py diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index 64fb37e9..28af7b83 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -17,16 +17,12 @@ async def child_read_shm( ctx: tractor.Context, msg_amount: int, token: RBToken, - buf_size: int, total_bytes: int, ) -> None: recvd_bytes = 0 await ctx.started() start_ts = time.time() - async with RingBuffReceiver( - token, - buf_size=buf_size, - ) as receiver: + async with RingBuffReceiver(token) as receiver: while recvd_bytes < total_bytes: msg = await receiver.receive_some() recvd_bytes += len(msg) @@ -51,7 +47,6 @@ async def child_write_shm( rand_min: int, rand_max: int, token: RBToken, - buf_size: int, ) -> None: msgs, total_bytes = generate_sample_messages( msg_amount, @@ -59,10 +54,7 @@ async def child_write_shm( rand_max=rand_max, ) await ctx.started(total_bytes) - async with RingBuffSender( - token, - buf_size=buf_size - ) as sender: + async with RingBuffSender(token) as sender: for msg in msgs: await sender.send_all(msg) @@ -103,7 +95,6 @@ def test_ringbuf( common_kwargs = { 'msg_amount': msg_amount, 'token': token, - 'buf_size': buf_size } async with tractor.open_nursery() as an: send_p = await an.start_actor( @@ -150,7 +141,7 @@ async def child_blocked_receiver( def test_ring_reader_cancel(): async def main(): - with open_ringbuf('test_ring_cancel') as token: + with open_ringbuf('test_ring_cancel_reader') as token: async with ( tractor.open_nursery() as an, RingBuffSender(token) as _sender, @@ -174,3 +165,41 @@ def test_ring_reader_cancel(): with pytest.raises(tractor._exceptions.ContextCancelled): trio.run(main) + + +@tractor.context +async def child_blocked_sender( + ctx: tractor.Context, + token: RBToken +): + async with RingBuffSender(token) as sender: + await ctx.started() + await sender.send_all(b'this will wrap') + + +def test_ring_sender_cancel(): + async def main(): + with open_ringbuf( + 'test_ring_cancel_sender', + buf_size=1 + ) as token: + async with tractor.open_nursery() as an: + recv_p = await an.start_actor( + 'ring_blocked_sender', + enable_modules=[__name__], + proc_kwargs={ + 'pass_fds': (token.write_eventfd, token.wrap_eventfd) + } + ) + async with ( + recv_p.open_context( + child_blocked_sender, + token=token + ) as (sctx, _sent), + ): + await trio.sleep(1) + await an.cancel() + + + with pytest.raises(tractor._exceptions.ContextCancelled): + trio.run(main) diff --git a/tractor/ipc/_mp_bs.py b/tractor/ipc/_mp_bs.py new file mode 100644 index 00000000..e51aa9ae --- /dev/null +++ b/tractor/ipc/_mp_bs.py @@ -0,0 +1,45 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . +''' +Utils to tame mp non-SC madeness + +''' +def disable_mantracker(): + ''' + 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 + + # Tell the "resource tracker" thing to fuck off. + class ManTracker(mantracker.ResourceTracker): + def register(self, name, rtype): + pass + + def unregister(self, name, rtype): + pass + + def ensure_running(self): + pass + + # "know your land and know your prey" + # https://www.dailymotion.com/video/x6ozzco + mantracker._resource_tracker = ManTracker() + mantracker.register = mantracker._resource_tracker.register + mantracker.ensure_running = mantracker._resource_tracker.ensure_running + mantracker.unregister = mantracker._resource_tracker.unregister + mantracker.getfd = mantracker._resource_tracker.getfd diff --git a/tractor/ipc/_ringbuf.py b/tractor/ipc/_ringbuf.py index c590e8e2..6337eea1 100644 --- a/tractor/ipc/_ringbuf.py +++ b/tractor/ipc/_ringbuf.py @@ -32,6 +32,10 @@ from ._linux import ( open_eventfd, EventFD ) +from ._mp_bs import disable_mantracker + + +disable_mantracker() class RBToken(Struct, frozen=True): @@ -43,6 +47,7 @@ class RBToken(Struct, frozen=True): shm_name: str write_eventfd: int wrap_eventfd: int + buf_size: int def as_msg(self): return to_builtins(self) @@ -67,13 +72,17 @@ def open_ringbuf( size=buf_size, create=True ) - token = RBToken( - shm_name=shm_name, - write_eventfd=open_eventfd(flags=write_efd_flags), - wrap_eventfd=open_eventfd(flags=wrap_efd_flags) - ) - yield token - shm.close() + try: + token = RBToken( + shm_name=shm_name, + write_eventfd=open_eventfd(flags=write_efd_flags), + wrap_eventfd=open_eventfd(flags=wrap_efd_flags), + buf_size=buf_size + ) + yield token + + finally: + shm.unlink() class RingBuffSender(trio.abc.SendStream): @@ -88,12 +97,11 @@ class RingBuffSender(trio.abc.SendStream): self, token: RBToken, start_ptr: int = 0, - buf_size: int = 10 * 1024, ): token = RBToken.from_msg(token) self._shm = SharedMemory( name=token.shm_name, - size=buf_size, + size=token.buf_size, create=False ) self._write_event = EventFD(token.write_eventfd, 'w') @@ -167,13 +175,12 @@ class RingBuffReceiver(trio.abc.ReceiveStream): self, token: RBToken, start_ptr: int = 0, - buf_size: int = 10 * 1024, flags: int = 0 ): token = RBToken.from_msg(token) self._shm = SharedMemory( name=token.shm_name, - size=buf_size, + size=token.buf_size, create=False ) self._write_event = EventFD(token.write_eventfd, 'w') diff --git a/tractor/ipc/_shm.py b/tractor/ipc/_shm.py index c101af30..9868ac73 100644 --- a/tractor/ipc/_shm.py +++ b/tractor/ipc/_shm.py @@ -38,6 +38,7 @@ from msgspec import ( ) import tractor +from tractor.ipc._mp_bs import disable_mantracker from tractor.log import get_logger @@ -57,34 +58,6 @@ except ImportError: log = get_logger(__name__) -def disable_mantracker(): - ''' - 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 - - # Tell the "resource tracker" thing to fuck off. - class ManTracker(mantracker.ResourceTracker): - def register(self, name, rtype): - pass - - def unregister(self, name, rtype): - pass - - def ensure_running(self): - pass - - # "know your land and know your prey" - # https://www.dailymotion.com/video/x6ozzco - mantracker._resource_tracker = ManTracker() - mantracker.register = mantracker._resource_tracker.register - mantracker.ensure_running = mantracker._resource_tracker.ensure_running - mantracker.unregister = mantracker._resource_tracker.unregister - mantracker.getfd = mantracker._resource_tracker.getfd - - disable_mantracker() From 6f1f198fb119cf50eaee896a5ef2a8c8843601b2 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sun, 16 Mar 2025 14:14:32 -0300 Subject: [PATCH 12/74] Break out transport protocol and tcp specifics into their own submodules under tractor.ipc --- tractor/ipc/__init__.py | 11 +- tractor/ipc/_chan.py | 434 +------------------------------------- tractor/ipc/_tcp.py | 406 +++++++++++++++++++++++++++++++++++ tractor/ipc/_transport.py | 74 +++++++ 4 files changed, 498 insertions(+), 427 deletions(-) create mode 100644 tractor/ipc/_tcp.py create mode 100644 tractor/ipc/_transport.py diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index ec6217a1..cd16a139 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -17,9 +17,16 @@ import platform +from ._transport import MsgTransport as MsgTransport + +from ._tcp import ( + get_stream_addrs as get_stream_addrs, + MsgpackTCPStream as MsgpackTCPStream +) + from ._chan import ( _connect_chan as _connect_chan, - MsgTransport as MsgTransport, + get_msg_transport as get_msg_transport, Channel as Channel ) @@ -39,5 +46,5 @@ if platform.system() == 'Linux': RBToken as RBToken, RingBuffSender as RingBuffSender, RingBuffReceiver as RingBuffReceiver, - open_ringbuf + open_ringbuf as open_ringbuf ) diff --git a/tractor/ipc/_chan.py b/tractor/ipc/_chan.py index 83186147..1b6ba29f 100644 --- a/tractor/ipc/_chan.py +++ b/tractor/ipc/_chan.py @@ -19,455 +19,39 @@ Inter-process comms abstractions """ from __future__ import annotations -from collections.abc import ( - AsyncGenerator, - AsyncIterator, -) +from collections.abc import AsyncGenerator from contextlib import ( asynccontextmanager as acm, contextmanager as cm, ) import platform from pprint import pformat -import struct import typing from typing import ( Any, - Callable, - runtime_checkable, - Protocol, - Type, - TypeVar, + Type ) -import msgspec -from tricycle import BufferedReceiveStream import trio +from tractor.ipc._transport import MsgTransport +from tractor.ipc._tcp import ( + MsgpackTCPStream, + get_stream_addrs +) from tractor.log import get_logger from tractor._exceptions import ( MsgTypeError, pack_from_raise, - TransportClosed, - _mk_send_mte, - _mk_recv_mte, -) -from tractor.msg import ( - _ctxvar_MsgCodec, - # _codec, XXX see `self._codec` sanity/debug checks - MsgCodec, - types as msgtypes, - pretty_struct, ) +from tractor.msg import MsgCodec + log = get_logger(__name__) _is_windows = platform.system() == 'Windows' -def get_stream_addrs( - stream: trio.SocketStream -) -> tuple[ - tuple[str, int], # local - tuple[str, int], # remote -]: - ''' - Return the `trio` streaming transport prot's socket-addrs for - both the local and remote sides as a pair. - - ''' - # rn, should both be IP sockets - lsockname = stream.socket.getsockname() - rsockname = stream.socket.getpeername() - return ( - tuple(lsockname[:2]), - tuple(rsockname[:2]), - ) - - -# from tractor.msg.types import MsgType -# ?TODO? this should be our `Union[*msgtypes.__spec__]` alias now right..? -# => BLEH, except can't bc prots must inherit typevar or param-spec -# vars.. -MsgType = TypeVar('MsgType') - - -# TODO: break up this mod into a subpkg so we can start adding new -# backends and move this type stuff into a dedicated file.. Bo -# -@runtime_checkable -class MsgTransport(Protocol[MsgType]): -# -# ^-TODO-^ consider using a generic def and indexing with our -# eventual msg definition/types? -# - https://docs.python.org/3/library/typing.html#typing.Protocol - - stream: trio.SocketStream - drained: list[MsgType] - - def __init__(self, stream: trio.SocketStream) -> None: - ... - - # XXX: should this instead be called `.sendall()`? - async def send(self, msg: MsgType) -> None: - ... - - async def recv(self) -> MsgType: - ... - - def __aiter__(self) -> MsgType: - ... - - def connected(self) -> bool: - ... - - # defining this sync otherwise it causes a mypy error because it - # can't figure out it's a generator i guess?..? - def drain(self) -> AsyncIterator[dict]: - ... - - @property - def laddr(self) -> tuple[str, int]: - ... - - @property - def raddr(self) -> tuple[str, int]: - ... - - -# TODO: typing oddity.. not sure why we have to inherit here, but it -# seems to be an issue with `get_msg_transport()` returning -# a `Type[Protocol]`; probably should make a `mypy` issue? -class MsgpackTCPStream(MsgTransport): - ''' - A ``trio.SocketStream`` delivering ``msgpack`` formatted data - using the ``msgspec`` codec lib. - - ''' - layer_key: int = 4 - name_key: str = 'tcp' - - # TODO: better naming for this? - # -[ ] check how libp2p does naming for such things? - codec_key: str = 'msgpack' - - def __init__( - self, - stream: trio.SocketStream, - prefix_size: int = 4, - - # XXX optionally provided codec pair for `msgspec`: - # https://jcristharif.com/msgspec/extending.html#mapping-to-from-native-types - # - # TODO: define this as a `Codec` struct which can be - # overriden dynamically by the application/runtime? - codec: tuple[ - Callable[[Any], Any]|None, # coder - Callable[[type, Any], Any]|None, # decoder - ]|None = None, - - ) -> None: - - self.stream = stream - assert self.stream.socket - - # should both be IP sockets - self._laddr, self._raddr = get_stream_addrs(stream) - - # create read loop instance - self._aiter_pkts = self._iter_packets() - self._send_lock = trio.StrictFIFOLock() - - # public i guess? - self.drained: list[dict] = [] - - self.recv_stream = BufferedReceiveStream( - transport_stream=stream - ) - self.prefix_size = prefix_size - - # allow for custom IPC msg interchange format - # dynamic override Bo - self._task = trio.lowlevel.current_task() - - # XXX for ctxvar debug only! - # self._codec: MsgCodec = ( - # codec - # or - # _codec._ctxvar_MsgCodec.get() - # ) - - async def _iter_packets(self) -> AsyncGenerator[dict, None]: - ''' - Yield `bytes`-blob decoded packets from the underlying TCP - stream using the current task's `MsgCodec`. - - This is a streaming routine implemented as an async generator - func (which was the original design, but could be changed?) - and is allocated by a `.__call__()` inside `.__init__()` where - it is assigned to the `._aiter_pkts` attr. - - ''' - decodes_failed: int = 0 - - while True: - try: - header: bytes = await self.recv_stream.receive_exactly(4) - except ( - ValueError, - ConnectionResetError, - - # not sure entirely why we need this but without it we - # seem to be getting racy failures here on - # arbiter/registry name subs.. - trio.BrokenResourceError, - - ) as trans_err: - - loglevel = 'transport' - match trans_err: - # case ( - # ConnectionResetError() - # ): - # loglevel = 'transport' - - # peer actor (graceful??) TCP EOF but `tricycle` - # seems to raise a 0-bytes-read? - case ValueError() if ( - 'unclean EOF' in trans_err.args[0] - ): - pass - - # peer actor (task) prolly shutdown quickly due - # to cancellation - case trio.BrokenResourceError() if ( - 'Connection reset by peer' in trans_err.args[0] - ): - pass - - # unless the disconnect condition falls under "a - # normal operation breakage" we usualy console warn - # about it. - case _: - loglevel: str = 'warning' - - - raise TransportClosed( - message=( - f'IPC transport already closed by peer\n' - f'x]> {type(trans_err)}\n' - f' |_{self}\n' - ), - loglevel=loglevel, - ) from trans_err - - # XXX definitely can happen if transport is closed - # manually by another `trio.lowlevel.Task` in the - # same actor; we use this in some simulated fault - # testing for ex, but generally should never happen - # under normal operation! - # - # NOTE: as such we always re-raise this error from the - # RPC msg loop! - except trio.ClosedResourceError as closure_err: - raise TransportClosed( - message=( - f'IPC transport already manually closed locally?\n' - f'x]> {type(closure_err)} \n' - f' |_{self}\n' - ), - loglevel='error', - raise_on_report=( - closure_err.args[0] == 'another task closed this fd' - or - closure_err.args[0] in ['another task closed this fd'] - ), - ) from closure_err - - # graceful TCP EOF disconnect - if header == b'': - raise TransportClosed( - message=( - f'IPC transport already gracefully closed\n' - f']>\n' - f' |_{self}\n' - ), - loglevel='transport', - # cause=??? # handy or no? - ) - - size: int - size, = struct.unpack(" None: - ''' - Send a msgpack encoded py-object-blob-as-msg over TCP. - - If `strict_types == True` then a `MsgTypeError` will be raised on any - invalid msg type - - ''' - __tracebackhide__: bool = hide_tb - - # XXX see `trio._sync.AsyncContextManagerMixin` for details - # on the `.acquire()`/`.release()` sequencing.. - async with self._send_lock: - - # NOTE: lookup the `trio.Task.context`'s var for - # the current `MsgCodec`. - codec: MsgCodec = _ctxvar_MsgCodec.get() - - # XXX for ctxvar debug only! - # if self._codec.pld_spec != codec.pld_spec: - # self._codec = codec - # log.runtime( - # f'Using new codec in {self}.send()\n' - # f'codec: {self._codec}\n\n' - # f'msg: {msg}\n' - # ) - - if type(msg) not in msgtypes.__msg_types__: - if strict_types: - raise _mk_send_mte( - msg, - codec=codec, - ) - else: - log.warning( - 'Sending non-`Msg`-spec msg?\n\n' - f'{msg}\n' - ) - - try: - bytes_data: bytes = codec.encode(msg) - except TypeError as _err: - typerr = _err - msgtyperr: MsgTypeError = _mk_send_mte( - msg, - codec=codec, - message=( - f'IPC-msg-spec violation in\n\n' - f'{pretty_struct.Struct.pformat(msg)}' - ), - src_type_error=typerr, - ) - raise msgtyperr from typerr - - # supposedly the fastest says, - # https://stackoverflow.com/a/54027962 - size: bytes = struct.pack(" - # except BaseException as _err: - # err = _err - # if not isinstance(err, MsgTypeError): - # __tracebackhide__: bool = False - # raise - - @property - def laddr(self) -> tuple[str, int]: - return self._laddr - - @property - def raddr(self) -> tuple[str, int]: - return self._raddr - - async def recv(self) -> Any: - return await self._aiter_pkts.asend(None) - - async def drain(self) -> AsyncIterator[dict]: - ''' - Drain the stream's remaining messages sent from - the far end until the connection is closed by - the peer. - - ''' - try: - async for msg in self._iter_packets(): - self.drained.append(msg) - except TransportClosed: - for msg in self.drained: - yield msg - - def __aiter__(self): - return self._aiter_pkts - - def connected(self) -> bool: - return self.stream.socket.fileno() != -1 - - def get_msg_transport( key: tuple[str, str], diff --git a/tractor/ipc/_tcp.py b/tractor/ipc/_tcp.py new file mode 100644 index 00000000..03185f82 --- /dev/null +++ b/tractor/ipc/_tcp.py @@ -0,0 +1,406 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . +''' +TCP implementation of tractor.ipc._transport.MsgTransport protocol + +''' +from __future__ import annotations +from collections.abc import ( + AsyncGenerator, + AsyncIterator, +) +import struct +from typing import ( + Any, + Callable, + Type, +) + +import msgspec +from tricycle import BufferedReceiveStream +import trio + +from tractor.log import get_logger +from tractor._exceptions import ( + MsgTypeError, + TransportClosed, + _mk_send_mte, + _mk_recv_mte, +) +from tractor.msg import ( + _ctxvar_MsgCodec, + # _codec, XXX see `self._codec` sanity/debug checks + MsgCodec, + types as msgtypes, + pretty_struct, +) +from tractor.ipc import MsgTransport + + +log = get_logger(__name__) + + +def get_stream_addrs( + stream: trio.SocketStream +) -> tuple[ + tuple[str, int], # local + tuple[str, int], # remote +]: + ''' + Return the `trio` streaming transport prot's socket-addrs for + both the local and remote sides as a pair. + + ''' + # rn, should both be IP sockets + lsockname = stream.socket.getsockname() + rsockname = stream.socket.getpeername() + return ( + tuple(lsockname[:2]), + tuple(rsockname[:2]), + ) + + +# TODO: typing oddity.. not sure why we have to inherit here, but it +# seems to be an issue with `get_msg_transport()` returning +# a `Type[Protocol]`; probably should make a `mypy` issue? +class MsgpackTCPStream(MsgTransport): + ''' + A ``trio.SocketStream`` delivering ``msgpack`` formatted data + using the ``msgspec`` codec lib. + + ''' + layer_key: int = 4 + name_key: str = 'tcp' + + # TODO: better naming for this? + # -[ ] check how libp2p does naming for such things? + codec_key: str = 'msgpack' + + def __init__( + self, + stream: trio.SocketStream, + prefix_size: int = 4, + + # XXX optionally provided codec pair for `msgspec`: + # https://jcristharif.com/msgspec/extending.html#mapping-to-from-native-types + # + # TODO: define this as a `Codec` struct which can be + # overriden dynamically by the application/runtime? + codec: tuple[ + Callable[[Any], Any]|None, # coder + Callable[[type, Any], Any]|None, # decoder + ]|None = None, + + ) -> None: + + self.stream = stream + assert self.stream.socket + + # should both be IP sockets + self._laddr, self._raddr = get_stream_addrs(stream) + + # create read loop instance + self._aiter_pkts = self._iter_packets() + self._send_lock = trio.StrictFIFOLock() + + # public i guess? + self.drained: list[dict] = [] + + self.recv_stream = BufferedReceiveStream( + transport_stream=stream + ) + self.prefix_size = prefix_size + + # allow for custom IPC msg interchange format + # dynamic override Bo + self._task = trio.lowlevel.current_task() + + # XXX for ctxvar debug only! + # self._codec: MsgCodec = ( + # codec + # or + # _codec._ctxvar_MsgCodec.get() + # ) + + async def _iter_packets(self) -> AsyncGenerator[dict, None]: + ''' + Yield `bytes`-blob decoded packets from the underlying TCP + stream using the current task's `MsgCodec`. + + This is a streaming routine implemented as an async generator + func (which was the original design, but could be changed?) + and is allocated by a `.__call__()` inside `.__init__()` where + it is assigned to the `._aiter_pkts` attr. + + ''' + decodes_failed: int = 0 + + while True: + try: + header: bytes = await self.recv_stream.receive_exactly(4) + except ( + ValueError, + ConnectionResetError, + + # not sure entirely why we need this but without it we + # seem to be getting racy failures here on + # arbiter/registry name subs.. + trio.BrokenResourceError, + + ) as trans_err: + + loglevel = 'transport' + match trans_err: + # case ( + # ConnectionResetError() + # ): + # loglevel = 'transport' + + # peer actor (graceful??) TCP EOF but `tricycle` + # seems to raise a 0-bytes-read? + case ValueError() if ( + 'unclean EOF' in trans_err.args[0] + ): + pass + + # peer actor (task) prolly shutdown quickly due + # to cancellation + case trio.BrokenResourceError() if ( + 'Connection reset by peer' in trans_err.args[0] + ): + pass + + # unless the disconnect condition falls under "a + # normal operation breakage" we usualy console warn + # about it. + case _: + loglevel: str = 'warning' + + + raise TransportClosed( + message=( + f'IPC transport already closed by peer\n' + f'x)> {type(trans_err)}\n' + f' |_{self}\n' + ), + loglevel=loglevel, + ) from trans_err + + # XXX definitely can happen if transport is closed + # manually by another `trio.lowlevel.Task` in the + # same actor; we use this in some simulated fault + # testing for ex, but generally should never happen + # under normal operation! + # + # NOTE: as such we always re-raise this error from the + # RPC msg loop! + except trio.ClosedResourceError as closure_err: + raise TransportClosed( + message=( + f'IPC transport already manually closed locally?\n' + f'x)> {type(closure_err)} \n' + f' |_{self}\n' + ), + loglevel='error', + raise_on_report=( + closure_err.args[0] == 'another task closed this fd' + or + closure_err.args[0] in ['another task closed this fd'] + ), + ) from closure_err + + # graceful TCP EOF disconnect + if header == b'': + raise TransportClosed( + message=( + f'IPC transport already gracefully closed\n' + f')>\n' + f'|_{self}\n' + ), + loglevel='transport', + # cause=??? # handy or no? + ) + + size: int + size, = struct.unpack(" None: + ''' + Send a msgpack encoded py-object-blob-as-msg over TCP. + + If `strict_types == True` then a `MsgTypeError` will be raised on any + invalid msg type + + ''' + __tracebackhide__: bool = hide_tb + + # XXX see `trio._sync.AsyncContextManagerMixin` for details + # on the `.acquire()`/`.release()` sequencing.. + async with self._send_lock: + + # NOTE: lookup the `trio.Task.context`'s var for + # the current `MsgCodec`. + codec: MsgCodec = _ctxvar_MsgCodec.get() + + # XXX for ctxvar debug only! + # if self._codec.pld_spec != codec.pld_spec: + # self._codec = codec + # log.runtime( + # f'Using new codec in {self}.send()\n' + # f'codec: {self._codec}\n\n' + # f'msg: {msg}\n' + # ) + + if type(msg) not in msgtypes.__msg_types__: + if strict_types: + raise _mk_send_mte( + msg, + codec=codec, + ) + else: + log.warning( + 'Sending non-`Msg`-spec msg?\n\n' + f'{msg}\n' + ) + + try: + bytes_data: bytes = codec.encode(msg) + except TypeError as _err: + typerr = _err + msgtyperr: MsgTypeError = _mk_send_mte( + msg, + codec=codec, + message=( + f'IPC-msg-spec violation in\n\n' + f'{pretty_struct.Struct.pformat(msg)}' + ), + src_type_error=typerr, + ) + raise msgtyperr from typerr + + # supposedly the fastest says, + # https://stackoverflow.com/a/54027962 + size: bytes = struct.pack(" + # except BaseException as _err: + # err = _err + # if not isinstance(err, MsgTypeError): + # __tracebackhide__: bool = False + # raise + + @property + def laddr(self) -> tuple[str, int]: + return self._laddr + + @property + def raddr(self) -> tuple[str, int]: + return self._raddr + + async def recv(self) -> Any: + return await self._aiter_pkts.asend(None) + + async def drain(self) -> AsyncIterator[dict]: + ''' + Drain the stream's remaining messages sent from + the far end until the connection is closed by + the peer. + + ''' + try: + async for msg in self._iter_packets(): + self.drained.append(msg) + except TransportClosed: + for msg in self.drained: + yield msg + + def __aiter__(self): + return self._aiter_pkts + + def connected(self) -> bool: + return self.stream.socket.fileno() != -1 diff --git a/tractor/ipc/_transport.py b/tractor/ipc/_transport.py new file mode 100644 index 00000000..24e03a90 --- /dev/null +++ b/tractor/ipc/_transport.py @@ -0,0 +1,74 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . +''' +typing.Protocol based generic msg API, implement this class to add backends for +tractor.ipc.Channel + +''' +import trio +from typing import ( + runtime_checkable, + Protocol, + TypeVar, +) +from collections.abc import AsyncIterator + + +# from tractor.msg.types import MsgType +# ?TODO? this should be our `Union[*msgtypes.__spec__]` alias now right..? +# => BLEH, except can't bc prots must inherit typevar or param-spec +# vars.. +MsgType = TypeVar('MsgType') + + +@runtime_checkable +class MsgTransport(Protocol[MsgType]): +# +# ^-TODO-^ consider using a generic def and indexing with our +# eventual msg definition/types? +# - https://docs.python.org/3/library/typing.html#typing.Protocol + + stream: trio.SocketStream + drained: list[MsgType] + + def __init__(self, stream: trio.SocketStream) -> None: + ... + + # XXX: should this instead be called `.sendall()`? + async def send(self, msg: MsgType) -> None: + ... + + async def recv(self) -> MsgType: + ... + + def __aiter__(self) -> MsgType: + ... + + def connected(self) -> bool: + ... + + # defining this sync otherwise it causes a mypy error because it + # can't figure out it's a generator i guess?..? + def drain(self) -> AsyncIterator[dict]: + ... + + @property + def laddr(self) -> tuple[str, int]: + ... + + @property + def raddr(self) -> tuple[str, int]: + ... From c7d5b021db4c6a3c2ad6445dcb58593e97862cb0 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sat, 22 Mar 2025 15:29:48 -0300 Subject: [PATCH 13/74] Starting to make `.ipc.Channel` work with multiple MsgTransports --- tractor/_discovery.py | 8 +- tractor/_root.py | 2 +- tractor/_runtime.py | 5 +- tractor/ipc/__init__.py | 20 +- tractor/ipc/_chan.py | 121 +++-------- tractor/ipc/_tcp.py | 410 +++++--------------------------------- tractor/ipc/_transport.py | 382 ++++++++++++++++++++++++++++++++++- tractor/ipc/_types.py | 101 ++++++++++ tractor/ipc/_uds.py | 84 ++++++++ 9 files changed, 657 insertions(+), 476 deletions(-) create mode 100644 tractor/ipc/_types.py create mode 100644 tractor/ipc/_uds.py diff --git a/tractor/_discovery.py b/tractor/_discovery.py index 7dcfeaaa..d94f8312 100644 --- a/tractor/_discovery.py +++ b/tractor/_discovery.py @@ -75,7 +75,7 @@ async def get_registry( # TODO: try to look pre-existing connection from # `Actor._peers` and use it instead? async with ( - _connect_chan(host, port) as chan, + _connect_chan((host, port)) as chan, open_portal(chan) as regstr_ptl, ): yield regstr_ptl @@ -93,7 +93,7 @@ async def get_root( assert host is not None async with ( - _connect_chan(host, port) as chan, + _connect_chan((host, port)) as chan, open_portal(chan, **kwargs) as portal, ): yield portal @@ -193,7 +193,7 @@ async def maybe_open_portal( pass if sockaddr: - async with _connect_chan(*sockaddr) as chan: + async with _connect_chan(sockaddr) as chan: async with open_portal(chan) as portal: yield portal else: @@ -316,6 +316,6 @@ async def wait_for_actor( # TODO: offer multi-portal yields in multi-homed case? sockaddr: tuple[str, int] = sockaddrs[-1] - async with _connect_chan(*sockaddr) as chan: + async with _connect_chan(sockaddr) as chan: async with open_portal(chan) as portal: yield portal diff --git a/tractor/_root.py b/tractor/_root.py index 35639c15..40682a7a 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -271,7 +271,7 @@ async def open_root_actor( # be better to eventually have a "discovery" protocol # with basic handshake instead? with trio.move_on_after(timeout): - async with _connect_chan(*addr): + async with _connect_chan(addr): ponged_addrs.append(addr) except OSError: diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 2c8dbbd9..eaab31b6 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -1040,10 +1040,7 @@ 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 = Channel( - destaddr=parent_addr, - ) - await chan.connect() + chan = await Channel.from_destaddr(parent_addr) # TODO: move this into a `Channel.handshake()`? # Initial handshake: swap names. diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index cd16a139..a2a49365 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -13,20 +13,26 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . - - import platform -from ._transport import MsgTransport as MsgTransport +from ._transport import ( + AddressType as AddressType, + MsgType as MsgType, + MsgTransport as MsgTransport, + MsgpackTransport as MsgpackTransport +) -from ._tcp import ( - get_stream_addrs as get_stream_addrs, - MsgpackTCPStream as MsgpackTCPStream +from ._tcp import MsgpackTCPStream as MsgpackTCPStream +from ._uds import MsgpackUDSStream as MsgpackUDSStream + +from ._types import ( + transport_from_destaddr as transport_from_destaddr, + transport_from_stream as transport_from_stream, + AddressTypes as AddressTypes ) from ._chan import ( _connect_chan as _connect_chan, - get_msg_transport as get_msg_transport, Channel as Channel ) diff --git a/tractor/ipc/_chan.py b/tractor/ipc/_chan.py index 1b6ba29f..ee259371 100644 --- a/tractor/ipc/_chan.py +++ b/tractor/ipc/_chan.py @@ -29,15 +29,15 @@ from pprint import pformat import typing from typing import ( Any, - Type ) import trio from tractor.ipc._transport import MsgTransport -from tractor.ipc._tcp import ( - MsgpackTCPStream, - get_stream_addrs +from tractor.ipc._types import ( + transport_from_destaddr, + transport_from_stream, + AddressTypes ) from tractor.log import get_logger from tractor._exceptions import ( @@ -52,17 +52,6 @@ log = get_logger(__name__) _is_windows = platform.system() == 'Windows' -def get_msg_transport( - - key: tuple[str, str], - -) -> Type[MsgTransport]: - - return { - ('msgpack', 'tcp'): MsgpackTCPStream, - }[key] - - class Channel: ''' An inter-process channel for communication between (remote) actors. @@ -77,10 +66,8 @@ class Channel: def __init__( self, - destaddr: tuple[str, int]|None, - - msg_transport_type_key: tuple[str, str] = ('msgpack', 'tcp'), - + destaddr: AddressTypes|None = None, + transport: MsgTransport|None = None, # TODO: optional reconnection support? # auto_reconnect: bool = False, # on_reconnect: typing.Callable[..., typing.Awaitable] = None, @@ -90,13 +77,11 @@ class Channel: # self._recon_seq = on_reconnect # self._autorecon = auto_reconnect - self._destaddr = destaddr - self._transport_key = msg_transport_type_key - # Either created in ``.connect()`` or passed in by # user in ``.from_stream()``. - self._stream: trio.SocketStream|None = None - self._transport: MsgTransport|None = None + self._transport: MsgTransport|None = transport + + self._destaddr = destaddr if destaddr else self._transport.raddr # set after handshake - always uid of far end self.uid: tuple[str, str]|None = None @@ -110,6 +95,10 @@ class Channel: # runtime. self._cancel_called: bool = False + @property + def stream(self) -> trio.abc.Stream | None: + return self._transport.stream if self._transport else None + @property def msgstream(self) -> MsgTransport: log.info( @@ -124,52 +113,31 @@ class Channel: @classmethod def from_stream( cls, - stream: trio.SocketStream, - **kwargs, - + stream: trio.abc.Stream, ) -> Channel: - - src, dst = get_stream_addrs(stream) - chan = Channel( - destaddr=dst, - **kwargs, + transport_cls = transport_from_stream(stream) + return Channel( + transport=transport_cls(stream) ) - # set immediately here from provided instance - chan._stream: trio.SocketStream = stream - chan.set_msg_transport(stream) - return chan + @classmethod + async def from_destaddr( + cls, + destaddr: AddressTypes, + **kwargs + ) -> Channel: + transport_cls = transport_from_destaddr(destaddr) + transport = await transport_cls.connect_to(destaddr, **kwargs) - def set_msg_transport( - self, - stream: trio.SocketStream, - type_key: tuple[str, str]|None = None, - - # XXX optionally provided codec pair for `msgspec`: - # https://jcristharif.com/msgspec/extending.html#mapping-to-from-native-types - codec: MsgCodec|None = None, - - ) -> MsgTransport: - type_key = ( - type_key - or - self._transport_key + log.transport( + f'Opened channel[{type(transport)}]: {transport.laddr} -> {transport.raddr}' ) - # get transport type, then - self._transport = get_msg_transport( - type_key - # instantiate an instance of the msg-transport - )( - stream, - codec=codec, - ) - return self._transport + return Channel(transport=transport) @cm def apply_codec( self, codec: MsgCodec, - ) -> None: ''' Temporarily override the underlying IPC msg codec for @@ -189,7 +157,7 @@ class Channel: return '' return repr( - self._transport.stream.socket._sock + self._transport ).replace( # type: ignore "socket.socket", "Channel", @@ -203,30 +171,6 @@ class Channel: def raddr(self) -> tuple[str, int]|None: return self._transport.raddr if self._transport else None - async def connect( - self, - destaddr: tuple[Any, ...] | None = None, - **kwargs - - ) -> MsgTransport: - - if self.connected(): - raise RuntimeError("channel is already connected?") - - destaddr = destaddr or self._destaddr - assert isinstance(destaddr, tuple) - - stream = await trio.open_tcp_stream( - *destaddr, - **kwargs - ) - transport = self.set_msg_transport(stream) - - log.transport( - f'Opened channel[{type(transport)}]: {self.laddr} -> {self.raddr}' - ) - return transport - # TODO: something like, # `pdbp.hideframe_on(errors=[MsgTypeError])` # instead of the `try/except` hack we have rn.. @@ -388,17 +332,14 @@ class Channel: @acm async def _connect_chan( - host: str, - port: int - + destaddr: AddressTypes ) -> typing.AsyncGenerator[Channel, None]: ''' Create and connect a channel with disconnect on context manager teardown. ''' - chan = Channel((host, port)) - await chan.connect() + chan = await Channel.from_destaddr(destaddr) yield chan with trio.CancelScope(shield=True): await chan.aclose() diff --git a/tractor/ipc/_tcp.py b/tractor/ipc/_tcp.py index 03185f82..71265f38 100644 --- a/tractor/ipc/_tcp.py +++ b/tractor/ipc/_tcp.py @@ -18,389 +18,75 @@ TCP implementation of tractor.ipc._transport.MsgTransport protocol ''' from __future__ import annotations -from collections.abc import ( - AsyncGenerator, - AsyncIterator, -) -import struct -from typing import ( - Any, - Callable, - Type, -) -import msgspec -from tricycle import BufferedReceiveStream import trio +from tractor.msg import MsgCodec from tractor.log import get_logger -from tractor._exceptions import ( - MsgTypeError, - TransportClosed, - _mk_send_mte, - _mk_recv_mte, -) -from tractor.msg import ( - _ctxvar_MsgCodec, - # _codec, XXX see `self._codec` sanity/debug checks - MsgCodec, - types as msgtypes, - pretty_struct, -) -from tractor.ipc import MsgTransport +from tractor.ipc._transport import MsgpackTransport log = get_logger(__name__) -def get_stream_addrs( - stream: trio.SocketStream -) -> tuple[ - tuple[str, int], # local - tuple[str, int], # remote -]: - ''' - Return the `trio` streaming transport prot's socket-addrs for - both the local and remote sides as a pair. - - ''' - # rn, should both be IP sockets - lsockname = stream.socket.getsockname() - rsockname = stream.socket.getpeername() - return ( - tuple(lsockname[:2]), - tuple(rsockname[:2]), - ) - - # TODO: typing oddity.. not sure why we have to inherit here, but it # seems to be an issue with `get_msg_transport()` returning # a `Type[Protocol]`; probably should make a `mypy` issue? -class MsgpackTCPStream(MsgTransport): +class MsgpackTCPStream(MsgpackTransport): ''' A ``trio.SocketStream`` delivering ``msgpack`` formatted data using the ``msgspec`` codec lib. ''' + address_type = tuple[str, int] layer_key: int = 4 name_key: str = 'tcp' - # TODO: better naming for this? - # -[ ] check how libp2p does naming for such things? - codec_key: str = 'msgpack' + # def __init__( + # self, + # stream: trio.SocketStream, + # prefix_size: int = 4, + # codec: CodecType = None, - def __init__( - self, - stream: trio.SocketStream, - prefix_size: int = 4, - - # XXX optionally provided codec pair for `msgspec`: - # https://jcristharif.com/msgspec/extending.html#mapping-to-from-native-types - # - # TODO: define this as a `Codec` struct which can be - # overriden dynamically by the application/runtime? - codec: tuple[ - Callable[[Any], Any]|None, # coder - Callable[[type, Any], Any]|None, # decoder - ]|None = None, - - ) -> None: - - self.stream = stream - assert self.stream.socket - - # should both be IP sockets - self._laddr, self._raddr = get_stream_addrs(stream) - - # create read loop instance - self._aiter_pkts = self._iter_packets() - self._send_lock = trio.StrictFIFOLock() - - # public i guess? - self.drained: list[dict] = [] - - self.recv_stream = BufferedReceiveStream( - transport_stream=stream - ) - self.prefix_size = prefix_size - - # allow for custom IPC msg interchange format - # dynamic override Bo - self._task = trio.lowlevel.current_task() - - # XXX for ctxvar debug only! - # self._codec: MsgCodec = ( - # codec - # or - # _codec._ctxvar_MsgCodec.get() - # ) - - async def _iter_packets(self) -> AsyncGenerator[dict, None]: - ''' - Yield `bytes`-blob decoded packets from the underlying TCP - stream using the current task's `MsgCodec`. - - This is a streaming routine implemented as an async generator - func (which was the original design, but could be changed?) - and is allocated by a `.__call__()` inside `.__init__()` where - it is assigned to the `._aiter_pkts` attr. - - ''' - decodes_failed: int = 0 - - while True: - try: - header: bytes = await self.recv_stream.receive_exactly(4) - except ( - ValueError, - ConnectionResetError, - - # not sure entirely why we need this but without it we - # seem to be getting racy failures here on - # arbiter/registry name subs.. - trio.BrokenResourceError, - - ) as trans_err: - - loglevel = 'transport' - match trans_err: - # case ( - # ConnectionResetError() - # ): - # loglevel = 'transport' - - # peer actor (graceful??) TCP EOF but `tricycle` - # seems to raise a 0-bytes-read? - case ValueError() if ( - 'unclean EOF' in trans_err.args[0] - ): - pass - - # peer actor (task) prolly shutdown quickly due - # to cancellation - case trio.BrokenResourceError() if ( - 'Connection reset by peer' in trans_err.args[0] - ): - pass - - # unless the disconnect condition falls under "a - # normal operation breakage" we usualy console warn - # about it. - case _: - loglevel: str = 'warning' - - - raise TransportClosed( - message=( - f'IPC transport already closed by peer\n' - f'x)> {type(trans_err)}\n' - f' |_{self}\n' - ), - loglevel=loglevel, - ) from trans_err - - # XXX definitely can happen if transport is closed - # manually by another `trio.lowlevel.Task` in the - # same actor; we use this in some simulated fault - # testing for ex, but generally should never happen - # under normal operation! - # - # NOTE: as such we always re-raise this error from the - # RPC msg loop! - except trio.ClosedResourceError as closure_err: - raise TransportClosed( - message=( - f'IPC transport already manually closed locally?\n' - f'x)> {type(closure_err)} \n' - f' |_{self}\n' - ), - loglevel='error', - raise_on_report=( - closure_err.args[0] == 'another task closed this fd' - or - closure_err.args[0] in ['another task closed this fd'] - ), - ) from closure_err - - # graceful TCP EOF disconnect - if header == b'': - raise TransportClosed( - message=( - f'IPC transport already gracefully closed\n' - f')>\n' - f'|_{self}\n' - ), - loglevel='transport', - # cause=??? # handy or no? - ) - - size: int - size, = struct.unpack(" None: - ''' - Send a msgpack encoded py-object-blob-as-msg over TCP. - - If `strict_types == True` then a `MsgTypeError` will be raised on any - invalid msg type - - ''' - __tracebackhide__: bool = hide_tb - - # XXX see `trio._sync.AsyncContextManagerMixin` for details - # on the `.acquire()`/`.release()` sequencing.. - async with self._send_lock: - - # NOTE: lookup the `trio.Task.context`'s var for - # the current `MsgCodec`. - codec: MsgCodec = _ctxvar_MsgCodec.get() - - # XXX for ctxvar debug only! - # if self._codec.pld_spec != codec.pld_spec: - # self._codec = codec - # log.runtime( - # f'Using new codec in {self}.send()\n' - # f'codec: {self._codec}\n\n' - # f'msg: {msg}\n' - # ) - - if type(msg) not in msgtypes.__msg_types__: - if strict_types: - raise _mk_send_mte( - msg, - codec=codec, - ) - else: - log.warning( - 'Sending non-`Msg`-spec msg?\n\n' - f'{msg}\n' - ) - - try: - bytes_data: bytes = codec.encode(msg) - except TypeError as _err: - typerr = _err - msgtyperr: MsgTypeError = _mk_send_mte( - msg, - codec=codec, - message=( - f'IPC-msg-spec violation in\n\n' - f'{pretty_struct.Struct.pformat(msg)}' - ), - src_type_error=typerr, - ) - raise msgtyperr from typerr - - # supposedly the fastest says, - # https://stackoverflow.com/a/54027962 - size: bytes = struct.pack(" - # except BaseException as _err: - # err = _err - # if not isinstance(err, MsgTypeError): - # __tracebackhide__: bool = False - # raise - - @property - def laddr(self) -> tuple[str, int]: - return self._laddr - - @property - def raddr(self) -> tuple[str, int]: - return self._raddr - - async def recv(self) -> Any: - return await self._aiter_pkts.asend(None) - - async def drain(self) -> AsyncIterator[dict]: - ''' - Drain the stream's remaining messages sent from - the far end until the connection is closed by - the peer. - - ''' - try: - async for msg in self._iter_packets(): - self.drained.append(msg) - except TransportClosed: - for msg in self.drained: - yield msg - - def __aiter__(self): - return self._aiter_pkts + # ) -> None: + # super().__init__( + # stream, + # prefix_size=prefix_size, + # codec=codec + # ) def connected(self) -> bool: return self.stream.socket.fileno() != -1 + + @classmethod + async def connect_to( + cls, + destaddr: tuple[str, int], + prefix_size: int = 4, + codec: MsgCodec|None = None, + **kwargs + ) -> MsgpackTCPStream: + stream = await trio.open_tcp_stream( + *destaddr, + **kwargs + ) + return MsgpackTCPStream( + stream, + prefix_size=prefix_size, + codec=codec + ) + + @classmethod + def get_stream_addrs( + cls, + stream: trio.SocketStream + ) -> tuple[ + tuple[str, int], + tuple[str, int] + ]: + lsockname = stream.socket.getsockname() + rsockname = stream.socket.getpeername() + return ( + tuple(lsockname[:2]), + tuple(rsockname[:2]), + ) diff --git a/tractor/ipc/_transport.py b/tractor/ipc/_transport.py index 24e03a90..bd7f2f54 100644 --- a/tractor/ipc/_transport.py +++ b/tractor/ipc/_transport.py @@ -18,24 +18,56 @@ typing.Protocol based generic msg API, implement this class to add backends for tractor.ipc.Channel ''' -import trio +from __future__ import annotations from typing import ( runtime_checkable, + Type, Protocol, TypeVar, + ClassVar ) -from collections.abc import AsyncIterator +from collections.abc import ( + AsyncGenerator, + AsyncIterator, +) +import struct +from typing import ( + Any, + Callable, +) + +import trio +import msgspec +from tricycle import BufferedReceiveStream + +from tractor.log import get_logger +from tractor._exceptions import ( + MsgTypeError, + TransportClosed, + _mk_send_mte, + _mk_recv_mte, +) +from tractor.msg import ( + _ctxvar_MsgCodec, + # _codec, XXX see `self._codec` sanity/debug checks + MsgCodec, + types as msgtypes, + pretty_struct, +) + +log = get_logger(__name__) # from tractor.msg.types import MsgType # ?TODO? this should be our `Union[*msgtypes.__spec__]` alias now right..? # => BLEH, except can't bc prots must inherit typevar or param-spec # vars.. +AddressType = TypeVar('AddressType') MsgType = TypeVar('MsgType') @runtime_checkable -class MsgTransport(Protocol[MsgType]): +class MsgTransport(Protocol[AddressType, MsgType]): # # ^-TODO-^ consider using a generic def and indexing with our # eventual msg definition/types? @@ -43,9 +75,7 @@ class MsgTransport(Protocol[MsgType]): stream: trio.SocketStream drained: list[MsgType] - - def __init__(self, stream: trio.SocketStream) -> None: - ... + address_type: ClassVar[Type[AddressType]] # XXX: should this instead be called `.sendall()`? async def send(self, msg: MsgType) -> None: @@ -66,9 +96,345 @@ class MsgTransport(Protocol[MsgType]): ... @property - def laddr(self) -> tuple[str, int]: + def laddr(self) -> AddressType: ... @property - def raddr(self) -> tuple[str, int]: + def raddr(self) -> AddressType: ... + + @classmethod + async def connect_to( + cls, + destaddr: AddressType, + **kwargs + ) -> MsgTransport: + ... + + @classmethod + def get_stream_addrs( + cls, + stream: trio.abc.Stream + ) -> tuple[ + AddressType, # local + AddressType # remote + ]: + ''' + Return the `trio` streaming transport prot's addrs for both + the local and remote sides as a pair. + + ''' + ... + + +class MsgpackTransport(MsgTransport): + + # TODO: better naming for this? + # -[ ] check how libp2p does naming for such things? + codec_key: str = 'msgpack' + + def __init__( + self, + stream: trio.abc.Stream, + prefix_size: int = 4, + + # XXX optionally provided codec pair for `msgspec`: + # https://jcristharif.com/msgspec/extending.html#mapping-to-from-native-types + # + # TODO: define this as a `Codec` struct which can be + # overriden dynamically by the application/runtime? + codec: MsgCodec = None, + + ) -> None: + self.stream = stream + self._laddr, self._raddr = self.get_stream_addrs(stream) + + # create read loop instance + self._aiter_pkts = self._iter_packets() + self._send_lock = trio.StrictFIFOLock() + + # public i guess? + self.drained: list[dict] = [] + + self.recv_stream = BufferedReceiveStream( + transport_stream=stream + ) + self.prefix_size = prefix_size + + # allow for custom IPC msg interchange format + # dynamic override Bo + self._task = trio.lowlevel.current_task() + + # XXX for ctxvar debug only! + # self._codec: MsgCodec = ( + # codec + # or + # _codec._ctxvar_MsgCodec.get() + # ) + + async def _iter_packets(self) -> AsyncGenerator[dict, None]: + ''' + Yield `bytes`-blob decoded packets from the underlying TCP + stream using the current task's `MsgCodec`. + + This is a streaming routine implemented as an async generator + func (which was the original design, but could be changed?) + and is allocated by a `.__call__()` inside `.__init__()` where + it is assigned to the `._aiter_pkts` attr. + + ''' + decodes_failed: int = 0 + + while True: + try: + header: bytes = await self.recv_stream.receive_exactly(4) + except ( + ValueError, + ConnectionResetError, + + # not sure entirely why we need this but without it we + # seem to be getting racy failures here on + # arbiter/registry name subs.. + trio.BrokenResourceError, + + ) as trans_err: + + loglevel = 'transport' + match trans_err: + # case ( + # ConnectionResetError() + # ): + # loglevel = 'transport' + + # peer actor (graceful??) TCP EOF but `tricycle` + # seems to raise a 0-bytes-read? + case ValueError() if ( + 'unclean EOF' in trans_err.args[0] + ): + pass + + # peer actor (task) prolly shutdown quickly due + # to cancellation + case trio.BrokenResourceError() if ( + 'Connection reset by peer' in trans_err.args[0] + ): + pass + + # unless the disconnect condition falls under "a + # normal operation breakage" we usualy console warn + # about it. + case _: + loglevel: str = 'warning' + + + raise TransportClosed( + message=( + f'IPC transport already closed by peer\n' + f'x)> {type(trans_err)}\n' + f' |_{self}\n' + ), + loglevel=loglevel, + ) from trans_err + + # XXX definitely can happen if transport is closed + # manually by another `trio.lowlevel.Task` in the + # same actor; we use this in some simulated fault + # testing for ex, but generally should never happen + # under normal operation! + # + # NOTE: as such we always re-raise this error from the + # RPC msg loop! + except trio.ClosedResourceError as closure_err: + raise TransportClosed( + message=( + f'IPC transport already manually closed locally?\n' + f'x)> {type(closure_err)} \n' + f' |_{self}\n' + ), + loglevel='error', + raise_on_report=( + closure_err.args[0] == 'another task closed this fd' + or + closure_err.args[0] in ['another task closed this fd'] + ), + ) from closure_err + + # graceful TCP EOF disconnect + if header == b'': + raise TransportClosed( + message=( + f'IPC transport already gracefully closed\n' + f')>\n' + f'|_{self}\n' + ), + loglevel='transport', + # cause=??? # handy or no? + ) + + size: int + size, = struct.unpack(" None: + ''' + Send a msgpack encoded py-object-blob-as-msg over TCP. + + If `strict_types == True` then a `MsgTypeError` will be raised on any + invalid msg type + + ''' + __tracebackhide__: bool = hide_tb + + # XXX see `trio._sync.AsyncContextManagerMixin` for details + # on the `.acquire()`/`.release()` sequencing.. + async with self._send_lock: + + # NOTE: lookup the `trio.Task.context`'s var for + # the current `MsgCodec`. + codec: MsgCodec = _ctxvar_MsgCodec.get() + + # XXX for ctxvar debug only! + # if self._codec.pld_spec != codec.pld_spec: + # self._codec = codec + # log.runtime( + # f'Using new codec in {self}.send()\n' + # f'codec: {self._codec}\n\n' + # f'msg: {msg}\n' + # ) + + if type(msg) not in msgtypes.__msg_types__: + if strict_types: + raise _mk_send_mte( + msg, + codec=codec, + ) + else: + log.warning( + 'Sending non-`Msg`-spec msg?\n\n' + f'{msg}\n' + ) + + try: + bytes_data: bytes = codec.encode(msg) + except TypeError as _err: + typerr = _err + msgtyperr: MsgTypeError = _mk_send_mte( + msg, + codec=codec, + message=( + f'IPC-msg-spec violation in\n\n' + f'{pretty_struct.Struct.pformat(msg)}' + ), + src_type_error=typerr, + ) + raise msgtyperr from typerr + + # supposedly the fastest says, + # https://stackoverflow.com/a/54027962 + size: bytes = struct.pack(" + # except BaseException as _err: + # err = _err + # if not isinstance(err, MsgTypeError): + # __tracebackhide__: bool = False + # raise + + async def recv(self) -> Any: + return await self._aiter_pkts.asend(None) + + async def drain(self) -> AsyncIterator[dict]: + ''' + Drain the stream's remaining messages sent from + the far end until the connection is closed by + the peer. + + ''' + try: + async for msg in self._iter_packets(): + self.drained.append(msg) + except TransportClosed: + for msg in self.drained: + yield msg + + def __aiter__(self): + return self._aiter_pkts + + @property + def laddr(self) -> AddressType: + return self._laddr + + @property + def raddr(self) -> AddressType: + return self._raddr diff --git a/tractor/ipc/_types.py b/tractor/ipc/_types.py new file mode 100644 index 00000000..93c5e3c9 --- /dev/null +++ b/tractor/ipc/_types.py @@ -0,0 +1,101 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . +from typing import Type, Union + +import trio +import socket + +from ._transport import MsgTransport +from ._tcp import MsgpackTCPStream +from ._uds import MsgpackUDSStream + + +# manually updated list of all supported codec+transport types +_msg_transports = { + ('msgpack', 'tcp'): MsgpackTCPStream, + ('msgpack', 'uds'): MsgpackUDSStream +} + + +# all different address py types we use +AddressTypes = Union[ + tuple([ + cls.address_type + for key, cls in _msg_transports.items() + ]) +] + + +def transport_from_destaddr( + destaddr: AddressTypes, + codec_key: str = 'msgpack', +) -> Type[MsgTransport]: + ''' + Given a destination address and a desired codec, find the + corresponding `MsgTransport` type. + + ''' + match destaddr: + case str(): + return MsgpackUDSStream + + case tuple(): + if ( + len(destaddr) == 2 + and + isinstance(destaddr[0], str) + and + isinstance(destaddr[1], int) + ): + return MsgpackTCPStream + + raise NotImplementedError( + f'No known transport for address {destaddr}' + ) + + +def transport_from_stream( + stream: trio.abc.Stream, + codec_key: str = 'msgpack' +) -> Type[MsgTransport]: + ''' + Given an arbitrary `trio.abc.Stream` and a desired codec, + find the corresponding `MsgTransport` type. + + ''' + transport = None + if isinstance(stream, trio.SocketStream): + sock = stream.socket + match sock.family: + case socket.AF_INET | socket.AF_INET6: + transport = 'tcp' + + case socket.AF_UNIX: + transport = 'uds' + + case _: + raise NotImplementedError( + f'Unsupported socket family: {sock.family}' + ) + + if not transport: + raise NotImplementedError( + f'Could not figure out transport type for stream type {type(stream)}' + ) + + key = (codec_key, transport) + + return _msg_transports[key] diff --git a/tractor/ipc/_uds.py b/tractor/ipc/_uds.py new file mode 100644 index 00000000..3b848898 --- /dev/null +++ b/tractor/ipc/_uds.py @@ -0,0 +1,84 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . +''' +Unix Domain Socket implementation of tractor.ipc._transport.MsgTransport protocol + +''' +from __future__ import annotations + +import trio + +from tractor.msg import MsgCodec +from tractor.log import get_logger +from tractor.ipc._transport import MsgpackTransport + + +log = get_logger(__name__) + + +class MsgpackUDSStream(MsgpackTransport): + ''' + A ``trio.SocketStream`` delivering ``msgpack`` formatted data + using the ``msgspec`` codec lib. + + ''' + address_type = str + layer_key: int = 7 + name_key: str = 'uds' + + # def __init__( + # self, + # stream: trio.SocketStream, + # prefix_size: int = 4, + # codec: CodecType = None, + + # ) -> None: + # super().__init__( + # stream, + # prefix_size=prefix_size, + # codec=codec + # ) + + def connected(self) -> bool: + return self.stream.socket.fileno() != -1 + + @classmethod + async def connect_to( + cls, + filename: str, + prefix_size: int = 4, + codec: MsgCodec|None = None, + **kwargs + ) -> MsgpackUDSStream: + stream = await trio.open_unix_socket( + filename, + **kwargs + ) + return MsgpackUDSStream( + stream, + prefix_size=prefix_size, + codec=codec + ) + + @classmethod + def get_stream_addrs( + cls, + stream: trio.SocketStream + ) -> tuple[str, str]: + return ( + stream.socket.getsockname(), + stream.socket.getpeername(), + ) From 5e2d456029bf9213e5cca0925793abf17890e95e Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sat, 22 Mar 2025 16:17:50 -0300 Subject: [PATCH 14/74] Add root and random addr getters on MsgTransport type --- tractor/ipc/__init__.py | 2 ++ tractor/ipc/_tcp.py | 8 ++++++++ tractor/ipc/_transport.py | 25 ++++++++++++++++++++----- tractor/ipc/_types.py | 25 ++++++++++++++++++++----- tractor/ipc/_uds.py | 10 ++++++++++ tractor/msg/types.py | 5 +++-- 6 files changed, 63 insertions(+), 12 deletions(-) diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index a2a49365..a6998ae2 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -16,6 +16,7 @@ import platform from ._transport import ( + MsgTransportKey as MsgTransportKey, AddressType as AddressType, MsgType as MsgType, MsgTransport as MsgTransport, @@ -26,6 +27,7 @@ from ._tcp import MsgpackTCPStream as MsgpackTCPStream from ._uds import MsgpackUDSStream as MsgpackUDSStream from ._types import ( + default_lo_addrs as default_lo_addrs, transport_from_destaddr as transport_from_destaddr, transport_from_stream as transport_from_stream, AddressTypes as AddressTypes diff --git a/tractor/ipc/_tcp.py b/tractor/ipc/_tcp.py index 71265f38..4a69ebbd 100644 --- a/tractor/ipc/_tcp.py +++ b/tractor/ipc/_tcp.py @@ -90,3 +90,11 @@ class MsgpackTCPStream(MsgpackTransport): tuple(lsockname[:2]), tuple(rsockname[:2]), ) + + @classmethod + def get_random_addr(self) -> tuple[str, int]: + return (None, 0) + + @classmethod + def get_root_addr(self) -> tuple[str, int]: + return ('127.0.0.1', 1616) diff --git a/tractor/ipc/_transport.py b/tractor/ipc/_transport.py index bd7f2f54..76fe048c 100644 --- a/tractor/ipc/_transport.py +++ b/tractor/ipc/_transport.py @@ -31,10 +31,6 @@ from collections.abc import ( AsyncIterator, ) import struct -from typing import ( - Any, - Callable, -) import trio import msgspec @@ -58,6 +54,10 @@ from tractor.msg import ( log = get_logger(__name__) +# (codec, transport) +MsgTransportKey = tuple[str, str] + + # from tractor.msg.types import MsgType # ?TODO? this should be our `Union[*msgtypes.__spec__]` alias now right..? # => BLEH, except can't bc prots must inherit typevar or param-spec @@ -77,6 +77,9 @@ class MsgTransport(Protocol[AddressType, MsgType]): drained: list[MsgType] address_type: ClassVar[Type[AddressType]] + codec_key: ClassVar[str] + name_key: ClassVar[str] + # XXX: should this instead be called `.sendall()`? async def send(self, msg: MsgType) -> None: ... @@ -95,6 +98,10 @@ class MsgTransport(Protocol[AddressType, MsgType]): def drain(self) -> AsyncIterator[dict]: ... + @classmethod + def key(cls) -> MsgTransportKey: + return cls.codec_key, cls.name_key + @property def laddr(self) -> AddressType: ... @@ -126,6 +133,14 @@ class MsgTransport(Protocol[AddressType, MsgType]): ''' ... + @classmethod + def get_random_addr(self) -> AddressType: + ... + + @classmethod + def get_root_addr(self) -> AddressType: + ... + class MsgpackTransport(MsgTransport): @@ -411,7 +426,7 @@ class MsgpackTransport(MsgTransport): # __tracebackhide__: bool = False # raise - async def recv(self) -> Any: + async def recv(self) -> msgtypes.MsgType: return await self._aiter_pkts.asend(None) async def drain(self) -> AsyncIterator[dict]: diff --git a/tractor/ipc/_types.py b/tractor/ipc/_types.py index 93c5e3c9..92d3af91 100644 --- a/tractor/ipc/_types.py +++ b/tractor/ipc/_types.py @@ -18,15 +18,24 @@ from typing import Type, Union import trio import socket -from ._transport import MsgTransport +from ._transport import ( + MsgTransportKey, + MsgTransport +) from ._tcp import MsgpackTCPStream from ._uds import MsgpackUDSStream +_msg_transports = [ + MsgpackTCPStream, + MsgpackUDSStream +] + + # manually updated list of all supported codec+transport types -_msg_transports = { - ('msgpack', 'tcp'): MsgpackTCPStream, - ('msgpack', 'uds'): MsgpackUDSStream +key_to_transport: dict[MsgTransportKey, Type[MsgTransport]] = { + cls.key(): cls + for cls in _msg_transports } @@ -34,11 +43,17 @@ _msg_transports = { AddressTypes = Union[ tuple([ cls.address_type - for key, cls in _msg_transports.items() + for cls in _msg_transports ]) ] +default_lo_addrs: dict[MsgTransportKey, AddressTypes] = { + cls.key(): cls.get_root_addr() + for cls in _msg_transports +} + + def transport_from_destaddr( destaddr: AddressTypes, codec_key: str = 'msgpack', diff --git a/tractor/ipc/_uds.py b/tractor/ipc/_uds.py index 3b848898..eb2e7f32 100644 --- a/tractor/ipc/_uds.py +++ b/tractor/ipc/_uds.py @@ -18,6 +18,8 @@ Unix Domain Socket implementation of tractor.ipc._transport.MsgTransport protoco ''' from __future__ import annotations +import tempfile +from uuid import uuid4 import trio @@ -82,3 +84,11 @@ class MsgpackUDSStream(MsgpackTransport): stream.socket.getsockname(), stream.socket.getpeername(), ) + + @classmethod + def get_random_addr(self) -> str: + return f'{tempfile.gettempdir()}/{uuid4()}.sock' + + @classmethod + def get_root_addr(self) -> str: + return 'tractor.sock' diff --git a/tractor/msg/types.py b/tractor/msg/types.py index 1cc8b78e..76d0bad6 100644 --- a/tractor/msg/types.py +++ b/tractor/msg/types.py @@ -46,6 +46,7 @@ from msgspec import ( from tractor.msg import ( pretty_struct, ) +from tractor.ipc import AddressTypes from tractor.log import get_logger @@ -167,8 +168,8 @@ class SpawnSpec( # TODO: not just sockaddr pairs? # -[ ] abstract into a `TransportAddr` type? - reg_addrs: list[tuple[str, int]] - bind_addrs: list[tuple[str, int]] + reg_addrs: list[AddressTypes] + bind_addrs: list[AddressTypes] # TODO: caps based RPC support in the payload? From 0be9f5f9073b70235ac832bc9422fd5c30fea7e5 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sun, 23 Mar 2025 00:14:04 -0300 Subject: [PATCH 15/74] Finally switch to using address protocol in all runtime --- examples/service_discovery.py | 2 +- tests/test_discovery.py | 6 +- tests/test_inter_peer_cancellation.py | 2 +- tests/test_local.py | 2 +- tests/test_multi_program.py | 4 +- tests/test_spawning.py | 2 +- tractor/_addr.py | 301 ++++++++++++++++++++++++++ tractor/_child.py | 3 +- tractor/_context.py | 11 +- tractor/_discovery.py | 62 +++--- tractor/_entry.py | 7 +- tractor/_root.py | 56 ++--- tractor/_runtime.py | 182 ++++++++-------- tractor/_spawn.py | 29 +-- tractor/_supervise.py | 19 +- tractor/ipc/__init__.py | 5 +- tractor/ipc/_chan.py | 32 +-- tractor/ipc/_tcp.py | 33 +-- tractor/ipc/_transport.py | 35 ++- tractor/ipc/_types.py | 57 ++--- tractor/ipc/_uds.py | 37 ++-- tractor/msg/types.py | 2 +- 22 files changed, 586 insertions(+), 303 deletions(-) create mode 100644 tractor/_addr.py diff --git a/examples/service_discovery.py b/examples/service_discovery.py index a0f37b88..64697e5b 100644 --- a/examples/service_discovery.py +++ b/examples/service_discovery.py @@ -9,7 +9,7 @@ async def main(service_name): async with tractor.open_nursery() as an: await an.start_actor(service_name) - async with tractor.get_registry('127.0.0.1', 1616) as portal: + async with tractor.get_registry(('127.0.0.1', 1616)) as portal: print(f"Arbiter is listening on {portal.channel}") async with tractor.wait_for_actor(service_name) as sockaddr: diff --git a/tests/test_discovery.py b/tests/test_discovery.py index 87455983..18b2aa1b 100644 --- a/tests/test_discovery.py +++ b/tests/test_discovery.py @@ -26,7 +26,7 @@ async def test_reg_then_unreg(reg_addr): portal = await n.start_actor('actor', enable_modules=[__name__]) uid = portal.channel.uid - async with tractor.get_registry(*reg_addr) as aportal: + async with tractor.get_registry(reg_addr) as aportal: # this local actor should be the arbiter assert actor is aportal.actor @@ -160,7 +160,7 @@ async def spawn_and_check_registry( async with tractor.open_root_actor( registry_addrs=[reg_addr], ): - async with tractor.get_registry(*reg_addr) as portal: + async with tractor.get_registry(reg_addr) as portal: # runtime needs to be up to call this actor = tractor.current_actor() @@ -300,7 +300,7 @@ async def close_chans_before_nursery( async with tractor.open_root_actor( registry_addrs=[reg_addr], ): - async with tractor.get_registry(*reg_addr) as aportal: + async with tractor.get_registry(reg_addr) as aportal: try: get_reg = partial(unpack_reg, aportal) diff --git a/tests/test_inter_peer_cancellation.py b/tests/test_inter_peer_cancellation.py index bac9a791..25935df2 100644 --- a/tests/test_inter_peer_cancellation.py +++ b/tests/test_inter_peer_cancellation.py @@ -871,7 +871,7 @@ async def serve_subactors( ) await ipc.send(( peer.chan.uid, - peer.chan.raddr, + peer.chan.raddr.unwrap(), )) print('Spawner exiting spawn serve loop!') diff --git a/tests/test_local.py b/tests/test_local.py index ecdad5fe..c6f5047a 100644 --- a/tests/test_local.py +++ b/tests/test_local.py @@ -38,7 +38,7 @@ async def test_self_is_registered_localportal(reg_addr): "Verify waiting on the arbiter to register itself using a local portal." actor = tractor.current_actor() assert actor.is_arbiter - async with tractor.get_registry(*reg_addr) as portal: + async with tractor.get_registry(reg_addr) as portal: assert isinstance(portal, tractor._portal.LocalPortal) with trio.fail_after(0.2): diff --git a/tests/test_multi_program.py b/tests/test_multi_program.py index 860eeebb..b0b145ee 100644 --- a/tests/test_multi_program.py +++ b/tests/test_multi_program.py @@ -32,7 +32,7 @@ def test_abort_on_sigint(daemon): @tractor_test async def test_cancel_remote_arbiter(daemon, reg_addr): assert not tractor.current_actor().is_arbiter - async with tractor.get_registry(*reg_addr) as portal: + async with tractor.get_registry(reg_addr) as portal: await portal.cancel_actor() time.sleep(0.1) @@ -41,7 +41,7 @@ async def test_cancel_remote_arbiter(daemon, reg_addr): # no arbiter socket should exist with pytest.raises(OSError): - async with tractor.get_registry(*reg_addr) as portal: + async with tractor.get_registry(reg_addr) as portal: pass diff --git a/tests/test_spawning.py b/tests/test_spawning.py index 99ec9abc..58aa955a 100644 --- a/tests/test_spawning.py +++ b/tests/test_spawning.py @@ -77,7 +77,7 @@ async def movie_theatre_question(): async def test_movie_theatre_convo(start_method): """The main ``tractor`` routine. """ - async with tractor.open_nursery() as n: + async with tractor.open_nursery(debug_mode=True) as n: portal = await n.start_actor( 'frank', diff --git a/tractor/_addr.py b/tractor/_addr.py new file mode 100644 index 00000000..0944c89d --- /dev/null +++ b/tractor/_addr.py @@ -0,0 +1,301 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . +from __future__ import annotations +import tempfile +from uuid import uuid4 +from typing import ( + Protocol, + ClassVar, + TypeVar, + Union, + Type +) + +import trio +from trio import socket + + +NamespaceType = TypeVar('NamespaceType') +AddressType = TypeVar('AddressType') +StreamType = TypeVar('StreamType') +ListenerType = TypeVar('ListenerType') + + +class Address(Protocol[ + NamespaceType, + AddressType, + StreamType, + ListenerType +]): + + name_key: ClassVar[str] + address_type: ClassVar[Type[AddressType]] + + @property + def is_valid(self) -> bool: + ... + + @property + def namespace(self) -> NamespaceType|None: + ... + + @classmethod + def from_addr(cls, addr: AddressType) -> Address: + ... + + def unwrap(self) -> AddressType: + ... + + @classmethod + def get_random(cls, namespace: NamespaceType | None = None) -> Address: + ... + + @classmethod + def get_root(cls) -> Address: + ... + + def __repr__(self) -> str: + ... + + def __eq__(self, other) -> bool: + ... + + async def open_stream(self, **kwargs) -> StreamType: + ... + + async def open_listener(self, **kwargs) -> ListenerType: + ... + + +class TCPAddress(Address[ + str, + tuple[str, int], + trio.SocketStream, + trio.SocketListener +]): + + name_key: str = 'tcp' + address_type: type = tuple[str, int] + + def __init__( + self, + host: str, + port: int + ): + if ( + not isinstance(host, str) + or + not isinstance(port, int) + ): + raise TypeError(f'Expected host {host} to be str and port {port} to be int') + self._host = host + self._port = port + + @property + def is_valid(self) -> bool: + return self._port != 0 + + @property + def namespace(self) -> str: + return self._host + + @classmethod + def from_addr(cls, addr: tuple[str, int]) -> TCPAddress: + return TCPAddress(addr[0], addr[1]) + + def unwrap(self) -> tuple[str, int]: + return self._host, self._port + + @classmethod + def get_random(cls, namespace: str = '127.0.0.1') -> TCPAddress: + return TCPAddress(namespace, 0) + + @classmethod + def get_root(cls) -> Address: + return TCPAddress('127.0.0.1', 1616) + + def __repr__(self) -> str: + return f'{type(self)} @ {self.unwrap()}' + + def __eq__(self, other) -> bool: + if not isinstance(other, TCPAddress): + raise TypeError( + f'Can not compare {type(other)} with {type(self)}' + ) + + return ( + self._host == other._host + and + self._port == other._port + ) + + async def open_stream(self, **kwargs) -> trio.SocketStream: + stream = await trio.open_tcp_stream( + self._host, + self._port, + **kwargs + ) + self._host, self._port = stream.socket.getsockname()[:2] + return stream + + async def open_listener(self, **kwargs) -> trio.SocketListener: + listeners = await trio.open_tcp_listeners( + host=self._host, + port=self._port, + **kwargs + ) + assert len(listeners) == 1 + listener = listeners[0] + self._host, self._port = listener.socket.getsockname()[:2] + return listener + + +class UDSAddress(Address[ + None, + str, + trio.SocketStream, + trio.SocketListener +]): + + name_key: str = 'uds' + address_type: type = str + + def __init__( + self, + filepath: str + ): + self._filepath = filepath + + @property + def is_valid(self) -> bool: + return True + + @property + def namespace(self) -> None: + return + + @classmethod + def from_addr(cls, filepath: str) -> UDSAddress: + return UDSAddress(filepath) + + def unwrap(self) -> str: + return self._filepath + + @classmethod + def get_random(cls, _ns: None = None) -> UDSAddress: + return UDSAddress(f'{tempfile.gettempdir()}/{uuid4().sock}') + + @classmethod + def get_root(cls) -> Address: + return UDSAddress('tractor.sock') + + def __repr__(self) -> str: + return f'{type(self)} @ {self._filepath}' + + def __eq__(self, other) -> bool: + if not isinstance(other, UDSAddress): + raise TypeError( + f'Can not compare {type(other)} with {type(self)}' + ) + + return self._filepath == other._filepath + + async def open_stream(self, **kwargs) -> trio.SocketStream: + stream = await trio.open_tcp_stream( + self._filepath, + **kwargs + ) + self._binded = True + return stream + + async def open_listener(self, **kwargs) -> trio.SocketListener: + sock = socket.socket(socket.AF_UNIX, socket.SOCK_STREAM) + sock.bind(self._filepath) + sock.listen() + self._binded = True + return trio.SocketListener(sock) + + +preferred_transport = 'tcp' + + +_address_types = ( + TCPAddress, + UDSAddress +) + + +_default_addrs: dict[str, Type[Address]] = { + cls.name_key: cls + for cls in _address_types +} + + +AddressTypes = Union[ + tuple([ + cls.address_type + for cls in _address_types + ]) +] + + +_default_lo_addrs: dict[ + str, + AddressTypes +] = { + cls.name_key: cls.get_root().unwrap() + for cls in _address_types +} + + +def get_address_cls(name: str) -> Type[Address]: + return _default_addrs[name] + + +def is_wrapped_addr(addr: any) -> bool: + return type(addr) in _address_types + + +def wrap_address(addr: AddressTypes) -> Address: + + if is_wrapped_addr(addr): + return addr + + cls = None + match addr: + case str(): + cls = UDSAddress + + case tuple() | list(): + cls = TCPAddress + + case None: + cls = get_address_cls(preferred_transport) + addr = cls.get_root().unwrap() + + case _: + raise TypeError( + f'Can not wrap addr {addr} of type {type(addr)}' + ) + + return cls.from_addr(addr) + + +def default_lo_addrs(transports: list[str]) -> list[AddressTypes]: + return [ + _default_lo_addrs[transport] + for transport in transports + ] diff --git a/tractor/_child.py b/tractor/_child.py index 4226ae90..69142889 100644 --- a/tractor/_child.py +++ b/tractor/_child.py @@ -31,8 +31,7 @@ def parse_uid(arg): return str(name), str(uuid) # ensures str encoding def parse_ipaddr(arg): - host, port = literal_eval(arg) - return (str(host), int(port)) + return literal_eval(arg) if __name__ == "__main__": diff --git a/tractor/_context.py b/tractor/_context.py index d93d7759..19f3daef 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -859,19 +859,10 @@ class Context: @property def dst_maddr(self) -> str: chan: Channel = self.chan - dst_addr, dst_port = chan.raddr trans: MsgTransport = chan.transport # cid: str = self.cid # cid_head, cid_tail = cid[:6], cid[-6:] - return ( - f'/ipv4/{dst_addr}' - f'/{trans.name_key}/{dst_port}' - # f'/{self.chan.uid[0]}' - # f'/{self.cid}' - - # f'/cid={cid_head}..{cid_tail}' - # TODO: ? not use this ^ right ? - ) + return trans.maddr dmaddr = dst_maddr diff --git a/tractor/_discovery.py b/tractor/_discovery.py index d94f8312..76d3cccd 100644 --- a/tractor/_discovery.py +++ b/tractor/_discovery.py @@ -30,6 +30,12 @@ from contextlib import asynccontextmanager as acm from tractor.log import get_logger from .trionics import gather_contexts from .ipc import _connect_chan, Channel +from ._addr import ( + AddressTypes, + Address, + preferred_transport, + wrap_address +) from ._portal import ( Portal, open_portal, @@ -48,11 +54,7 @@ log = get_logger(__name__) @acm -async def get_registry( - host: str, - port: int, - -) -> AsyncGenerator[ +async def get_registry(addr: AddressTypes) -> AsyncGenerator[ Portal | LocalPortal | None, None, ]: @@ -69,13 +71,13 @@ async def get_registry( # (likely a re-entrant call from the arbiter actor) yield LocalPortal( actor, - Channel((host, port)) + await Channel.from_addr(addr) ) else: # TODO: try to look pre-existing connection from # `Actor._peers` and use it instead? async with ( - _connect_chan((host, port)) as chan, + _connect_chan(addr) as chan, open_portal(chan) as regstr_ptl, ): yield regstr_ptl @@ -89,11 +91,10 @@ async def get_root( # TODO: rename mailbox to `_root_maddr` when we finally # add and impl libp2p multi-addrs? - host, port = _runtime_vars['_root_mailbox'] - assert host is not None + addr = _runtime_vars['_root_mailbox'] async with ( - _connect_chan((host, port)) as chan, + _connect_chan(addr) as chan, open_portal(chan, **kwargs) as portal, ): yield portal @@ -140,10 +141,10 @@ def get_peer_by_name( @acm async def query_actor( name: str, - regaddr: tuple[str, int]|None = None, + regaddr: AddressTypes|None = None, ) -> AsyncGenerator[ - tuple[str, int]|None, + AddressTypes|None, None, ]: ''' @@ -169,31 +170,31 @@ async def query_actor( return reg_portal: Portal - regaddr: tuple[str, int] = regaddr or actor.reg_addrs[0] - async with get_registry(*regaddr) as reg_portal: + regaddr: Address = wrap_address(regaddr) or actor.reg_addrs[0] + async with get_registry(regaddr) as reg_portal: # TODO: return portals to all available actors - for now # just the last one that registered - sockaddr: tuple[str, int] = await reg_portal.run_from_ns( + addr: AddressTypes = await reg_portal.run_from_ns( 'self', 'find_actor', name=name, ) - yield sockaddr + yield addr @acm async def maybe_open_portal( - addr: tuple[str, int], + addr: AddressTypes, name: str, ): async with query_actor( name=name, regaddr=addr, - ) as sockaddr: + ) as addr: pass - if sockaddr: - async with _connect_chan(sockaddr) as chan: + if addr: + async with _connect_chan(addr) as chan: async with open_portal(chan) as portal: yield portal else: @@ -203,7 +204,8 @@ async def maybe_open_portal( @acm async def find_actor( name: str, - registry_addrs: list[tuple[str, int]]|None = None, + registry_addrs: list[AddressTypes]|None = None, + enable_transports: list[str] = [preferred_transport], only_first: bool = True, raise_on_none: bool = False, @@ -230,15 +232,15 @@ async def find_actor( # XXX NOTE: make sure to dynamically read the value on # every call since something may change it globally (eg. # like in our discovery test suite)! - from . import _root + from ._addr import default_lo_addrs registry_addrs = ( _runtime_vars['_registry_addrs'] or - _root._default_lo_addrs + default_lo_addrs(enable_transports) ) maybe_portals: list[ - AsyncContextManager[tuple[str, int]] + AsyncContextManager[AddressTypes] ] = list( maybe_open_portal( addr=addr, @@ -280,7 +282,7 @@ async def find_actor( @acm async def wait_for_actor( name: str, - registry_addr: tuple[str, int] | None = None, + registry_addr: AddressTypes | None = None, ) -> AsyncGenerator[Portal, None]: ''' @@ -297,7 +299,7 @@ async def wait_for_actor( yield peer_portal return - regaddr: tuple[str, int] = ( + regaddr: AddressTypes = ( registry_addr or actor.reg_addrs[0] @@ -305,8 +307,8 @@ async def wait_for_actor( # TODO: use `.trionics.gather_contexts()` like # above in `find_actor()` as well? reg_portal: Portal - async with get_registry(*regaddr) as reg_portal: - sockaddrs = await reg_portal.run_from_ns( + async with get_registry(regaddr) as reg_portal: + addrs = await reg_portal.run_from_ns( 'self', 'wait_for_actor', name=name, @@ -314,8 +316,8 @@ async def wait_for_actor( # get latest registered addr by default? # TODO: offer multi-portal yields in multi-homed case? - sockaddr: tuple[str, int] = sockaddrs[-1] + addr: AddressTypes = addrs[-1] - async with _connect_chan(sockaddr) as chan: + async with _connect_chan(addr) as chan: async with open_portal(chan) as portal: yield portal diff --git a/tractor/_entry.py b/tractor/_entry.py index 8156d25f..1328aa45 100644 --- a/tractor/_entry.py +++ b/tractor/_entry.py @@ -37,6 +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 ._runtime import ( async_main, Actor, @@ -52,10 +53,10 @@ log = get_logger(__name__) def _mp_main( actor: Actor, - accept_addrs: list[tuple[str, int]], + accept_addrs: list[AddressTypes], forkserver_info: tuple[Any, Any, Any, Any, Any], start_method: SpawnMethodKey, - parent_addr: tuple[str, int] | None = None, + parent_addr: AddressTypes | None = None, infect_asyncio: bool = False, ) -> None: @@ -206,7 +207,7 @@ def nest_from_op( def _trio_main( actor: Actor, *, - parent_addr: tuple[str, int] | None = None, + parent_addr: AddressTypes | None = None, infect_asyncio: bool = False, ) -> None: diff --git a/tractor/_root.py b/tractor/_root.py index 40682a7a..e9cac3f2 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -43,21 +43,18 @@ from .devx import _debug from . import _spawn from . import _state from . import log -from .ipc import _connect_chan +from .ipc import ( + _connect_chan, +) +from ._addr import ( + AddressTypes, + wrap_address, + preferred_transport, + default_lo_addrs +) from ._exceptions import is_multi_cancelled -# set at startup and after forks -_default_host: str = '127.0.0.1' -_default_port: int = 1616 - -# default registry always on localhost -_default_lo_addrs: list[tuple[str, int]] = [( - _default_host, - _default_port, -)] - - logger = log.get_logger('tractor') @@ -66,10 +63,12 @@ async def open_root_actor( *, # defaults are above - registry_addrs: list[tuple[str, int]]|None = None, + registry_addrs: list[AddressTypes]|None = None, # defaults are above - arbiter_addr: tuple[str, int]|None = None, + arbiter_addr: tuple[AddressTypes]|None = None, + + enable_transports: list[str] = [preferred_transport], name: str|None = 'root', @@ -195,11 +194,9 @@ async def open_root_actor( ) registry_addrs = [arbiter_addr] - registry_addrs: list[tuple[str, int]] = ( - registry_addrs - or - _default_lo_addrs - ) + if not registry_addrs: + registry_addrs: list[AddressTypes] = default_lo_addrs(enable_transports) + assert registry_addrs loglevel = ( @@ -248,10 +245,10 @@ async def open_root_actor( enable_stack_on_sig() # closed into below ping task-func - ponged_addrs: list[tuple[str, int]] = [] + ponged_addrs: list[AddressTypes] = [] async def ping_tpt_socket( - addr: tuple[str, int], + addr: AddressTypes, timeout: float = 1, ) -> None: ''' @@ -284,10 +281,10 @@ async def open_root_actor( for addr in registry_addrs: tn.start_soon( ping_tpt_socket, - tuple(addr), # TODO: just drop this requirement? + addr, ) - trans_bind_addrs: list[tuple[str, int]] = [] + trans_bind_addrs: list[AddressTypes] = [] # Create a new local root-actor instance which IS NOT THE # REGISTRAR @@ -311,9 +308,12 @@ async def open_root_actor( ) # DO NOT use the registry_addrs as the transport server # addrs for this new non-registar, root-actor. - for host, port in ponged_addrs: - # NOTE: zero triggers dynamic OS port allocation - trans_bind_addrs.append((host, 0)) + for addr in ponged_addrs: + waddr = wrap_address(addr) + print(waddr) + trans_bind_addrs.append( + waddr.get_random(namespace=waddr.namespace) + ) # Start this local actor as the "registrar", aka a regular # actor who manages the local registry of "mailboxes" of @@ -322,7 +322,7 @@ async def open_root_actor( # NOTE that if the current actor IS THE REGISTAR, the # following init steps are taken: - # - the tranport layer server is bound to each (host, port) + # - the tranport layer server is bound to each addr # pair defined in provided registry_addrs, or the default. trans_bind_addrs = registry_addrs @@ -462,7 +462,7 @@ def run_daemon( # runtime kwargs name: str | None = 'root', - registry_addrs: list[tuple[str, int]] = _default_lo_addrs, + registry_addrs: list[AddressTypes]|None = None, start_method: str | None = None, debug_mode: bool = False, diff --git a/tractor/_runtime.py b/tractor/_runtime.py index eaab31b6..e755d5ce 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -74,6 +74,12 @@ from tractor.msg import ( types as msgtypes, ) from .ipc import Channel +from ._addr import ( + AddressTypes, + Address, + TCPAddress, + wrap_address, +) from ._context import ( mk_context, Context, @@ -179,11 +185,11 @@ class Actor: enable_modules: list[str] = [], uid: str|None = None, loglevel: str|None = None, - registry_addrs: list[tuple[str, int]]|None = None, + registry_addrs: list[AddressTypes]|None = None, spawn_method: str|None = None, # TODO: remove! - arbiter_addr: tuple[str, int]|None = None, + arbiter_addr: AddressTypes|None = None, ) -> None: ''' @@ -223,7 +229,7 @@ class Actor: DeprecationWarning, stacklevel=2, ) - registry_addrs: list[tuple[str, int]] = [arbiter_addr] + registry_addrs: list[AddressTypes] = [arbiter_addr] # marked by the process spawning backend at startup # will be None for the parent most process started manually @@ -257,6 +263,7 @@ class Actor: ] = {} self._listeners: list[trio.abc.Listener] = [] + self._listen_addrs: list[Address] = [] self._parent_chan: Channel|None = None self._forkserver_info: tuple|None = None @@ -269,13 +276,13 @@ class Actor: # when provided, init the registry addresses property from # input via the validator. - self._reg_addrs: list[tuple[str, int]] = [] + self._reg_addrs: list[AddressTypes] = [] if registry_addrs: - self.reg_addrs: list[tuple[str, int]] = registry_addrs + self.reg_addrs: list[AddressTypes] = registry_addrs _state._runtime_vars['_registry_addrs'] = registry_addrs @property - def reg_addrs(self) -> list[tuple[str, int]]: + def reg_addrs(self) -> list[AddressTypes]: ''' List of (socket) addresses for all known (and contactable) registry actors. @@ -286,7 +293,7 @@ class Actor: @reg_addrs.setter def reg_addrs( self, - addrs: list[tuple[str, int]], + addrs: list[AddressTypes], ) -> None: if not addrs: log.warning( @@ -295,16 +302,7 @@ class Actor: ) return - # always sanity check the input list since it's critical - # that addrs are correct for discovery sys operation. - for addr in addrs: - if not isinstance(addr, tuple): - raise ValueError( - 'Expected `Actor.reg_addrs: list[tuple[str, int]]`\n' - f'Got {addrs}' - ) - - self._reg_addrs = addrs + self._reg_addrs = addrs async def wait_for_peer( self, @@ -1024,11 +1022,11 @@ class Actor: async def _from_parent( self, - parent_addr: tuple[str, int]|None, + parent_addr: AddressTypes|None, ) -> tuple[ Channel, - list[tuple[str, int]]|None, + list[AddressTypes]|None, ]: ''' Bootstrap this local actor's runtime config from its parent by @@ -1040,13 +1038,13 @@ 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_destaddr(parent_addr) + chan = await Channel.from_addr(wrap_address(parent_addr)) # TODO: move this into a `Channel.handshake()`? # Initial handshake: swap names. await self._do_handshake(chan) - accept_addrs: list[tuple[str, int]]|None = None + accept_addrs: list[AddressTypes]|None = None if self._spawn_method == "trio": @@ -1063,7 +1061,7 @@ class Actor: # if "trace"/"util" mode is enabled? f'{pretty_struct.pformat(spawnspec)}\n' ) - accept_addrs: list[tuple[str, int]] = spawnspec.bind_addrs + accept_addrs: list[AddressTypes] = spawnspec.bind_addrs # TODO: another `Struct` for rtvs.. rvs: dict[str, Any] = spawnspec._runtime_vars @@ -1170,8 +1168,7 @@ class Actor: self, handler_nursery: Nursery, *, - # (host, port) to bind for channel server - listen_sockaddrs: list[tuple[str, int]]|None = None, + listen_addrs: list[AddressTypes]|None = None, task_status: TaskStatus[Nursery] = trio.TASK_STATUS_IGNORED, ) -> None: @@ -1183,37 +1180,39 @@ class Actor: `.cancel_server()` is called. ''' - if listen_sockaddrs is None: - listen_sockaddrs = [(None, 0)] + if listen_addrs is None: + listen_addrs = [TCPAddress.get_random()] + + else: + listen_addrs: list[Address] = [ + wrap_address(a) for a in listen_addrs + ] self._server_down = trio.Event() try: async with trio.open_nursery() as server_n: + listeners: list[trio.abc.Listener] = [ + await addr.open_listener() + for addr in listen_addrs + ] + await server_n.start( + partial( + trio.serve_listeners, + handler=self._stream_handler, + listeners=listeners, - for host, port in listen_sockaddrs: - listeners: list[trio.abc.Listener] = await server_n.start( - partial( - trio.serve_tcp, - - handler=self._stream_handler, - port=port, - host=host, - - # NOTE: configured such that new - # connections will stay alive even if - # this server is cancelled! - handler_nursery=handler_nursery, - ) + # NOTE: configured such that new + # connections will stay alive even if + # this server is cancelled! + handler_nursery=handler_nursery ) - sockets: list[trio.socket] = [ - getattr(listener, 'socket', 'unknown socket') - for listener in listeners - ] - log.runtime( - 'Started TCP server(s)\n' - f'|_{sockets}\n' - ) - self._listeners.extend(listeners) + ) + log.runtime( + 'Started server(s)\n' + '\n'.join([f'|_{addr}' for addr in listen_addrs]) + ) + self._listen_addrs.extend(listen_addrs) + self._listeners.extend(listeners) task_status.started(server_n) @@ -1576,26 +1575,21 @@ class Actor: return False @property - def accept_addrs(self) -> list[tuple[str, int]]: + def accept_addrs(self) -> list[AddressTypes]: ''' All addresses to which the transport-channel server binds and listens for new connections. ''' - # throws OSError on failure - return [ - listener.socket.getsockname() - for listener in self._listeners - ] # type: ignore + return [a.unwrap() for a in self._listen_addrs] @property - def accept_addr(self) -> tuple[str, int]: + def accept_addr(self) -> AddressTypes: ''' Primary address to which the IPC transport server is bound and listening for new connections. ''' - # throws OSError on failure return self.accept_addrs[0] def get_parent(self) -> Portal: @@ -1667,7 +1661,7 @@ class Actor: async def async_main( actor: Actor, - accept_addrs: tuple[str, int]|None = None, + accept_addrs: AddressTypes|None = None, # XXX: currently ``parent_addr`` is only needed for the # ``multiprocessing`` backend (which pickles state sent to @@ -1676,7 +1670,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: tuple[str, int]|None = None, + parent_addr: AddressTypes|None = None, task_status: TaskStatus[None] = trio.TASK_STATUS_IGNORED, ) -> None: @@ -1766,7 +1760,7 @@ async def async_main( partial( actor._serve_forever, service_nursery, - listen_sockaddrs=accept_addrs, + listen_addrs=accept_addrs, ) ) except OSError as oserr: @@ -1782,7 +1776,7 @@ async def async_main( raise - accept_addrs: list[tuple[str, int]] = actor.accept_addrs + accept_addrs: list[AddressTypes] = actor.accept_addrs # NOTE: only set the loopback addr for the # process-tree-global "root" mailbox since @@ -1790,9 +1784,8 @@ async def async_main( # their root actor over that channel. if _state._runtime_vars['_is_root']: for addr in accept_addrs: - host, _ = addr - # TODO: generic 'lo' detector predicate - if '127.0.0.1' in host: + waddr = wrap_address(addr) + if waddr == waddr.get_root(): _state._runtime_vars['_root_mailbox'] = addr # Register with the arbiter if we're told its addr @@ -1807,24 +1800,21 @@ async def async_main( # only on unique actor uids? for addr in actor.reg_addrs: try: - assert isinstance(addr, tuple) - assert addr[1] # non-zero after bind + waddr = wrap_address(addr) + assert waddr.is_valid except AssertionError: await _debug.pause() - async with get_registry(*addr) as reg_portal: + async with get_registry(addr) as reg_portal: for accept_addr in accept_addrs: - - if not accept_addr[1]: - await _debug.pause() - - assert accept_addr[1] + accept_addr = wrap_address(accept_addr) + assert accept_addr.is_valid await reg_portal.run_from_ns( 'self', 'register_actor', uid=actor.uid, - sockaddr=accept_addr, + addr=accept_addr.unwrap(), ) is_registered: bool = True @@ -1951,12 +1941,13 @@ async def async_main( ): failed: bool = False for addr in actor.reg_addrs: - assert isinstance(addr, tuple) + waddr = wrap_address(addr) + assert waddr.is_valid with trio.move_on_after(0.5) as cs: cs.shield = True try: async with get_registry( - *addr, + addr, ) as reg_portal: await reg_portal.run_from_ns( 'self', @@ -2034,7 +2025,7 @@ class Arbiter(Actor): self._registry: dict[ tuple[str, str], - tuple[str, int], + AddressTypes, ] = {} self._waiters: dict[ str, @@ -2050,18 +2041,18 @@ class Arbiter(Actor): self, name: str, - ) -> tuple[str, int]|None: + ) -> AddressTypes|None: - for uid, sockaddr in self._registry.items(): + for uid, addr in self._registry.items(): if name in uid: - return sockaddr + return addr return None async def get_registry( self - ) -> dict[str, tuple[str, int]]: + ) -> dict[str, AddressTypes]: ''' Return current name registry. @@ -2081,7 +2072,7 @@ class Arbiter(Actor): self, name: str, - ) -> list[tuple[str, int]]: + ) -> list[AddressTypes]: ''' Wait for a particular actor to register. @@ -2089,44 +2080,41 @@ class Arbiter(Actor): registered. ''' - sockaddrs: list[tuple[str, int]] = [] - sockaddr: tuple[str, int] + addrs: list[AddressTypes] = [] + addr: AddressTypes mailbox_info: str = 'Actor registry contact infos:\n' - for uid, sockaddr in self._registry.items(): + for uid, addr in self._registry.items(): mailbox_info += ( f'|_uid: {uid}\n' - f'|_sockaddr: {sockaddr}\n\n' + f'|_addr: {addr}\n\n' ) if name == uid[0]: - sockaddrs.append(sockaddr) + addrs.append(addr) - if not sockaddrs: + if not addrs: waiter = trio.Event() self._waiters.setdefault(name, []).append(waiter) await waiter.wait() for uid in self._waiters[name]: if not isinstance(uid, trio.Event): - sockaddrs.append(self._registry[uid]) + addrs.append(self._registry[uid]) log.runtime(mailbox_info) - return sockaddrs + return addrs async def register_actor( self, uid: tuple[str, str], - sockaddr: tuple[str, int] - + addr: AddressTypes ) -> None: uid = name, hash = (str(uid[0]), str(uid[1])) - addr = (host, port) = ( - str(sockaddr[0]), - int(sockaddr[1]), - ) - if port == 0: + waddr: Address = wrap_address(addr) + if not waddr.is_valid: + # should never be 0-dynamic-os-alloc await _debug.pause() - assert port # should never be 0-dynamic-os-alloc + self._registry[uid] = addr # pop and signal all waiter events diff --git a/tractor/_spawn.py b/tractor/_spawn.py index dc2429d9..d1eb7f37 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -46,6 +46,7 @@ from tractor._state import ( _runtime_vars, ) from tractor.log import get_logger +from tractor._addr import AddressTypes from tractor._portal import Portal from tractor._runtime import Actor from tractor._entry import _mp_main @@ -392,8 +393,8 @@ async def new_proc( errors: dict[tuple[str, str], Exception], # passed through to actor main - bind_addrs: list[tuple[str, int]], - parent_addr: tuple[str, int], + bind_addrs: list[AddressTypes], + parent_addr: AddressTypes, _runtime_vars: dict[str, Any], # serialized and sent to _child *, @@ -431,8 +432,8 @@ async def trio_proc( errors: dict[tuple[str, str], Exception], # passed through to actor main - bind_addrs: list[tuple[str, int]], - parent_addr: tuple[str, int], + bind_addrs: list[AddressTypes], + parent_addr: AddressTypes, _runtime_vars: dict[str, Any], # serialized and sent to _child *, infect_asyncio: bool = False, @@ -520,15 +521,15 @@ async def trio_proc( # send a "spawning specification" which configures the # initial runtime state of the child. - await chan.send( - SpawnSpec( - _parent_main_data=subactor._parent_main_data, - enable_modules=subactor.enable_modules, - reg_addrs=subactor.reg_addrs, - bind_addrs=bind_addrs, - _runtime_vars=_runtime_vars, - ) + sspec = SpawnSpec( + _parent_main_data=subactor._parent_main_data, + enable_modules=subactor.enable_modules, + reg_addrs=subactor.reg_addrs, + bind_addrs=bind_addrs, + _runtime_vars=_runtime_vars, ) + log.runtime(f'Sending spawn spec: {str(sspec)}') + await chan.send(sspec) # track subactor in current nursery curr_actor: Actor = current_actor() @@ -638,8 +639,8 @@ async def mp_proc( subactor: Actor, errors: dict[tuple[str, str], Exception], # passed through to actor main - bind_addrs: list[tuple[str, int]], - parent_addr: tuple[str, int], + bind_addrs: list[AddressTypes], + parent_addr: AddressTypes, _runtime_vars: dict[str, Any], # serialized and sent to _child *, infect_asyncio: bool = False, diff --git a/tractor/_supervise.py b/tractor/_supervise.py index 052a5f4c..2a3842f7 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -28,7 +28,13 @@ import warnings import trio + from .devx._debug import maybe_wait_for_debugger +from ._addr import ( + AddressTypes, + preferred_transport, + get_address_cls +) from ._state import current_actor, is_main_process from .log import get_logger, get_loglevel from ._runtime import Actor @@ -47,8 +53,6 @@ if TYPE_CHECKING: log = get_logger(__name__) -_default_bind_addr: tuple[str, int] = ('127.0.0.1', 0) - class ActorNursery: ''' @@ -130,8 +134,9 @@ class ActorNursery: *, - bind_addrs: list[tuple[str, int]] = [_default_bind_addr], + bind_addrs: list[AddressTypes]|None = None, rpc_module_paths: list[str]|None = None, + enable_transports: list[str] = [preferred_transport], enable_modules: list[str]|None = None, loglevel: str|None = None, # set log level per subactor debug_mode: bool|None = None, @@ -156,6 +161,12 @@ 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 @@ -224,7 +235,7 @@ class ActorNursery: *, name: str | None = None, - bind_addrs: tuple[str, int] = [_default_bind_addr], + bind_addrs: AddressTypes|None = None, rpc_module_paths: list[str] | None = None, enable_modules: list[str] | None = None, loglevel: str | None = None, # set log level per subactor diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index a6998ae2..f1cb8e8b 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -17,7 +17,6 @@ import platform from ._transport import ( MsgTransportKey as MsgTransportKey, - AddressType as AddressType, MsgType as MsgType, MsgTransport as MsgTransport, MsgpackTransport as MsgpackTransport @@ -27,10 +26,8 @@ from ._tcp import MsgpackTCPStream as MsgpackTCPStream from ._uds import MsgpackUDSStream as MsgpackUDSStream from ._types import ( - default_lo_addrs as default_lo_addrs, - transport_from_destaddr as transport_from_destaddr, + transport_from_addr as transport_from_addr, transport_from_stream as transport_from_stream, - AddressTypes as AddressTypes ) from ._chan import ( diff --git a/tractor/ipc/_chan.py b/tractor/ipc/_chan.py index ee259371..93f17132 100644 --- a/tractor/ipc/_chan.py +++ b/tractor/ipc/_chan.py @@ -35,8 +35,12 @@ import trio from tractor.ipc._transport import MsgTransport from tractor.ipc._types import ( - transport_from_destaddr, + transport_from_addr, transport_from_stream, +) +from tractor._addr import ( + wrap_address, + Address, AddressTypes ) from tractor.log import get_logger @@ -66,7 +70,6 @@ class Channel: def __init__( self, - destaddr: AddressTypes|None = None, transport: MsgTransport|None = None, # TODO: optional reconnection support? # auto_reconnect: bool = False, @@ -81,8 +84,6 @@ class Channel: # user in ``.from_stream()``. self._transport: MsgTransport|None = transport - self._destaddr = destaddr if destaddr else self._transport.raddr - # set after handshake - always uid of far end self.uid: tuple[str, str]|None = None @@ -121,13 +122,14 @@ class Channel: ) @classmethod - async def from_destaddr( + async def from_addr( cls, - destaddr: AddressTypes, + addr: AddressTypes, **kwargs ) -> Channel: - transport_cls = transport_from_destaddr(destaddr) - transport = await transport_cls.connect_to(destaddr, **kwargs) + addr: Address = wrap_address(addr) + transport_cls = transport_from_addr(addr) + transport = await transport_cls.connect_to(addr, **kwargs) log.transport( f'Opened channel[{type(transport)}]: {transport.laddr} -> {transport.raddr}' @@ -164,11 +166,11 @@ class Channel: ) @property - def laddr(self) -> tuple[str, int]|None: + def laddr(self) -> Address|None: return self._transport.laddr if self._transport else None @property - def raddr(self) -> tuple[str, int]|None: + def raddr(self) -> Address|None: return self._transport.raddr if self._transport else None # TODO: something like, @@ -205,7 +207,11 @@ class Channel: # assert err __tracebackhide__: bool = False else: - assert err.cid + try: + assert err.cid + + except KeyError: + raise err raise @@ -332,14 +338,14 @@ class Channel: @acm async def _connect_chan( - destaddr: AddressTypes + addr: AddressTypes ) -> typing.AsyncGenerator[Channel, None]: ''' Create and connect a channel with disconnect on context manager teardown. ''' - chan = await Channel.from_destaddr(destaddr) + chan = await Channel.from_addr(addr) yield chan with trio.CancelScope(shield=True): await chan.aclose() diff --git a/tractor/ipc/_tcp.py b/tractor/ipc/_tcp.py index 4a69ebbd..a8008519 100644 --- a/tractor/ipc/_tcp.py +++ b/tractor/ipc/_tcp.py @@ -23,6 +23,7 @@ import trio from tractor.msg import MsgCodec from tractor.log import get_logger +from tractor._addr import TCPAddress from tractor.ipc._transport import MsgpackTransport @@ -38,9 +39,8 @@ class MsgpackTCPStream(MsgpackTransport): using the ``msgspec`` codec lib. ''' - address_type = tuple[str, int] + address_type = TCPAddress layer_key: int = 4 - name_key: str = 'tcp' # def __init__( # self, @@ -55,19 +55,32 @@ class MsgpackTCPStream(MsgpackTransport): # codec=codec # ) + @property + def maddr(self) -> str: + host, port = self.raddr.unwrap() + return ( + f'/ipv4/{host}' + f'/{self.address_type.name_key}/{port}' + # f'/{self.chan.uid[0]}' + # f'/{self.cid}' + + # f'/cid={cid_head}..{cid_tail}' + # TODO: ? not use this ^ right ? + ) + def connected(self) -> bool: return self.stream.socket.fileno() != -1 @classmethod async def connect_to( cls, - destaddr: tuple[str, int], + destaddr: TCPAddress, prefix_size: int = 4, codec: MsgCodec|None = None, **kwargs ) -> MsgpackTCPStream: stream = await trio.open_tcp_stream( - *destaddr, + *destaddr.unwrap(), **kwargs ) return MsgpackTCPStream( @@ -87,14 +100,6 @@ class MsgpackTCPStream(MsgpackTransport): lsockname = stream.socket.getsockname() rsockname = stream.socket.getpeername() return ( - tuple(lsockname[:2]), - tuple(rsockname[:2]), + TCPAddress.from_addr(tuple(lsockname[:2])), + TCPAddress.from_addr(tuple(rsockname[:2])), ) - - @classmethod - def get_random_addr(self) -> tuple[str, int]: - return (None, 0) - - @classmethod - def get_root_addr(self) -> tuple[str, int]: - return ('127.0.0.1', 1616) diff --git a/tractor/ipc/_transport.py b/tractor/ipc/_transport.py index 76fe048c..1440d453 100644 --- a/tractor/ipc/_transport.py +++ b/tractor/ipc/_transport.py @@ -50,6 +50,7 @@ from tractor.msg import ( types as msgtypes, pretty_struct, ) +from tractor._addr import Address log = get_logger(__name__) @@ -62,12 +63,11 @@ MsgTransportKey = tuple[str, str] # ?TODO? this should be our `Union[*msgtypes.__spec__]` alias now right..? # => BLEH, except can't bc prots must inherit typevar or param-spec # vars.. -AddressType = TypeVar('AddressType') MsgType = TypeVar('MsgType') @runtime_checkable -class MsgTransport(Protocol[AddressType, MsgType]): +class MsgTransport(Protocol[MsgType]): # # ^-TODO-^ consider using a generic def and indexing with our # eventual msg definition/types? @@ -75,10 +75,9 @@ class MsgTransport(Protocol[AddressType, MsgType]): stream: trio.SocketStream drained: list[MsgType] - address_type: ClassVar[Type[AddressType]] + address_type: ClassVar[Type[Address]] codec_key: ClassVar[str] - name_key: ClassVar[str] # XXX: should this instead be called `.sendall()`? async def send(self, msg: MsgType) -> None: @@ -100,20 +99,24 @@ class MsgTransport(Protocol[AddressType, MsgType]): @classmethod def key(cls) -> MsgTransportKey: - return cls.codec_key, cls.name_key + return cls.codec_key, cls.address_type.name_key @property - def laddr(self) -> AddressType: + def laddr(self) -> Address: ... @property - def raddr(self) -> AddressType: + def raddr(self) -> Address: + ... + + @property + def maddr(self) -> str: ... @classmethod async def connect_to( cls, - destaddr: AddressType, + addr: Address, **kwargs ) -> MsgTransport: ... @@ -123,8 +126,8 @@ class MsgTransport(Protocol[AddressType, MsgType]): cls, stream: trio.abc.Stream ) -> tuple[ - AddressType, # local - AddressType # remote + Address, # local + Address # remote ]: ''' Return the `trio` streaming transport prot's addrs for both @@ -133,14 +136,6 @@ class MsgTransport(Protocol[AddressType, MsgType]): ''' ... - @classmethod - def get_random_addr(self) -> AddressType: - ... - - @classmethod - def get_root_addr(self) -> AddressType: - ... - class MsgpackTransport(MsgTransport): @@ -447,9 +442,9 @@ class MsgpackTransport(MsgTransport): return self._aiter_pkts @property - def laddr(self) -> AddressType: + def laddr(self) -> Address: return self._laddr @property - def raddr(self) -> AddressType: + def raddr(self) -> Address: return self._raddr diff --git a/tractor/ipc/_types.py b/tractor/ipc/_types.py index 92d3af91..3e0e43e5 100644 --- a/tractor/ipc/_types.py +++ b/tractor/ipc/_types.py @@ -13,49 +13,42 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . -from typing import Type, Union +from typing import Type import trio import socket -from ._transport import ( +from tractor._addr import Address +from tractor.ipc._transport import ( MsgTransportKey, MsgTransport ) -from ._tcp import MsgpackTCPStream -from ._uds import MsgpackUDSStream +from tractor.ipc._tcp import MsgpackTCPStream +from tractor.ipc._uds import MsgpackUDSStream +# manually updated list of all supported msg transport types _msg_transports = [ MsgpackTCPStream, MsgpackUDSStream ] -# manually updated list of all supported codec+transport types -key_to_transport: dict[MsgTransportKey, Type[MsgTransport]] = { +# convert a MsgTransportKey to the corresponding transport type +_key_to_transport: dict[MsgTransportKey, Type[MsgTransport]] = { cls.key(): cls for cls in _msg_transports } - -# all different address py types we use -AddressTypes = Union[ - tuple([ - cls.address_type - for cls in _msg_transports - ]) -] - - -default_lo_addrs: dict[MsgTransportKey, AddressTypes] = { - cls.key(): cls.get_root_addr() +# convert an Address wrapper to its corresponding transport type +_addr_to_transport: dict[Type[Address], Type[MsgTransport]] = { + cls.address_type: cls for cls in _msg_transports } -def transport_from_destaddr( - destaddr: AddressTypes, +def transport_from_addr( + addr: Address, codec_key: str = 'msgpack', ) -> Type[MsgTransport]: ''' @@ -63,23 +56,13 @@ def transport_from_destaddr( corresponding `MsgTransport` type. ''' - match destaddr: - case str(): - return MsgpackUDSStream + try: + return _addr_to_transport[type(addr)] - case tuple(): - if ( - len(destaddr) == 2 - and - isinstance(destaddr[0], str) - and - isinstance(destaddr[1], int) - ): - return MsgpackTCPStream - - raise NotImplementedError( - f'No known transport for address {destaddr}' - ) + except KeyError: + raise NotImplementedError( + f'No known transport for address {repr(addr)}' + ) def transport_from_stream( @@ -113,4 +96,4 @@ def transport_from_stream( key = (codec_key, transport) - return _msg_transports[key] + return _key_to_transport[key] diff --git a/tractor/ipc/_uds.py b/tractor/ipc/_uds.py index eb2e7f32..ee147d42 100644 --- a/tractor/ipc/_uds.py +++ b/tractor/ipc/_uds.py @@ -18,13 +18,12 @@ Unix Domain Socket implementation of tractor.ipc._transport.MsgTransport protoco ''' from __future__ import annotations -import tempfile -from uuid import uuid4 import trio from tractor.msg import MsgCodec from tractor.log import get_logger +from tractor._addr import UDSAddress from tractor.ipc._transport import MsgpackTransport @@ -37,9 +36,8 @@ class MsgpackUDSStream(MsgpackTransport): using the ``msgspec`` codec lib. ''' - address_type = str + address_type = UDSAddress layer_key: int = 7 - name_key: str = 'uds' # def __init__( # self, @@ -54,19 +52,32 @@ class MsgpackUDSStream(MsgpackTransport): # codec=codec # ) + @property + def maddr(self) -> str: + filepath = self.raddr.unwrap() + return ( + f'/ipv4/localhost' + f'/{self.address_type.name_key}/{filepath}' + # f'/{self.chan.uid[0]}' + # f'/{self.cid}' + + # f'/cid={cid_head}..{cid_tail}' + # TODO: ? not use this ^ right ? + ) + def connected(self) -> bool: return self.stream.socket.fileno() != -1 @classmethod async def connect_to( cls, - filename: str, + addr: UDSAddress, prefix_size: int = 4, codec: MsgCodec|None = None, **kwargs ) -> MsgpackUDSStream: stream = await trio.open_unix_socket( - filename, + addr.unwrap(), **kwargs ) return MsgpackUDSStream( @@ -79,16 +90,8 @@ class MsgpackUDSStream(MsgpackTransport): def get_stream_addrs( cls, stream: trio.SocketStream - ) -> tuple[str, str]: + ) -> tuple[UDSAddress, UDSAddress]: return ( - stream.socket.getsockname(), - stream.socket.getpeername(), + UDSAddress.from_addr(stream.socket.getsockname()), + UDSAddress.from_addr(stream.socket.getsockname()), ) - - @classmethod - def get_random_addr(self) -> str: - return f'{tempfile.gettempdir()}/{uuid4()}.sock' - - @classmethod - def get_root_addr(self) -> str: - return 'tractor.sock' diff --git a/tractor/msg/types.py b/tractor/msg/types.py index 76d0bad6..3e58ae3a 100644 --- a/tractor/msg/types.py +++ b/tractor/msg/types.py @@ -46,8 +46,8 @@ from msgspec import ( from tractor.msg import ( pretty_struct, ) -from tractor.ipc import AddressTypes from tractor.log import get_logger +from tractor._addr import AddressTypes log = get_logger('tractor.msgspec') From f67e19a85220d91503785b678cce9e3693c22a64 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sun, 23 Mar 2025 02:18:01 -0300 Subject: [PATCH 16/74] Trying to make full suite pass with uds --- default.nix | 1 + examples/service_discovery.py | 2 +- tests/test_docs_examples.py | 15 +++++++++++---- tractor/_addr.py | 29 +++++++++++++++++++---------- tractor/_child.py | 7 ++++++- tractor/_discovery.py | 2 +- tractor/_runtime.py | 7 +++++-- 7 files changed, 44 insertions(+), 19 deletions(-) diff --git a/default.nix b/default.nix index 5a936971..08e46d06 100644 --- a/default.nix +++ b/default.nix @@ -10,6 +10,7 @@ pkgs.mkShell { inherit nativeBuildInputs; LD_LIBRARY_PATH = pkgs.lib.makeLibraryPath nativeBuildInputs; + TMPDIR = "/tmp"; shellHook = '' set -e diff --git a/examples/service_discovery.py b/examples/service_discovery.py index 64697e5b..1219f0c1 100644 --- a/examples/service_discovery.py +++ b/examples/service_discovery.py @@ -9,7 +9,7 @@ async def main(service_name): async with tractor.open_nursery() as an: await an.start_actor(service_name) - async with tractor.get_registry(('127.0.0.1', 1616)) as portal: + async with tractor.get_registry() as portal: print(f"Arbiter is listening on {portal.channel}") async with tractor.wait_for_actor(service_name) as sockaddr: diff --git a/tests/test_docs_examples.py b/tests/test_docs_examples.py index cc4904f8..6250e0aa 100644 --- a/tests/test_docs_examples.py +++ b/tests/test_docs_examples.py @@ -66,6 +66,9 @@ def run_example_in_subproc( # due to backpressure!!! proc = testdir.popen( cmdargs, + stdin=subprocess.PIPE, + stdout=subprocess.PIPE, + stderr=subprocess.PIPE, **kwargs, ) assert not proc.returncode @@ -119,10 +122,14 @@ def test_example( code = ex.read() with run_example_in_subproc(code) as proc: - proc.wait() - err, _ = proc.stderr.read(), proc.stdout.read() - # print(f'STDERR: {err}') - # print(f'STDOUT: {out}') + err = None + try: + if not proc.poll(): + _, err = proc.communicate(timeout=15) + + except subprocess.TimeoutExpired as e: + proc.kill() + err = e.stderr # if we get some gnarly output let's aggregate and raise if err: diff --git a/tractor/_addr.py b/tractor/_addr.py index 0944c89d..f59ad542 100644 --- a/tractor/_addr.py +++ b/tractor/_addr.py @@ -14,6 +14,7 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . from __future__ import annotations +import os import tempfile from uuid import uuid4 from typing import ( @@ -79,6 +80,9 @@ class Address(Protocol[ async def open_listener(self, **kwargs) -> ListenerType: ... + async def close_listener(self): + ... + class TCPAddress(Address[ str, @@ -162,6 +166,9 @@ class TCPAddress(Address[ self._host, self._port = listener.socket.getsockname()[:2] return listener + async def close_listener(self): + ... + class UDSAddress(Address[ None, @@ -195,8 +202,8 @@ class UDSAddress(Address[ return self._filepath @classmethod - def get_random(cls, _ns: None = None) -> UDSAddress: - return UDSAddress(f'{tempfile.gettempdir()}/{uuid4().sock}') + def get_random(cls, namespace: None = None) -> UDSAddress: + return UDSAddress(f'{tempfile.gettempdir()}/{uuid4()}.sock') @classmethod def get_root(cls) -> Address: @@ -214,22 +221,24 @@ class UDSAddress(Address[ return self._filepath == other._filepath async def open_stream(self, **kwargs) -> trio.SocketStream: - stream = await trio.open_tcp_stream( + stream = await trio.open_unix_socket( self._filepath, **kwargs ) - self._binded = True return stream async def open_listener(self, **kwargs) -> trio.SocketListener: - sock = socket.socket(socket.AF_UNIX, socket.SOCK_STREAM) - sock.bind(self._filepath) - sock.listen() - self._binded = True - return trio.SocketListener(sock) + self._sock = socket.socket(socket.AF_UNIX, socket.SOCK_STREAM) + await self._sock.bind(self._filepath) + self._sock.listen(1) + return trio.SocketListener(self._sock) + + async def close_listener(self): + self._sock.close() + os.unlink(self._filepath) -preferred_transport = 'tcp' +preferred_transport = 'uds' _address_types = ( diff --git a/tractor/_child.py b/tractor/_child.py index 69142889..4666e1fa 100644 --- a/tractor/_child.py +++ b/tractor/_child.py @@ -31,7 +31,12 @@ def parse_uid(arg): return str(name), str(uuid) # ensures str encoding def parse_ipaddr(arg): - return literal_eval(arg) + try: + return literal_eval(arg) + + except (ValueError, SyntaxError): + # UDS: try to interpret as a straight up str + return arg if __name__ == "__main__": diff --git a/tractor/_discovery.py b/tractor/_discovery.py index 76d3cccd..36b4c020 100644 --- a/tractor/_discovery.py +++ b/tractor/_discovery.py @@ -54,7 +54,7 @@ log = get_logger(__name__) @acm -async def get_registry(addr: AddressTypes) -> AsyncGenerator[ +async def get_registry(addr: AddressTypes | None = None) -> AsyncGenerator[ Portal | LocalPortal | None, None, ]: diff --git a/tractor/_runtime.py b/tractor/_runtime.py index e755d5ce..cb46e953 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -77,8 +77,9 @@ from .ipc import Channel from ._addr import ( AddressTypes, Address, - TCPAddress, wrap_address, + preferred_transport, + default_lo_addrs ) from ._context import ( mk_context, @@ -1181,7 +1182,7 @@ class Actor: ''' if listen_addrs is None: - listen_addrs = [TCPAddress.get_random()] + listen_addrs = default_lo_addrs([preferred_transport]) else: listen_addrs: list[Address] = [ @@ -1217,6 +1218,8 @@ class Actor: task_status.started(server_n) finally: + for addr in listen_addrs: + await addr.close_listener() # signal the server is down since nursery above terminated self._server_down.set() From 568fb18d01c998018f4f5d35d6403bb681abb77e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 30 Mar 2025 18:30:43 -0400 Subject: [PATCH 17/74] Rework/simplify transport addressing A few things that can fundamentally change, - UDS addresses now always encapsulate the local and remote pid such that it denotes each side's process much like a TCP *port*. |_ `.__init__()` takes a new `maybe_pid: int`. |_ this required changes to the `.ipc._uds` backend which will come in an subsequent commit! |_ `UDSAddress.address_type` becomes a `tuple[str, int]` just like the TCP case. |_ adjust `wrap_address()` to match. - use a new `_state.get_rt_dir() -> Path` as the default location for UDS socket file: now under `XDG_RUNTIME_DIR'/tractor/` subdir by default. - re-implement `USDAddress.get_random()` to use both the local `Actor.uid` (if available) and at least the pid for its socket file name. Removals, - drop the loop generated `_default_addrs`, simplify to just `_default_lo_addrs` for per-transport default registry addresses. |_ change to `_address_types: dict[str, Type[Address]]` instead of separate types `list`. |_ adjust `is_wrapped_addr()` to just check `in _addr_types.values()`. - comment out `Address.open_stream()` it's unused and i think the wrong place for this API. Renames, - from `AddressTypes` -> `UnwrappedAddress`, since it's a simple type union and all this type set is, is the simple python data-structures we encode to for the wire. |_ see note about possibly implementing the `.[un]wrap()` stuff as `msgspec` codec `enc/dec_hook()`s instead! Additions, - add a `mk_uuid()` to be used throughout the runtime including for generating the `Aid.uuid` part. - tons of notes around follow up refinements! --- tractor/_addr.py | 278 +++++++++++++++++++++++++++++++++++----------- tractor/_state.py | 27 ++++- 2 files changed, 238 insertions(+), 67 deletions(-) diff --git a/tractor/_addr.py b/tractor/_addr.py index f59ad542..f6bb5e2a 100644 --- a/tractor/_addr.py +++ b/tractor/_addr.py @@ -14,20 +14,31 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . from __future__ import annotations +from pathlib import Path import os -import tempfile +# import tempfile from uuid import uuid4 from typing import ( Protocol, ClassVar, TypeVar, Union, - Type + Type, + TYPE_CHECKING, ) +from bidict import bidict import trio from trio import socket +from ._state import ( + get_rt_dir, + current_actor, +) + +if TYPE_CHECKING: + from ._runtime import Actor + NamespaceType = TypeVar('NamespaceType') AddressType = TypeVar('AddressType') @@ -58,12 +69,24 @@ class Address(Protocol[ ... def unwrap(self) -> AddressType: + ''' + Deliver the underying minimum field set in + a primitive python data type-structure. + ''' ... @classmethod def get_random(cls, namespace: NamespaceType | None = None) -> Address: ... + # TODO, this should be something like a `.get_def_registar_addr()` + # or similar since, + # - it should be a **host singleton** (not root/tree singleton) + # - we **only need this value** when one isn't provided to the + # runtime at boot and we want to implicitly provide a host-wide + # registrar. + # - each rooted-actor-tree should likely have its own + # micro-registry (likely the root being it), also see @classmethod def get_root(cls) -> Address: ... @@ -74,8 +97,13 @@ class Address(Protocol[ def __eq__(self, other) -> bool: ... - async def open_stream(self, **kwargs) -> StreamType: - ... + # async def open_stream(self, **kwargs) -> StreamType: + # ''' + # Open a connection *TO* this address and deliver back a + # `trio.SocketStream` wrapping the underlying transport. + + # ''' + # ... async def open_listener(self, **kwargs) -> ListenerType: ... @@ -104,9 +132,12 @@ class TCPAddress(Address[ or not isinstance(port, int) ): - raise TypeError(f'Expected host {host} to be str and port {port} to be int') - self._host = host - self._port = port + raise TypeError( + f'Expected host {host!r} to be str and port {port!r} to be int' + ) + + self._host: str = host + self._port: int = port @property def is_valid(self) -> bool: @@ -117,14 +148,23 @@ class TCPAddress(Address[ return self._host @classmethod - def from_addr(cls, addr: tuple[str, int]) -> TCPAddress: + def from_addr( + cls, + addr: tuple[str, int] + ) -> TCPAddress: return TCPAddress(addr[0], addr[1]) def unwrap(self) -> tuple[str, int]: - return self._host, self._port + return ( + self._host, + self._port, + ) @classmethod - def get_random(cls, namespace: str = '127.0.0.1') -> TCPAddress: + def get_random( + cls, + namespace: str = '127.0.0.1', + ) -> TCPAddress: return TCPAddress(namespace, 0) @classmethod @@ -132,7 +172,9 @@ class TCPAddress(Address[ return TCPAddress('127.0.0.1', 1616) def __repr__(self) -> str: - return f'{type(self)} @ {self.unwrap()}' + return ( + f'{type(self).__name__}[{self.unwrap()}]' + ) def __eq__(self, other) -> bool: if not isinstance(other, TCPAddress): @@ -146,14 +188,14 @@ class TCPAddress(Address[ self._port == other._port ) - async def open_stream(self, **kwargs) -> trio.SocketStream: - stream = await trio.open_tcp_stream( - self._host, - self._port, - **kwargs - ) - self._host, self._port = stream.socket.getsockname()[:2] - return stream + # async def open_stream(self, **kwargs) -> trio.SocketStream: + # stream = await trio.open_tcp_stream( + # self._host, + # self._port, + # **kwargs + # ) + # self._host, self._port = stream.socket.getsockname()[:2] + # return stream async def open_listener(self, **kwargs) -> trio.SocketListener: listeners = await trio.open_tcp_listeners( @@ -177,14 +219,23 @@ class UDSAddress(Address[ trio.SocketListener ]): + # TODO, maybe we should use 'unix' instead? + # -[ ] need to check what other mult-transport frameworks do + # like zmq, nng, uri-spec et al! name_key: str = 'uds' - address_type: type = str + address_type: type = tuple[str, int] def __init__( self, - filepath: str + filepath: str|Path, + maybe_pid: int, + # ^XXX, in the sense you can also pass + # a "non-real-world-process-id" such as is handy to represent + # our host-local default "port-like" key for the very first + # root actor to create a registry address. ): - self._filepath = filepath + self._filepath: Path = Path(filepath).absolute() + self._pid: int = maybe_pid @property def is_valid(self) -> bool: @@ -195,22 +246,65 @@ class UDSAddress(Address[ return @classmethod - def from_addr(cls, filepath: str) -> UDSAddress: - return UDSAddress(filepath) + def from_addr( + cls, + addr: tuple[Path, int] + ) -> UDSAddress: + return UDSAddress( + filepath=addr[0], + maybe_pid=addr[1], + ) - def unwrap(self) -> str: - return self._filepath + def unwrap(self) -> tuple[Path, int]: + return ( + str(self._filepath), + # XXX NOTE, since this gets passed DIRECTLY to + # `trio.open_unix_ + self._pid, + ) @classmethod - def get_random(cls, namespace: None = None) -> UDSAddress: - return UDSAddress(f'{tempfile.gettempdir()}/{uuid4()}.sock') + def get_random( + cls, + namespace: None = None, # unused + ) -> UDSAddress: + + rt_dir: Path = get_rt_dir() + pid: int = os.getpid() + actor: Actor|None = current_actor( + err_on_no_runtime=False, + ) + if actor: + sockname: str = '::'.join(actor.uid) + f'@{pid}' + else: + sockname: str = f'@{pid}' + + sockpath: Path = Path(f'{rt_dir}/{sockname}.sock') + return UDSAddress( + # filename=f'{tempfile.gettempdir()}/{uuid4()}.sock' + filepath=sockpath, + maybe_pid=pid, + ) @classmethod def get_root(cls) -> Address: - return UDSAddress('tractor.sock') + def_uds_filepath: Path = ( + get_rt_dir() + / + 'registry@1616.sock' + ) + return UDSAddress( + filepath=def_uds_filepath, + maybe_pid=1616 + ) def __repr__(self) -> str: - return f'{type(self)} @ {self._filepath}' + return ( + f'{type(self).__name__}' + f'[' + f'({self._filepath}, {self._pid})' + f']' + ) def __eq__(self, other) -> bool: if not isinstance(other, UDSAddress): @@ -220,15 +314,23 @@ class UDSAddress(Address[ return self._filepath == other._filepath - async def open_stream(self, **kwargs) -> trio.SocketStream: - stream = await trio.open_unix_socket( - self._filepath, - **kwargs - ) - return stream + # TODO? remove right, it's never used? + # + # async def open_stream( + # self, + # **kwargs, + # ) -> trio.SocketStream: + # stream: trio.SocketStream = await trio.open_unix_socket( + # self._filepath, + # **kwargs + # ) + # return stream async def open_listener(self, **kwargs) -> trio.SocketListener: - self._sock = socket.socket(socket.AF_UNIX, socket.SOCK_STREAM) + self._sock = socket.socket( + socket.AF_UNIX, + socket.SOCK_STREAM + ) await self._sock.bind(self._filepath) self._sock.listen(1) return trio.SocketListener(self._sock) @@ -238,72 +340,120 @@ class UDSAddress(Address[ os.unlink(self._filepath) -preferred_transport = 'uds' +preferred_transport: str = 'uds' -_address_types = ( - TCPAddress, - UDSAddress -) - - -_default_addrs: dict[str, Type[Address]] = { - cls.name_key: cls - for cls in _address_types +_address_types: bidict[str, Type[Address]] = { + 'tcp': TCPAddress, + 'uds': UDSAddress } -AddressTypes = Union[ - tuple([ - cls.address_type - for cls in _address_types - ]) +# TODO, can't we just use a type alias +# for this? namely just some `tuple[str, int, str, str]`? +# +# -[ ] would also just be simpler to keep this as SockAddr[tuple] +# or something, implying it's just a simple pair of values which can +# presumably be mapped to all transports? +# -[ ] `pydoc socket.socket.getsockname()` delivers a 4-tuple for +# ipv6 `(hostaddr, port, flowinfo, scope_id)`.. so how should we +# handle that? +# -[ ] as a further alternative to this wrap()/unwrap() approach we +# could just implement `enc/dec_hook()`s for the `Address`-types +# and just deal with our internal objs directly and always and +# leave it to the codec layer to figure out marshalling? +# |_ would mean only one spot to do the `.unwrap()` (which we may +# end up needing to call from the hook()s anyway?) +# -[x] rename to `UnwrappedAddress[Descriptor]` ?? +# seems like the right name as per, +# https://www.geeksforgeeks.org/introduction-to-address-descriptor/ +# +UnwrappedAddress = Union[ + tuple[ + str, # (net/cgroup-)namespace/host-domain + int, # (p)id/port + ] # tcp/udp/uds + + # ?TODO? should we also include another 2 fields from + # our `Aid` msg such that we include the runtime `Actor.uid` + # of `.name` and `.uuid`? + # - would ensure uniqueness across entire net? + # - allows for easier runtime-level filtering of "actors by + # service name" ] +# TODO! really these are discovery sys default addrs ONLY useful for +# when none is provided to a root actor on first boot. _default_lo_addrs: dict[ str, - AddressTypes + UnwrappedAddress ] = { - cls.name_key: cls.get_root().unwrap() - for cls in _address_types + 'tcp': TCPAddress( + host='127.0.0.1', + port=1616, + ).unwrap(), + 'uds': UDSAddress.get_root().unwrap(), } def get_address_cls(name: str) -> Type[Address]: - return _default_addrs[name] + return _address_types[name] def is_wrapped_addr(addr: any) -> bool: - return type(addr) in _address_types + return type(addr) in _address_types.values() -def wrap_address(addr: AddressTypes) -> Address: +def mk_uuid() -> str: + ''' + Encapsulate creation of a uuid4 as `str` as used + for creating `Actor.uid: tuple[str, str]` and/or + `.msg.types.Aid`. + + ''' + return str(uuid4()) + + +def wrap_address( + addr: UnwrappedAddress +) -> Address: if is_wrapped_addr(addr): return addr - cls = None + cls: Type|None = None match addr: - case str(): + case ( + str()|Path(), + int(), + ): cls = UDSAddress case tuple() | list(): cls = TCPAddress case None: - cls = get_address_cls(preferred_transport) - addr = cls.get_root().unwrap() + cls: Type[Address] = get_address_cls(preferred_transport) + addr: AddressType = cls.get_root().unwrap() case _: raise TypeError( - f'Can not wrap addr {addr} of type {type(addr)}' + f'Can not wrap address {type(addr)}\n' + f'{addr!r}\n' ) return cls.from_addr(addr) -def default_lo_addrs(transports: list[str]) -> list[AddressTypes]: +def default_lo_addrs( + transports: list[str], +) -> list[Type[Address]]: + ''' + Return the default, host-singleton, registry address + for an input transport key set. + + ''' return [ _default_lo_addrs[transport] for transport in transports diff --git a/tractor/_state.py b/tractor/_state.py index 79c8bdea..4cb7e784 100644 --- a/tractor/_state.py +++ b/tractor/_state.py @@ -14,14 +14,16 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . -""" -Per process state +''' +Per actor-process runtime state mgmt APIs. -""" +''' from __future__ import annotations from contextvars import ( ContextVar, ) +import os +from pathlib import Path from typing import ( Any, TYPE_CHECKING, @@ -143,3 +145,22 @@ def current_ipc_ctx( f'|_{current_task()}\n' ) return ctx + + +# std ODE (mutable) app state location +_rtdir: Path = Path(os.environ['XDG_RUNTIME_DIR']) + + +def get_rt_dir( + subdir: str = 'tractor' +) -> Path: + ''' + Return the user "runtime dir" where most userspace apps stick + their IPC and cache related system util-files; we take hold + of a `'XDG_RUNTIME_DIR'/tractor/` subdir by default. + + ''' + rtdir: Path = _rtdir / subdir + if not rtdir.is_dir(): + rtdir.mkdir() + return rtdir From 63fac5a80971ad63ed2bfe87fea6f2cf4c1d2c15 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 30 Mar 2025 21:00:36 -0400 Subject: [PATCH 18/74] 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`. --- tractor/ipc/_tcp.py | 28 +++----- tractor/ipc/_uds.py | 162 +++++++++++++++++++++++++++++++++++++------- 2 files changed, 148 insertions(+), 42 deletions(-) diff --git a/tractor/ipc/_tcp.py b/tractor/ipc/_tcp.py index a8008519..eb2003ec 100644 --- a/tractor/ipc/_tcp.py +++ b/tractor/ipc/_tcp.py @@ -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), ) diff --git a/tractor/ipc/_uds.py b/tractor/ipc/_uds.py index ee147d42..894e3fbc 100644 --- a/tractor/ipc/_uds.py +++ b/tractor/ipc/_uds.py @@ -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 '' + + 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) From 2540d1f9e0896538f20be0c7c0e5f0a5bb953313 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 30 Mar 2025 21:21:10 -0400 Subject: [PATCH 19/74] Adjust imports to use new `UnwrappedAddress` For those mods where it's just a type-alias (name) import change. --- tractor/_discovery.py | 28 ++++++++++++++++------------ tractor/_entry.py | 8 ++++---- tractor/_spawn.py | 14 +++++++------- tractor/msg/types.py | 19 +++++++++++++------ 4 files changed, 40 insertions(+), 29 deletions(-) diff --git a/tractor/_discovery.py b/tractor/_discovery.py index 36b4c020..5d9b6cd1 100644 --- a/tractor/_discovery.py +++ b/tractor/_discovery.py @@ -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 @@ -141,10 +145,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, ]: ''' @@ -174,7 +178,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, @@ -184,7 +188,7 @@ async def query_actor( @acm async def maybe_open_portal( - addr: AddressTypes, + addr: UnwrappedAddress, name: str, ): async with query_actor( @@ -204,7 +208,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, @@ -240,7 +244,7 @@ async def find_actor( ) maybe_portals: list[ - AsyncContextManager[AddressTypes] + AsyncContextManager[UnwrappedAddress] ] = list( maybe_open_portal( addr=addr, @@ -282,7 +286,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]: ''' @@ -299,7 +303,7 @@ async def wait_for_actor( yield peer_portal return - regaddr: AddressTypes = ( + regaddr: UnwrappedAddress = ( registry_addr or actor.reg_addrs[0] @@ -316,7 +320,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: diff --git a/tractor/_entry.py b/tractor/_entry.py index 1328aa45..0f6a91c7 100644 --- a/tractor/_entry.py +++ b/tractor/_entry.py @@ -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: diff --git a/tractor/_spawn.py b/tractor/_spawn.py index d1eb7f37..db83ee57 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -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, diff --git a/tractor/msg/types.py b/tractor/msg/types.py index 3e58ae3a..e082d950 100644 --- a/tractor/msg/types.py +++ b/tractor/msg/types.py @@ -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? From bbae2c91fd3c044b68e69e48f97848af5cf3eabf Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 30 Mar 2025 21:36:45 -0400 Subject: [PATCH 20/74] 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. --- tractor/_root.py | 27 +++++---- tractor/_runtime.py | 132 ++++++++++++++++++++++++++++-------------- tractor/_supervise.py | 24 ++++---- 3 files changed, 115 insertions(+), 68 deletions(-) diff --git a/tractor/_root.py b/tractor/_root.py index e9cac3f2..711bd442 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -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, diff --git a/tractor/_runtime.py b/tractor/_runtime.py index cb46e953..f0489814 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -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) diff --git a/tractor/_supervise.py b/tractor/_supervise.py index 2a3842f7..e4017c44 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -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 From 7b3324b2406f37c5ee13f63826324da9f205e94a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 30 Mar 2025 22:35:35 -0400 Subject: [PATCH 21/74] Add a big boi `Channel.pformat()/__repr__()` Much like how `Context` has been implemented, try to give tons of high level details on all the lower level encapsulated primitives, namely the `.msgstream/.transport` and any useful runtime state. B) Impl deats, - adjust `.from_addr()` to only call `._addr.wrap_address()` when we detect `addr` is unwrapped. - add another `log.runtime()` using the new `.__repr__()` in `Channel.from_addr()`. - change to `UnwrappedAddress` as in prior commits. --- tractor/ipc/_chan.py | 78 +++++++++++++++++++++++++++++++++++--------- 1 file changed, 62 insertions(+), 16 deletions(-) diff --git a/tractor/ipc/_chan.py b/tractor/ipc/_chan.py index 93f17132..1175cbb6 100644 --- a/tractor/ipc/_chan.py +++ b/tractor/ipc/_chan.py @@ -24,6 +24,7 @@ from contextlib import ( asynccontextmanager as acm, contextmanager as cm, ) +import os import platform from pprint import pformat import typing @@ -39,9 +40,10 @@ from tractor.ipc._types import ( transport_from_stream, ) from tractor._addr import ( + is_wrapped_addr, wrap_address, Address, - AddressTypes + UnwrappedAddress, ) from tractor.log import get_logger from tractor._exceptions import ( @@ -88,7 +90,8 @@ class Channel: self.uid: tuple[str, str]|None = None self._aiter_msgs = self._iter_msgs() - self._exc: Exception|None = None # set if far end actor errors + self._exc: Exception|None = None + # ^XXX! ONLY set if a remote actor sends an `Error`-msg self._closed: bool = False # flag set by ``Portal.cancel_actor()`` indicating remote @@ -124,17 +127,26 @@ class Channel: @classmethod async def from_addr( cls, - addr: AddressTypes, + addr: UnwrappedAddress, **kwargs ) -> Channel: - addr: Address = wrap_address(addr) - transport_cls = transport_from_addr(addr) - transport = await transport_cls.connect_to(addr, **kwargs) - log.transport( - f'Opened channel[{type(transport)}]: {transport.laddr} -> {transport.raddr}' + if not is_wrapped_addr(addr): + addr: Address = wrap_address(addr) + + transport_cls = transport_from_addr(addr) + transport = await transport_cls.connect_to( + addr, + **kwargs, ) - return Channel(transport=transport) + assert transport.raddr == addr + chan = Channel(transport=transport) + log.runtime( + f'Connected channel IPC transport\n' + f'[>\n' + f' |_{chan}\n' + ) + return chan @cm def apply_codec( @@ -154,16 +166,50 @@ class Channel: self._transport.codec = orig # TODO: do a .src/.dst: str for maddrs? - def __repr__(self) -> str: + def pformat(self) -> str: if not self._transport: return '' - return repr( - self._transport - ).replace( # type: ignore - "socket.socket", - "Channel", + tpt: MsgTransport = self._transport + tpt_name: str = type(tpt).__name__ + tpt_status: str = ( + 'connected' if self.connected() + else 'closed' ) + return ( + f'\n' + ) + + # NOTE: making this return a value that can be passed to + # `eval()` is entirely **optional** FYI! + # https://docs.python.org/3/library/functions.html#repr + # https://docs.python.org/3/reference/datamodel.html#object.__repr__ + # + # Currently we target **readability** from a (console) + # logging perspective over `eval()`-ability since we do NOT + # target serializing non-struct instances! + # def __repr__(self) -> str: + __str__ = pformat + __repr__ = pformat @property def laddr(self) -> Address|None: @@ -338,7 +384,7 @@ class Channel: @acm async def _connect_chan( - addr: AddressTypes + addr: UnwrappedAddress ) -> typing.AsyncGenerator[Channel, None]: ''' Create and connect a channel with disconnect on context manager From abdaf7bf1ffff56ee7a85cf7a8b4cbb37ea5a95c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 30 Mar 2025 22:42:51 -0400 Subject: [PATCH 22/74] Slight typing and multi-line styling tweaks in `.ipc` sugpkg --- tractor/ipc/_shm.py | 5 ++++- tractor/ipc/_transport.py | 5 ++++- tractor/ipc/_types.py | 2 +- 3 files changed, 9 insertions(+), 3 deletions(-) diff --git a/tractor/ipc/_shm.py b/tractor/ipc/_shm.py index 9868ac73..62b26e79 100644 --- a/tractor/ipc/_shm.py +++ b/tractor/ipc/_shm.py @@ -50,7 +50,10 @@ if _USE_POSIX: try: import numpy as np from numpy.lib import recfunctions as rfn - import nptyping + # TODO ruff complains with, + # warning| F401: `nptyping` imported but unused; consider using + # `importlib.util.find_spec` to test for availability + import nptyping # noqa except ImportError: pass diff --git a/tractor/ipc/_transport.py b/tractor/ipc/_transport.py index 1440d453..9dd09a9b 100644 --- a/tractor/ipc/_transport.py +++ b/tractor/ipc/_transport.py @@ -157,7 +157,10 @@ class MsgpackTransport(MsgTransport): ) -> None: self.stream = stream - self._laddr, self._raddr = self.get_stream_addrs(stream) + ( + self._laddr, + self._raddr, + ) = self.get_stream_addrs(stream) # create read loop instance self._aiter_pkts = self._iter_packets() diff --git a/tractor/ipc/_types.py b/tractor/ipc/_types.py index 3e0e43e5..1b86636d 100644 --- a/tractor/ipc/_types.py +++ b/tractor/ipc/_types.py @@ -76,7 +76,7 @@ def transport_from_stream( ''' transport = None if isinstance(stream, trio.SocketStream): - sock = stream.socket + sock: socket.socket = stream.socket match sock.family: case socket.AF_INET | socket.AF_INET6: transport = 'tcp' From a6fefcc2a844f5a390f3377b8c0035100ad898a5 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 30 Mar 2025 22:45:44 -0400 Subject: [PATCH 23/74] Adjust lowlevel-tb hiding logic for `MsgStream` Such that whenev the `self._ctx.chan._exc is trans_err` we suppress. I.e. when the `Channel._exc: Exception|None` error **is the same as** set by the `._rpc.process_messages()` loop (that is, set to the underlying transport layer error), we suppress the lowlevel tb, otherwise we deliver the full tb since likely something at the lowlevel that we aren't detecting changed/signalled/is-relevant! --- tractor/_streaming.py | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/tractor/_streaming.py b/tractor/_streaming.py index 21e59214..e6001c7c 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -595,8 +595,17 @@ class MsgStream(trio.abc.Channel): trio.ClosedResourceError, trio.BrokenResourceError, BrokenPipeError, - ) as trans_err: - if hide_tb: + ) as _trans_err: + trans_err = _trans_err + if ( + hide_tb + and + self._ctx.chan._exc is trans_err + # ^XXX, IOW, only if the channel is marked errored + # for the same reason as whatever its underlying + # transport raised, do we keep the full low-level tb + # suppressed from the user. + ): raise type(trans_err)( *trans_err.args ) from trans_err From 8c0d9614bcd413a864cc72fd149f4f3339ba5437 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 30 Mar 2025 22:52:46 -0400 Subject: [PATCH 24/74] Add `bidict` pkg as dep since used in `._addr` for now --- pyproject.toml | 1 + uv.lock | 11 +++++++++++ 2 files changed, 12 insertions(+) diff --git a/pyproject.toml b/pyproject.toml index fd67bff2..74d3afcb 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -46,6 +46,7 @@ dependencies = [ # typed IPC msging "msgspec>=0.19.0", "cffi>=1.17.1", + "bidict>=0.23.1", ] # ------ project ------ diff --git a/uv.lock b/uv.lock index 76b22243..50a168b6 100644 --- a/uv.lock +++ b/uv.lock @@ -11,6 +11,15 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/89/aa/ab0f7891a01eeb2d2e338ae8fecbe57fcebea1a24dbb64d45801bfab481d/attrs-24.3.0-py3-none-any.whl", hash = "sha256:ac96cd038792094f438ad1f6ff80837353805ac950cd2aa0e0625ef19850c308", size = 63397 }, ] +[[package]] +name = "bidict" +version = "0.23.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/9a/6e/026678aa5a830e07cd9498a05d3e7e650a4f56a42f267a53d22bcda1bdc9/bidict-0.23.1.tar.gz", hash = "sha256:03069d763bc387bbd20e7d49914e75fc4132a41937fa3405417e1a5a2d006d71", size = 29093 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/99/37/e8730c3587a65eb5645d4aba2d27aae48e8003614d6aaf15dda67f702f1f/bidict-0.23.1-py3-none-any.whl", hash = "sha256:5dae8d4d79b552a71cbabc7deb25dfe8ce710b17ff41711e13010ead2abfc3e5", size = 32764 }, +] + [[package]] name = "cffi" version = "1.17.1" @@ -349,6 +358,7 @@ name = "tractor" version = "0.1.0a6.dev0" source = { editable = "." } dependencies = [ + { name = "bidict" }, { name = "cffi" }, { name = "colorlog" }, { name = "msgspec" }, @@ -371,6 +381,7 @@ dev = [ [package.metadata] requires-dist = [ + { name = "bidict", specifier = ">=0.23.1" }, { name = "cffi", specifier = ">=1.17.1" }, { name = "colorlog", specifier = ">=6.8.2,<7" }, { name = "msgspec", specifier = ">=0.19.0" }, From a504d925367ad75c791539550c437946ed6ea556 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 30 Mar 2025 22:53:36 -0400 Subject: [PATCH 25/74] Adjust `._child` instantiation of `Actor` to use newly named `uuid` arg --- tractor/_child.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tractor/_child.py b/tractor/_child.py index 4666e1fa..d2f03f55 100644 --- a/tractor/_child.py +++ b/tractor/_child.py @@ -50,8 +50,8 @@ if __name__ == "__main__": args = parser.parse_args() subactor = Actor( - args.uid[0], - uid=args.uid[1], + name=args.uid[0], + uuid=args.uid[1], loglevel=args.loglevel, spawn_method="trio" ) From b4ce618e33c0ec3091cdce8334a68f2e90e494d6 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 30 Mar 2025 22:54:19 -0400 Subject: [PATCH 26/74] Bit of multi-line styling for `LocalPortal` --- tractor/_portal.py | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/tractor/_portal.py b/tractor/_portal.py index c8a781a7..0efcbd0b 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -107,6 +107,10 @@ class Portal: # point. self._expect_result_ctx: Context|None = None self._streams: set[MsgStream] = set() + + # TODO, this should be PRIVATE (and never used publicly)! since it's just + # a cached ref to the local runtime instead of calling + # `current_actor()` everywhere.. XD self.actor: Actor = current_actor() @property @@ -504,8 +508,12 @@ class LocalPortal: return it's result. ''' - obj = self.actor if ns == 'self' else importlib.import_module(ns) - func = getattr(obj, func_name) + obj = ( + self.actor + if ns == 'self' + else importlib.import_module(ns) + ) + func: Callable = getattr(obj, func_name) return await func(**kwargs) From 44c9da1c917ed971ed2ce036056698fab481e260 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 30 Mar 2025 23:57:52 -0400 Subject: [PATCH 27/74] Emphasize internal error block header-comment a bit --- tractor/_rpc.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tractor/_rpc.py b/tractor/_rpc.py index 6dfecd38..67681bee 100644 --- a/tractor/_rpc.py +++ b/tractor/_rpc.py @@ -1156,7 +1156,7 @@ async def process_messages( trio.Event(), ) - # runtime-scoped remote (internal) error + # XXX RUNTIME-SCOPED! remote (likely internal) error # (^- bc no `Error.cid` -^) # # NOTE: this is the non-rpc error case, that From 31196b9cb476c6fa2a45fefdff0030ff29426199 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 1 Apr 2025 12:56:28 -0400 Subject: [PATCH 28/74] Handle broken-pipes from `MsgpackTransport.send()` Much like we already do in the `._iter_packets()` async-generator which delivers to `.recv()` and `async for`, handle the `''[Errno 32] Broken pipe'` case that can show up with unix-domain-socket usage. Seems like the cause is due to how fast the socket can be torn down during a registry addr channel ping where, - the sending side can break the connection faster then the pong side can prep its handshake msg, - the pong side tries to send it's handshake pkt via `.SocketStream.send_all()` after the breakage and then raises `trio.BrokenResourceError`. --- tractor/ipc/_transport.py | 28 +++++++++++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/tractor/ipc/_transport.py b/tractor/ipc/_transport.py index 9dd09a9b..d94fda6d 100644 --- a/tractor/ipc/_transport.py +++ b/tractor/ipc/_transport.py @@ -412,7 +412,33 @@ class MsgpackTransport(MsgTransport): # supposedly the fastest says, # https://stackoverflow.com/a/54027962 size: bytes = struct.pack(" {type(trans_err)}\n' + f' |_{self}\n' + ), + loglevel=loglevel, + ) from trans_err + + # unless the disconnect condition falls under "a + # normal operation breakage" we usualy console warn + # about it. + case _: + raise trans_err # ?TODO? does it help ever to dynamically show this # frame? From 7a3c9d04586db4c903760048f486de032965b8d0 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 1 Apr 2025 21:53:03 -0400 Subject: [PATCH 29/74] Even more `tractor._addr.Address` simplifying Namely reducing the duplication of class-fields and `TypeVar`s used for parametrizing the `Address` protocol type, - drop all of the `TypeVar` types and just stick with all concrete addrs types inheriting from `Address` only. - rename `Address.name_key` -> `.proto_key`. - rename `Address.address_type` -> `.unwrapped_type` - rename `.namespace` -> `.bindspace` to better reflect that this "part" of the address represents the possible "space for binding endpoints". |_ also linux already uses "namespace" to mean the `netns` and i'd prefer to stick with their semantics for that. - add `TCPAddress/UDSAddress.def_bindspace` values. - drop commented `.open_stream()` method; never used. - simplify `UnwrappedAdress` to just a `tuple` of union types. - add logging to `USDAddress.open_listener()` for now. - adjust `tractor.ipc/_uds/tcp` transport to use new addr field names. --- tractor/_addr.py | 289 ++++++++++++++++++++++++------------------- tractor/ipc/_chan.py | 2 +- tractor/ipc/_tcp.py | 2 +- tractor/ipc/_uds.py | 10 +- 4 files changed, 166 insertions(+), 137 deletions(-) diff --git a/tractor/_addr.py b/tractor/_addr.py index f6bb5e2a..d5b8c81b 100644 --- a/tractor/_addr.py +++ b/tractor/_addr.py @@ -21,54 +21,121 @@ from uuid import uuid4 from typing import ( Protocol, ClassVar, - TypeVar, - Union, + # TypeVar, + # Union, Type, TYPE_CHECKING, ) from bidict import bidict -import trio -from trio import socket +# import trio +from trio import ( + socket, + SocketListener, + open_tcp_listeners, +) +from .log import get_logger from ._state import ( get_rt_dir, current_actor, + is_root_process, ) if TYPE_CHECKING: from ._runtime import Actor - -NamespaceType = TypeVar('NamespaceType') -AddressType = TypeVar('AddressType') -StreamType = TypeVar('StreamType') -ListenerType = TypeVar('ListenerType') +log = get_logger(__name__) -class Address(Protocol[ - NamespaceType, - AddressType, - StreamType, - ListenerType -]): +# TODO, maybe breakout the netns key to a struct? +# class NetNs(Struct)[str, int]: +# ... - name_key: ClassVar[str] - address_type: ClassVar[Type[AddressType]] +# TODO, can't we just use a type alias +# for this? namely just some `tuple[str, int, str, str]`? +# +# -[ ] would also just be simpler to keep this as SockAddr[tuple] +# or something, implying it's just a simple pair of values which can +# presumably be mapped to all transports? +# -[ ] `pydoc socket.socket.getsockname()` delivers a 4-tuple for +# ipv6 `(hostaddr, port, flowinfo, scope_id)`.. so how should we +# handle that? +# -[ ] as a further alternative to this wrap()/unwrap() approach we +# could just implement `enc/dec_hook()`s for the `Address`-types +# and just deal with our internal objs directly and always and +# leave it to the codec layer to figure out marshalling? +# |_ would mean only one spot to do the `.unwrap()` (which we may +# end up needing to call from the hook()s anyway?) +# -[x] rename to `UnwrappedAddress[Descriptor]` ?? +# seems like the right name as per, +# https://www.geeksforgeeks.org/introduction-to-address-descriptor/ +# +UnwrappedAddress = ( + # tcp/udp/uds + tuple[ + str, # host/domain(tcp), filesys-dir(uds) + int|str, # port/path(uds) + ] + # ?TODO? should we also include another 2 fields from + # our `Aid` msg such that we include the runtime `Actor.uid` + # of `.name` and `.uuid`? + # - would ensure uniqueness across entire net? + # - allows for easier runtime-level filtering of "actors by + # service name" +) + +# TODO, maybe rename to `SocketAddress`? +class Address(Protocol): + proto_key: ClassVar[str] + unwrapped_type: ClassVar[UnwrappedAddress] + + # TODO, i feel like an `.is_bound()` is a better thing to + # support? + # Lke, what use does this have besides a noop and if it's not + # valid why aren't we erroring on creation/use? @property def is_valid(self) -> bool: ... + # TODO, maybe `.netns` is a better name? @property - def namespace(self) -> NamespaceType|None: + def namespace(self) -> tuple[str, int]|None: + ''' + The if-available, OS-specific "network namespace" key. + + ''' + ... + + @property + def bindspace(self) -> str: + ''' + Deliver the socket address' "bindable space" from + a `socket.socket.bind()` and thus from the perspective of + specific transport protocol domain. + + I.e. for most (layer-4) network-socket protocols this is + normally the ipv4/6 address, for UDS this is normally + a filesystem (sub-directory). + + For (distributed) network protocols this is normally the routing + layer's domain/(ip-)address, though it might also include a "network namespace" + key different then the default. + + For local-host-only transports this is either an explicit + namespace (with types defined by the OS: netns, Cgroup, IPC, + pid, etc. on linux) or failing that the sub-directory in the + filesys in which socket/shm files are located *under*. + + ''' ... @classmethod - def from_addr(cls, addr: AddressType) -> Address: + def from_addr(cls, addr: UnwrappedAddress) -> Address: ... - def unwrap(self) -> AddressType: + def unwrap(self) -> UnwrappedAddress: ''' Deliver the underying minimum field set in a primitive python data type-structure. @@ -76,7 +143,11 @@ class Address(Protocol[ ... @classmethod - def get_random(cls, namespace: NamespaceType | None = None) -> Address: + def get_random( + cls, + current_actor: Actor, + bindspace: str|None = None, + ) -> Address: ... # TODO, this should be something like a `.get_def_registar_addr()` @@ -97,30 +168,20 @@ class Address(Protocol[ def __eq__(self, other) -> bool: ... - # async def open_stream(self, **kwargs) -> StreamType: - # ''' - # Open a connection *TO* this address and deliver back a - # `trio.SocketStream` wrapping the underlying transport. - - # ''' - # ... - - async def open_listener(self, **kwargs) -> ListenerType: + async def open_listener( + self, + **kwargs, + ) -> SocketListener: ... async def close_listener(self): ... -class TCPAddress(Address[ - str, - tuple[str, int], - trio.SocketStream, - trio.SocketListener -]): - - name_key: str = 'tcp' - address_type: type = tuple[str, int] +class TCPAddress(Address): + proto_key: str = 'tcp' + unwrapped_type: type = tuple[str, int] + def_bindspace: str = '127.0.0.1' def __init__( self, @@ -144,7 +205,11 @@ class TCPAddress(Address[ return self._port != 0 @property - def namespace(self) -> str: + def bindspace(self) -> str: + return self._host + + @property + def domain(self) -> str: return self._host @classmethod @@ -163,13 +228,17 @@ class TCPAddress(Address[ @classmethod def get_random( cls, - namespace: str = '127.0.0.1', + current_actor: Actor, + bindspace: str = def_bindspace, ) -> TCPAddress: - return TCPAddress(namespace, 0) + return TCPAddress(bindspace, 0) @classmethod def get_root(cls) -> Address: - return TCPAddress('127.0.0.1', 1616) + return TCPAddress( + '127.0.0.1', + 1616, + ) def __repr__(self) -> str: return ( @@ -188,17 +257,11 @@ class TCPAddress(Address[ self._port == other._port ) - # async def open_stream(self, **kwargs) -> trio.SocketStream: - # stream = await trio.open_tcp_stream( - # self._host, - # self._port, - # **kwargs - # ) - # self._host, self._port = stream.socket.getsockname()[:2] - # return stream - - async def open_listener(self, **kwargs) -> trio.SocketListener: - listeners = await trio.open_tcp_listeners( + async def open_listener( + self, + **kwargs, + ) -> SocketListener: + listeners: list[SocketListener] = await open_tcp_listeners( host=self._host, port=self._port, **kwargs @@ -212,18 +275,15 @@ class TCPAddress(Address[ ... -class UDSAddress(Address[ - None, - str, - trio.SocketStream, - trio.SocketListener -]): - - # TODO, maybe we should use 'unix' instead? +class UDSAddress(Address): + # TODO, maybe we should use better field and value + # -[x] really this is a `.protocol_key` not a "name" of anything. + # -[ ] consider a 'unix' proto-key instead? # -[ ] need to check what other mult-transport frameworks do # like zmq, nng, uri-spec et al! - name_key: str = 'uds' - address_type: type = tuple[str, int] + proto_key: str = 'uds' + unwrapped_type: type = tuple[str, int] + def_bindspace: Path = get_rt_dir() def __init__( self, @@ -239,11 +299,20 @@ class UDSAddress(Address[ @property def is_valid(self) -> bool: - return True + ''' + We block socket files not allocated under the runtime subdir. + + ''' + return self.bindspace in self._filepath.parents @property - def namespace(self) -> None: - return + def bindspace(self) -> Path: + ''' + We replicate the "ip-set-of-hosts" part of a UDS socket as + just the sub-directory in which we allocate socket files. + + ''' + return self.def_bindspace @classmethod def from_addr( @@ -259,17 +328,17 @@ class UDSAddress(Address[ return ( str(self._filepath), # XXX NOTE, since this gets passed DIRECTLY to - # `trio.open_unix_ + # `open_unix_socket_w_passcred()` above! self._pid, ) @classmethod def get_random( cls, - namespace: None = None, # unused + bindspace: Path|None = None, # default netns ) -> UDSAddress: - rt_dir: Path = get_rt_dir() + bs: Path = bindspace or get_rt_dir() pid: int = os.getpid() actor: Actor|None = current_actor( err_on_no_runtime=False, @@ -277,9 +346,12 @@ class UDSAddress(Address[ if actor: sockname: str = '::'.join(actor.uid) + f'@{pid}' else: - sockname: str = f'@{pid}' + prefix: str = '' + if is_root_process(): + prefix: str = 'root' + sockname: str = f'{prefix}@{pid}' - sockpath: Path = Path(f'{rt_dir}/{sockname}.sock') + sockpath: Path = Path(f'{bs}/{sockname}.sock') return UDSAddress( # filename=f'{tempfile.gettempdir()}/{uuid4()}.sock' filepath=sockpath, @@ -314,28 +386,30 @@ class UDSAddress(Address[ return self._filepath == other._filepath - # TODO? remove right, it's never used? - # - # async def open_stream( - # self, - # **kwargs, - # ) -> trio.SocketStream: - # stream: trio.SocketStream = await trio.open_unix_socket( - # self._filepath, - # **kwargs - # ) - # return stream - - async def open_listener(self, **kwargs) -> trio.SocketListener: + # async def open_listener(self, **kwargs) -> SocketListener: + async def open_listener( + self, + **kwargs, + ) -> SocketListener: self._sock = socket.socket( socket.AF_UNIX, socket.SOCK_STREAM ) + log.info( + f'Attempting to bind UDS socket\n' + f'>[\n' + f'|_{self}\n' + ) await self._sock.bind(self._filepath) self._sock.listen(1) - return trio.SocketListener(self._sock) + log.info( + f'Listening on UDS socket\n' + f'[>\n' + f' |_{self}\n' + ) + return SocketListener(self._sock) - async def close_listener(self): + def close_listener(self): self._sock.close() os.unlink(self._filepath) @@ -349,50 +423,13 @@ _address_types: bidict[str, Type[Address]] = { } -# TODO, can't we just use a type alias -# for this? namely just some `tuple[str, int, str, str]`? -# -# -[ ] would also just be simpler to keep this as SockAddr[tuple] -# or something, implying it's just a simple pair of values which can -# presumably be mapped to all transports? -# -[ ] `pydoc socket.socket.getsockname()` delivers a 4-tuple for -# ipv6 `(hostaddr, port, flowinfo, scope_id)`.. so how should we -# handle that? -# -[ ] as a further alternative to this wrap()/unwrap() approach we -# could just implement `enc/dec_hook()`s for the `Address`-types -# and just deal with our internal objs directly and always and -# leave it to the codec layer to figure out marshalling? -# |_ would mean only one spot to do the `.unwrap()` (which we may -# end up needing to call from the hook()s anyway?) -# -[x] rename to `UnwrappedAddress[Descriptor]` ?? -# seems like the right name as per, -# https://www.geeksforgeeks.org/introduction-to-address-descriptor/ -# -UnwrappedAddress = Union[ - tuple[ - str, # (net/cgroup-)namespace/host-domain - int, # (p)id/port - ] # tcp/udp/uds - - # ?TODO? should we also include another 2 fields from - # our `Aid` msg such that we include the runtime `Actor.uid` - # of `.name` and `.uuid`? - # - would ensure uniqueness across entire net? - # - allows for easier runtime-level filtering of "actors by - # service name" -] - - # TODO! really these are discovery sys default addrs ONLY useful for # when none is provided to a root actor on first boot. _default_lo_addrs: dict[ str, UnwrappedAddress ] = { - 'tcp': TCPAddress( - host='127.0.0.1', - port=1616, - ).unwrap(), + 'tcp': TCPAddress.get_root().unwrap(), 'uds': UDSAddress.get_root().unwrap(), } @@ -435,7 +472,7 @@ def wrap_address( case None: cls: Type[Address] = get_address_cls(preferred_transport) - addr: AddressType = cls.get_root().unwrap() + addr: UnwrappedAddress = cls.get_root().unwrap() case _: raise TypeError( diff --git a/tractor/ipc/_chan.py b/tractor/ipc/_chan.py index 1175cbb6..01baf1e1 100644 --- a/tractor/ipc/_chan.py +++ b/tractor/ipc/_chan.py @@ -187,7 +187,7 @@ class Channel: f' uid={self.uid}\n' f'\n' f' |_msgstream: {tpt_name}\n' - f' proto={tpt.laddr.name_key!r}\n' + f' proto={tpt.laddr.proto_key!r}\n' f' layer={tpt.layer_key!r}\n' f' laddr={tpt.laddr}\n' f' raddr={tpt.raddr}\n' diff --git a/tractor/ipc/_tcp.py b/tractor/ipc/_tcp.py index eb2003ec..61e6f3e6 100644 --- a/tractor/ipc/_tcp.py +++ b/tractor/ipc/_tcp.py @@ -51,7 +51,7 @@ class MsgpackTCPStream(MsgpackTransport): # choosing the routing prefix part. f'/ipv4/{host}' - f'/{self.address_type.name_key}/{port}' + f'/{self.address_type.proto_key}/{port}' # f'/{self.chan.uid[0]}' # f'/{self.cid}' diff --git a/tractor/ipc/_uds.py b/tractor/ipc/_uds.py index 894e3fbc..918e930c 100644 --- a/tractor/ipc/_uds.py +++ b/tractor/ipc/_uds.py @@ -119,7 +119,7 @@ class MsgpackUDSStream(MsgpackTransport): filepath: Path = Path(self.raddr.unwrap()[0]) return ( - f'/{self.address_type.name_key}/{filepath}' + f'/{self.address_type.proto_key}/{filepath}' # f'/{self.chan.uid[0]}' # f'/{self.cid}' @@ -190,14 +190,6 @@ class MsgpackUDSStream(MsgpackTransport): 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(), From 1c85338ff8ef1275b78049c91786861f5d965452 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 1 Apr 2025 22:01:51 -0400 Subject: [PATCH 30/74] Add a `MsgpackTransport.pformat()` And map `.__repr__/__str__` to it. Also adjust to new `Address.proto_key` and add a #TODO for a `.get_peers()`. --- tractor/ipc/_transport.py | 37 +++++++++++++++++++++++++++++++++---- 1 file changed, 33 insertions(+), 4 deletions(-) diff --git a/tractor/ipc/_transport.py b/tractor/ipc/_transport.py index d94fda6d..98403c1f 100644 --- a/tractor/ipc/_transport.py +++ b/tractor/ipc/_transport.py @@ -99,7 +99,7 @@ class MsgTransport(Protocol[MsgType]): @classmethod def key(cls) -> MsgTransportKey: - return cls.codec_key, cls.address_type.name_key + return cls.codec_key, cls.address_type.proto_key @property def laddr(self) -> Address: @@ -136,6 +136,16 @@ class MsgTransport(Protocol[MsgType]): ''' ... + # TODO, such that all `.raddr`s for each `SocketStream` are + # delivered? + # -[ ] move `.open_listener()` here and internally track the + # listener set, per address? + # def get_peers( + # self, + # ) -> list[Address]: + # ... + + class MsgpackTransport(MsgTransport): @@ -421,9 +431,12 @@ class MsgpackTransport(MsgTransport): match trans_err: case trio.BrokenResourceError() if ( '[Errno 32] Broken pipe' in trans_err.args[0] - # ^XXX, specifc to UDS transport afaik? - # likely todo with races related to how fast - # the socket is setup/torn-down on linux.. + # ^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. ): raise TransportClosed( message=( @@ -438,6 +451,9 @@ class MsgpackTransport(MsgTransport): # normal operation breakage" we usualy console warn # about it. case _: + log.exception( + 'Transport layer failed for {self.transport!r} ?\n' + ) raise trans_err # ?TODO? does it help ever to dynamically show this @@ -477,3 +493,16 @@ class MsgpackTransport(MsgTransport): @property def raddr(self) -> Address: return self._raddr + + def pformat(self) -> str: + return ( + f'<{type(self).__name__}(\n' + f' |_task: {self._task}\n' + f'\n' + f' |_peers: 2\n' + f' laddr: {self._laddr}\n' + f' raddr: {self._raddr}\n' + f')>\n' + ) + + __repr__ = __str__ = pformat From 7d1512e03a6aaaa98d6b293243108b2dd3653a4e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 1 Apr 2025 22:08:56 -0400 Subject: [PATCH 31/74] Add an `Actor.pformat()` And map `.__repr__/__str__` to it and add various new fields to fill it out, - drop `self.uid` as var and instead add `Actor._aid: Aid` and proxy to it for the various `.name/.uid/.pid` properties as well as a new `.aid` field. |_ the `Aid.pid` addition is also included. Other improvements, - flip to a sync call to `Address.close_listener()`. - track the `async_main()` parent task as `Actor._task`. - add exception logging around failure to bind due to already-in-use when calling `add.open_listener()` in `._stream_forever()`; sometimes the error might be overridden by something else during the runtime-failure unwind.. --- tractor/_runtime.py | 131 ++++++++++++++++++++++++++++++++++++++----- tractor/msg/types.py | 1 + 2 files changed, 117 insertions(+), 15 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index f0489814..bb2ac579 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -200,9 +200,14 @@ class Actor: phase (aka before a new process is executed). ''' - self.name = name - self.uid = (name, uuid) + self._aid = msgtypes.Aid( + name=name, + uuid=uuid, + pid=os.getpid(), + ) + self._task: trio.Task|None = None + # state self._cancel_complete = trio.Event() self._cancel_called_by_remote: tuple[str, tuple]|None = None self._cancel_called: bool = False @@ -281,6 +286,77 @@ class Actor: self.reg_addrs: list[UnwrappedAddress] = registry_addrs _state._runtime_vars['_registry_addrs'] = registry_addrs + @property + def aid(self) -> msgtypes.Aid: + ''' + This process-singleton-actor's "unique ID" in struct form. + + ''' + return self._aid + + @property + def name(self) -> str: + return self._aid.name + + @property + def uid(self) -> tuple[str, str]: + ''' + This process-singleton's "unique (cross-host) ID". + + Delivered from the `.Aid.name/.uuid` fields as a `tuple` pair + and should be multi-host unique despite a large distributed + process plane. + + ''' + return ( + self._aid.name, + self._aid.uuid, + ) + + @property + def pid(self) -> int: + return self._aid.pid + + def pformat(self) -> str: + ds: str = '=' + parent_uid: tuple|None = None + if rent_chan := self._parent_chan: + parent_uid = rent_chan.uid + peers: list[tuple] = list(self._peer_connected) + listen_addrs: str = pformat(self._listen_addrs) + fmtstr: str = ( + f' |_id: {self.aid!r}\n' + # f" aid{ds}{self.aid!r}\n" + f" parent{ds}{parent_uid}\n" + f'\n' + f' |_ipc: {len(peers)!r} connected peers\n' + f" peers{ds}{peers!r}\n" + f" _listen_addrs{ds}'{listen_addrs}'\n" + f" _listeners{ds}'{self._listeners}'\n" + f'\n' + f' |_rpc: {len(self._rpc_tasks)} tasks\n' + f" ctxs{ds}{len(self._contexts)}\n" + f'\n' + f' |_runtime: ._task{ds}{self._task!r}\n' + f' _spawn_method{ds}{self._spawn_method}\n' + f' _actoruid2nursery{ds}{self._actoruid2nursery}\n' + f' _forkserver_info{ds}{self._forkserver_info}\n' + f'\n' + f' |_state: "TODO: .repr_state()"\n' + f' _cancel_complete{ds}{self._cancel_complete}\n' + f' _cancel_called_by_remote{ds}{self._cancel_called_by_remote}\n' + f' _cancel_called{ds}{self._cancel_called}\n' + ) + return ( + '\n' + ) + + __repr__ = pformat + @property def reg_addrs(self) -> list[UnwrappedAddress]: ''' @@ -421,12 +497,19 @@ class Actor: try: uid: tuple|None = await self._do_handshake(chan) except ( - # we need this for ``msgspec`` for some reason? - # for now, it's been put in the stream backend. + TransportClosed, + # ^XXX NOTE, the above wraps `trio` exc types raised + # during various `SocketStream.send/receive_xx()` calls + # under different fault conditions such as, + # # trio.BrokenResourceError, # trio.ClosedResourceError, - - TransportClosed, + # + # Inside our `.ipc._transport` layer we absorb and + # re-raise our own `TransportClosed` exc such that this + # higher level runtime code can only worry one + # "kinda-error" that we expect to tolerate during + # discovery-sys related pings, queires, DoS etc. ): # XXX: This may propagate up from `Channel._aiter_recv()` # and `MsgpackStream._inter_packets()` on a read from the @@ -1205,7 +1288,8 @@ class Actor: task_status: TaskStatus[Nursery] = trio.TASK_STATUS_IGNORED, ) -> None: ''' - Start the IPC transport server, begin listening for new connections. + Start the IPC transport server, begin listening/accepting new + `trio.SocketStream` connections. This will cause an actor to continue living (and thus blocking at the process/OS-thread level) until @@ -1223,10 +1307,24 @@ class Actor: self._server_down = trio.Event() try: async with trio.open_nursery() as server_n: - listeners: list[trio.abc.Listener] = [ - await addr.open_listener() - for addr in listen_addrs - ] + + listeners: list[trio.abc.Listener] = [] + for addr in listen_addrs: + try: + listener: trio.abc.Listener = await addr.open_listener() + except OSError as oserr: + if ( + '[Errno 98] Address already in use' + in + oserr.args[0] + ): + log.exception( + f'Address already in use?\n' + f'{addr}\n' + ) + raise + listeners.append(listener) + await server_n.start( partial( trio.serve_listeners, @@ -1249,8 +1347,10 @@ class Actor: task_status.started(server_n) finally: + addr: Address for addr in listen_addrs: - await addr.close_listener() + addr.close_listener() + # signal the server is down since nursery above terminated self._server_down.set() @@ -1717,6 +1817,8 @@ async def async_main( the actor's "runtime" and all thus all ongoing RPC tasks. ''' + actor._task: trio.Task = trio.lowlevel.current_task() + # attempt to retreive ``trio``'s sigint handler and stash it # on our debugger state. _debug.DebugStatus._trio_handler = signal.getsignal(signal.SIGINT) @@ -1726,18 +1828,17 @@ async def async_main( # establish primary connection with immediate parent actor._parent_chan: Channel|None = None - if parent_addr is not None: + if parent_addr is not None: ( actor._parent_chan, set_accept_addr_says_rent, maybe_preferred_transports_says_rent, ) = await actor._from_parent(parent_addr) - + accept_addrs: list[UnwrappedAddress] = [] # 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 diff --git a/tractor/msg/types.py b/tractor/msg/types.py index e082d950..d71fb7e0 100644 --- a/tractor/msg/types.py +++ b/tractor/msg/types.py @@ -143,6 +143,7 @@ class Aid( ''' name: str uuid: str + pid: int|None = None # TODO? can/should we extend this field set? # -[ ] use built-in support for UUIDs? `uuid.UUID` which has From 81bf810fbbd449cc4fd8a61091462496d8bedf69 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 1 Apr 2025 22:21:51 -0400 Subject: [PATCH 32/74] Factor `breakpoint()` blocking into `@acm` Call it `maybe_block_bp()` can wrap the `open_root_actor()` body with it. Main reason is to guarantee we can bp inside actor runtime bootup as needed when debugging internals! Prolly should factor this to another module tho? ALSO, ensure we RTE on recurrent entries to `open_root_actor()` from within an existing tree! There was actually `test_spawning` test somehow getting away with this!? Should never be possible or allowed! --- tractor/_root.py | 709 +++++++++++++++++++++++++---------------------- 1 file changed, 378 insertions(+), 331 deletions(-) diff --git a/tractor/_root.py b/tractor/_root.py index 711bd442..50773056 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -18,7 +18,9 @@ Root actor runtime ignition(s). ''' -from contextlib import asynccontextmanager as acm +from contextlib import ( + asynccontextmanager as acm, +) from functools import partial import importlib import inspect @@ -26,7 +28,10 @@ import logging import os import signal import sys -from typing import Callable +from typing import ( + Any, + Callable, +) import warnings @@ -47,21 +52,95 @@ from .ipc import ( _connect_chan, ) from ._addr import ( + Address, UnwrappedAddress, default_lo_addrs, mk_uuid, preferred_transport, wrap_address, ) -from ._exceptions import is_multi_cancelled +from ._exceptions import ( + ActorFailure, + is_multi_cancelled, +) logger = log.get_logger('tractor') +# TODO: stick this in a `@acm` defined in `devx._debug`? +# -[ ] also maybe consider making this a `wrapt`-deco to +# save an indent level? +# +@acm +async def maybe_block_bp( + debug_mode: bool, + maybe_enable_greenback: bool, +) -> bool: + # Override the global debugger hook to make it play nice with + # ``trio``, see much discussion in: + # https://github.com/python-trio/trio/issues/1155#issuecomment-742964018 + builtin_bp_handler: Callable = sys.breakpointhook + orig_bp_path: str|None = os.environ.get( + 'PYTHONBREAKPOINT', + None, + ) + bp_blocked: bool + if ( + debug_mode + and maybe_enable_greenback + and ( + maybe_mod := await _debug.maybe_init_greenback( + raise_not_found=False, + ) + ) + ): + logger.info( + f'Found `greenback` installed @ {maybe_mod}\n' + 'Enabling `tractor.pause_from_sync()` support!\n' + ) + os.environ['PYTHONBREAKPOINT'] = ( + 'tractor.devx._debug._sync_pause_from_builtin' + ) + _state._runtime_vars['use_greenback'] = True + bp_blocked = False + + else: + # TODO: disable `breakpoint()` by default (without + # `greenback`) since it will break any multi-actor + # usage by a clobbered TTY's stdstreams! + def block_bps(*args, **kwargs): + raise RuntimeError( + 'Trying to use `breakpoint()` eh?\n\n' + 'Welp, `tractor` blocks `breakpoint()` built-in calls by default!\n' + 'If you need to use it please install `greenback` and set ' + '`debug_mode=True` when opening the runtime ' + '(either via `.open_nursery()` or `open_root_actor()`)\n' + ) + + sys.breakpointhook = block_bps + # lol ok, + # https://docs.python.org/3/library/sys.html#sys.breakpointhook + os.environ['PYTHONBREAKPOINT'] = "0" + bp_blocked = True + + try: + yield bp_blocked + finally: + # restore any prior built-in `breakpoint()` hook state + if builtin_bp_handler is not None: + sys.breakpointhook = builtin_bp_handler + + if orig_bp_path is not None: + os.environ['PYTHONBREAKPOINT'] = orig_bp_path + + else: + # clear env back to having no entry + os.environ.pop('PYTHONBREAKPOINT', None) + + @acm async def open_root_actor( - *, # defaults are above registry_addrs: list[UnwrappedAddress]|None = None, @@ -111,355 +190,323 @@ async def open_root_actor( Runtime init entry point for ``tractor``. ''' - _debug.hide_runtime_frames() - __tracebackhide__: bool = hide_tb - - # TODO: stick this in a `@cm` defined in `devx._debug`? - # - # Override the global debugger hook to make it play nice with - # ``trio``, see much discussion in: - # https://github.com/python-trio/trio/issues/1155#issuecomment-742964018 - builtin_bp_handler: Callable = sys.breakpointhook - orig_bp_path: str|None = os.environ.get( - 'PYTHONBREAKPOINT', - None, - ) - if ( - debug_mode - and maybe_enable_greenback - and ( - maybe_mod := await _debug.maybe_init_greenback( - raise_not_found=False, - ) + # XXX NEVER allow nested actor-trees! + if already_actor := _state.current_actor(err_on_no_runtime=False): + rtvs: dict[str, Any] = _state._runtime_vars + root_mailbox: list[str, int] = rtvs['_root_mailbox'] + registry_addrs: list[list[str, int]] = rtvs['_registry_addrs'] + raise ActorFailure( + f'A current actor already exists !?\n' + f'({already_actor}\n' + f'\n' + f'You can NOT open a second root actor from within ' + f'an existing tree and the current root of this ' + f'already exists !!\n' + f'\n' + f'_root_mailbox: {root_mailbox!r}\n' + f'_registry_addrs: {registry_addrs!r}\n' ) + + async with maybe_block_bp( + debug_mode=debug_mode, + maybe_enable_greenback=maybe_enable_greenback, ): - logger.info( - f'Found `greenback` installed @ {maybe_mod}\n' - 'Enabling `tractor.pause_from_sync()` support!\n' - ) - os.environ['PYTHONBREAKPOINT'] = ( - 'tractor.devx._debug._sync_pause_from_builtin' - ) - _state._runtime_vars['use_greenback'] = True + _debug.hide_runtime_frames() + __tracebackhide__: bool = hide_tb - else: - # TODO: disable `breakpoint()` by default (without - # `greenback`) since it will break any multi-actor - # usage by a clobbered TTY's stdstreams! - def block_bps(*args, **kwargs): - raise RuntimeError( - 'Trying to use `breakpoint()` eh?\n\n' - 'Welp, `tractor` blocks `breakpoint()` built-in calls by default!\n' - 'If you need to use it please install `greenback` and set ' - '`debug_mode=True` when opening the runtime ' - '(either via `.open_nursery()` or `open_root_actor()`)\n' + # attempt to retreive ``trio``'s sigint handler and stash it + # on our debugger lock state. + _debug.DebugStatus._trio_handler = signal.getsignal(signal.SIGINT) + + # mark top most level process as root actor + _state._runtime_vars['_is_root'] = True + + # caps based rpc list + enable_modules = ( + enable_modules + or + [] + ) + + if rpc_module_paths: + warnings.warn( + "`rpc_module_paths` is now deprecated, use " + " `enable_modules` instead.", + DeprecationWarning, + stacklevel=2, + ) + enable_modules.extend(rpc_module_paths) + + if start_method is not None: + _spawn.try_set_start_method(start_method) + + # TODO! remove this ASAP! + if arbiter_addr is not None: + warnings.warn( + '`arbiter_addr` is now deprecated\n' + 'Use `registry_addrs: list[tuple]` instead..', + DeprecationWarning, + stacklevel=2, + ) + registry_addrs = [arbiter_addr] + + if not registry_addrs: + registry_addrs: list[UnwrappedAddress] = default_lo_addrs( + enable_transports ) - sys.breakpointhook = block_bps - # lol ok, - # https://docs.python.org/3/library/sys.html#sys.breakpointhook - os.environ['PYTHONBREAKPOINT'] = "0" + assert registry_addrs - # attempt to retreive ``trio``'s sigint handler and stash it - # on our debugger lock state. - _debug.DebugStatus._trio_handler = signal.getsignal(signal.SIGINT) + loglevel = ( + loglevel + or log._default_loglevel + ).upper() - # mark top most level process as root actor - _state._runtime_vars['_is_root'] = True - - # caps based rpc list - enable_modules = ( - enable_modules - or - [] - ) - - if rpc_module_paths: - warnings.warn( - "`rpc_module_paths` is now deprecated, use " - " `enable_modules` instead.", - DeprecationWarning, - stacklevel=2, - ) - enable_modules.extend(rpc_module_paths) - - if start_method is not None: - _spawn.try_set_start_method(start_method) - - if arbiter_addr is not None: - warnings.warn( - '`arbiter_addr` is now deprecated\n' - 'Use `registry_addrs: list[tuple]` instead..', - DeprecationWarning, - stacklevel=2, - ) - registry_addrs = [arbiter_addr] - - if not registry_addrs: - registry_addrs: list[UnwrappedAddress] = default_lo_addrs( - enable_transports - ) - - assert registry_addrs - - loglevel = ( - loglevel - or log._default_loglevel - ).upper() - - if ( - debug_mode - and _spawn._spawn_method == 'trio' - ): - _state._runtime_vars['_debug_mode'] = True - - # expose internal debug module to every actor allowing for - # use of ``await tractor.pause()`` - enable_modules.append('tractor.devx._debug') - - # if debug mode get's enabled *at least* use that level of - # logging for some informative console prompts. if ( - logging.getLevelName( - # lul, need the upper case for the -> int map? - # sweet "dynamic function behaviour" stdlib... - loglevel, - ) > logging.getLevelName('PDB') + debug_mode + and _spawn._spawn_method == 'trio' ): - loglevel = 'PDB' + _state._runtime_vars['_debug_mode'] = True + + # expose internal debug module to every actor allowing for + # use of ``await tractor.pause()`` + enable_modules.append('tractor.devx._debug') + + # if debug mode get's enabled *at least* use that level of + # logging for some informative console prompts. + if ( + logging.getLevelName( + # lul, need the upper case for the -> int map? + # sweet "dynamic function behaviour" stdlib... + loglevel, + ) > logging.getLevelName('PDB') + ): + loglevel = 'PDB' - elif debug_mode: - raise RuntimeError( - "Debug mode is only supported for the `trio` backend!" - ) - - assert loglevel - _log = log.get_console_log(loglevel) - assert _log - - # TODO: factor this into `.devx._stackscope`!! - if ( - debug_mode - and - enable_stack_on_sig - ): - from .devx._stackscope import enable_stack_on_sig - enable_stack_on_sig() - - # closed into below ping task-func - ponged_addrs: list[UnwrappedAddress] = [] - - async def ping_tpt_socket( - addr: UnwrappedAddress, - timeout: float = 1, - ) -> None: - ''' - Attempt temporary connection to see if a registry is - listening at the requested address by a tranport layer - ping. - - If a connection can't be made quickly we assume none no - server is listening at that addr. - - ''' - try: - # TODO: this connect-and-bail forces us to have to - # carefully rewrap TCP 104-connection-reset errors as - # EOF so as to avoid propagating cancel-causing errors - # to the channel-msg loop machinery. Likely it would - # be better to eventually have a "discovery" protocol - # with basic handshake instead? - with trio.move_on_after(timeout): - async with _connect_chan(addr): - ponged_addrs.append(addr) - - except OSError: - # TODO: make this a "discovery" log level? - logger.info( - f'No actor registry found @ {addr}\n' - ) - - async with trio.open_nursery() as tn: - for addr in registry_addrs: - tn.start_soon( - ping_tpt_socket, - addr, - ) - - trans_bind_addrs: list[UnwrappedAddress] = [] - - # Create a new local root-actor instance which IS NOT THE - # REGISTRAR - if ponged_addrs: - if ensure_registry: + elif debug_mode: raise RuntimeError( - f'Failed to open `{name}`@{ponged_addrs}: ' - 'registry socket(s) already bound' + "Debug mode is only supported for the `trio` backend!" ) - # we were able to connect to an arbiter - logger.info( - f'Registry(s) seem(s) to exist @ {ponged_addrs}' - ) + assert loglevel + _log = log.get_console_log(loglevel) + assert _log - actor = Actor( - name=name or 'anonymous', - uuid=mk_uuid(), - registry_addrs=ponged_addrs, - loglevel=loglevel, - enable_modules=enable_modules, - ) - # DO NOT use the registry_addrs as the transport server - # addrs for this new non-registar, root-actor. - for addr in ponged_addrs: - waddr = wrap_address(addr) - print(waddr) - trans_bind_addrs.append( - waddr.get_random(namespace=waddr.namespace) - ) - - # Start this local actor as the "registrar", aka a regular - # actor who manages the local registry of "mailboxes" of - # other process-tree-local sub-actors. - else: - - # NOTE that if the current actor IS THE REGISTAR, the - # following init steps are taken: - # - the tranport layer server is bound to each addr - # pair defined in provided registry_addrs, or the default. - trans_bind_addrs = registry_addrs - - # - it is normally desirable for any registrar to stay up - # indefinitely until either all registered (child/sub) - # actors are terminated (via SC supervision) or, - # a re-election process has taken place. - # NOTE: all of ^ which is not implemented yet - see: - # https://github.com/goodboy/tractor/issues/216 - # https://github.com/goodboy/tractor/pull/348 - # https://github.com/goodboy/tractor/issues/296 - - actor = Arbiter( - name=name or 'registrar', - uuid=mk_uuid(), - registry_addrs=registry_addrs, - loglevel=loglevel, - enable_modules=enable_modules, - ) - # XXX, in case the root actor runtime was actually run from - # `tractor.to_asyncio.run_as_asyncio_guest()` and NOt - # `.trio.run()`. - actor._infected_aio = _state._runtime_vars['_is_infected_aio'] - - # Start up main task set via core actor-runtime nurseries. - try: - # assign process-local actor - _state._current_actor = actor - - # start local channel-server and fake the portal API - # NOTE: this won't block since we provide the nursery - ml_addrs_str: str = '\n'.join( - f'@{addr}' for addr in trans_bind_addrs - ) - logger.info( - f'Starting local {actor.uid} on the following transport addrs:\n' - f'{ml_addrs_str}' - ) - - # start the actor runtime in a new task - async with trio.open_nursery( - strict_exception_groups=False, - # ^XXX^ TODO? instead unpack any RAE as per "loose" style? - ) as nursery: - - # ``_runtime.async_main()`` creates an internal nursery - # and blocks here until any underlying actor(-process) - # tree has terminated thereby conducting so called - # "end-to-end" structured concurrency throughout an - # entire hierarchical python sub-process set; all - # "actor runtime" primitives are SC-compat and thus all - # transitively spawned actors/processes must be as - # well. - await nursery.start( - partial( - async_main, - actor, - accept_addrs=trans_bind_addrs, - parent_addr=None - ) - ) - try: - yield actor - except ( - Exception, - BaseExceptionGroup, - ) as err: - - # TODO, in beginning to handle the subsubactor with - # crashed grandparent cases.. - # - # was_locked: bool = await _debug.maybe_wait_for_debugger( - # child_in_debug=True, - # ) - # XXX NOTE XXX see equiv note inside - # `._runtime.Actor._stream_handler()` where in the - # non-root or root-that-opened-this-mahually case we - # wait for the local actor-nursery to exit before - # exiting the transport channel handler. - entered: bool = await _debug._maybe_enter_pm( - err, - api_frame=inspect.currentframe(), - debug_filter=debug_filter, - ) - - if ( - not entered - and - not is_multi_cancelled( - err, - ) - ): - logger.exception('Root actor crashed\n') - - # ALWAYS re-raise any error bubbled up from the - # runtime! - raise - - finally: - # NOTE: not sure if we'll ever need this but it's - # possibly better for even more determinism? - # logger.cancel( - # f'Waiting on {len(nurseries)} nurseries in root..') - # nurseries = actor._actoruid2nursery.values() - # async with trio.open_nursery() as tempn: - # for an in nurseries: - # tempn.start_soon(an.exited.wait) - - logger.info( - 'Closing down root actor' - ) - await actor.cancel(None) # self cancel - finally: - _state._current_actor = None - _state._last_actor_terminated = actor - - # restore built-in `breakpoint()` hook state + # TODO: factor this into `.devx._stackscope`!! if ( debug_mode and - maybe_enable_greenback + enable_stack_on_sig ): - if builtin_bp_handler is not None: - sys.breakpointhook = builtin_bp_handler + from .devx._stackscope import enable_stack_on_sig + enable_stack_on_sig() - if orig_bp_path is not None: - os.environ['PYTHONBREAKPOINT'] = orig_bp_path + # closed into below ping task-func + ponged_addrs: list[UnwrappedAddress] = [] - else: - # clear env back to having no entry - os.environ.pop('PYTHONBREAKPOINT', None) + async def ping_tpt_socket( + addr: UnwrappedAddress, + timeout: float = 1, + ) -> None: + ''' + Attempt temporary connection to see if a registry is + listening at the requested address by a tranport layer + ping. - logger.runtime("Root actor terminated") + If a connection can't be made quickly we assume none no + server is listening at that addr. + + ''' + try: + # TODO: this connect-and-bail forces us to have to + # carefully rewrap TCP 104-connection-reset errors as + # EOF so as to avoid propagating cancel-causing errors + # to the channel-msg loop machinery. Likely it would + # be better to eventually have a "discovery" protocol + # with basic handshake instead? + with trio.move_on_after(timeout): + async with _connect_chan(addr): + ponged_addrs.append(addr) + + except OSError: + # TODO: make this a "discovery" log level? + logger.info( + f'No actor registry found @ {addr}\n' + ) + + async with trio.open_nursery() as tn: + for addr in registry_addrs: + tn.start_soon( + ping_tpt_socket, + addr, + ) + + trans_bind_addrs: list[UnwrappedAddress] = [] + + # Create a new local root-actor instance which IS NOT THE + # REGISTRAR + if ponged_addrs: + if ensure_registry: + raise RuntimeError( + f'Failed to open `{name}`@{ponged_addrs}: ' + 'registry socket(s) already bound' + ) + + # we were able to connect to an arbiter + logger.info( + f'Registry(s) seem(s) to exist @ {ponged_addrs}' + ) + + actor = Actor( + name=name or 'anonymous', + uuid=mk_uuid(), + registry_addrs=ponged_addrs, + loglevel=loglevel, + enable_modules=enable_modules, + ) + # DO NOT use the registry_addrs as the transport server + # addrs for this new non-registar, root-actor. + for addr in ponged_addrs: + waddr: Address = wrap_address(addr) + trans_bind_addrs.append( + waddr.get_random(bindspace=waddr.bindspace) + ) + + # Start this local actor as the "registrar", aka a regular + # actor who manages the local registry of "mailboxes" of + # other process-tree-local sub-actors. + else: + + # NOTE that if the current actor IS THE REGISTAR, the + # following init steps are taken: + # - the tranport layer server is bound to each addr + # pair defined in provided registry_addrs, or the default. + trans_bind_addrs = registry_addrs + + # - it is normally desirable for any registrar to stay up + # indefinitely until either all registered (child/sub) + # actors are terminated (via SC supervision) or, + # a re-election process has taken place. + # NOTE: all of ^ which is not implemented yet - see: + # https://github.com/goodboy/tractor/issues/216 + # https://github.com/goodboy/tractor/pull/348 + # https://github.com/goodboy/tractor/issues/296 + + actor = Arbiter( + name=name or 'registrar', + uuid=mk_uuid(), + registry_addrs=registry_addrs, + loglevel=loglevel, + enable_modules=enable_modules, + ) + # XXX, in case the root actor runtime was actually run from + # `tractor.to_asyncio.run_as_asyncio_guest()` and NOt + # `.trio.run()`. + actor._infected_aio = _state._runtime_vars['_is_infected_aio'] + + # Start up main task set via core actor-runtime nurseries. + try: + # assign process-local actor + _state._current_actor = actor + + # start local channel-server and fake the portal API + # NOTE: this won't block since we provide the nursery + ml_addrs_str: str = '\n'.join( + f'@{addr}' for addr in trans_bind_addrs + ) + logger.info( + f'Starting local {actor.uid} on the following transport addrs:\n' + f'{ml_addrs_str}' + ) + + # start the actor runtime in a new task + async with trio.open_nursery( + strict_exception_groups=False, + # ^XXX^ TODO? instead unpack any RAE as per "loose" style? + ) as nursery: + + # ``_runtime.async_main()`` creates an internal nursery + # and blocks here until any underlying actor(-process) + # tree has terminated thereby conducting so called + # "end-to-end" structured concurrency throughout an + # entire hierarchical python sub-process set; all + # "actor runtime" primitives are SC-compat and thus all + # transitively spawned actors/processes must be as + # well. + await nursery.start( + partial( + async_main, + actor, + accept_addrs=trans_bind_addrs, + parent_addr=None + ) + ) + try: + yield actor + except ( + Exception, + BaseExceptionGroup, + ) as err: + + # TODO, in beginning to handle the subsubactor with + # crashed grandparent cases.. + # + # was_locked: bool = await _debug.maybe_wait_for_debugger( + # child_in_debug=True, + # ) + # XXX NOTE XXX see equiv note inside + # `._runtime.Actor._stream_handler()` where in the + # non-root or root-that-opened-this-mahually case we + # wait for the local actor-nursery to exit before + # exiting the transport channel handler. + entered: bool = await _debug._maybe_enter_pm( + err, + api_frame=inspect.currentframe(), + debug_filter=debug_filter, + ) + + if ( + not entered + and + not is_multi_cancelled( + err, + ) + ): + logger.exception( + 'Root actor crashed\n' + f'>x)\n' + f' |_{actor}\n' + ) + + # ALWAYS re-raise any error bubbled up from the + # runtime! + raise + + finally: + # NOTE: not sure if we'll ever need this but it's + # possibly better for even more determinism? + # logger.cancel( + # f'Waiting on {len(nurseries)} nurseries in root..') + # nurseries = actor._actoruid2nursery.values() + # async with trio.open_nursery() as tempn: + # for an in nurseries: + # tempn.start_soon(an.exited.wait) + + logger.info( + f'Closing down root actor\n' + f'>)\n' + f'|_{actor}\n' + ) + await actor.cancel(None) # self cancel + finally: + _state._current_actor = None + _state._last_actor_terminated = actor + logger.runtime( + f'Root actor terminated\n' + f')>\n' + f' |_{actor}\n' + ) def run_daemon( From 078e507774fa5cee34c28978411495d0ed41ca78 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 1 Apr 2025 22:26:56 -0400 Subject: [PATCH 33/74] Add `psutil` to `--dev` / testing deps --- pyproject.toml | 1 + uv.lock | 17 +++++++++++++++++ 2 files changed, 18 insertions(+) diff --git a/pyproject.toml b/pyproject.toml index 74d3afcb..da3105ff 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -64,6 +64,7 @@ dev = [ "pyperclip>=1.9.0", "prompt-toolkit>=3.0.50", "xonsh>=0.19.2", + "psutil>=7.0.0", ] # TODO, add these with sane versions; were originally in # `requirements-docs.txt`.. diff --git a/uv.lock b/uv.lock index 50a168b6..b5d06153 100644 --- a/uv.lock +++ b/uv.lock @@ -257,6 +257,21 @@ wheels = [ { url = "https://files.pythonhosted.org/packages/e4/ea/d836f008d33151c7a1f62caf3d8dd782e4d15f6a43897f64480c2b8de2ad/prompt_toolkit-3.0.50-py3-none-any.whl", hash = "sha256:9b6427eb19e479d98acff65196a307c555eb567989e6d88ebbb1b509d9779198", size = 387816 }, ] +[[package]] +name = "psutil" +version = "7.0.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/2a/80/336820c1ad9286a4ded7e845b2eccfcb27851ab8ac6abece774a6ff4d3de/psutil-7.0.0.tar.gz", hash = "sha256:7be9c3eba38beccb6495ea33afd982a44074b78f28c434a1f51cc07fd315c456", size = 497003 } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ed/e6/2d26234410f8b8abdbf891c9da62bee396583f713fb9f3325a4760875d22/psutil-7.0.0-cp36-abi3-macosx_10_9_x86_64.whl", hash = "sha256:101d71dc322e3cffd7cea0650b09b3d08b8e7c4109dd6809fe452dfd00e58b25", size = 238051 }, + { url = "https://files.pythonhosted.org/packages/04/8b/30f930733afe425e3cbfc0e1468a30a18942350c1a8816acfade80c005c4/psutil-7.0.0-cp36-abi3-macosx_11_0_arm64.whl", hash = "sha256:39db632f6bb862eeccf56660871433e111b6ea58f2caea825571951d4b6aa3da", size = 239535 }, + { url = "https://files.pythonhosted.org/packages/2a/ed/d362e84620dd22876b55389248e522338ed1bf134a5edd3b8231d7207f6d/psutil-7.0.0-cp36-abi3-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1fcee592b4c6f146991ca55919ea3d1f8926497a713ed7faaf8225e174581e91", size = 275004 }, + { url = "https://files.pythonhosted.org/packages/bf/b9/b0eb3f3cbcb734d930fdf839431606844a825b23eaf9a6ab371edac8162c/psutil-7.0.0-cp36-abi3-manylinux_2_12_x86_64.manylinux2010_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4b1388a4f6875d7e2aff5c4ca1cc16c545ed41dd8bb596cefea80111db353a34", size = 277986 }, + { url = "https://files.pythonhosted.org/packages/eb/a2/709e0fe2f093556c17fbafda93ac032257242cabcc7ff3369e2cb76a97aa/psutil-7.0.0-cp36-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a5f098451abc2828f7dc6b58d44b532b22f2088f4999a937557b603ce72b1993", size = 279544 }, + { url = "https://files.pythonhosted.org/packages/50/e6/eecf58810b9d12e6427369784efe814a1eec0f492084ce8eb8f4d89d6d61/psutil-7.0.0-cp37-abi3-win32.whl", hash = "sha256:ba3fcef7523064a6c9da440fc4d6bd07da93ac726b5733c29027d7dc95b39d99", size = 241053 }, + { url = "https://files.pythonhosted.org/packages/50/1b/6921afe68c74868b4c9fa424dad3be35b095e16687989ebbb50ce4fceb7c/psutil-7.0.0-cp37-abi3-win_amd64.whl", hash = "sha256:4cf3d4eb1aa9b348dec30105c55cd9b7d4629285735a102beb4441e38db90553", size = 244885 }, +] + [[package]] name = "ptyprocess" version = "0.7.0" @@ -373,6 +388,7 @@ dev = [ { name = "greenback" }, { name = "pexpect" }, { name = "prompt-toolkit" }, + { name = "psutil" }, { name = "pyperclip" }, { name = "pytest" }, { name = "stackscope" }, @@ -396,6 +412,7 @@ dev = [ { name = "greenback", specifier = ">=1.2.1,<2" }, { name = "pexpect", specifier = ">=4.9.0,<5" }, { name = "prompt-toolkit", specifier = ">=3.0.50" }, + { name = "psutil", specifier = ">=7.0.0" }, { name = "pyperclip", specifier = ">=1.9.0" }, { name = "pytest", specifier = ">=8.3.5" }, { name = "stackscope", specifier = ">=0.2.2,<0.3" }, From acac605c37e832276ced752f83ba928de259e84b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 2 Apr 2025 22:50:03 -0400 Subject: [PATCH 34/74] Move `DebugRequestError` to `._exceptions` --- tractor/_exceptions.py | 13 ++++++++++++- tractor/devx/_debug.py | 8 +------- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 8442ecfd..c9449a7b 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -126,6 +126,12 @@ class TrioTaskExited(Exception): ''' +class DebugRequestError(RuntimeError): + ''' + Failed to request stdio lock from root actor! + + ''' + # NOTE: more or less should be close to these: # 'boxed_type', # 'src_type', @@ -191,6 +197,8 @@ def get_err_type(type_name: str) -> BaseException|None: ): return type_ref + return None + def pack_from_raise( local_err: ( @@ -1009,7 +1017,10 @@ class TransportClosed(trio.BrokenResourceError): f' {cause}\n' # exc repr ) - getattr(log, self._loglevel)(message) + getattr( + log, + self._loglevel + )(message) # some errors we want to blow up from # inside the RPC msg loop diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index b95640dc..77753ff0 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -73,6 +73,7 @@ from tractor.log import get_logger from tractor._context import Context from tractor import _state from tractor._exceptions import ( + DebugRequestError, InternalError, NoRuntime, is_multi_cancelled, @@ -1740,13 +1741,6 @@ def sigint_shield( _pause_msg: str = 'Opening a pdb REPL in paused actor' -class DebugRequestError(RuntimeError): - ''' - Failed to request stdio lock from root actor! - - ''' - - _repl_fail_msg: str|None = ( 'Failed to REPl via `_pause()` ' ) From ad211f8c2ce55776560b328725a3f7d91ad0c8f7 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 3 Apr 2025 10:10:18 -0400 Subject: [PATCH 35/74] More `._addr` boxing refinements The more I think about it, it seems @guille's orig approach of unwrapping UDS socket-file addresses to strings (or `Path`) is making the most sense. I had originally thought that pairing it with the listening side's pid would add clarity (and it definitely does for introspection/debug/logging) but since we don't end up passing that pid to the eventual `.connect()` call on the client side, it doesn't make much sense to wrap it for the wire just to discard.. Further, the `tuple[str, int]` makes `wrap_address()` break for TCP since it will always match on uds first. So, on that note this patch refines a few things in prep for going back to that original `UnwrappedAddress` as `str` type though longer run i think the more "builtin approach" would be to add `msgspec` codec hooks for these types to avoid all the `.wrap()`/`.unwrap()` calls throughout the runtime. Down-low deats, - add `wrap_address()` doc string, detailed (todo) comments and handle the `[None, None]` case that can come directly from `._state._runtime_vars['_root_mailbox']`. - buncha adjustments to `UDSAddress`, - add a `filedir`, chng `filepath` -> `filename` and mk `maybe_pid` optional. - the intent `filedir` is act as the equivalent of the host part in a network proto's socket address and when it's null use the `.def_bindspace = get_rt_dir()`. - always ensure the `filedir / filename` is an absolute path and expose it as a new `.sockpath: Path` property. - mk `.is_valid` actually verify the `.sockpath` is in the valid `.bindspace: namely just checking it's in the expected dir. - add pedantic `match:`ing to `.from_addr()` such that we error on unexpected `type(addr)` inputs and otherwise parse any `sockpath: Path` inputs using a new `unwrap_sockpath()` which simply splits an abs file path to dir, file-name parts. - `.unwrap()` now just `str`-ifies the `.sockpath: Path` - adjust `.open/close_listener()` to use `.sockpath`. --- tractor/_addr.py | 150 +++++++++++++++++++++++++++++++++++------------ 1 file changed, 113 insertions(+), 37 deletions(-) diff --git a/tractor/_addr.py b/tractor/_addr.py index d5b8c81b..66fc49fc 100644 --- a/tractor/_addr.py +++ b/tractor/_addr.py @@ -217,7 +217,14 @@ class TCPAddress(Address): cls, addr: tuple[str, int] ) -> TCPAddress: - return TCPAddress(addr[0], addr[1]) + match addr: + case (str(), int()): + return TCPAddress(addr[0], addr[1]) + case _: + raise ValueError( + f'Invalid unwrapped address for {cls}\n' + f'{addr}\n' + ) def unwrap(self) -> tuple[str, int]: return ( @@ -228,7 +235,6 @@ class TCPAddress(Address): @classmethod def get_random( cls, - current_actor: Actor, bindspace: str = def_bindspace, ) -> TCPAddress: return TCPAddress(bindspace, 0) @@ -275,6 +281,15 @@ class TCPAddress(Address): ... +def unwrap_sockpath( + sockpath: Path, +) -> tuple[Path, Path]: + return ( + sockpath.parent, + sockpath.name, + ) + + class UDSAddress(Address): # TODO, maybe we should use better field and value # -[x] really this is a `.protocol_key` not a "name" of anything. @@ -287,23 +302,36 @@ class UDSAddress(Address): def __init__( self, - filepath: str|Path, - maybe_pid: int, - # ^XXX, in the sense you can also pass + filedir: Path|str|None, + # TODO, i think i want `.filename` here? + filename: str|Path, + + # XXX, in the sense you can also pass # a "non-real-world-process-id" such as is handy to represent # our host-local default "port-like" key for the very first # root actor to create a registry address. + maybe_pid: int|None = None, ): - self._filepath: Path = Path(filepath).absolute() + fdir = self._filedir = Path(filedir or self.def_bindspace).absolute() + fpath = self._filepath = Path(filename) + fp: Path = fdir / fpath + assert fp.is_absolute() + + # to track which "side" is the peer process by reading socket + # credentials-info. self._pid: int = maybe_pid + @property + def sockpath(self) -> Path: + return self._filedir / self._filepath + @property def is_valid(self) -> bool: ''' We block socket files not allocated under the runtime subdir. ''' - return self.bindspace in self._filepath.parents + return self.bindspace in self.sockpath.parents @property def bindspace(self) -> Path: @@ -312,23 +340,43 @@ class UDSAddress(Address): just the sub-directory in which we allocate socket files. ''' - return self.def_bindspace + return self._filedir or self.def_bindspace @classmethod def from_addr( cls, - addr: tuple[Path, int] + addr: ( + tuple[Path|str|None, int] + |Path|str + ), ) -> UDSAddress: - return UDSAddress( - filepath=addr[0], - maybe_pid=addr[1], - ) + match addr: + case tuple()|list(): + sockpath: Path = Path(addr[0]) + filedir, filename = unwrap_sockpath(sockpath) + pid: int = addr[1] + return UDSAddress( + filedir=filedir, + filename=filename, + maybe_pid=pid, + ) + # NOTE, in case we ever decide to just `.unwrap()` + # to a `Path|str`? + case str()|Path(): + sockpath: Path = Path(addr) + return UDSAddress(*unwrap_sockpath(sockpath)) + case _: + # import pdbp; pdbp.set_trace() + raise TypeError( + f'Bad unwrapped-address for {cls} !\n' + f'{addr!r}\n' + ) - def unwrap(self) -> tuple[Path, int]: + def unwrap(self) -> tuple[str, int]: + # XXX NOTE, since this gets passed DIRECTLY to + # `.ipc._uds.open_unix_socket_w_passcred()` return ( - str(self._filepath), - # XXX NOTE, since this gets passed DIRECTLY to - # `open_unix_socket_w_passcred()` above! + str(self.sockpath), self._pid, ) @@ -338,7 +386,7 @@ class UDSAddress(Address): bindspace: Path|None = None, # default netns ) -> UDSAddress: - bs: Path = bindspace or get_rt_dir() + filedir: Path = bindspace or cls.def_bindspace pid: int = os.getpid() actor: Actor|None = current_actor( err_on_no_runtime=False, @@ -351,30 +399,27 @@ class UDSAddress(Address): prefix: str = 'root' sockname: str = f'{prefix}@{pid}' - sockpath: Path = Path(f'{bs}/{sockname}.sock') + sockpath: Path = Path(f'{sockname}.sock') return UDSAddress( - # filename=f'{tempfile.gettempdir()}/{uuid4()}.sock' - filepath=sockpath, + filedir=filedir, + filename=sockpath, maybe_pid=pid, ) @classmethod def get_root(cls) -> Address: - def_uds_filepath: Path = ( - get_rt_dir() - / - 'registry@1616.sock' - ) + def_uds_filepath: Path = 'registry@1616.sock' return UDSAddress( - filepath=def_uds_filepath, - maybe_pid=1616 + filedir=None, + filename=def_uds_filepath, + maybe_pid=1616, ) def __repr__(self) -> str: return ( f'{type(self).__name__}' f'[' - f'({self._filepath}, {self._pid})' + f'({self.sockpath}, {self._pid})' f']' ) @@ -391,7 +436,7 @@ class UDSAddress(Address): self, **kwargs, ) -> SocketListener: - self._sock = socket.socket( + sock = self._sock = socket.socket( socket.AF_UNIX, socket.SOCK_STREAM ) @@ -400,8 +445,10 @@ class UDSAddress(Address): f'>[\n' f'|_{self}\n' ) - await self._sock.bind(self._filepath) - self._sock.listen(1) + + bindpath: Path = self.sockpath + await sock.bind(str(bindpath)) + sock.listen(1) log.info( f'Listening on UDS socket\n' f'[>\n' @@ -411,7 +458,7 @@ class UDSAddress(Address): def close_listener(self): self._sock.close() - os.unlink(self._filepath) + os.unlink(self.sockpath) preferred_transport: str = 'uds' @@ -455,26 +502,55 @@ def mk_uuid() -> str: def wrap_address( addr: UnwrappedAddress ) -> Address: + ''' + Wrap an `UnwrappedAddress` as an `Address`-type based + on matching builtin python data-structures which we adhoc + use for each. + XXX NOTE, careful care must be placed to ensure + `UnwrappedAddress` cases are **definitely unique** otherwise the + wrong transport backend may be loaded and will break many + low-level things in our runtime in a not-fun-to-debug way! + + XD + + ''' if is_wrapped_addr(addr): return addr cls: Type|None = None + # if 'sock' in addr[0]: + # import pdbp; pdbp.set_trace() match addr: - case ( + # TODO! BUT THIS WILL MATCH FOR TCP !... + # -[ ] so prolly go back to what guille had orig XD + # a plain ol' `str`? + case (( str()|Path(), int(), - ): + )): cls = UDSAddress - case tuple() | list(): + # classic network socket-address as tuple/list + case ( + (str(), int()) + | + [str(), int()] + ): cls = TCPAddress - case None: + # likely an unset UDS or TCP reg address as defaulted in + # `_state._runtime_vars['_root_mailbox']` + case ( + None + | + [None, None] + ): cls: Type[Address] = get_address_cls(preferred_transport) addr: UnwrappedAddress = cls.get_root().unwrap() case _: + # import pdbp; pdbp.set_trace() raise TypeError( f'Can not wrap address {type(addr)}\n' f'{addr!r}\n' From f15bbb30cc2a617ff41f26f3869aa8abd4a179bd Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 3 Apr 2025 12:16:29 -0400 Subject: [PATCH 36/74] UDS: translate file dne to connection-error For the case where there's clearly no socket file created/bound obviously the `trio.socket.connect()` call will raise `FileNotFoundError`, so just translate this to a builtin-`ConnectionError` at the transport layer so we can report the guilty `UDSAddress`. --- tractor/ipc/_uds.py | 34 +++++++++++++++++++++------------- 1 file changed, 21 insertions(+), 13 deletions(-) diff --git a/tractor/ipc/_uds.py b/tractor/ipc/_uds.py index 918e930c..9b0a4bff 100644 --- a/tractor/ipc/_uds.py +++ b/tractor/ipc/_uds.py @@ -139,20 +139,28 @@ class MsgpackUDSStream(MsgpackTransport): **kwargs ) -> MsgpackUDSStream: - 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 - ) + sockpath: Path = addr.sockpath + # + # ^XXX NOTE, we don't provide any out-of-band `.pid` info + # (like, over the socket as extra msgs) since the (augmented) + # `.setsockopt()` call tells the OS provide it; the client + # pid can then be read on server/listen() side via + # `get_peer_info()` above. + try: + 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( stream, prefix_size=prefix_size, From 3cc835c21502421cfb1c2d976e77b28176b98798 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 3 Apr 2025 12:22:21 -0400 Subject: [PATCH 37/74] Mv `Actor._do_handshake()` to `Channel`, add `.aid` Finally.. i've been meaning todo this for ages since the actor-id-swap-as-handshake is better layered as part of the IPC msg-ing machinery and then let's us encapsulate the connection-time-assignment of a remote peer's `Aid` as a new `Channel.aid: Aid`. For now we continue to offer the `.uid: tuple[str, str]` attr (by delegating to the `.uid` field) since there's still a few things relying on it in the runtime and ctx layers Nice bonuses from this, - it's very easy to get the peer's `Aid.pid: int` from anywhere in an IPC ctx by just reading it from the chan. - we aren't saving more then the wire struct-msg received. Also add deprecation warnings around usage to get us moving on porting the rest of consuming runtime code to the new attr! --- tractor/_runtime.py | 87 +++++++++++++++++++++----------------------- tractor/ipc/_chan.py | 63 ++++++++++++++++++++++++++++---- 2 files changed, 97 insertions(+), 53 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index bb2ac579..3db7ad11 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -289,7 +289,9 @@ class Actor: @property def aid(self) -> msgtypes.Aid: ''' - This process-singleton-actor's "unique ID" in struct form. + This process-singleton-actor's "unique actor ID" in struct form. + + See the `tractor.msg.Aid` struct for details. ''' return self._aid @@ -308,6 +310,17 @@ class Actor: process plane. ''' + msg: str = ( + f'`{type(self).__name__}.uid` is now deprecated.\n' + 'Use the new `.aid: tractor.msg.Aid` (struct) instead ' + 'which also provides additional named (optional) fields ' + 'beyond just the `.name` and `.uuid`.' + ) + warnings.warn( + msg, + DeprecationWarning, + stacklevel=2, + ) return ( self._aid.name, self._aid.uuid, @@ -495,7 +508,9 @@ class Actor: # send/receive initial handshake response try: - uid: tuple|None = await self._do_handshake(chan) + peer_aid: msgtypes.Aid = await chan._do_handshake( + aid=self.aid, + ) except ( TransportClosed, # ^XXX NOTE, the above wraps `trio` exc types raised @@ -524,6 +539,12 @@ class Actor: ) return + uid: tuple[str, str] = ( + peer_aid.name, + peer_aid.uuid, + ) + # TODO, can we make this downstream peer tracking use the + # `peer_aid` instead? familiar: str = 'new-peer' if _pre_chan := self._peers.get(uid): familiar: str = 'pre-existing-peer' @@ -1127,9 +1148,8 @@ class Actor: ) assert isinstance(chan, Channel) - # TODO: move this into a `Channel.handshake()`? # Initial handshake: swap names. - await self._do_handshake(chan) + await chan._do_handshake(aid=self.aid) accept_addrs: list[UnwrappedAddress]|None = None @@ -1270,11 +1290,16 @@ class Actor: # -[ ] need to extend the `SpawnSpec` tho! ) - except OSError: # failed to connect + # failed to connect back? + except ( + OSError, + ConnectionError, + ): log.warning( f'Failed to connect to spawning parent actor!?\n' + f'\n' f'x=> {parent_addr}\n' - f'|_{self}\n\n' + f' |_{self}\n\n' ) await self.cancel(req_chan=None) # self cancel raise @@ -1316,13 +1341,13 @@ class Actor: if ( '[Errno 98] Address already in use' in - oserr.args[0] + oserr.args#[0] ): log.exception( f'Address already in use?\n' f'{addr}\n' ) - raise + raise listeners.append(listener) await server_n.start( @@ -1337,8 +1362,10 @@ class Actor: handler_nursery=handler_nursery ) ) - log.runtime( + # TODO, wow make this message better! XD + log.info( 'Started server(s)\n' + + '\n'.join([f'|_{addr}' for addr in listen_addrs]) ) self._listen_addrs.extend(listen_addrs) @@ -1457,8 +1484,13 @@ class Actor: if self._server_down is not None: await self._server_down.wait() else: + tpt_protos: list[str] = [] + addr: Address + for addr in self._listen_addrs: + tpt_protos.append(addr.proto_key) log.warning( - 'Transport[TCP] server was cancelled start?' + 'Transport server(s) may have been cancelled before started?\n' + f'protos: {tpt_protos!r}\n' ) # cancel all rpc tasks permanently @@ -1745,41 +1777,6 @@ class Actor: ''' return self._peers[uid] - # TODO: move to `Channel.handshake(uid)` - async def _do_handshake( - self, - chan: Channel - - ) -> msgtypes.Aid: - ''' - Exchange `(name, UUIDs)` identifiers as the first - communication step with any (peer) remote `Actor`. - - These are essentially the "mailbox addresses" found in - "actor model" parlance. - - ''' - name, uuid = self.uid - await chan.send( - msgtypes.Aid( - name=name, - uuid=uuid, - ) - ) - aid: msgtypes.Aid = await chan.recv() - chan.aid = aid - - uid: tuple[str, str] = ( - aid.name, - aid.uuid, - ) - - if not isinstance(uid, tuple): - raise ValueError(f"{uid} is not a valid uid?!") - - chan.uid = uid - return uid - def is_infected_aio(self) -> bool: ''' If `True`, this actor is running `trio` in guest mode on diff --git a/tractor/ipc/_chan.py b/tractor/ipc/_chan.py index 01baf1e1..f6a50cc1 100644 --- a/tractor/ipc/_chan.py +++ b/tractor/ipc/_chan.py @@ -24,13 +24,13 @@ from contextlib import ( asynccontextmanager as acm, contextmanager as cm, ) -import os import platform from pprint import pformat import typing from typing import ( Any, ) +import warnings import trio @@ -50,7 +50,10 @@ from tractor._exceptions import ( MsgTypeError, pack_from_raise, ) -from tractor.msg import MsgCodec +from tractor.msg import ( + Aid, + MsgCodec, +) log = get_logger(__name__) @@ -86,8 +89,8 @@ class Channel: # user in ``.from_stream()``. self._transport: MsgTransport|None = transport - # set after handshake - always uid of far end - self.uid: tuple[str, str]|None = None + # set after handshake - always info from peer end + self.aid: Aid|None = None self._aiter_msgs = self._iter_msgs() self._exc: Exception|None = None @@ -99,6 +102,29 @@ class Channel: # runtime. self._cancel_called: bool = False + @property + def uid(self) -> tuple[str, str]: + ''' + Peer actor's unique id. + + ''' + msg: str = ( + f'`{type(self).__name__}.uid` is now deprecated.\n' + 'Use the new `.aid: tractor.msg.Aid` (struct) instead ' + 'which also provides additional named (optional) fields ' + 'beyond just the `.name` and `.uuid`.' + ) + warnings.warn( + msg, + DeprecationWarning, + stacklevel=2, + ) + peer_aid: Aid = self.aid + return ( + peer_aid.name, + peer_aid.uuid, + ) + @property def stream(self) -> trio.abc.Stream | None: return self._transport.stream if self._transport else None @@ -182,9 +208,7 @@ class Channel: f' _closed={self._closed}\n' f' _cancel_called={self._cancel_called}\n' f'\n' - f' |_runtime: Actor\n' - f' pid={os.getpid()}\n' - f' uid={self.uid}\n' + f' |_peer: {self.aid}\n' f'\n' f' |_msgstream: {tpt_name}\n' f' proto={tpt.laddr.proto_key!r}\n' @@ -281,7 +305,7 @@ class Channel: async def aclose(self) -> None: log.transport( - f'Closing channel to {self.uid} ' + f'Closing channel to {self.aid} ' f'{self.laddr} -> {self.raddr}' ) assert self._transport @@ -381,6 +405,29 @@ class Channel: def connected(self) -> bool: return self._transport.connected() if self._transport else False + async def _do_handshake( + self, + aid: Aid, + + ) -> Aid: + ''' + Exchange `(name, UUIDs)` identifiers as the first + communication step with any (peer) remote `Actor`. + + These are essentially the "mailbox addresses" found in + "actor model" parlance. + + ''' + await self.send(aid) + peer_aid: Aid = await self.recv() + log.runtime( + f'Received hanshake with peer actor,\n' + f'{peer_aid}\n' + ) + # NOTE, we always are referencing the remote peer! + self.aid = peer_aid + return peer_aid + @acm async def _connect_chan( From 412c66d000e5f48c1522060b63dd50c0a7bb115f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 3 Apr 2025 13:14:58 -0400 Subject: [PATCH 38/74] Mv to `Channel._do_handshake()` in `open_portal()` As per the method migration in the last commit. Also adjust all `.uid` usage to the new `.aid`. --- tractor/_portal.py | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/tractor/_portal.py b/tractor/_portal.py index 0efcbd0b..d0fd3cc3 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -175,7 +175,7 @@ class Portal: # not expecting a "main" result if self._expect_result_ctx is None: log.warning( - f"Portal for {self.channel.uid} not expecting a final" + f"Portal for {self.channel.aid} not expecting a final" " result?\nresult() should only be called if subactor" " was spawned with `ActorNursery.run_in_actor()`") return NoResult @@ -222,7 +222,7 @@ class Portal: # IPC calls if self._streams: log.cancel( - f"Cancelling all streams with {self.channel.uid}") + f"Cancelling all streams with {self.channel.aid}") for stream in self._streams.copy(): try: await stream.aclose() @@ -267,7 +267,7 @@ class Portal: return False reminfo: str = ( - f'c)=> {self.channel.uid}\n' + f'c)=> {self.channel.aid}\n' f' |_{chan}\n' ) log.cancel( @@ -310,7 +310,7 @@ class Portal: ): log.debug( 'IPC chan for actor already closed or broken?\n\n' - f'{self.channel.uid}\n' + f'{self.channel.aid}\n' f' |_{self.channel}\n' ) return False @@ -551,8 +551,10 @@ async def open_portal( await channel.connect() was_connected = True - if channel.uid is None: - await actor._do_handshake(channel) + if channel.aid is None: + await channel._do_handshake( + aid=actor.aid, + ) msg_loop_cs: trio.CancelScope|None = None if start_msg_loop: From 0fcbedd2be2d5ebca4fe46538e98bd0ec339fda9 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 3 Apr 2025 13:19:19 -0400 Subject: [PATCH 39/74] Change some low-hanging `.uid`s to `.aid` Throughout `_context` and `_spawn` where it causes no big disruption. Still lots to work out for things like how to pass `--uid ` to spawned subactors and whether we want a diff name for the minimum `tuple` required to distinguish a subactor pre-process-ID allocation by the OS. --- tractor/_context.py | 8 ++++---- tractor/_spawn.py | 17 +++++++++++------ 2 files changed, 15 insertions(+), 10 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index 19f3daef..53f5b233 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -366,7 +366,7 @@ class Context: # f' ---\n' f' |_ipc: {self.dst_maddr}\n' # f' dst_maddr{ds}{self.dst_maddr}\n' - f" uid{ds}'{self.chan.uid}'\n" + f" uid{ds}'{self.chan.aid}'\n" f" cid{ds}'{self.cid}'\n" # f' ---\n' f'\n' @@ -945,10 +945,10 @@ class Context: reminfo: str = ( # ' =>\n' # f'Context.cancel() => {self.chan.uid}\n' + f'\n' f'c)=> {self.chan.uid}\n' - # f'{self.chan.uid}\n' - f' |_ @{self.dst_maddr}\n' - f' >> {self.repr_rpc}\n' + f' |_[{self.dst_maddr}\n' + f' >>{self.repr_rpc}\n' # f' >> {self._nsf}() -> {codec}[dict]:\n\n' # TODO: pull msg-type from spec re #320 ) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index db83ee57..5df8125a 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -52,6 +52,7 @@ from tractor._runtime import Actor from tractor._entry import _mp_main from tractor._exceptions import ActorFailure from tractor.msg.types import ( + Aid, SpawnSpec, ) @@ -164,7 +165,7 @@ async def exhaust_portal( # TODO: merge with above? log.warning( 'Cancelled portal result waiter task:\n' - f'uid: {portal.channel.uid}\n' + f'uid: {portal.channel.aid}\n' f'error: {err}\n' ) return err @@ -172,7 +173,7 @@ async def exhaust_portal( else: log.debug( f'Returning final result from portal:\n' - f'uid: {portal.channel.uid}\n' + f'uid: {portal.channel.aid}\n' f'result: {final}\n' ) return final @@ -325,12 +326,12 @@ async def soft_kill( see `.hard_kill()`). ''' - uid: tuple[str, str] = portal.channel.uid + peer_aid: Aid = portal.channel.aid try: log.cancel( f'Soft killing sub-actor via portal request\n' f'\n' - f'(c=> {portal.chan.uid}\n' + f'(c=> {peer_aid}\n' f' |_{proc}\n' ) # wait on sub-proc to signal termination @@ -379,7 +380,7 @@ async def soft_kill( if proc.poll() is None: # type: ignore log.warning( 'Subactor still alive after cancel request?\n\n' - f'uid: {uid}\n' + f'uid: {peer_aid}\n' f'|_{proc}\n' ) n.cancel_scope.cancel() @@ -460,6 +461,9 @@ async def trio_proc( # the OS; it otherwise can be passed via the parent channel if # we prefer in the future (for privacy). "--uid", + # TODO, how to pass this over "wire" encodings like + # cmdline args? + # -[ ] maybe we can add an `Aid.min_tuple()` ? str(subactor.uid), # Address the child must connect to on startup "--parent_addr", @@ -725,7 +729,8 @@ async def mp_proc( # channel should have handshake completed by the # local actor by the time we get a ref to it event, chan = await actor_nursery._actor.wait_for_peer( - subactor.uid) + subactor.uid, + ) # XXX: monkey patch poll API to match the ``subprocess`` API.. # not sure why they don't expose this but kk. From d6d0112d95b38bc60477bc369f608ebd6a0ad703 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 3 Apr 2025 13:28:36 -0400 Subject: [PATCH 40/74] Some more log message tweaks - aggregate the `MsgStream.aclose()` "reader tasks" stats content into a common `message: str` before emit. - tweak an `_rpc.process_messages()` emit per new `Channel.__repr__()`. --- tractor/_rpc.py | 6 ++++-- tractor/_streaming.py | 16 ++++++++-------- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/tractor/_rpc.py b/tractor/_rpc.py index 67681bee..3fc11db7 100644 --- a/tractor/_rpc.py +++ b/tractor/_rpc.py @@ -1219,8 +1219,10 @@ async def process_messages( # -[ ] figure out how this will break with other transports? tc.report_n_maybe_raise( message=( - f'peer IPC channel closed abruptly?\n\n' - f'<=x {chan}\n' + f'peer IPC channel closed abruptly?\n' + f'\n' + f'<=x[\n' + f' {chan}\n' f' |_{chan.raddr}\n\n' ) + diff --git a/tractor/_streaming.py b/tractor/_streaming.py index e6001c7c..f9d1e7ef 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -437,22 +437,23 @@ class MsgStream(trio.abc.Channel): message: str = ( f'Stream self-closed by {this_side!r}-side before EoC from {peer_side!r}\n' # } bc a stream is a "scope"/msging-phase inside an IPC - f'x}}>\n' + f'c}}>\n' f' |_{self}\n' ) - log.cancel(message) - self._eoc = trio.EndOfChannel(message) - if ( (rx_chan := self._rx_chan) and (stats := rx_chan.statistics()).tasks_waiting_receive ): - log.cancel( - f'Msg-stream is closing but there is still reader tasks,\n' + message += ( + f'AND there is still reader tasks,\n' + f'\n' f'{stats}\n' ) + log.cancel(message) + self._eoc = trio.EndOfChannel(message) + # ?XXX WAIT, why do we not close the local mem chan `._rx_chan` XXX? # => NO, DEFINITELY NOT! <= # if we're a bi-dir `MsgStream` BECAUSE this same @@ -811,13 +812,12 @@ async def open_stream_from_ctx( # sanity, can remove? assert eoc is stream._eoc - log.warning( + log.runtime( 'Stream was terminated by EoC\n\n' # NOTE: won't show the error but # does show txt followed by IPC msg. f'{str(eoc)}\n' ) - finally: if ctx._portal: try: From 708ce4a051eb5112348270341445004887d2ddc9 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 3 Apr 2025 16:15:53 -0400 Subject: [PATCH 41/74] Repair weird spawn test, start `test_root_runtime` There was a very strange legacy test `test_spawning.test_local_arbiter_subactor_global_state` which was causing unforseen hangs/errors on the UDS tpt and looking deeper this test was already doing root-actor things that should never have been valid XD So rework that test to properly demonstrate something of value (i guess..) and add a new suite which start more rigorously auditing our `open_root_actor()` permitted usage. For the old test, - since the main point of this test seemed to be the ability to invoke the same function in both the parent and child actor (using the very legacy `ActorNursery.run_in_actor()`.. due to be deprecated) rename it to `test_run_in_actor_same_func_in_child`, - don't re-enter `.open_root_actor()` since that's invalid usage (tested in new suite see below), - adjust some `spawn()` arg/var naming and ensure we only return in the child. For the new suite add tests for, - ensuring the implicit `open_root_actor()` call under `open_nursery()`. - double open of `open_root_actor()` from within the same process tree both from a root and sub. Intro some new `_exceptions` used in the new suite, - a top level `RuntimeFailure` for generically expressing faults not of our own doing that prevent successful operation; this is what we now (changed in this commit) raise on attempts to open a 2nd root. - mk `ActorFailure` derive from the former; it's already used from `._spawn` when subprocs fail to boot. --- tests/test_root_runtime.py | 85 +++++++++++++++++++++++++++++ tests/test_spawning.py | 108 ++++++++++++++++++++++--------------- tractor/_exceptions.py | 18 ++++++- tractor/_root.py | 4 +- 4 files changed, 168 insertions(+), 47 deletions(-) create mode 100644 tests/test_root_runtime.py diff --git a/tests/test_root_runtime.py b/tests/test_root_runtime.py new file mode 100644 index 00000000..c97d4eba --- /dev/null +++ b/tests/test_root_runtime.py @@ -0,0 +1,85 @@ +''' +Runtime boot/init sanity. + +''' + +import pytest +import trio + +import tractor +from tractor._exceptions import RuntimeFailure + + +@tractor.context +async def open_new_root_in_sub( + ctx: tractor.Context, +) -> None: + + async with tractor.open_root_actor(): + pass + + +@pytest.mark.parametrize( + 'open_root_in', + ['root', 'sub'], + ids='open_2nd_root_in={}'.format, +) +def test_only_one_root_actor( + open_root_in: str, + reg_addr: tuple, + debug_mode: bool +): + ''' + Verify we specially fail whenever more then one root actor + is attempted to be opened within an already opened tree. + + ''' + async def main(): + async with tractor.open_nursery() as an: + + if open_root_in == 'root': + async with tractor.open_root_actor( + registry_addrs=[reg_addr], + ): + pass + + ptl: tractor.Portal = await an.start_actor( + name='bad_rooty_boi', + enable_modules=[__name__], + ) + + async with ptl.open_context( + open_new_root_in_sub, + ) as (ctx, first): + pass + + if open_root_in == 'root': + with pytest.raises( + RuntimeFailure + ) as excinfo: + trio.run(main) + + else: + with pytest.raises( + tractor.RemoteActorError, + ) as excinfo: + trio.run(main) + + assert excinfo.value.boxed_type is RuntimeFailure + + +def test_implicit_root_via_first_nursery( + reg_addr: tuple, + debug_mode: bool +): + ''' + The first `ActorNursery` open should implicitly call + `_root.open_root_actor()`. + + ''' + async def main(): + async with tractor.open_nursery() as an: + assert an._implicit_runtime_started + assert tractor.current_actor().aid.name == 'root' + + trio.run(main) diff --git a/tests/test_spawning.py b/tests/test_spawning.py index 58aa955a..cf373ada 100644 --- a/tests/test_spawning.py +++ b/tests/test_spawning.py @@ -2,6 +2,7 @@ Spawning basics """ +from functools import partial from typing import ( Any, ) @@ -12,74 +13,95 @@ import tractor from tractor._testing import tractor_test -data_to_pass_down = {'doggy': 10, 'kitty': 4} +data_to_pass_down = { + 'doggy': 10, + 'kitty': 4, +} async def spawn( - is_arbiter: bool, + should_be_root: bool, data: dict, reg_addr: tuple[str, int], + + debug_mode: bool = False, ): - namespaces = [__name__] - await trio.sleep(0.1) + actor = tractor.current_actor(err_on_no_runtime=False) - async with tractor.open_root_actor( - arbiter_addr=reg_addr, - ): - actor = tractor.current_actor() - assert actor.is_arbiter == is_arbiter - data = data_to_pass_down + if should_be_root: + assert actor is None # no runtime yet + async with ( + tractor.open_root_actor( + arbiter_addr=reg_addr, + ), + tractor.open_nursery() as an, + ): + # now runtime exists + actor: tractor.Actor = tractor.current_actor() + assert actor.is_arbiter == should_be_root - if actor.is_arbiter: - async with tractor.open_nursery() as nursery: + # spawns subproc here + portal: tractor.Portal = await an.run_in_actor( + fn=spawn, - # forks here - portal = await nursery.run_in_actor( - spawn, - is_arbiter=False, - name='sub-actor', - data=data, - reg_addr=reg_addr, - enable_modules=namespaces, - ) + # spawning args + name='sub-actor', + enable_modules=[__name__], - assert len(nursery._children) == 1 - assert portal.channel.uid in tractor.current_actor()._peers - # be sure we can still get the result - result = await portal.result() - assert result == 10 - return result - else: - return 10 + # passed to a subactor-recursive RPC invoke + # of this same `spawn()` fn. + should_be_root=False, + data=data_to_pass_down, + reg_addr=reg_addr, + ) + + assert len(an._children) == 1 + assert portal.channel.uid in tractor.current_actor()._peers + + # get result from child subactor + result = await portal.result() + assert result == 10 + return result + else: + assert actor.is_arbiter == should_be_root + return 10 -def test_local_arbiter_subactor_global_state( - reg_addr, +def test_run_in_actor_same_func_in_child( + reg_addr: tuple, + debug_mode: bool, ): result = trio.run( - spawn, - True, - data_to_pass_down, - reg_addr, + partial( + spawn, + should_be_root=True, + data=data_to_pass_down, + reg_addr=reg_addr, + debug_mode=debug_mode, + ) ) assert result == 10 async def movie_theatre_question(): - """A question asked in a dark theatre, in a tangent + ''' + A question asked in a dark theatre, in a tangent (errr, I mean different) process. - """ + + ''' return 'have you ever seen a portal?' @tractor_test async def test_movie_theatre_convo(start_method): - """The main ``tractor`` routine. - """ - async with tractor.open_nursery(debug_mode=True) as n: + ''' + The main ``tractor`` routine. - portal = await n.start_actor( + ''' + async with tractor.open_nursery(debug_mode=True) as an: + + portal = await an.start_actor( 'frank', # enable the actor to run funcs from this current module enable_modules=[__name__], @@ -118,8 +140,8 @@ async def test_most_beautiful_word( with trio.fail_after(1): async with tractor.open_nursery( debug_mode=debug_mode, - ) as n: - portal = await n.run_in_actor( + ) as an: + portal = await an.run_in_actor( cellar_door, return_value=return_value, name='some_linguist', diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index c9449a7b..0b4e8196 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -72,8 +72,22 @@ log = get_logger('tractor') _this_mod = importlib.import_module(__name__) -class ActorFailure(Exception): - "General actor failure" +class RuntimeFailure(RuntimeError): + ''' + General `Actor`-runtime failure due to, + + - a bad runtime-env, + - falied spawning (bad input to process), + - API usage. + + ''' + + +class ActorFailure(RuntimeFailure): + ''' + `Actor` failed to boot before/after spawn + + ''' class InternalError(RuntimeError): diff --git a/tractor/_root.py b/tractor/_root.py index 50773056..77344013 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -60,7 +60,7 @@ from ._addr import ( wrap_address, ) from ._exceptions import ( - ActorFailure, + RuntimeFailure, is_multi_cancelled, ) @@ -195,7 +195,7 @@ async def open_root_actor( rtvs: dict[str, Any] = _state._runtime_vars root_mailbox: list[str, int] = rtvs['_root_mailbox'] registry_addrs: list[list[str, int]] = rtvs['_registry_addrs'] - raise ActorFailure( + raise RuntimeFailure( f'A current actor already exists !?\n' f'({already_actor}\n' f'\n' From d9aee98db268e09cb72179d05285e18e5776d682 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 3 Apr 2025 16:35:33 -0400 Subject: [PATCH 42/74] Add `Arbiter.is_registry()` in prep for proper `.discovery._registry` --- tractor/_runtime.py | 20 +++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 3db7ad11..080768a2 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -2134,15 +2134,15 @@ async def async_main( log.info(teardown_report) -# TODO: rename to `Registry` and move to `._discovery`! +# TODO: rename to `Registry` and move to `.discovery._registry`! class Arbiter(Actor): ''' - A special registrar actor who can contact all other actors - within its immediate process tree and possibly keeps a registry - of others meant to be discoverable in a distributed - application. Normally the registrar is also the "root actor" - and thus always has access to the top-most-level actor - (process) nursery. + A special registrar (and for now..) `Actor` who can contact all + other actors within its immediate process tree and possibly keeps + a registry of others meant to be discoverable in a distributed + application. Normally the registrar is also the "root actor" and + thus always has access to the top-most-level actor (process) + nursery. By default, the registrar is always initialized when and if no other registrar socket addrs have been specified to runtime @@ -2162,6 +2162,12 @@ class Arbiter(Actor): ''' is_arbiter = True + # TODO, implement this as a read on there existing a `._state` of + # some sort setup by whenever we impl this all as + # a `.discovery._registry.open_registry()` API + def is_registry(self) -> bool: + return self.is_arbiter + def __init__( self, *args, From be1d8bf6fa1264b9c56c9ef2a2b08ee031e60e0c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 3 Apr 2025 20:12:30 -0400 Subject: [PATCH 43/74] s/`._addr.preferred_transport`/`_state._def_tpt_proto` Such that the "global-ish" setting (actor-local) is managed with the others per actor-process and type it as a `Literal['tcp', 'uds']` of the currently support protocol keys. Here obvi `_tpt` is some kinda shorthand for "transport" and `_proto` is for "protocol" Bp Change imports and refs in all dependent modules. Oh right, and disable UDS in `wrap_address()` for the moment while i figure out how to avoid the unwrapped type collision.. --- tractor/_addr.py | 18 +++++++++--------- tractor/_discovery.py | 4 ++-- tractor/_root.py | 6 ++++-- tractor/_runtime.py | 7 ++++--- tractor/_state.py | 9 +++++++++ tractor/_supervise.py | 7 ++++--- 6 files changed, 32 insertions(+), 19 deletions(-) diff --git a/tractor/_addr.py b/tractor/_addr.py index 66fc49fc..e7b92461 100644 --- a/tractor/_addr.py +++ b/tractor/_addr.py @@ -40,6 +40,7 @@ from ._state import ( get_rt_dir, current_actor, is_root_process, + _def_tpt_proto, ) if TYPE_CHECKING: @@ -461,9 +462,6 @@ class UDSAddress(Address): os.unlink(self.sockpath) -preferred_transport: str = 'uds' - - _address_types: bidict[str, Type[Address]] = { 'tcp': TCPAddress, 'uds': UDSAddress @@ -525,11 +523,11 @@ def wrap_address( # TODO! BUT THIS WILL MATCH FOR TCP !... # -[ ] so prolly go back to what guille had orig XD # a plain ol' `str`? - case (( - str()|Path(), - int(), - )): - cls = UDSAddress + # case (( + # str()|Path(), + # int(), + # )): + # cls = UDSAddress # classic network socket-address as tuple/list case ( @@ -541,12 +539,14 @@ def wrap_address( # likely an unset UDS or TCP reg address as defaulted in # `_state._runtime_vars['_root_mailbox']` + # + # TODO? figure out when/if we even need this? case ( None | [None, None] ): - cls: Type[Address] = get_address_cls(preferred_transport) + cls: Type[Address] = get_address_cls(_def_tpt_proto) addr: UnwrappedAddress = cls.get_root().unwrap() case _: diff --git a/tractor/_discovery.py b/tractor/_discovery.py index 5d9b6cd1..aa487b26 100644 --- a/tractor/_discovery.py +++ b/tractor/_discovery.py @@ -33,7 +33,6 @@ from .ipc import _connect_chan, Channel from ._addr import ( UnwrappedAddress, Address, - preferred_transport, wrap_address ) from ._portal import ( @@ -44,6 +43,7 @@ from ._portal import ( from ._state import ( current_actor, _runtime_vars, + _def_tpt_proto, ) if TYPE_CHECKING: @@ -209,7 +209,7 @@ async def maybe_open_portal( async def find_actor( name: str, registry_addrs: list[UnwrappedAddress]|None = None, - enable_transports: list[str] = [preferred_transport], + enable_transports: list[str] = [_def_tpt_proto], only_first: bool = True, raise_on_none: bool = False, diff --git a/tractor/_root.py b/tractor/_root.py index 77344013..f67d746a 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -56,7 +56,6 @@ from ._addr import ( UnwrappedAddress, default_lo_addrs, mk_uuid, - preferred_transport, wrap_address, ) from ._exceptions import ( @@ -139,6 +138,7 @@ async def maybe_block_bp( os.environ.pop('PYTHONBREAKPOINT', None) + @acm async def open_root_actor( *, @@ -148,7 +148,9 @@ async def open_root_actor( # defaults are above arbiter_addr: tuple[UnwrappedAddress]|None = None, - enable_transports: list[str] = [preferred_transport], + enable_transports: list[ + _state.TransportProtocolKey, + ] = [_state._def_tpt_proto], name: str|None = 'root', diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 080768a2..f8356582 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -80,7 +80,6 @@ from ._addr import ( Address, default_lo_addrs, get_address_cls, - preferred_transport, wrap_address, ) from ._context import ( @@ -1322,7 +1321,9 @@ class Actor: ''' if listen_addrs is None: - listen_addrs = default_lo_addrs([preferred_transport]) + listen_addrs = default_lo_addrs([ + _state._def_tpt_proto + ]) else: listen_addrs: list[Address] = [ @@ -1846,7 +1847,7 @@ async def async_main( enable_transports: list[str] = ( maybe_preferred_transports_says_rent or - [preferred_transport] + [_state._def_tpt_proto] ) for transport_key in enable_transports: transport_cls: Type[Address] = get_address_cls( diff --git a/tractor/_state.py b/tractor/_state.py index 4cb7e784..0ec4577e 100644 --- a/tractor/_state.py +++ b/tractor/_state.py @@ -26,6 +26,7 @@ import os from pathlib import Path from typing import ( Any, + Literal, TYPE_CHECKING, ) @@ -164,3 +165,11 @@ def get_rt_dir( if not rtdir.is_dir(): rtdir.mkdir() return rtdir + + +# default IPC transport protocol settings +TransportProtocolKey = Literal[ + 'tcp', + 'uds', +] +_def_tpt_proto: TransportProtocolKey = 'tcp' diff --git a/tractor/_supervise.py b/tractor/_supervise.py index e4017c44..ab3a2a94 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -34,7 +34,6 @@ import trio from .devx._debug import maybe_wait_for_debugger from ._addr import ( UnwrappedAddress, - preferred_transport, mk_uuid, ) from ._state import current_actor, is_main_process @@ -45,7 +44,9 @@ from ._exceptions import ( is_multi_cancelled, ContextCancelled, ) -from ._root import open_root_actor +from ._root import ( + open_root_actor, +) from . import _state from . import _spawn @@ -138,7 +139,7 @@ class ActorNursery: bind_addrs: list[UnwrappedAddress]|None = None, rpc_module_paths: list[str]|None = None, - enable_transports: list[str] = [preferred_transport], + enable_transports: list[str] = [_state._def_tpt_proto], enable_modules: list[str]|None = None, loglevel: str|None = None, # set log level per subactor debug_mode: bool|None = None, From bedde076d911f449a5930d39f0ae0d05ab8f66c5 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 3 Apr 2025 22:24:24 -0400 Subject: [PATCH 44/74] Unwrap `UDSAddress` as `tuple[str, str]`, i.e. sin pid Since in hindsight the real analog of a net-proto's "bindspace" (normally its routing layer's addresses-port-set) is more akin to the "location in the file-system" for a UDS socket file (aka the file's parent directory) determines whether or not the "port" (aka it's file-name) collides with any other. So the `._filedir: Path` is like the allocated "address" and, the `._filename: Path|str` is basically the "port", at least in my mind.. Bp Thinking about fs dirs like a "host address" means you can get essentially the same benefits/behaviour of say an (ip) addresses-port-space but using the (current process-namespace's) filesys-tree. Note that for UDS sockets in particular the network-namespace is what would normally isolate so called "abstract sockets" (i.e. UDS sockets that do NOT use file-paths by setting `struct sockaddr_un.sun_path = 'abstract', see `man unix`); using directories is even easier and definitely more explicit/readable/immediately-obvious as a human-user. As such this reworks all the necessary `UDSAddress` meths, - `.unwrap()` now returns a `tuple(str(._filedir, str(._filename))`, - `wrap_address()` now matches UDS on a 2nd tuple `str()` element, - `.get_root()` no longer passes `maybe_pid`. AND adjusts `MsgpackUDSStream` to, - use the new `unwrap_sockpath()` on the `socket.get[sock/peer]name()` output before passing directly as `UDSAddress.__init__(filedir, filename)` instead of via `.from_addr()`. - also pass `maybe_pid`s to init since no longer included in the unwrapped-type form. --- tractor/_addr.py | 64 ++++++++++++++++++++++++++------------------- tractor/ipc/_uds.py | 31 +++++++++++++--------- 2 files changed, 56 insertions(+), 39 deletions(-) diff --git a/tractor/_addr.py b/tractor/_addr.py index e7b92461..eaf4c202 100644 --- a/tractor/_addr.py +++ b/tractor/_addr.py @@ -313,10 +313,18 @@ class UDSAddress(Address): # root actor to create a registry address. maybe_pid: int|None = None, ): - fdir = self._filedir = Path(filedir or self.def_bindspace).absolute() - fpath = self._filepath = Path(filename) + fdir = self._filedir = Path( + filedir + or + self.def_bindspace + ).absolute() + fpath = self._filename = Path(filename) fp: Path = fdir / fpath - assert fp.is_absolute() + assert ( + fp.is_absolute() + and + fp == self.sockpath + ) # to track which "side" is the peer process by reading socket # credentials-info. @@ -324,7 +332,7 @@ class UDSAddress(Address): @property def sockpath(self) -> Path: - return self._filedir / self._filepath + return self._filedir / self._filename @property def is_valid(self) -> bool: @@ -347,19 +355,20 @@ class UDSAddress(Address): def from_addr( cls, addr: ( - tuple[Path|str|None, int] - |Path|str + tuple[Path|str, Path|str]|Path|str ), ) -> UDSAddress: match addr: case tuple()|list(): - sockpath: Path = Path(addr[0]) - filedir, filename = unwrap_sockpath(sockpath) - pid: int = addr[1] + filedir = Path(addr[0]) + filename = Path(addr[1]) + # sockpath: Path = Path(addr[0]) + # filedir, filename = unwrap_sockpath(sockpath) + # pid: int = addr[1] return UDSAddress( filedir=filedir, filename=filename, - maybe_pid=pid, + # maybe_pid=pid, ) # NOTE, in case we ever decide to just `.unwrap()` # to a `Path|str`? @@ -377,8 +386,8 @@ class UDSAddress(Address): # XXX NOTE, since this gets passed DIRECTLY to # `.ipc._uds.open_unix_socket_w_passcred()` return ( - str(self.sockpath), - self._pid, + str(self._filedir), + str(self._filename), ) @classmethod @@ -409,18 +418,18 @@ class UDSAddress(Address): @classmethod def get_root(cls) -> Address: - def_uds_filepath: Path = 'registry@1616.sock' + def_uds_filename: Path = 'registry@1616.sock' return UDSAddress( filedir=None, - filename=def_uds_filepath, - maybe_pid=1616, + filename=def_uds_filename, + # maybe_pid=1616, ) def __repr__(self) -> str: return ( f'{type(self).__name__}' f'[' - f'({self.sockpath}, {self._pid})' + f'({self._filedir}, {self._filename})' f']' ) @@ -430,7 +439,7 @@ class UDSAddress(Address): f'Can not compare {type(other)} with {type(self)}' ) - return self._filepath == other._filepath + return self.sockpath == other.sockpath # async def open_listener(self, **kwargs) -> SocketListener: async def open_listener( @@ -520,14 +529,6 @@ def wrap_address( # if 'sock' in addr[0]: # import pdbp; pdbp.set_trace() match addr: - # TODO! BUT THIS WILL MATCH FOR TCP !... - # -[ ] so prolly go back to what guille had orig XD - # a plain ol' `str`? - # case (( - # str()|Path(), - # int(), - # )): - # cls = UDSAddress # classic network socket-address as tuple/list case ( @@ -537,6 +538,14 @@ def wrap_address( ): cls = TCPAddress + case ( + # (str()|Path(), str()|Path()), + # ^TODO? uhh why doesn't this work!? + + (_, filename) + ) if type(filename) is str: + cls = UDSAddress + # likely an unset UDS or TCP reg address as defaulted in # `_state._runtime_vars['_root_mailbox']` # @@ -552,8 +561,9 @@ def wrap_address( case _: # import pdbp; pdbp.set_trace() raise TypeError( - f'Can not wrap address {type(addr)}\n' - f'{addr!r}\n' + f'Can not wrap unwrapped-address ??\n' + f'type(addr): {type(addr)!r}\n' + f'addr: {addr!r}\n' ) return cls.from_addr(addr) diff --git a/tractor/ipc/_uds.py b/tractor/ipc/_uds.py index 9b0a4bff..3d24447b 100644 --- a/tractor/ipc/_uds.py +++ b/tractor/ipc/_uds.py @@ -38,7 +38,10 @@ from trio._highlevel_open_unix_stream import ( from tractor.msg import MsgCodec from tractor.log import get_logger -from tractor._addr import UDSAddress +from tractor._addr import ( + UDSAddress, + unwrap_sockpath, +) from tractor.ipc._transport import MsgpackTransport @@ -194,16 +197,20 @@ class MsgpackUDSStream(MsgpackTransport): case (bytes(), str()): sock_path: Path = Path(sockname) ( - pid, - uid, - gid, + peer_pid, + _, + _, ) = get_peer_info(sock) - laddr = UDSAddress.from_addr(( - sock_path, - os.getpid(), - )) - raddr = UDSAddress.from_addr(( - sock_path, - pid - )) + + filedir, filename = unwrap_sockpath(sock_path) + laddr = UDSAddress( + filedir=filedir, + filename=filename, + maybe_pid=os.getpid(), + ) + raddr = UDSAddress( + filedir=filedir, + filename=filename, + maybe_pid=peer_pid + ) return (laddr, raddr) From 00d8a2a0990a4a3ed36ef267b1465d181582d4a6 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 6 Apr 2025 13:54:10 -0400 Subject: [PATCH 45/74] Improve `TransportClosed.__repr__()`, add `src_exc` By borrowing from the implementation of `RemoteActorError.pformat()` which is now factored into a new `.devx.pformat_exc()` and re-used for both error types while maintaining the same func-sig. Obviously delegate `RemoteActorError.pformat()` to the new helper accordingly and keeping the prior `body` generation from `.devx.pformat_boxed_tb()` as before. The new helper allows for, - passing any of a `header|message|body: str` which are all combined in that order in the final output. - getting the `exc.message` as the default `message` part. - generating an objecty-looking "type-name" header to be rendered by default when `header` is not overridden. - "first-line-of `message`" processing which we split-off and then re-inject as a `f'<{type(exc).__name__}( {first} )>'` top line header. - an optional `tail: str = '>'` to "close the object"-look only added when `with_type_header: bool = True`. Adjustments to `TransportClosed` around this include, - replacing the init `cause` arg for a `src_exc` which is now always assigned to a same named instance var. - displaying that new `.src_exc` in the `body: str` arg to the `.devx.pformat.pformat_exc()` call so you can always see the underlying (normally `trio`) source error. - just make it inherit from `Exception` not `trio.BrokenResourceError` to avoid handlers catching `TransportClosed` as the former particularly in testing when we want to sometimes to distinguish them. --- tractor/_exceptions.py | 110 ++++++++++++++------------------------ tractor/devx/pformat.py | 80 +++++++++++++++++++++++++++ tractor/ipc/_transport.py | 24 ++++----- 3 files changed, 130 insertions(+), 84 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 0b4e8196..1238220e 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -23,7 +23,6 @@ import builtins import importlib from pprint import pformat from pdb import bdb -import sys from types import ( TracebackType, ) @@ -543,7 +542,6 @@ class RemoteActorError(Exception): if val: _repr += f'{key}={val_str}{end_char}' - return _repr def reprol(self) -> str: @@ -622,56 +620,9 @@ class RemoteActorError(Exception): the type name is already implicitly shown by python). ''' - header: str = '' - body: str = '' - message: str = '' - - # XXX when the currently raised exception is this instance, - # we do not ever use the "type header" style repr. - is_being_raised: bool = False - if ( - (exc := sys.exception()) - and - exc is self - ): - is_being_raised: bool = True - - with_type_header: bool = ( - with_type_header - and - not is_being_raised - ) - - # style - if with_type_header: - header: str = f'<{type(self).__name__}(' - - if message := self._message: - - # split off the first line so, if needed, it isn't - # indented the same like the "boxed content" which - # since there is no `.tb_str` is just the `.message`. - lines: list[str] = message.splitlines() - first: str = lines[0] - message: str = message.removeprefix(first) - - # with a type-style header we, - # - have no special message "first line" extraction/handling - # - place the message a space in from the header: - # `MsgTypeError( ..` - # ^-here - # - indent the `.message` inside the type body. - if with_type_header: - first = f' {first} )>' - - message: str = textwrap.indent( - message, - prefix=' '*2, - ) - message: str = first + message - # IFF there is an embedded traceback-str we always # draw the ascii-box around it. + body: str = '' if tb_str := self.tb_str: fields: str = self._mk_fields_str( _body_fields @@ -692,21 +643,15 @@ class RemoteActorError(Exception): boxer_header=self.relay_uid, ) - tail = '' - if ( - with_type_header - and not message - ): - tail: str = '>' - - return ( - header - + - message - + - f'{body}' - + - tail + # !TODO, it'd be nice to import these top level without + # cycles! + from tractor.devx.pformat import ( + pformat_exc, + ) + return pformat_exc( + exc=self, + with_type_header=with_type_header, + body=body, ) __repr__ = pformat @@ -984,7 +929,7 @@ class StreamOverrun( ''' -class TransportClosed(trio.BrokenResourceError): +class TransportClosed(Exception): ''' IPC transport (protocol) connection was closed or broke and indicates that the wrapping communication `Channel` can no longer @@ -995,16 +940,21 @@ class TransportClosed(trio.BrokenResourceError): self, message: str, loglevel: str = 'transport', - cause: BaseException|None = None, + src_exc: Exception|None = None, raise_on_report: bool = False, ) -> None: self.message: str = message - self._loglevel = loglevel + self._loglevel: str = loglevel super().__init__(message) - if cause is not None: - self.__cause__ = cause + self.src_exc = src_exc + if ( + src_exc is not None + and + not self.__cause__ + ): + self.__cause__ = src_exc # flag to toggle whether the msg loop should raise # the exc in its `TransportClosed` handler block. @@ -1041,6 +991,26 @@ class TransportClosed(trio.BrokenResourceError): if self._raise_on_report: raise self from cause + def pformat(self) -> str: + from tractor.devx.pformat import ( + pformat_exc, + ) + src_err: Exception|None = self.src_exc or '' + src_msg: tuple[str] = src_err.args + src_exc_repr: str = ( + f'{type(src_err).__name__}[ {src_msg} ]' + ) + return pformat_exc( + exc=self, + # message=self.message, # implicit! + body=( + f' |_src_exc: {src_exc_repr}\n' + ), + ) + + # delegate to `str`-ified pformat + __repr__ = pformat + class NoResult(RuntimeError): "No final result is expected for this actor" diff --git a/tractor/devx/pformat.py b/tractor/devx/pformat.py index 1530ef02..e04b4fe8 100644 --- a/tractor/devx/pformat.py +++ b/tractor/devx/pformat.py @@ -19,6 +19,7 @@ Pretty formatters for use throughout the code base. Mostly handy for logging and exception message content. ''' +import sys import textwrap import traceback @@ -115,6 +116,85 @@ def pformat_boxed_tb( ) +def pformat_exc( + exc: Exception, + header: str = '', + message: str = '', + body: str = '', + with_type_header: bool = True, +) -> str: + + # XXX when the currently raised exception is this instance, + # we do not ever use the "type header" style repr. + is_being_raised: bool = False + if ( + (curr_exc := sys.exception()) + and + curr_exc is exc + ): + is_being_raised: bool = True + + with_type_header: bool = ( + with_type_header + and + not is_being_raised + ) + + # style + if ( + with_type_header + and + not header + ): + header: str = f'<{type(exc).__name__}(' + + message: str = ( + message + or + exc.message + ) + if message: + # split off the first line so, if needed, it isn't + # indented the same like the "boxed content" which + # since there is no `.tb_str` is just the `.message`. + lines: list[str] = message.splitlines() + first: str = lines[0] + message: str = message.removeprefix(first) + + # with a type-style header we, + # - have no special message "first line" extraction/handling + # - place the message a space in from the header: + # `MsgTypeError( ..` + # ^-here + # - indent the `.message` inside the type body. + if with_type_header: + first = f' {first} )>' + + message: str = textwrap.indent( + message, + prefix=' '*2, + ) + message: str = first + message + + tail: str = '' + if ( + with_type_header + and + not message + ): + tail: str = '>' + + return ( + header + + + message + + + f'{body}' + + + tail + ) + + def pformat_caller_frame( stack_limit: int = 1, box_tb: bool = True, diff --git a/tractor/ipc/_transport.py b/tractor/ipc/_transport.py index 98403c1f..160423c8 100644 --- a/tractor/ipc/_transport.py +++ b/tractor/ipc/_transport.py @@ -208,6 +208,7 @@ class MsgpackTransport(MsgTransport): ''' decodes_failed: int = 0 + tpt_name: str = f'{type(self).__name__!r}' while True: try: header: bytes = await self.recv_stream.receive_exactly(4) @@ -252,10 +253,9 @@ class MsgpackTransport(MsgTransport): raise TransportClosed( message=( - f'IPC transport already closed by peer\n' - f'x)> {type(trans_err)}\n' - f' |_{self}\n' + f'{tpt_name} already closed by peer\n' ), + src_exc=trans_err, loglevel=loglevel, ) from trans_err @@ -267,18 +267,17 @@ class MsgpackTransport(MsgTransport): # # NOTE: as such we always re-raise this error from the # RPC msg loop! - except trio.ClosedResourceError as closure_err: + except trio.ClosedResourceError as cre: + closure_err = cre + raise TransportClosed( message=( - f'IPC transport already manually closed locally?\n' - f'x)> {type(closure_err)} \n' - f' |_{self}\n' + f'{tpt_name} was already closed locally ?\n' ), + src_exc=closure_err, loglevel='error', raise_on_report=( - closure_err.args[0] == 'another task closed this fd' - or - closure_err.args[0] in ['another task closed this fd'] + 'another task closed this fd' in closure_err.args ), ) from closure_err @@ -286,12 +285,9 @@ class MsgpackTransport(MsgTransport): if header == b'': raise TransportClosed( message=( - f'IPC transport already gracefully closed\n' - f')>\n' - f'|_{self}\n' + f'{tpt_name} already gracefully closed\n' ), loglevel='transport', - # cause=??? # handy or no? ) size: int From ba45c03e141dd7eb9af47a40b94d1847b79db5a7 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 6 Apr 2025 14:44:40 -0400 Subject: [PATCH 46/74] Skip the ringbuf test mod for now since data-gen is a bit "heavy/laggy" atm --- tests/test_ringbuf.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index 28af7b83..287a0501 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -11,6 +11,9 @@ from tractor.ipc import ( ) from tractor._testing.samples import generate_sample_messages +# in case you don't want to melt your cores, uncomment dis! +pytestmark = pytest.mark.skip + @tractor.context async def child_read_shm( From 094447787e7437bc136f112ec5a6ded995feb13c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 6 Apr 2025 14:46:25 -0400 Subject: [PATCH 47/74] Add API-modernize-todo on `experimental._pubsub.fan_out_to_ctxs` --- tractor/experimental/_pubsub.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tractor/experimental/_pubsub.py b/tractor/experimental/_pubsub.py index b894ed49..bc5881e1 100644 --- a/tractor/experimental/_pubsub.py +++ b/tractor/experimental/_pubsub.py @@ -45,6 +45,8 @@ __all__ = ['pub'] log = get_logger('messaging') +# TODO! this needs to reworked to use the modern +# `Context`/`MsgStream` APIs!! async def fan_out_to_ctxs( pub_async_gen_func: typing.Callable, # it's an async gen ... gd mypy topics2ctxs: dict[str, list], From 61df10b333232a17a52043b95b75ec0e8a922aa2 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 7 Apr 2025 18:07:58 -0400 Subject: [PATCH 48/74] Move concrete `Address`es to each tpt module That is moving from `._addr`, - `TCPAddress` to `.ipc._tcp` - `UDSAddress` to `.ipc._uds` Obviously this requires adjusting a buncha stuff in `._addr` to avoid import cycles (the original reason the module was not also included in the new `.ipc` subpkg) including, - avoiding "unnecessary" imports of `[Unwrapped]Address` in various modules. * since `Address` is a protocol and the main point is that it **does not need to be inherited** per (https://typing.python.org/en/latest/spec/protocol.html#terminology) thus I removed the need for it in both transport submods. * and `UnwrappedAddress` is a type alias for tuples.. so we don't really always need to be importing it since it also kinda obfuscates what the underlying pairs are. - not exporting everything in submods at the `.ipc` top level and importing from specific submods by default. - only importing various types under a `if typing.TYPE_CHECKING:` guard as needed. --- tests/test_ringbuf.py | 7 +- tractor/_addr.py | 305 +------------------------------------- tractor/_context.py | 2 +- tractor/ipc/__init__.py | 37 +---- tractor/ipc/_chan.py | 7 +- tractor/ipc/_fd_share.py | 163 ++++++++++++++++++++ tractor/ipc/_tcp.py | 108 +++++++++++++- tractor/ipc/_transport.py | 19 ++- tractor/ipc/_types.py | 31 +++- tractor/ipc/_uds.py | 212 +++++++++++++++++++++++++- tractor/msg/types.py | 6 +- 11 files changed, 535 insertions(+), 362 deletions(-) create mode 100644 tractor/ipc/_fd_share.py diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index 287a0501..0d3b420b 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -2,14 +2,17 @@ import time import trio import pytest + import tractor -from tractor.ipc import ( +from tractor.ipc._ringbuf import ( open_ringbuf, RBToken, RingBuffSender, RingBuffReceiver ) -from tractor._testing.samples import generate_sample_messages +from tractor._testing.samples import ( + generate_sample_messages, +) # in case you don't want to melt your cores, uncomment dis! pytestmark = pytest.mark.skip diff --git a/tractor/_addr.py b/tractor/_addr.py index eaf4c202..d8d11227 100644 --- a/tractor/_addr.py +++ b/tractor/_addr.py @@ -14,34 +14,25 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . from __future__ import annotations -from pathlib import Path -import os -# import tempfile from uuid import uuid4 from typing import ( Protocol, ClassVar, - # TypeVar, - # Union, Type, TYPE_CHECKING, ) from bidict import bidict -# import trio from trio import ( - socket, SocketListener, - open_tcp_listeners, ) from .log import get_logger from ._state import ( - get_rt_dir, - current_actor, - is_root_process, _def_tpt_proto, ) +from .ipc._tcp import TCPAddress +from .ipc._uds import UDSAddress if TYPE_CHECKING: from ._runtime import Actor @@ -179,298 +170,6 @@ class Address(Protocol): ... -class TCPAddress(Address): - proto_key: str = 'tcp' - unwrapped_type: type = tuple[str, int] - def_bindspace: str = '127.0.0.1' - - def __init__( - self, - host: str, - port: int - ): - if ( - not isinstance(host, str) - or - not isinstance(port, int) - ): - raise TypeError( - f'Expected host {host!r} to be str and port {port!r} to be int' - ) - - self._host: str = host - self._port: int = port - - @property - def is_valid(self) -> bool: - return self._port != 0 - - @property - def bindspace(self) -> str: - return self._host - - @property - def domain(self) -> str: - return self._host - - @classmethod - def from_addr( - cls, - addr: tuple[str, int] - ) -> TCPAddress: - match addr: - case (str(), int()): - return TCPAddress(addr[0], addr[1]) - case _: - raise ValueError( - f'Invalid unwrapped address for {cls}\n' - f'{addr}\n' - ) - - def unwrap(self) -> tuple[str, int]: - return ( - self._host, - self._port, - ) - - @classmethod - def get_random( - cls, - bindspace: str = def_bindspace, - ) -> TCPAddress: - return TCPAddress(bindspace, 0) - - @classmethod - def get_root(cls) -> Address: - return TCPAddress( - '127.0.0.1', - 1616, - ) - - def __repr__(self) -> str: - return ( - f'{type(self).__name__}[{self.unwrap()}]' - ) - - def __eq__(self, other) -> bool: - if not isinstance(other, TCPAddress): - raise TypeError( - f'Can not compare {type(other)} with {type(self)}' - ) - - return ( - self._host == other._host - and - self._port == other._port - ) - - async def open_listener( - self, - **kwargs, - ) -> SocketListener: - listeners: list[SocketListener] = await open_tcp_listeners( - host=self._host, - port=self._port, - **kwargs - ) - assert len(listeners) == 1 - listener = listeners[0] - self._host, self._port = listener.socket.getsockname()[:2] - return listener - - async def close_listener(self): - ... - - -def unwrap_sockpath( - sockpath: Path, -) -> tuple[Path, Path]: - return ( - sockpath.parent, - sockpath.name, - ) - - -class UDSAddress(Address): - # TODO, maybe we should use better field and value - # -[x] really this is a `.protocol_key` not a "name" of anything. - # -[ ] consider a 'unix' proto-key instead? - # -[ ] need to check what other mult-transport frameworks do - # like zmq, nng, uri-spec et al! - proto_key: str = 'uds' - unwrapped_type: type = tuple[str, int] - def_bindspace: Path = get_rt_dir() - - def __init__( - self, - filedir: Path|str|None, - # TODO, i think i want `.filename` here? - filename: str|Path, - - # XXX, in the sense you can also pass - # a "non-real-world-process-id" such as is handy to represent - # our host-local default "port-like" key for the very first - # root actor to create a registry address. - maybe_pid: int|None = None, - ): - fdir = self._filedir = Path( - filedir - or - self.def_bindspace - ).absolute() - fpath = self._filename = Path(filename) - fp: Path = fdir / fpath - assert ( - fp.is_absolute() - and - fp == self.sockpath - ) - - # to track which "side" is the peer process by reading socket - # credentials-info. - self._pid: int = maybe_pid - - @property - def sockpath(self) -> Path: - return self._filedir / self._filename - - @property - def is_valid(self) -> bool: - ''' - We block socket files not allocated under the runtime subdir. - - ''' - return self.bindspace in self.sockpath.parents - - @property - def bindspace(self) -> Path: - ''' - We replicate the "ip-set-of-hosts" part of a UDS socket as - just the sub-directory in which we allocate socket files. - - ''' - return self._filedir or self.def_bindspace - - @classmethod - def from_addr( - cls, - addr: ( - tuple[Path|str, Path|str]|Path|str - ), - ) -> UDSAddress: - match addr: - case tuple()|list(): - filedir = Path(addr[0]) - filename = Path(addr[1]) - # sockpath: Path = Path(addr[0]) - # filedir, filename = unwrap_sockpath(sockpath) - # pid: int = addr[1] - return UDSAddress( - filedir=filedir, - filename=filename, - # maybe_pid=pid, - ) - # NOTE, in case we ever decide to just `.unwrap()` - # to a `Path|str`? - case str()|Path(): - sockpath: Path = Path(addr) - return UDSAddress(*unwrap_sockpath(sockpath)) - case _: - # import pdbp; pdbp.set_trace() - raise TypeError( - f'Bad unwrapped-address for {cls} !\n' - f'{addr!r}\n' - ) - - def unwrap(self) -> tuple[str, int]: - # XXX NOTE, since this gets passed DIRECTLY to - # `.ipc._uds.open_unix_socket_w_passcred()` - return ( - str(self._filedir), - str(self._filename), - ) - - @classmethod - def get_random( - cls, - bindspace: Path|None = None, # default netns - ) -> UDSAddress: - - filedir: Path = bindspace or cls.def_bindspace - pid: int = os.getpid() - actor: Actor|None = current_actor( - err_on_no_runtime=False, - ) - if actor: - sockname: str = '::'.join(actor.uid) + f'@{pid}' - else: - prefix: str = '' - if is_root_process(): - prefix: str = 'root' - sockname: str = f'{prefix}@{pid}' - - sockpath: Path = Path(f'{sockname}.sock') - return UDSAddress( - filedir=filedir, - filename=sockpath, - maybe_pid=pid, - ) - - @classmethod - def get_root(cls) -> Address: - def_uds_filename: Path = 'registry@1616.sock' - return UDSAddress( - filedir=None, - filename=def_uds_filename, - # maybe_pid=1616, - ) - - def __repr__(self) -> str: - return ( - f'{type(self).__name__}' - f'[' - f'({self._filedir}, {self._filename})' - f']' - ) - - def __eq__(self, other) -> bool: - if not isinstance(other, UDSAddress): - raise TypeError( - f'Can not compare {type(other)} with {type(self)}' - ) - - return self.sockpath == other.sockpath - - # async def open_listener(self, **kwargs) -> SocketListener: - async def open_listener( - self, - **kwargs, - ) -> SocketListener: - sock = self._sock = socket.socket( - socket.AF_UNIX, - socket.SOCK_STREAM - ) - log.info( - f'Attempting to bind UDS socket\n' - f'>[\n' - f'|_{self}\n' - ) - - bindpath: Path = self.sockpath - await sock.bind(str(bindpath)) - sock.listen(1) - log.info( - f'Listening on UDS socket\n' - f'[>\n' - f' |_{self}\n' - ) - return SocketListener(self._sock) - - def close_listener(self): - self._sock.close() - os.unlink(self.sockpath) - - _address_types: bidict[str, Type[Address]] = { 'tcp': TCPAddress, 'uds': UDSAddress diff --git a/tractor/_context.py b/tractor/_context.py index 53f5b233..e5cce1ec 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -105,7 +105,7 @@ from ._state import ( if TYPE_CHECKING: from ._portal import Portal from ._runtime import Actor - from .ipc import MsgTransport + from .ipc._transport import MsgTransport from .devx._frame_stack import ( CallerInfo, ) diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index f1cb8e8b..2c6c3b5d 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -13,43 +13,12 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . -import platform -from ._transport import ( - MsgTransportKey as MsgTransportKey, - MsgType as MsgType, - MsgTransport as MsgTransport, - MsgpackTransport as MsgpackTransport -) - -from ._tcp import MsgpackTCPStream as MsgpackTCPStream -from ._uds import MsgpackUDSStream as MsgpackUDSStream - -from ._types import ( - transport_from_addr as transport_from_addr, - transport_from_stream as transport_from_stream, -) +''' +A modular IPC layer supporting the power of cross-process SC! +''' from ._chan import ( _connect_chan as _connect_chan, Channel as Channel ) - -if platform.system() == 'Linux': - from ._linux import ( - EFD_SEMAPHORE as EFD_SEMAPHORE, - EFD_CLOEXEC as EFD_CLOEXEC, - EFD_NONBLOCK as EFD_NONBLOCK, - open_eventfd as open_eventfd, - write_eventfd as write_eventfd, - read_eventfd as read_eventfd, - close_eventfd as close_eventfd, - EventFD as EventFD, - ) - - from ._ringbuf import ( - RBToken as RBToken, - RingBuffSender as RingBuffSender, - RingBuffReceiver as RingBuffReceiver, - open_ringbuf as open_ringbuf - ) diff --git a/tractor/ipc/_chan.py b/tractor/ipc/_chan.py index f6a50cc1..00c749e1 100644 --- a/tractor/ipc/_chan.py +++ b/tractor/ipc/_chan.py @@ -29,13 +29,13 @@ from pprint import pformat import typing from typing import ( Any, + TYPE_CHECKING, ) import warnings import trio -from tractor.ipc._transport import MsgTransport -from tractor.ipc._types import ( +from ._types import ( transport_from_addr, transport_from_stream, ) @@ -55,6 +55,9 @@ from tractor.msg import ( MsgCodec, ) +if TYPE_CHECKING: + from ._transport import MsgTransport + log = get_logger(__name__) diff --git a/tractor/ipc/_fd_share.py b/tractor/ipc/_fd_share.py new file mode 100644 index 00000000..e51069ba --- /dev/null +++ b/tractor/ipc/_fd_share.py @@ -0,0 +1,163 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . +''' +File-descriptor-sharing on `linux` by "wilhelm_of_bohemia". + +''' +from __future__ import annotations +import os +import array +import socket +import tempfile +from pathlib import Path +from contextlib import ExitStack + +import trio +import tractor +from tractor.ipc import RBToken + + +actor_name = 'ringd' + + +_rings: dict[str, dict] = {} + + +async def _attach_to_ring( + ring_name: str +) -> tuple[int, int, int]: + actor = tractor.current_actor() + + fd_amount = 3 + sock_path = ( + Path(tempfile.gettempdir()) + / + f'{os.getpid()}-pass-ring-fds-{ring_name}-to-{actor.name}.sock' + ) + sock = socket.socket(socket.AF_UNIX, socket.SOCK_STREAM) + sock.bind(sock_path) + sock.listen(1) + + async with ( + tractor.find_actor(actor_name) as ringd, + ringd.open_context( + _pass_fds, + name=ring_name, + sock_path=sock_path + ) as (ctx, _sent) + ): + # prepare array to receive FD + fds = array.array("i", [0] * fd_amount) + + conn, _ = sock.accept() + + # receive FD + msg, ancdata, flags, addr = conn.recvmsg( + 1024, + socket.CMSG_LEN(fds.itemsize * fd_amount) + ) + + for ( + cmsg_level, + cmsg_type, + cmsg_data, + ) in ancdata: + if ( + cmsg_level == socket.SOL_SOCKET + and + cmsg_type == socket.SCM_RIGHTS + ): + fds.frombytes(cmsg_data[:fds.itemsize * fd_amount]) + break + else: + raise RuntimeError("Receiver: No FDs received") + + conn.close() + sock.close() + sock_path.unlink() + + return RBToken.from_msg( + await ctx.wait_for_result() + ) + + +@tractor.context +async def _pass_fds( + ctx: tractor.Context, + name: str, + sock_path: str +) -> RBToken: + global _rings + token = _rings[name] + client = socket.socket(socket.AF_UNIX, socket.SOCK_STREAM) + client.connect(sock_path) + await ctx.started() + fds = array.array('i', token.fds) + client.sendmsg([b'FDs'], [(socket.SOL_SOCKET, socket.SCM_RIGHTS, fds)]) + client.close() + return token + + +@tractor.context +async def _open_ringbuf( + ctx: tractor.Context, + name: str, + buf_size: int +) -> RBToken: + global _rings + is_owner = False + if name not in _rings: + stack = ExitStack() + token = stack.enter_context( + tractor.open_ringbuf( + name, + buf_size=buf_size + ) + ) + _rings[name] = { + 'token': token, + 'stack': stack, + } + is_owner = True + + ring = _rings[name] + await ctx.started() + + try: + await trio.sleep_forever() + + except tractor.ContextCancelled: + ... + + finally: + if is_owner: + ring['stack'].close() + + +async def open_ringbuf( + name: str, + buf_size: int +) -> RBToken: + async with ( + tractor.find_actor(actor_name) as ringd, + ringd.open_context( + _open_ringbuf, + name=name, + buf_size=buf_size + ) as (rd_ctx, _) + ): + yield await _attach_to_ring(name) + await rd_ctx.cancel() diff --git a/tractor/ipc/_tcp.py b/tractor/ipc/_tcp.py index 61e6f3e6..dbecdf5e 100644 --- a/tractor/ipc/_tcp.py +++ b/tractor/ipc/_tcp.py @@ -20,16 +20,122 @@ TCP implementation of tractor.ipc._transport.MsgTransport protocol from __future__ import annotations import trio +from trio import ( + SocketListener, + open_tcp_listeners, +) from tractor.msg import MsgCodec from tractor.log import get_logger -from tractor._addr import TCPAddress from tractor.ipc._transport import MsgpackTransport log = get_logger(__name__) +class TCPAddress: + proto_key: str = 'tcp' + unwrapped_type: type = tuple[str, int] + def_bindspace: str = '127.0.0.1' + + def __init__( + self, + host: str, + port: int + ): + if ( + not isinstance(host, str) + or + not isinstance(port, int) + ): + raise TypeError( + f'Expected host {host!r} to be str and port {port!r} to be int' + ) + + self._host: str = host + self._port: int = port + + @property + def is_valid(self) -> bool: + return self._port != 0 + + @property + def bindspace(self) -> str: + return self._host + + @property + def domain(self) -> str: + return self._host + + @classmethod + def from_addr( + cls, + addr: tuple[str, int] + ) -> TCPAddress: + match addr: + case (str(), int()): + return TCPAddress(addr[0], addr[1]) + case _: + raise ValueError( + f'Invalid unwrapped address for {cls}\n' + f'{addr}\n' + ) + + def unwrap(self) -> tuple[str, int]: + return ( + self._host, + self._port, + ) + + @classmethod + def get_random( + cls, + bindspace: str = def_bindspace, + ) -> TCPAddress: + return TCPAddress(bindspace, 0) + + @classmethod + def get_root(cls) -> TCPAddress: + return TCPAddress( + '127.0.0.1', + 1616, + ) + + def __repr__(self) -> str: + return ( + f'{type(self).__name__}[{self.unwrap()}]' + ) + + def __eq__(self, other) -> bool: + if not isinstance(other, TCPAddress): + raise TypeError( + f'Can not compare {type(other)} with {type(self)}' + ) + + return ( + self._host == other._host + and + self._port == other._port + ) + + async def open_listener( + self, + **kwargs, + ) -> SocketListener: + listeners: list[SocketListener] = await open_tcp_listeners( + host=self._host, + port=self._port, + **kwargs + ) + assert len(listeners) == 1 + listener = listeners[0] + self._host, self._port = listener.socket.getsockname()[:2] + return listener + + async def close_listener(self): + ... + + # TODO: typing oddity.. not sure why we have to inherit here, but it # seems to be an issue with `get_msg_transport()` returning # a `Type[Protocol]`; probably should make a `mypy` issue? diff --git a/tractor/ipc/_transport.py b/tractor/ipc/_transport.py index 160423c8..2a9926f9 100644 --- a/tractor/ipc/_transport.py +++ b/tractor/ipc/_transport.py @@ -14,8 +14,8 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . ''' -typing.Protocol based generic msg API, implement this class to add backends for -tractor.ipc.Channel +typing.Protocol based generic msg API, implement this class to add +backends for tractor.ipc.Channel ''' from __future__ import annotations @@ -23,8 +23,9 @@ from typing import ( runtime_checkable, Type, Protocol, - TypeVar, - ClassVar + # TypeVar, + ClassVar, + TYPE_CHECKING, ) from collections.abc import ( AsyncGenerator, @@ -47,10 +48,13 @@ from tractor.msg import ( _ctxvar_MsgCodec, # _codec, XXX see `self._codec` sanity/debug checks MsgCodec, + MsgType, types as msgtypes, pretty_struct, ) -from tractor._addr import Address + +if TYPE_CHECKING: + from tractor._addr import Address log = get_logger(__name__) @@ -63,12 +67,13 @@ MsgTransportKey = tuple[str, str] # ?TODO? this should be our `Union[*msgtypes.__spec__]` alias now right..? # => BLEH, except can't bc prots must inherit typevar or param-spec # vars.. -MsgType = TypeVar('MsgType') +# MsgType = TypeVar('MsgType') @runtime_checkable -class MsgTransport(Protocol[MsgType]): +class MsgTransport(Protocol): # +# class MsgTransport(Protocol[MsgType]): # ^-TODO-^ consider using a generic def and indexing with our # eventual msg definition/types? # - https://docs.python.org/3/library/typing.html#typing.Protocol diff --git a/tractor/ipc/_types.py b/tractor/ipc/_types.py index 1b86636d..8d543d9d 100644 --- a/tractor/ipc/_types.py +++ b/tractor/ipc/_types.py @@ -13,19 +13,37 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . -from typing import Type + +''' +IPC subsys type-lookup helpers? + +''' +from typing import ( + Type, + # TYPE_CHECKING, +) import trio import socket -from tractor._addr import Address from tractor.ipc._transport import ( MsgTransportKey, MsgTransport ) -from tractor.ipc._tcp import MsgpackTCPStream -from tractor.ipc._uds import MsgpackUDSStream +from tractor.ipc._tcp import ( + TCPAddress, + MsgpackTCPStream, +) +from tractor.ipc._uds import ( + UDSAddress, + MsgpackUDSStream, +) +# if TYPE_CHECKING: +# from tractor._addr import Address + + +Address = TCPAddress|UDSAddress # manually updated list of all supported msg transport types _msg_transports = [ @@ -41,7 +59,10 @@ _key_to_transport: dict[MsgTransportKey, Type[MsgTransport]] = { } # convert an Address wrapper to its corresponding transport type -_addr_to_transport: dict[Type[Address], Type[MsgTransport]] = { +_addr_to_transport: dict[ + Type[TCPAddress|UDSAddress], + Type[MsgTransport] +] = { cls.address_type: cls for cls in _msg_transports } diff --git a/tractor/ipc/_uds.py b/tractor/ipc/_uds.py index 3d24447b..33843f6a 100644 --- a/tractor/ipc/_uds.py +++ b/tractor/ipc/_uds.py @@ -29,8 +29,15 @@ from socket import ( SOL_SOCKET, ) import struct +from typing import ( + TYPE_CHECKING, +) import trio +from trio import ( + socket, + SocketListener, +) from trio._highlevel_open_unix_stream import ( close_on_error, has_unix, @@ -38,16 +45,211 @@ from trio._highlevel_open_unix_stream import ( from tractor.msg import MsgCodec from tractor.log import get_logger -from tractor._addr import ( - UDSAddress, - unwrap_sockpath, +from tractor.ipc._transport import ( + MsgpackTransport, ) -from tractor.ipc._transport import MsgpackTransport +from .._state import ( + get_rt_dir, + current_actor, + is_root_process, +) + +if TYPE_CHECKING: + from ._runtime import Actor log = get_logger(__name__) +def unwrap_sockpath( + sockpath: Path, +) -> tuple[Path, Path]: + return ( + sockpath.parent, + sockpath.name, + ) + + +class UDSAddress: + # TODO, maybe we should use better field and value + # -[x] really this is a `.protocol_key` not a "name" of anything. + # -[ ] consider a 'unix' proto-key instead? + # -[ ] need to check what other mult-transport frameworks do + # like zmq, nng, uri-spec et al! + proto_key: str = 'uds' + unwrapped_type: type = tuple[str, int] + def_bindspace: Path = get_rt_dir() + + def __init__( + self, + filedir: Path|str|None, + # TODO, i think i want `.filename` here? + filename: str|Path, + + # XXX, in the sense you can also pass + # a "non-real-world-process-id" such as is handy to represent + # our host-local default "port-like" key for the very first + # root actor to create a registry address. + maybe_pid: int|None = None, + ): + fdir = self._filedir = Path( + filedir + or + self.def_bindspace + ).absolute() + fpath = self._filename = Path(filename) + fp: Path = fdir / fpath + assert ( + fp.is_absolute() + and + fp == self.sockpath + ) + + # to track which "side" is the peer process by reading socket + # credentials-info. + self._pid: int = maybe_pid + + @property + def sockpath(self) -> Path: + return self._filedir / self._filename + + @property + def is_valid(self) -> bool: + ''' + We block socket files not allocated under the runtime subdir. + + ''' + return self.bindspace in self.sockpath.parents + + @property + def bindspace(self) -> Path: + ''' + We replicate the "ip-set-of-hosts" part of a UDS socket as + just the sub-directory in which we allocate socket files. + + ''' + return self._filedir or self.def_bindspace + + @classmethod + def from_addr( + cls, + addr: ( + tuple[Path|str, Path|str]|Path|str + ), + ) -> UDSAddress: + match addr: + case tuple()|list(): + filedir = Path(addr[0]) + filename = Path(addr[1]) + # sockpath: Path = Path(addr[0]) + # filedir, filename = unwrap_sockpath(sockpath) + # pid: int = addr[1] + return UDSAddress( + filedir=filedir, + filename=filename, + # maybe_pid=pid, + ) + # NOTE, in case we ever decide to just `.unwrap()` + # to a `Path|str`? + case str()|Path(): + sockpath: Path = Path(addr) + return UDSAddress(*unwrap_sockpath(sockpath)) + case _: + # import pdbp; pdbp.set_trace() + raise TypeError( + f'Bad unwrapped-address for {cls} !\n' + f'{addr!r}\n' + ) + + def unwrap(self) -> tuple[str, int]: + # XXX NOTE, since this gets passed DIRECTLY to + # `.ipc._uds.open_unix_socket_w_passcred()` + return ( + str(self._filedir), + str(self._filename), + ) + + @classmethod + def get_random( + cls, + bindspace: Path|None = None, # default netns + ) -> UDSAddress: + + filedir: Path = bindspace or cls.def_bindspace + pid: int = os.getpid() + actor: Actor|None = current_actor( + err_on_no_runtime=False, + ) + if actor: + sockname: str = '::'.join(actor.uid) + f'@{pid}' + else: + prefix: str = '' + if is_root_process(): + prefix: str = 'root' + sockname: str = f'{prefix}@{pid}' + + sockpath: Path = Path(f'{sockname}.sock') + return UDSAddress( + filedir=filedir, + filename=sockpath, + maybe_pid=pid, + ) + + @classmethod + def get_root(cls) -> UDSAddress: + def_uds_filename: Path = 'registry@1616.sock' + return UDSAddress( + filedir=None, + filename=def_uds_filename, + # maybe_pid=1616, + ) + + def __repr__(self) -> str: + return ( + f'{type(self).__name__}' + f'[' + f'({self._filedir}, {self._filename})' + f']' + ) + + def __eq__(self, other) -> bool: + if not isinstance(other, UDSAddress): + raise TypeError( + f'Can not compare {type(other)} with {type(self)}' + ) + + return self.sockpath == other.sockpath + + # async def open_listener(self, **kwargs) -> SocketListener: + async def open_listener( + self, + **kwargs, + ) -> SocketListener: + sock = self._sock = socket.socket( + socket.AF_UNIX, + socket.SOCK_STREAM + ) + log.info( + f'Attempting to bind UDS socket\n' + f'>[\n' + f'|_{self}\n' + ) + + bindpath: Path = self.sockpath + await sock.bind(str(bindpath)) + sock.listen(1) + log.info( + f'Listening on UDS socket\n' + f'[>\n' + f' |_{self}\n' + ) + return SocketListener(self._sock) + + def close_listener(self): + self._sock.close() + os.unlink(self.sockpath) + + async def open_unix_socket_w_passcred( filename: str|bytes|os.PathLike[str]|os.PathLike[bytes], ) -> trio.SocketStream: @@ -214,3 +416,5 @@ class MsgpackUDSStream(MsgpackTransport): maybe_pid=peer_pid ) return (laddr, raddr) + + diff --git a/tractor/msg/types.py b/tractor/msg/types.py index d71fb7e0..86752aba 100644 --- a/tractor/msg/types.py +++ b/tractor/msg/types.py @@ -48,7 +48,7 @@ from tractor.msg import ( pretty_struct, ) from tractor.log import get_logger -from tractor._addr import UnwrappedAddress +# from tractor._addr import UnwrappedAddress log = get_logger('tractor.msgspec') @@ -176,8 +176,8 @@ class SpawnSpec( # TODO: not just sockaddr pairs? # -[ ] abstract into a `TransportAddr` type? - reg_addrs: list[UnwrappedAddress] - bind_addrs: list[UnwrappedAddress]|None + reg_addrs: list[tuple[str, str|int]] + bind_addrs: list[tuple[str, str|int]]|None # TODO: caps based RPC support in the payload? From 98c4614a3613978877a68d23508acb58be6a8f0f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 10 Apr 2025 18:06:12 -0400 Subject: [PATCH 49/74] Factor actor-embedded IPC-tpt-server to `ipc` subsys Primarily moving the `Actor._serve_forever()`-task-as-method and supporting actor-instance attributes to a new `.ipo._server` sub-mod which now encapsulates, - the coupling various `trio.Nursery`s (and their independent lifetime mgmt) to different `trio.serve_listener()`s tasks and `SocketStream` handler scopes. - `Address` and `SocketListener` mgmt and tracking through the idea of an "IPC endpoint": each "bound-and-active instance" of a served-listener for some (varied transport protocol's socket) address. - start and shutdown of the entire server's lifetime via an `@acm`. - delegation of starting/stopping tpt-protocol-specific `trio.abc.Listener`s to the corresponding `.ipc._` sub-module (newly defined mod-top-level instead of `Address` method) `start/close_listener()` funcs. Impl details of the `.ipc._server` sub-sys, - add new `IPCServer`, allocated with `open_ipc_server()`, and which encapsulates starting multiple-transport-proto-`trio.abc.Listener`s from an input set of `._addr.Address`s using, |_`IPCServer.listen_on()` which internally spawns tasks that delegate to a new `_serve_ipc_eps()`, a rework of what was (effectively) `Actor._serve_forever()` and which now, * allocates a new `IPCEndpoint`-struct (see below) for each address-listener pair alongside the specified listener-serving/stream-handling `trio.Nursery`s provided by the caller. * starts and stops each transport (socket's) listener by calling `IPCEndpoint.start/close_listener()` which in turn delegates to the underlying `inspect.getmodule(IPCEndpoint.addr)` backend tpt module's equivalent impl. * tracks all created endpoints in a `._endpoints: list[IPCEndpoint]` which is further exposed through public properties for introspection of served transport-protocols and their addresses. |_`IPCServer._[parent/stream_handler]_tn: Nursery`s which are either allocated (in which case, as the same instance) or provided by the caller of `open_ipc_server()` such that the same nursery-cancel-scope controls offered by `trio.serve_listeners(handler_nursery=)` are offered where the `._parent_tn` is used to spawn `_serve_ipc_eps()` tasks, and `._stream_handler_tn` is passed verbatim as `handler_nursery`. - a new `IPCEndpoint`-struct (as mentioned) which wraps each transport-proto's address + listener + allocated-supervising-nursery to encapsulate the "lifetime of a server IPC endpoint" such that eventually we can track and managed per-protocol/address/`.listen_on()`-call scoped starts/stops/restarts for the purposes of filtering/banning peer traffic. |_ also included is an unused `.peer_tpts` table which we can hopefully use to replace `Actor._peers` in a `Channel`-tracking transport-proto-aware way! Surrounding changes to `.ipc.*` primitives to match, - make `[TCP|UDS]Address` types `msgspec.Struct(frozen=True)` and thus drop any-and-all `addr._host =` style mutation throughout. |_ as such also drop their `.__init__()` and `.__eq__()` meths. |_ UDS tweaks to field names and thus `.__repr__()`. - move `[TCP|UDS]Address.[start/close]_listener()` meths to be mod-level equiv `start|close_listener()` funcs. - just hard code the `.ipc._types._key_to_transport/._addr_to_transport` table entries instead of all the prior fancy dynamic class property reading stuff (remember, "explicit is better then implicit"). Modified in `._runtime.Actor` internals, - drop the `._serve_forever()` and `.cancel_server()`, methods and `._server_down` waiting logic from `.cancel_soon()` - add `.[_]ipc_server` which is opened just after the `._service_n` and delegate to it for any equivalent publicly exposed instance attributes/properties. --- tractor/_runtime.py | 213 +++++++---------- tractor/ipc/_server.py | 467 ++++++++++++++++++++++++++++++++++++++ tractor/ipc/_tcp.py | 99 ++++---- tractor/ipc/_transport.py | 9 +- tractor/ipc/_types.py | 13 +- tractor/ipc/_uds.py | 178 ++++++++------- 6 files changed, 701 insertions(+), 278 deletions(-) create mode 100644 tractor/ipc/_server.py diff --git a/tractor/_runtime.py b/tractor/_runtime.py index f8356582..4daa4742 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -74,11 +74,14 @@ from tractor.msg import ( pretty_struct, types as msgtypes, ) -from .ipc import Channel +from .ipc import ( + Channel, + _server, +) from ._addr import ( UnwrappedAddress, Address, - default_lo_addrs, + # default_lo_addrs, get_address_cls, wrap_address, ) @@ -157,16 +160,24 @@ class Actor: # nursery placeholders filled in by `async_main()` after fork _root_n: Nursery|None = None _service_n: Nursery|None = None - _server_n: Nursery|None = None + + # XXX moving to IPCServer! + _ipc_server: _server.IPCServer|None = None + + @property + def ipc_server(self) -> _server.IPCServer: + ''' + The IPC transport-server for this actor; normally + a process-singleton. + + ''' + return self._ipc_server # Information about `__main__` from parent _parent_main_data: dict[str, str] _parent_chan_cs: CancelScope|None = None _spawn_spec: msgtypes.SpawnSpec|None = None - # syncs for setup/teardown sequences - _server_down: trio.Event|None = None - # if started on ``asycio`` running ``trio`` in guest mode _infected_aio: bool = False @@ -266,8 +277,6 @@ class Actor: Context ] = {} - self._listeners: list[trio.abc.Listener] = [] - self._listen_addrs: list[Address] = [] self._parent_chan: Channel|None = None self._forkserver_info: tuple|None = None @@ -335,7 +344,6 @@ class Actor: if rent_chan := self._parent_chan: parent_uid = rent_chan.uid peers: list[tuple] = list(self._peer_connected) - listen_addrs: str = pformat(self._listen_addrs) fmtstr: str = ( f' |_id: {self.aid!r}\n' # f" aid{ds}{self.aid!r}\n" @@ -343,8 +351,7 @@ class Actor: f'\n' f' |_ipc: {len(peers)!r} connected peers\n' f" peers{ds}{peers!r}\n" - f" _listen_addrs{ds}'{listen_addrs}'\n" - f" _listeners{ds}'{self._listeners}'\n" + f" ipc_server{ds}{self._ipc_server}\n" f'\n' f' |_rpc: {len(self._rpc_tasks)} tasks\n' f" ctxs{ds}{len(self._contexts)}\n" @@ -499,6 +506,9 @@ class Actor: ''' self._no_more_peers = trio.Event() # unset by making new + # with _debug.maybe_open_crash_handler( + # pdb=True, + # ) as boxerr: chan = Channel.from_stream(stream) con_status: str = ( 'New inbound IPC connection <=\n' @@ -1303,85 +1313,6 @@ class Actor: await self.cancel(req_chan=None) # self cancel raise - async def _serve_forever( - self, - handler_nursery: Nursery, - *, - listen_addrs: list[UnwrappedAddress]|None = None, - - task_status: TaskStatus[Nursery] = trio.TASK_STATUS_IGNORED, - ) -> None: - ''' - Start the IPC transport server, begin listening/accepting new - `trio.SocketStream` connections. - - This will cause an actor to continue living (and thus - blocking at the process/OS-thread level) until - `.cancel_server()` is called. - - ''' - if listen_addrs is None: - listen_addrs = default_lo_addrs([ - _state._def_tpt_proto - ]) - - else: - listen_addrs: list[Address] = [ - wrap_address(a) for a in listen_addrs - ] - - self._server_down = trio.Event() - try: - async with trio.open_nursery() as server_n: - - listeners: list[trio.abc.Listener] = [] - for addr in listen_addrs: - try: - listener: trio.abc.Listener = await addr.open_listener() - except OSError as oserr: - if ( - '[Errno 98] Address already in use' - in - oserr.args#[0] - ): - log.exception( - f'Address already in use?\n' - f'{addr}\n' - ) - raise - listeners.append(listener) - - await server_n.start( - partial( - trio.serve_listeners, - handler=self._stream_handler, - listeners=listeners, - - # NOTE: configured such that new - # connections will stay alive even if - # this server is cancelled! - handler_nursery=handler_nursery - ) - ) - # TODO, wow make this message better! XD - log.info( - 'Started server(s)\n' - + - '\n'.join([f'|_{addr}' for addr in listen_addrs]) - ) - self._listen_addrs.extend(listen_addrs) - self._listeners.extend(listeners) - - task_status.started(server_n) - - finally: - addr: Address - for addr in listen_addrs: - addr.close_listener() - - # signal the server is down since nursery above terminated - self._server_down.set() - def cancel_soon(self) -> None: ''' Cancel this actor asap; can be called from a sync context. @@ -1481,18 +1412,9 @@ class Actor: ) # stop channel server - self.cancel_server() - if self._server_down is not None: - await self._server_down.wait() - else: - tpt_protos: list[str] = [] - addr: Address - for addr in self._listen_addrs: - tpt_protos.append(addr.proto_key) - log.warning( - 'Transport server(s) may have been cancelled before started?\n' - f'protos: {tpt_protos!r}\n' - ) + if ipc_server := self.ipc_server: + ipc_server.cancel() + await ipc_server.wait_for_shutdown() # cancel all rpc tasks permanently if self._service_n: @@ -1723,24 +1645,6 @@ class Actor: ) await self._ongoing_rpc_tasks.wait() - def cancel_server(self) -> bool: - ''' - Cancel the internal IPC transport server nursery thereby - preventing any new inbound IPC connections establishing. - - ''' - if self._server_n: - # 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() - return True - - return False - @property def accept_addrs(self) -> list[UnwrappedAddress]: ''' @@ -1748,7 +1652,7 @@ class Actor: and listens for new connections. ''' - return [a.unwrap() for a in self._listen_addrs] + return self._ipc_server.accept_addrs @property def accept_addr(self) -> UnwrappedAddress: @@ -1856,6 +1760,7 @@ async def async_main( addr: Address = transport_cls.get_random() accept_addrs.append(addr.unwrap()) + assert accept_addrs # The "root" nursery ensures the channel with the immediate # parent is kept alive as a resilient service until # cancellation steps have (mostly) occurred in @@ -1866,15 +1771,37 @@ async def async_main( actor._root_n = root_nursery assert actor._root_n - async with trio.open_nursery( - strict_exception_groups=False, - ) as service_nursery: + ipc_server: _server.IPCServer + async with ( + trio.open_nursery( + strict_exception_groups=False, + ) as service_nursery, + + _server.open_ipc_server( + actor=actor, + parent_tn=service_nursery, + stream_handler_tn=service_nursery, + ) as ipc_server, + # ) as actor._ipc_server, + # ^TODO? prettier? + + ): # This nursery is used to handle all inbound # connections to us such that if the TCP server # is killed, connections can continue to process # in the background until this nursery is cancelled. actor._service_n = service_nursery - assert actor._service_n + actor._ipc_server = ipc_server + assert ( + actor._service_n + and ( + actor._service_n + is + actor._ipc_server._parent_tn + is + ipc_server._stream_handler_tn + ) + ) # load exposed/allowed RPC modules # XXX: do this **after** establishing a channel to the parent @@ -1898,30 +1825,42 @@ async def async_main( # - subactor: the bind address is sent by our parent # over our established channel # - root actor: the ``accept_addr`` passed to this method - assert accept_addrs + # TODO: why is this not with the root nursery? try: - # TODO: why is this not with the root nursery? - actor._server_n = await service_nursery.start( - partial( - actor._serve_forever, - service_nursery, - listen_addrs=accept_addrs, - ) + log.runtime( + 'Booting IPC server' ) + eps: list = await ipc_server.listen_on( + actor=actor, + accept_addrs=accept_addrs, + stream_handler_nursery=service_nursery, + ) + log.runtime( + f'Booted IPC server\n' + f'{ipc_server}\n' + ) + assert ( + (eps[0].listen_tn) + is not service_nursery + ) + except OSError as oserr: # NOTE: always allow runtime hackers to debug # tranport address bind errors - normally it's # something silly like the wrong socket-address # passed via a config or CLI Bo - entered_debug: bool = await _debug._maybe_enter_pm(oserr) + entered_debug: bool = await _debug._maybe_enter_pm( + oserr, + ) if not entered_debug: - log.exception('Failed to init IPC channel server !?\n') + log.exception('Failed to init IPC server !?\n') else: log.runtime('Exited debug REPL..') raise + # TODO, just read direct from ipc_server? accept_addrs: list[UnwrappedAddress] = actor.accept_addrs # NOTE: only set the loopback addr for the @@ -1954,7 +1893,9 @@ async def async_main( async with get_registry(addr) as reg_portal: for accept_addr in accept_addrs: accept_addr = wrap_address(accept_addr) - assert accept_addr.is_valid + + if not accept_addr.is_valid: + breakpoint() await reg_portal.run_from_ns( 'self', diff --git a/tractor/ipc/_server.py b/tractor/ipc/_server.py new file mode 100644 index 00000000..f23cf697 --- /dev/null +++ b/tractor/ipc/_server.py @@ -0,0 +1,467 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . +''' +High-level "IPC server" encapsulation for all your +multi-transport-protcol needs! + +''' +from __future__ import annotations +from contextlib import ( + asynccontextmanager as acm, +) +from functools import partial +import inspect +from types import ( + ModuleType, +) +from typing import ( + Callable, + TYPE_CHECKING, +) + +import trio +from trio import ( + EventStatistics, + Nursery, + TaskStatus, + SocketListener, +) + +from ..msg import Struct +from ..trionics import maybe_open_nursery +from .. import ( + _state, + log, +) +from .._addr import Address +from ._transport import MsgTransport +from ._uds import UDSAddress +from ._tcp import TCPAddress + +if TYPE_CHECKING: + from .._runtime import Actor + + +log = log.get_logger(__name__) + + +class IPCEndpoint(Struct): + ''' + An instance of an IPC "bound" address where the lifetime of the + "ability to accept connections" (from clients) and then handle + those inbound sessions or sequences-of-packets is determined by + a (maybe pair of) nurser(y/ies). + + ''' + addr: Address + listen_tn: Nursery + stream_handler_tn: Nursery|None = None + + # NOTE, normally filled in by calling `.start_listener()` + _listener: SocketListener|None = None + + # ?TODO, mk stream_handler hook into this ep instance so that we + # always keep track of all `SocketStream` instances per + # listener/ep? + peer_tpts: dict[ + UDSAddress|TCPAddress, # peer addr + MsgTransport, # handle to encoded-msg transport stream + ] = {} + + async def start_listener(self) -> SocketListener: + tpt_mod: ModuleType = inspect.getmodule(self.addr) + lstnr: SocketListener = await tpt_mod.start_listener( + addr=self.addr, + ) + + # NOTE, for handling the resolved non-0 port for + # TCP/UDP network sockets. + if ( + (unwrapped := lstnr.socket.getsockname()) + != + self.addr.unwrap() + ): + self.addr=self.addr.from_addr(unwrapped) + + self._listener = lstnr + return lstnr + + def close_listener( + self, + ) -> bool: + tpt_mod: ModuleType = inspect.getmodule(self.addr) + closer: Callable = getattr( + tpt_mod, + 'close_listener', + False, + ) + # when no defined closing is implicit! + if not closer: + return True + return closer( + addr=self.addr, + lstnr=self._listener, + ) + + +class IPCServer(Struct): + _parent_tn: Nursery + _stream_handler_tn: Nursery + _endpoints: list[IPCEndpoint] = [] + + # syncs for setup/teardown sequences + _shutdown: trio.Event|None = None + + # TODO, maybe just make `._endpoints: list[IPCEndpoint]` and + # provide dict-views onto it? + # @property + # def addrs2eps(self) -> dict[Address, IPCEndpoint]: + # ... + + @property + def proto_keys(self) -> list[str]: + return [ + ep.addr.proto_key + for ep in self._endpoints + ] + + # def cancel_server(self) -> bool: + def cancel( + self, + + # !TODO, suport just shutting down accepting new clients, + # not existing ones! + # only_listeners: str|None = None + + ) -> bool: + ''' + Cancel this IPC transport server nursery thereby + preventing any new inbound IPC connections establishing. + + ''' + if self._parent_tn: + # TODO: obvi a different server type when we eventually + # support some others XD + log.runtime( + f'Cancelling server(s) for\n' + f'{self.proto_keys!r}\n' + ) + self._parent_tn.cancel_scope.cancel() + return True + + log.warning( + 'No IPC server started before cancelling ?' + ) + return False + + async def wait_for_shutdown( + self, + ) -> bool: + if self._shutdown is not None: + await self._shutdown.wait() + else: + tpt_protos: list[str] = [] + ep: IPCEndpoint + for ep in self._endpoints: + tpt_protos.append(ep.addr.proto_key) + + log.warning( + 'Transport server(s) may have been cancelled before started?\n' + f'protos: {tpt_protos!r}\n' + ) + + @property + def addrs(self) -> list[Address]: + return [ep.addr for ep in self._endpoints] + + @property + def accept_addrs(self) -> list[str, str|int]: + ''' + The `list` of `Address.unwrap()`-ed active IPC endpoint addrs. + + ''' + return [ep.addr.unwrap() for ep in self._endpoints] + + def epsdict(self) -> dict[ + Address, + IPCEndpoint, + ]: + return { + ep.addr: ep + for ep in self._endpoints + } + + def is_shutdown(self) -> bool: + if (ev := self._shutdown) is None: + return False + + return ev.is_set() + + def pformat(self) -> str: + + fmtstr: str = ( + f' |_endpoints: {self._endpoints}\n' + ) + if self._shutdown is not None: + shutdown_stats: EventStatistics = self._shutdown.statistics() + fmtstr += ( + f'\n' + f' |_shutdown: {shutdown_stats}\n' + ) + + return ( + f'\n' + ) + + __repr__ = pformat + + # TODO? maybe allow shutting down a `.listen_on()`s worth of + # listeners by cancelling the corresponding + # `IPCEndpoint._listen_tn` only ? + # -[ ] in theory you could use this to + # "boot-and-wait-for-reconnect" of all current and connecting + # peers? + # |_ would require that the stream-handler is intercepted so we + # can intercept every `MsgTransport` (stream) and track per + # `IPCEndpoint` likely? + # + # async def unlisten( + # self, + # listener: SocketListener, + # ) -> bool: + # ... + + async def listen_on( + self, + *, + actor: Actor, + accept_addrs: list[tuple[str, int|str]]|None = None, + stream_handler_nursery: Nursery|None = None, + ) -> list[IPCEndpoint]: + ''' + Start `SocketListeners` (i.e. bind and call `socket.listen()`) + for all IPC-transport-protocol specific `Address`-types + in `accept_addrs`. + + ''' + from .._addr import ( + default_lo_addrs, + wrap_address, + ) + if accept_addrs is None: + accept_addrs = default_lo_addrs([ + _state._def_tpt_proto + ]) + + else: + accept_addrs: list[Address] = [ + wrap_address(a) for a in accept_addrs + ] + + if self._shutdown is None: + self._shutdown = trio.Event() + + elif self.is_shutdown(): + raise RuntimeError( + f'IPC server has already terminated ?\n' + f'{self}\n' + ) + + log.info( + f'Binding to endpoints for,\n' + f'{accept_addrs}\n' + ) + eps: list[IPCEndpoint] = await self._parent_tn.start( + partial( + _serve_ipc_eps, + actor=actor, + server=self, + stream_handler_tn=stream_handler_nursery, + listen_addrs=accept_addrs, + ) + ) + log.info( + f'Started IPC endpoints\n' + f'{eps}\n' + ) + + self._endpoints.extend(eps) + # XXX, just a little bit of sanity + group_tn: Nursery|None = None + ep: IPCEndpoint + for ep in eps: + if ep.addr not in self.addrs: + breakpoint() + + if group_tn is None: + group_tn = ep.listen_tn + else: + assert group_tn is ep.listen_tn + + return eps + + +async def _serve_ipc_eps( + *, + actor: Actor, + server: IPCServer, + stream_handler_tn: Nursery, + listen_addrs: list[tuple[str, int|str]], + + task_status: TaskStatus[ + Nursery, + ] = trio.TASK_STATUS_IGNORED, +) -> None: + ''' + Start IPC transport server(s) for the actor, begin + listening/accepting new `trio.SocketStream` connections + from peer actors via a `SocketListener`. + + This will cause an actor to continue living (and thus + blocking at the process/OS-thread level) until + `.cancel_server()` is called. + + ''' + try: + listen_tn: Nursery + async with trio.open_nursery() as listen_tn: + + eps: list[IPCEndpoint] = [] + # XXX NOTE, required to call `serve_listeners()` below. + # ?TODO, maybe just pass `list(eps.values()` tho? + listeners: list[trio.abc.Listener] = [] + for addr in listen_addrs: + ep = IPCEndpoint( + addr=addr, + listen_tn=listen_tn, + stream_handler_tn=stream_handler_tn, + ) + try: + log.info( + f'Starting new endpoint listener\n' + f'{ep}\n' + ) + listener: trio.abc.Listener = await ep.start_listener() + assert listener is ep._listener + # if actor.is_registry: + # import pdbp; pdbp.set_trace() + + except OSError as oserr: + if ( + '[Errno 98] Address already in use' + in + oserr.args#[0] + ): + log.exception( + f'Address already in use?\n' + f'{addr}\n' + ) + raise + + listeners.append(listener) + eps.append(ep) + + _listeners: list[SocketListener] = await listen_tn.start( + partial( + trio.serve_listeners, + handler=actor._stream_handler, + listeners=listeners, + + # NOTE: configured such that new + # connections will stay alive even if + # this server is cancelled! + handler_nursery=stream_handler_tn + ) + ) + # TODO, wow make this message better! XD + log.info( + 'Started server(s)\n' + + + '\n'.join([f'|_{addr}' for addr in listen_addrs]) + ) + + log.info( + f'Started IPC endpoints\n' + f'{eps}\n' + ) + task_status.started( + eps, + ) + + finally: + if eps: + addr: Address + ep: IPCEndpoint + for addr, ep in server.epsdict().items(): + ep.close_listener() + server._endpoints.remove(ep) + + # if actor.is_arbiter: + # import pdbp; pdbp.set_trace() + + # signal the server is "shutdown"/"terminated" + # since no more active endpoints are active. + if not server._endpoints: + server._shutdown.set() + +@acm +async def open_ipc_server( + actor: Actor, + parent_tn: Nursery|None = None, + stream_handler_tn: Nursery|None = None, + +) -> IPCServer: + + async with maybe_open_nursery( + nursery=parent_tn, + ) as rent_tn: + ipc_server = IPCServer( + _parent_tn=rent_tn, + _stream_handler_tn=stream_handler_tn or rent_tn, + ) + try: + yield ipc_server + + # except BaseException as berr: + # log.exception( + # 'IPC server crashed on exit ?' + # ) + # raise berr + + finally: + # ?TODO, maybe we can ensure the endpoints are torndown + # (and thus their managed listeners) beforehand to ensure + # super graceful RPC mechanics? + # + # -[ ] but aren't we doing that already per-`listen_tn` + # inside `_serve_ipc_eps()` above? + # + # if not ipc_server.is_shutdown(): + # ipc_server.cancel() + # await ipc_server.wait_for_shutdown() + # assert ipc_server.is_shutdown() + pass + + # !XXX TODO! lol so classic, the below code is rekt! + # + # XXX here is a perfect example of suppressing errors with + # `trio.Cancelled` as per our demonstrating example, + # `test_trioisms::test_acm_embedded_nursery_propagates_enter_err + # + # with trio.CancelScope(shield=True): + # await ipc_server.wait_for_shutdown() diff --git a/tractor/ipc/_tcp.py b/tractor/ipc/_tcp.py index dbecdf5e..b534b143 100644 --- a/tractor/ipc/_tcp.py +++ b/tractor/ipc/_tcp.py @@ -18,7 +18,14 @@ TCP implementation of tractor.ipc._transport.MsgTransport protocol ''' from __future__ import annotations +from typing import ( + ClassVar, +) +# from contextlib import ( +# asynccontextmanager as acm, +# ) +import msgspec import trio from trio import ( SocketListener, @@ -27,33 +34,25 @@ from trio import ( from tractor.msg import MsgCodec from tractor.log import get_logger -from tractor.ipc._transport import MsgpackTransport +from tractor.ipc._transport import ( + MsgTransport, + MsgpackTransport, +) log = get_logger(__name__) -class TCPAddress: - proto_key: str = 'tcp' - unwrapped_type: type = tuple[str, int] - def_bindspace: str = '127.0.0.1' +class TCPAddress( + msgspec.Struct, + frozen=True, +): + _host: str + _port: int - def __init__( - self, - host: str, - port: int - ): - if ( - not isinstance(host, str) - or - not isinstance(port, int) - ): - raise TypeError( - f'Expected host {host!r} to be str and port {port!r} to be int' - ) - - self._host: str = host - self._port: int = port + proto_key: ClassVar[str] = 'tcp' + unwrapped_type: ClassVar[type] = tuple[str, int] + def_bindspace: ClassVar[str] = '127.0.0.1' @property def is_valid(self) -> bool: @@ -106,34 +105,42 @@ class TCPAddress: f'{type(self).__name__}[{self.unwrap()}]' ) - def __eq__(self, other) -> bool: - if not isinstance(other, TCPAddress): - raise TypeError( - f'Can not compare {type(other)} with {type(self)}' - ) + @classmethod + def get_transport( + cls, + codec: str = 'msgpack', + ) -> MsgTransport: + match codec: + case 'msgspack': + return MsgpackTCPStream + case _: + raise ValueError( + f'No IPC transport with {codec!r} supported !' + ) - return ( - self._host == other._host - and - self._port == other._port - ) - async def open_listener( - self, - **kwargs, - ) -> SocketListener: - listeners: list[SocketListener] = await open_tcp_listeners( - host=self._host, - port=self._port, - **kwargs - ) - assert len(listeners) == 1 - listener = listeners[0] - self._host, self._port = listener.socket.getsockname()[:2] - return listener +async def start_listener( + addr: TCPAddress, + **kwargs, +) -> SocketListener: + ''' + Start a TCP socket listener on the given `TCPAddress`. - async def close_listener(self): - ... + ''' + # ?TODO, maybe we should just change the lower-level call this is + # using internall per-listener? + listeners: list[SocketListener] = await open_tcp_listeners( + host=addr._host, + port=addr._port, + **kwargs + ) + # NOTE, for now we don't expect non-singleton-resolving + # domain-addresses/multi-homed-hosts. + # (though it is supported by `open_tcp_listeners()`) + assert len(listeners) == 1 + listener = listeners[0] + host, port = listener.socket.getsockname()[:2] + return listener # TODO: typing oddity.. not sure why we have to inherit here, but it diff --git a/tractor/ipc/_transport.py b/tractor/ipc/_transport.py index 2a9926f9..ec3c442c 100644 --- a/tractor/ipc/_transport.py +++ b/tractor/ipc/_transport.py @@ -104,7 +104,10 @@ class MsgTransport(Protocol): @classmethod def key(cls) -> MsgTransportKey: - return cls.codec_key, cls.address_type.proto_key + return ( + cls.codec_key, + cls.address_type.proto_key, + ) @property def laddr(self) -> Address: @@ -135,8 +138,8 @@ class MsgTransport(Protocol): Address # remote ]: ''' - Return the `trio` streaming transport prot's addrs for both - the local and remote sides as a pair. + Return the transport protocol's address pair for the local + and remote-peer side. ''' ... diff --git a/tractor/ipc/_types.py b/tractor/ipc/_types.py index 8d543d9d..59653b17 100644 --- a/tractor/ipc/_types.py +++ b/tractor/ipc/_types.py @@ -53,9 +53,12 @@ _msg_transports = [ # convert a MsgTransportKey to the corresponding transport type -_key_to_transport: dict[MsgTransportKey, Type[MsgTransport]] = { - cls.key(): cls - for cls in _msg_transports +_key_to_transport: dict[ + MsgTransportKey, + Type[MsgTransport], +] = { + ('msgpack', 'tcp'): MsgpackTCPStream, + ('msgpack', 'uds'): MsgpackUDSStream, } # convert an Address wrapper to its corresponding transport type @@ -63,8 +66,8 @@ _addr_to_transport: dict[ Type[TCPAddress|UDSAddress], Type[MsgTransport] ] = { - cls.address_type: cls - for cls in _msg_transports + TCPAddress: MsgpackTCPStream, + UDSAddress: MsgpackUDSStream, } diff --git a/tractor/ipc/_uds.py b/tractor/ipc/_uds.py index 33843f6a..604802f3 100644 --- a/tractor/ipc/_uds.py +++ b/tractor/ipc/_uds.py @@ -21,7 +21,6 @@ from __future__ import annotations from pathlib import Path import os from socket import ( - # socket, AF_UNIX, SOCK_STREAM, SO_PASSCRED, @@ -31,8 +30,10 @@ from socket import ( import struct from typing import ( TYPE_CHECKING, + ClassVar, ) +import msgspec import trio from trio import ( socket, @@ -70,56 +71,22 @@ def unwrap_sockpath( ) -class UDSAddress: +class UDSAddress( + msgspec.Struct, + frozen=True, +): + filedir: str|Path|None + filename: str|Path + maybe_pid: int|None = None + # TODO, maybe we should use better field and value # -[x] really this is a `.protocol_key` not a "name" of anything. # -[ ] consider a 'unix' proto-key instead? # -[ ] need to check what other mult-transport frameworks do # like zmq, nng, uri-spec et al! - proto_key: str = 'uds' - unwrapped_type: type = tuple[str, int] - def_bindspace: Path = get_rt_dir() - - def __init__( - self, - filedir: Path|str|None, - # TODO, i think i want `.filename` here? - filename: str|Path, - - # XXX, in the sense you can also pass - # a "non-real-world-process-id" such as is handy to represent - # our host-local default "port-like" key for the very first - # root actor to create a registry address. - maybe_pid: int|None = None, - ): - fdir = self._filedir = Path( - filedir - or - self.def_bindspace - ).absolute() - fpath = self._filename = Path(filename) - fp: Path = fdir / fpath - assert ( - fp.is_absolute() - and - fp == self.sockpath - ) - - # to track which "side" is the peer process by reading socket - # credentials-info. - self._pid: int = maybe_pid - - @property - def sockpath(self) -> Path: - return self._filedir / self._filename - - @property - def is_valid(self) -> bool: - ''' - We block socket files not allocated under the runtime subdir. - - ''' - return self.bindspace in self.sockpath.parents + proto_key: ClassVar[str] = 'uds' + unwrapped_type: ClassVar[type] = tuple[str, int] + def_bindspace: ClassVar[Path] = get_rt_dir() @property def bindspace(self) -> Path: @@ -128,7 +95,25 @@ class UDSAddress: just the sub-directory in which we allocate socket files. ''' - return self._filedir or self.def_bindspace + return ( + self.filedir + or + self.def_bindspace + # or + # get_rt_dir() + ) + + @property + def sockpath(self) -> Path: + return self.bindspace / self.filename + + @property + def is_valid(self) -> bool: + ''' + We block socket files not allocated under the runtime subdir. + + ''' + return self.bindspace in self.sockpath.parents @classmethod def from_addr( @@ -141,9 +126,6 @@ class UDSAddress: case tuple()|list(): filedir = Path(addr[0]) filename = Path(addr[1]) - # sockpath: Path = Path(addr[0]) - # filedir, filename = unwrap_sockpath(sockpath) - # pid: int = addr[1] return UDSAddress( filedir=filedir, filename=filename, @@ -165,8 +147,8 @@ class UDSAddress: # XXX NOTE, since this gets passed DIRECTLY to # `.ipc._uds.open_unix_socket_w_passcred()` return ( - str(self._filedir), - str(self._filename), + str(self.filedir), + str(self.filename), ) @classmethod @@ -199,55 +181,77 @@ class UDSAddress: def get_root(cls) -> UDSAddress: def_uds_filename: Path = 'registry@1616.sock' return UDSAddress( - filedir=None, + filedir=cls.def_bindspace, filename=def_uds_filename, # maybe_pid=1616, ) + # ?TODO, maybe we should just our .msg.pretty_struct.Struct` for + # this instead? + # -[ ] is it too "multi-line"y tho? + # the compact tuple/.unwrapped() form is simple enough? + # def __repr__(self) -> str: + if not (pid := self.maybe_pid): + pid: str = '' + + body: str = ( + f'({self.filedir}, {self.filename}, {pid})' + ) return ( f'{type(self).__name__}' f'[' - f'({self._filedir}, {self._filename})' + f'{body}' f']' ) - def __eq__(self, other) -> bool: - if not isinstance(other, UDSAddress): - raise TypeError( - f'Can not compare {type(other)} with {type(self)}' - ) - return self.sockpath == other.sockpath +async def start_listener( + addr: UDSAddress, + **kwargs, +) -> SocketListener: + # sock = addr._sock = socket.socket( + sock = socket.socket( + socket.AF_UNIX, + socket.SOCK_STREAM + ) + log.info( + f'Attempting to bind UDS socket\n' + f'>[\n' + f'|_{addr}\n' + ) - # async def open_listener(self, **kwargs) -> SocketListener: - async def open_listener( - self, - **kwargs, - ) -> SocketListener: - sock = self._sock = socket.socket( - socket.AF_UNIX, - socket.SOCK_STREAM - ) - log.info( - f'Attempting to bind UDS socket\n' - f'>[\n' - f'|_{self}\n' - ) - - bindpath: Path = self.sockpath + bindpath: Path = addr.sockpath + try: await sock.bind(str(bindpath)) - sock.listen(1) - log.info( - f'Listening on UDS socket\n' - f'[>\n' - f' |_{self}\n' - ) - return SocketListener(self._sock) + except ( + FileNotFoundError, + ) as fdne: + raise ConnectionError( + f'Bad UDS socket-filepath-as-address ??\n' + f'{addr}\n' + f' |_sockpath: {addr.sockpath}\n' + ) from fdne - def close_listener(self): - self._sock.close() - os.unlink(self.sockpath) + sock.listen(1) + log.info( + f'Listening on UDS socket\n' + f'[>\n' + f' |_{addr}\n' + ) + return SocketListener(sock) + + +def close_listener( + addr: UDSAddress, + lstnr: SocketListener, +) -> None: + ''' + Close and remove the listening unix socket's path. + + ''' + lstnr.socket.close() + os.unlink(addr.sockpath) async def open_unix_socket_w_passcred( @@ -416,5 +420,3 @@ class MsgpackUDSStream(MsgpackTransport): maybe_pid=peer_pid ) return (laddr, raddr) - - From 79d0c17f6b568ff81f29067bd14cda72cb4cdd2f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 10 Apr 2025 23:37:16 -0400 Subject: [PATCH 50/74] Add `TransportClosed.from_src_exc()` Such that re-wrapping/raising from a low-level `trio` resource error is simpler and includes the `.src_exc` in the `__repr__()` and `.message/.args` rendered at higher layers (like from `Channel` and `._rpc` machinery). Impl deats, - mainly leverages packing in a new cls-method `.repr_src_exc() -> str:` repr of the underlying error before an optional `body: str` all as handled by the previously augmented `.pformat()`'s delegation to `pformat_exc()`. - change `.src_exc` to be a property around a renamed `._src_exc`. But wait, why? - use it inside `MsgpackTransport.send()` to rewrap any `trio.BrokenResourceError`s so we always see the underlying `trio`-src-exc just like in the `.recv()._iter_packets()` handlers. --- tractor/_exceptions.py | 83 +++++++++++++++++++++++++++++++++------ tractor/ipc/_transport.py | 26 ++++++------ 2 files changed, 86 insertions(+), 23 deletions(-) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 1238220e..3561c7c6 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -948,7 +948,8 @@ class TransportClosed(Exception): self._loglevel: str = loglevel super().__init__(message) - self.src_exc = src_exc + self._src_exc = src_exc + # set the cause manually if not already set by python if ( src_exc is not None and @@ -960,9 +961,18 @@ class TransportClosed(Exception): # the exc in its `TransportClosed` handler block. self._raise_on_report = raise_on_report + @property + def src_exc(self) -> Exception: + return ( + self.__cause__ + or + self._src_exc + ) + def report_n_maybe_raise( self, message: str|None = None, + hide_tb: bool = True, ) -> None: ''' @@ -970,9 +980,10 @@ class TransportClosed(Exception): for this error. ''' + __tracebackhide__: bool = hide_tb message: str = message or self.message # when a cause is set, slap it onto the log emission. - if cause := self.__cause__: + if cause := self.src_exc: cause_tb_str: str = ''.join( traceback.format_tb(cause.__traceback__) ) @@ -991,26 +1002,76 @@ class TransportClosed(Exception): if self._raise_on_report: raise self from cause + @classmethod + def repr_src_exc( + self, + src_exc: Exception|None = None, + ) -> str: + + if src_exc is None: + return '' + + src_msg: tuple[str] = src_exc.args + src_exc_repr: str = ( + f'{type(src_exc).__name__}[ {src_msg} ]' + ) + return src_exc_repr + def pformat(self) -> str: from tractor.devx.pformat import ( pformat_exc, ) - src_err: Exception|None = self.src_exc or '' - src_msg: tuple[str] = src_err.args - src_exc_repr: str = ( - f'{type(src_err).__name__}[ {src_msg} ]' - ) return pformat_exc( exc=self, - # message=self.message, # implicit! - body=( - f' |_src_exc: {src_exc_repr}\n' - ), ) # delegate to `str`-ified pformat __repr__ = pformat + @classmethod + def from_src_exc( + cls, + src_exc: ( + Exception| + trio.ClosedResource| + trio.BrokenResourceError + ), + message: str, + body: str = '', + **init_kws, + ) -> TransportClosed: + ''' + Convenience constructor for creation from an underlying + `trio`-sourced async-resource/chan/stream error. + + Embeds the original `src_exc`'s repr within the + `Exception.args` via a first-line-in-`.message`-put-in-header + pre-processing and allows inserting additional content beyond + the main message via a `body: str`. + + ''' + repr_src_exc: str = cls.repr_src_exc( + src_exc, + ) + next_line: str = f' src_exc: {repr_src_exc}\n' + if body: + body: str = textwrap.indent( + body, + prefix=' '*2, + ) + + return TransportClosed( + message=( + message + + + next_line + + + body + ), + src_exc=src_exc, + **init_kws, + ) + class NoResult(RuntimeError): "No final result is expected for this actor" diff --git a/tractor/ipc/_transport.py b/tractor/ipc/_transport.py index ec3c442c..6bfa5f6a 100644 --- a/tractor/ipc/_transport.py +++ b/tractor/ipc/_transport.py @@ -367,7 +367,7 @@ class MsgpackTransport(MsgTransport): msg: msgtypes.MsgType, strict_types: bool = True, - hide_tb: bool = False, + hide_tb: bool = True, ) -> None: ''' @@ -430,8 +430,9 @@ class MsgpackTransport(MsgTransport): return await self.stream.send_all(size + bytes_data) except ( trio.BrokenResourceError, - ) as trans_err: - loglevel = 'transport' + ) as bre: + trans_err = bre + tpt_name: str = f'{type(self).__name__!r}' match trans_err: case trio.BrokenResourceError() if ( '[Errno 32] Broken pipe' in trans_err.args[0] @@ -442,21 +443,22 @@ class MsgpackTransport(MsgTransport): # as it pertains to rando pings from the # `.discovery` subsys and protos. ): - raise TransportClosed( + raise TransportClosed.from_src_exc( message=( - f'IPC transport already closed by peer\n' - f'x)> {type(trans_err)}\n' - f' |_{self}\n' + f'{tpt_name} already closed by peer\n' ), - loglevel=loglevel, - ) from trans_err + body=f'{self}\n', + src_exc=trans_err, + raise_on_report=True, + loglevel='transport', + ) from bre # unless the disconnect condition falls under "a # normal operation breakage" we usualy console warn # about it. case _: log.exception( - 'Transport layer failed for {self.transport!r} ?\n' + '{tpt_name} layer failed pre-send ??\n' ) raise trans_err @@ -501,11 +503,11 @@ class MsgpackTransport(MsgTransport): def pformat(self) -> str: return ( f'<{type(self).__name__}(\n' - f' |_task: {self._task}\n' - f'\n' f' |_peers: 2\n' f' laddr: {self._laddr}\n' f' raddr: {self._raddr}\n' + # f'\n' + f' |_task: {self._task}\n' f')>\n' ) From a7d4bcdfb99f25a4f0c60011904ecde4189524f0 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 10 Apr 2025 23:49:36 -0400 Subject: [PATCH 51/74] Absorb `TransportClosed` in `Portal.cancel_actor()` Just like we *were* for the `trio`-resource-errors it normally wraps since we now also do the same wrapping in `MsgpackTransport.send()` and we don't normally care to raise tpt-closure-errors on graceful actor cancel requests. Also, warn-report any non-tpt-closed low-level `trio` errors we haven't yet re-wrapped (likely bc they haven't shown up). --- tractor/_portal.py | 28 ++++++++++++++++++++++++---- 1 file changed, 24 insertions(+), 4 deletions(-) diff --git a/tractor/_portal.py b/tractor/_portal.py index d0fd3cc3..a4ae431a 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -52,8 +52,8 @@ from .msg import ( Return, ) from ._exceptions import ( - # unpack_error, NoResult, + TransportClosed, ) from ._context import ( Context, @@ -305,14 +305,34 @@ class Portal: return False except ( + # XXX, should never really get raised unless we aren't + # wrapping them in the below type by mistake? + # + # Leaving the catch here for now until we're very sure + # all the cases (for various tpt protos) have indeed been + # re-wrapped ;p trio.ClosedResourceError, trio.BrokenResourceError, - ): - log.debug( - 'IPC chan for actor already closed or broken?\n\n' + + TransportClosed, + ) as tpt_err: + report: str = ( + f'IPC chan for actor already closed or broken?\n\n' f'{self.channel.aid}\n' f' |_{self.channel}\n' ) + match tpt_err: + case TransportClosed(): + log.debug(report) + case _: + report += ( + f'\n' + f'Unhandled low-level transport-closed/error during\n' + f'Portal.cancel_actor()` request?\n' + f'<{type(tpt_err).__name__}( {tpt_err} )>\n' + ) + log.warning(report) + return False # TODO: do we still need this for low level `Actor`-runtime From 52b5efd78d78f092dbc42e3220c16b2a89c0ff4b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 10 Apr 2025 23:53:35 -0400 Subject: [PATCH 52/74] Another `tn` eg-loosify inside `ActorNursery.cancel()`.. --- tractor/_supervise.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tractor/_supervise.py b/tractor/_supervise.py index ab3a2a94..07ae8bb3 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -316,7 +316,9 @@ class ActorNursery: child_count: int = len(children) msg: str = f'Cancelling actor nursery with {child_count} children\n' with trio.move_on_after(3) as cs: - async with trio.open_nursery() as tn: + async with trio.open_nursery( + strict_exception_groups=False, + ) as tn: subactor: Actor proc: trio.Process From 56ef4cba23415e81bd35e7fadc4522311488d6d1 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 10 Apr 2025 23:53:44 -0400 Subject: [PATCH 53/74] Add `_state.current_ipc_protos()` For now just wrapping wtv the `._def_tpt_proto` per-actor setting is. --- tractor/_state.py | 23 ++++++++++++++++++++--- 1 file changed, 20 insertions(+), 3 deletions(-) diff --git a/tractor/_state.py b/tractor/_state.py index 0ec4577e..bc12d0de 100644 --- a/tractor/_state.py +++ b/tractor/_state.py @@ -102,7 +102,7 @@ def current_actor( return _current_actor -def is_main_process() -> bool: +def is_root_process() -> bool: ''' Bool determining if this actor is running in the top-most process. @@ -111,8 +111,10 @@ def is_main_process() -> bool: return mp.current_process().name == 'MainProcess' -# TODO, more verby name? -def debug_mode() -> bool: +is_main_process = is_root_process + + +def is_debug_mode() -> bool: ''' Bool determining if "debug mode" is on which enables remote subactor pdb entry on crashes. @@ -121,6 +123,9 @@ def debug_mode() -> bool: return bool(_runtime_vars['_debug_mode']) +debug_mode = is_debug_mode + + def is_root_process() -> bool: return _runtime_vars['_is_root'] @@ -173,3 +178,15 @@ TransportProtocolKey = Literal[ 'uds', ] _def_tpt_proto: TransportProtocolKey = 'tcp' + + +def current_ipc_protos() -> list[str]: + ''' + Return the list of IPC transport protocol keys currently + in use by this actor. + + The keys are as declared by `MsgTransport` and `Address` + concrete-backend sub-types defined throughout `tractor.ipc`. + + ''' + return [_def_tpt_proto] From 01d6f111f66cede7a90511804e7bae8c3d14e7d9 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 10 Apr 2025 23:55:41 -0400 Subject: [PATCH 54/74] Use `current_ipc_protos()` as the `enable_transports`-default-when-`None` Also ensure we assertion-error whenever the list is > 1 entry for now! --- tractor/_root.py | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/tractor/_root.py b/tractor/_root.py index f67d746a..1bcfa00b 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -149,8 +149,10 @@ async def open_root_actor( arbiter_addr: tuple[UnwrappedAddress]|None = None, enable_transports: list[ + # TODO, this should eventually be the pairs as + # defined by (codec, proto) as on `MsgTransport. _state.TransportProtocolKey, - ] = [_state._def_tpt_proto], + ]|None = None, name: str|None = 'root', @@ -213,6 +215,14 @@ async def open_root_actor( debug_mode=debug_mode, maybe_enable_greenback=maybe_enable_greenback, ): + if enable_transports is None: + enable_transports: list[str] = _state.current_ipc_protos() + + # TODO! support multi-tpts per actor! Bo + assert ( + len(enable_transports) == 1 + ), 'No multi-tpt support yet!' + _debug.hide_runtime_frames() __tracebackhide__: bool = hide_tb From 0477a62ac3e58ffc93b801f470ebf093ac4c3ed4 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 11 Apr 2025 00:00:12 -0400 Subject: [PATCH 55/74] Never hide non-[msgtype/tpt-closed] error tbs in `Channel.send()` --- tractor/ipc/_chan.py | 32 +++++++++++++++++++++----------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/tractor/ipc/_chan.py b/tractor/ipc/_chan.py index 00c749e1..2c3374c2 100644 --- a/tractor/ipc/_chan.py +++ b/tractor/ipc/_chan.py @@ -49,6 +49,7 @@ from tractor.log import get_logger from tractor._exceptions import ( MsgTypeError, pack_from_raise, + TransportClosed, ) from tractor.msg import ( Aid, @@ -256,7 +257,7 @@ class Channel: self, payload: Any, - hide_tb: bool = False, + hide_tb: bool = True, ) -> None: ''' @@ -274,18 +275,27 @@ class Channel: payload, hide_tb=hide_tb, ) - except BaseException as _err: + except ( + BaseException, + MsgTypeError, + TransportClosed, + )as _err: err = _err # bind for introspection - if not isinstance(_err, MsgTypeError): - # assert err - __tracebackhide__: bool = False - else: - try: - assert err.cid - - except KeyError: - raise err + match err: + case MsgTypeError(): + try: + assert err.cid + except KeyError: + raise err + case TransportClosed(): + log.transport( + f'Transport stream closed due to\n' + f'{err.repr_src_exc()}\n' + ) + case _: + # never suppress non-tpt sources + __tracebackhide__: bool = False raise async def recv(self) -> Any: From 0711576678fcacd1c4fdf00487e9657acd1d8ceb Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 11 Apr 2025 01:16:46 -0400 Subject: [PATCH 56/74] Passthrough `_pause()` kwargs from `_maybe_enter_pm()` --- tractor/devx/_debug.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 77753ff0..0f05534b 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -3003,6 +3003,7 @@ async def _maybe_enter_pm( [BaseException|BaseExceptionGroup], bool, ] = lambda err: not is_multi_cancelled(err), + **_pause_kws, ): if ( @@ -3029,6 +3030,7 @@ async def _maybe_enter_pm( await post_mortem( api_frame=api_frame, tb=tb, + **_pause_kws, ) return True From 26fedec6a14f57265cd1113b44687d3cd47a8bce Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 11 Apr 2025 14:30:21 -0400 Subject: [PATCH 57/74] Mv `Actor._stream_handler()` to `.ipc._server` func Call it `handle_stream_from_peer()` and bind in the `actor: Actor` via a `handler=partial()` to `trio.serve_listeners()`. With this (minus the `Actor._peers/._peer_connected/._no_more_peers` attrs ofc) we get nearly full separation of IPC-connection-processing (concerns) from `Actor` state. Thus it's a first look at modularizing the low-level runtime into isolated subsystems which will hopefully improve the entire code base's grok-ability and ease any new feature design discussions especially pertaining to introducing and/or composing-together any new transport protocols. --- tractor/_portal.py | 4 +- tractor/_runtime.py | 439 +------------------------------------- tractor/ipc/_server.py | 468 ++++++++++++++++++++++++++++++++++++++++- 3 files changed, 471 insertions(+), 440 deletions(-) diff --git a/tractor/_portal.py b/tractor/_portal.py index a4ae431a..d5dd8369 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -578,10 +578,10 @@ async def open_portal( msg_loop_cs: trio.CancelScope|None = None if start_msg_loop: - from ._runtime import process_messages + from . import _rpc msg_loop_cs = await tn.start( partial( - process_messages, + _rpc.process_messages, actor, channel, # if the local task is cancelled we want to keep diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 4daa4742..b21ed0ee 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -96,18 +96,13 @@ from ._exceptions import ( ModuleNotExposed, MsgTypeError, unpack_error, - TransportClosed, ) from .devx import _debug from ._discovery import get_registry from ._portal import Portal from . import _state from . import _mp_fixup_main -from ._rpc import ( - process_messages, - try_ship_error_to_remote, -) - +from . import _rpc if TYPE_CHECKING: from ._supervise import ActorNursery @@ -493,434 +488,6 @@ class Actor: raise mne - # TODO: maybe change to mod-func and rename for implied - # multi-transport semantics? - async def _stream_handler( - self, - stream: trio.SocketStream, - - ) -> None: - ''' - Entry point for new inbound IPC connections on a specific - transport server. - - ''' - self._no_more_peers = trio.Event() # unset by making new - # with _debug.maybe_open_crash_handler( - # pdb=True, - # ) as boxerr: - chan = Channel.from_stream(stream) - con_status: str = ( - 'New inbound IPC connection <=\n' - f'|_{chan}\n' - ) - - # send/receive initial handshake response - try: - peer_aid: msgtypes.Aid = await chan._do_handshake( - aid=self.aid, - ) - except ( - TransportClosed, - # ^XXX NOTE, the above wraps `trio` exc types raised - # during various `SocketStream.send/receive_xx()` calls - # under different fault conditions such as, - # - # trio.BrokenResourceError, - # trio.ClosedResourceError, - # - # Inside our `.ipc._transport` layer we absorb and - # re-raise our own `TransportClosed` exc such that this - # higher level runtime code can only worry one - # "kinda-error" that we expect to tolerate during - # discovery-sys related pings, queires, DoS etc. - ): - # XXX: This may propagate up from `Channel._aiter_recv()` - # and `MsgpackStream._inter_packets()` on a read from the - # stream particularly when the runtime is first starting up - # inside `open_root_actor()` where there is a check for - # a bound listener on the "arbiter" addr. the reset will be - # because the handshake was never meant took place. - log.runtime( - con_status - + - ' -> But failed to handshake? Ignoring..\n' - ) - return - - uid: tuple[str, str] = ( - peer_aid.name, - peer_aid.uuid, - ) - # TODO, can we make this downstream peer tracking use the - # `peer_aid` instead? - familiar: str = 'new-peer' - if _pre_chan := self._peers.get(uid): - familiar: str = 'pre-existing-peer' - uid_short: str = f'{uid[0]}[{uid[1][-6:]}]' - con_status += ( - f' -> Handshake with {familiar} `{uid_short}` complete\n' - ) - - if _pre_chan: - # con_status += ( - # ^TODO^ swap once we minimize conn duplication - # -[ ] last thing might be reg/unreg runtime reqs? - # log.warning( - log.debug( - f'?Wait?\n' - f'We already have IPC with peer {uid_short!r}\n' - f'|_{_pre_chan}\n' - ) - - # IPC connection tracking for both peers and new children: - # - if this is a new channel to a locally spawned - # sub-actor there will be a spawn wait even registered - # by a call to `.wait_for_peer()`. - # - if a peer is connecting no such event will exit. - event: trio.Event|None = self._peer_connected.pop( - uid, - None, - ) - if event: - con_status += ( - ' -> Waking subactor spawn waiters: ' - f'{event.statistics().tasks_waiting}\n' - f' -> Registered IPC chan for child actor {uid}@{chan.raddr}\n' - # f' {event}\n' - # f' |{event.statistics()}\n' - ) - # wake tasks waiting on this IPC-transport "connect-back" - event.set() - - else: - con_status += ( - f' -> Registered IPC chan for peer actor {uid}@{chan.raddr}\n' - ) # type: ignore - - chans: list[Channel] = self._peers[uid] - # if chans: - # # TODO: re-use channels for new connections instead - # # of always new ones? - # # => will require changing all the discovery funcs.. - - # append new channel - # TODO: can we just use list-ref directly? - chans.append(chan) - - con_status += ' -> Entering RPC msg loop..\n' - log.runtime(con_status) - - # Begin channel management - respond to remote requests and - # process received reponses. - disconnected: bool = False - last_msg: MsgType - try: - ( - disconnected, - last_msg, - ) = await process_messages( - self, - chan, - ) - except trio.Cancelled: - log.cancel( - 'IPC transport msg loop was cancelled\n' - f'c)>\n' - f' |_{chan}\n' - ) - raise - - finally: - local_nursery: ( - ActorNursery|None - ) = self._actoruid2nursery.get(uid) - - # This is set in ``Portal.cancel_actor()``. So if - # the peer was cancelled we try to wait for them - # to tear down their side of the connection before - # moving on with closing our own side. - if ( - local_nursery - and ( - self._cancel_called - or - chan._cancel_called - ) - # - # ^-TODO-^ along with this is there another condition - # that we should filter with to avoid entering this - # waiting block needlessly? - # -[ ] maybe `and local_nursery.cancelled` and/or - # only if the `._children` table is empty or has - # only `Portal`s with .chan._cancel_called == - # True` as per what we had below; the MAIN DIFF - # BEING that just bc one `Portal.cancel_actor()` - # was called, doesn't mean the whole actor-nurse - # is gonna exit any time soon right!? - # - # or - # all(chan._cancel_called for chan in chans) - - ): - log.cancel( - 'Waiting on cancel request to peer..\n' - f'c)=>\n' - f' |_{chan.uid}\n' - ) - - # XXX: this is a soft wait on the channel (and its - # underlying transport protocol) to close from the - # remote peer side since we presume that any channel - # which is mapped to a sub-actor (i.e. it's managed - # by local actor-nursery) has a message that is sent - # to the peer likely by this actor (which may be in - # a shutdown sequence due to cancellation) when the - # local runtime here is now cancelled while - # (presumably) in the middle of msg loop processing. - chan_info: str = ( - f'{chan.uid}\n' - f'|_{chan}\n' - f' |_{chan.transport}\n\n' - ) - with trio.move_on_after(0.5) as drain_cs: - drain_cs.shield = True - - # attempt to wait for the far end to close the - # channel and bail after timeout (a 2-generals - # problem on closure). - assert chan.transport - async for msg in chan.transport.drain(): - - # try to deliver any lingering msgs - # before we destroy the channel. - # This accomplishes deterministic - # ``Portal.cancel_actor()`` cancellation by - # making sure any RPC response to that call is - # delivered the local calling task. - # TODO: factor this into a helper? - log.warning( - 'Draining msg from disconnected peer\n' - f'{chan_info}' - f'{pformat(msg)}\n' - ) - # cid: str|None = msg.get('cid') - cid: str|None = msg.cid - if cid: - # deliver response to local caller/waiter - await self._deliver_ctx_payload( - chan, - cid, - msg, - ) - if drain_cs.cancelled_caught: - log.warning( - 'Timed out waiting on IPC transport channel to drain?\n' - f'{chan_info}' - ) - - # XXX NOTE XXX when no explicit call to - # `open_root_actor()` was made by the application - # (normally we implicitly make that call inside - # the first `.open_nursery()` in root-actor - # user/app code), we can assume that either we - # are NOT the root actor or are root but the - # runtime was started manually. and thus DO have - # to wait for the nursery-enterer to exit before - # shutting down the local runtime to avoid - # clobbering any ongoing subactor - # teardown/debugging/graceful-cancel. - # - # see matching note inside `._supervise.open_nursery()` - # - # TODO: should we have a separate cs + timeout - # block here? - if ( - # XXX SO either, - # - not root OR, - # - is root but `open_root_actor()` was - # entered manually (in which case we do - # the equiv wait there using the - # `devx._debug` sub-sys APIs). - not local_nursery._implicit_runtime_started - ): - log.runtime( - 'Waiting on local actor nursery to exit..\n' - f'|_{local_nursery}\n' - ) - with trio.move_on_after(0.5) as an_exit_cs: - an_exit_cs.shield = True - await local_nursery.exited.wait() - - # TODO: currently this is always triggering for every - # sub-daemon spawned from the `piker.services._mngr`? - # -[ ] how do we ensure that the IPC is supposed to - # be long lived and isn't just a register? - # |_ in the register case how can we signal that the - # ephemeral msg loop was intentional? - if ( - # not local_nursery._implicit_runtime_started - # and - an_exit_cs.cancelled_caught - ): - report: str = ( - 'Timed out waiting on local actor-nursery to exit?\n' - f'c)>\n' - f' |_{local_nursery}\n' - ) - if children := local_nursery._children: - # indent from above local-nurse repr - report += ( - f' |_{pformat(children)}\n' - ) - - log.warning(report) - - if disconnected: - # if the transport died and this actor is still - # registered within a local nursery, we report - # that the IPC layer may have failed - # unexpectedly since it may be the cause of - # other downstream errors. - entry: tuple|None = local_nursery._children.get(uid) - if entry: - proc: trio.Process - _, proc, _ = entry - - if ( - (poll := getattr(proc, 'poll', None)) - and - poll() is None # proc still alive - ): - # 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'<=x {chan.uid}@{chan.raddr}\n' - f' |_{proc}\n' - ) - - # ``Channel`` teardown and closure sequence - # drop ref to channel so it can be gc-ed and disconnected - con_teardown_status: str = ( - f'IPC channel disconnected:\n' - f'<=x uid: {chan.uid}\n' - f' |_{pformat(chan)}\n\n' - ) - chans.remove(chan) - - # TODO: do we need to be this pedantic? - if not chans: - con_teardown_status += ( - f'-> No more channels with {chan.uid}' - ) - self._peers.pop(uid, None) - - peers_str: str = '' - for uid, chans in self._peers.items(): - peers_str += ( - f'uid: {uid}\n' - ) - for i, chan in enumerate(chans): - peers_str += ( - f' |_[{i}] {pformat(chan)}\n' - ) - - con_teardown_status += ( - f'-> Remaining IPC {len(self._peers)} peers: {peers_str}\n' - ) - - # No more channels to other actors (at all) registered - # as connected. - if not self._peers: - con_teardown_status += ( - 'Signalling no more peer channel connections' - ) - self._no_more_peers.set() - - # NOTE: block this actor from acquiring the - # debugger-TTY-lock since we have no way to know if we - # cancelled it and further there is no way to ensure the - # lock will be released if acquired due to having no - # more active IPC channels. - if _state.is_root_process(): - pdb_lock = _debug.Lock - pdb_lock._blocked.add(uid) - - # TODO: NEEEDS TO BE TESTED! - # actually, no idea if this ever even enters.. XD - # - # XXX => YES IT DOES, when i was testing ctl-c - # from broken debug TTY locking due to - # msg-spec races on application using RunVar... - if ( - (ctx_in_debug := pdb_lock.ctx_in_debug) - and - (pdb_user_uid := ctx_in_debug.chan.uid) - and - local_nursery - ): - entry: tuple|None = local_nursery._children.get( - tuple(pdb_user_uid) - ) - if entry: - proc: trio.Process - _, proc, _ = entry - - if ( - (poll := getattr(proc, 'poll', None)) - and poll() is None - ): - log.cancel( - 'Root actor reports no-more-peers, BUT\n' - 'a DISCONNECTED child still has the debug ' - 'lock!\n\n' - # f'root uid: {self.uid}\n' - f'last disconnected child uid: {uid}\n' - f'locking child uid: {pdb_user_uid}\n' - ) - await _debug.maybe_wait_for_debugger( - child_in_debug=True - ) - - # TODO: just bc a child's transport dropped - # doesn't mean it's not still using the pdb - # REPL! so, - # -[ ] ideally we can check out child proc - # tree to ensure that its alive (and - # actually using the REPL) before we cancel - # it's lock acquire by doing the below! - # -[ ] create a way to read the tree of each actor's - # grandchildren such that when an - # intermediary parent is cancelled but their - # child has locked the tty, the grandparent - # will not allow the parent to cancel or - # zombie reap the child! see open issue: - # - https://github.com/goodboy/tractor/issues/320 - # ------ - ------ - # if a now stale local task has the TTY lock still - # we cancel it to allow servicing other requests for - # the lock. - if ( - (db_cs := pdb_lock.get_locking_task_cs()) - and not db_cs.cancel_called - and uid == pdb_user_uid - ): - log.critical( - f'STALE DEBUG LOCK DETECTED FOR {uid}' - ) - # TODO: figure out why this breaks tests.. - db_cs.cancel() - - log.runtime(con_teardown_status) - # finally block closure - # TODO: rename to `._deliver_payload()` since this handles # more then just `result` msgs now obvi XD async def _deliver_ctx_payload( @@ -1916,7 +1483,7 @@ async def async_main( if actor._parent_chan: await root_nursery.start( partial( - process_messages, + _rpc.process_messages, actor, actor._parent_chan, shield=True, @@ -1959,7 +1526,7 @@ async def async_main( log.exception(err_report) if actor._parent_chan: - await try_ship_error_to_remote( + await _rpc.try_ship_error_to_remote( actor._parent_chan, internal_err, ) diff --git a/tractor/ipc/_server.py b/tractor/ipc/_server.py index f23cf697..7d2ab217 100644 --- a/tractor/ipc/_server.py +++ b/tractor/ipc/_server.py @@ -24,6 +24,7 @@ from contextlib import ( ) from functools import partial import inspect +from pprint import pformat from types import ( ModuleType, ) @@ -40,24 +41,484 @@ from trio import ( SocketListener, ) -from ..msg import Struct +from ..devx import _debug +from .._exceptions import ( + TransportClosed, +) +from .. import _rpc +from ..msg import ( + MsgType, + Struct, + types as msgtypes, +) from ..trionics import maybe_open_nursery from .. import ( _state, log, ) from .._addr import Address +from ._chan import Channel from ._transport import MsgTransport from ._uds import UDSAddress from ._tcp import TCPAddress if TYPE_CHECKING: from .._runtime import Actor + from .._supervise import ActorNursery log = log.get_logger(__name__) +# TODO multi-tpt support with per-proto peer tracking? +# +# -[x] maybe change to mod-func and rename for implied +# multi-transport semantics? +# +# -[ ] register each stream/tpt/chan with the owning `IPCEndpoint` +# so that we can query per tpt all peer contact infos? +# |_[ ] possibly provide a global viewing via a +# `collections.ChainMap`? +# +async def handle_stream_from_peer( + stream: trio.SocketStream, + actor: Actor, + +) -> None: + ''' + Top-level `trio.abc.Stream` (i.e. normally `trio.SocketStream`) + handler-callback as spawn-invoked by `trio.serve_listeners()`. + + Note that each call to this handler is as a spawned task inside + any `IPCServer.listen_on()` passed `stream_handler_tn: Nursery` + such that it is invoked as, + + IPCEndpoint.stream_handler_tn.start_soon( + handle_stream, + stream, + ) + + ''' + actor._no_more_peers = trio.Event() # unset by making new + + # TODO, debug_mode tooling for when hackin this lower layer? + # with _debug.maybe_open_crash_handler( + # pdb=True, + # ) as boxerr: + + chan = Channel.from_stream(stream) + con_status: str = ( + 'New inbound IPC connection <=\n' + f'|_{chan}\n' + ) + + # initial handshake with peer phase + try: + peer_aid: msgtypes.Aid = await chan._do_handshake( + aid=actor.aid, + ) + except ( + TransportClosed, + # ^XXX NOTE, the above wraps `trio` exc types raised + # during various `SocketStream.send/receive_xx()` calls + # under different fault conditions such as, + # + # trio.BrokenResourceError, + # trio.ClosedResourceError, + # + # Inside our `.ipc._transport` layer we absorb and + # re-raise our own `TransportClosed` exc such that this + # higher level runtime code can only worry one + # "kinda-error" that we expect to tolerate during + # discovery-sys related pings, queires, DoS etc. + ): + # XXX: This may propagate up from `Channel._aiter_recv()` + # and `MsgpackStream._inter_packets()` on a read from the + # stream particularly when the runtime is first starting up + # inside `open_root_actor()` where there is a check for + # a bound listener on the "arbiter" addr. the reset will be + # because the handshake was never meant took place. + log.runtime( + con_status + + + ' -> But failed to handshake? Ignoring..\n' + ) + return + + uid: tuple[str, str] = ( + peer_aid.name, + peer_aid.uuid, + ) + # TODO, can we make this downstream peer tracking use the + # `peer_aid` instead? + familiar: str = 'new-peer' + if _pre_chan := actor._peers.get(uid): + familiar: str = 'pre-existing-peer' + uid_short: str = f'{uid[0]}[{uid[1][-6:]}]' + con_status += ( + f' -> Handshake with {familiar} `{uid_short}` complete\n' + ) + + if _pre_chan: + # con_status += ( + # ^TODO^ swap once we minimize conn duplication + # -[ ] last thing might be reg/unreg runtime reqs? + # log.warning( + log.debug( + f'?Wait?\n' + f'We already have IPC with peer {uid_short!r}\n' + f'|_{_pre_chan}\n' + ) + + # IPC connection tracking for both peers and new children: + # - if this is a new channel to a locally spawned + # sub-actor there will be a spawn wait even registered + # by a call to `.wait_for_peer()`. + # - if a peer is connecting no such event will exit. + event: trio.Event|None = actor._peer_connected.pop( + uid, + None, + ) + if event: + con_status += ( + ' -> Waking subactor spawn waiters: ' + f'{event.statistics().tasks_waiting}\n' + f' -> Registered IPC chan for child actor {uid}@{chan.raddr}\n' + # f' {event}\n' + # f' |{event.statistics()}\n' + ) + # wake tasks waiting on this IPC-transport "connect-back" + event.set() + + else: + con_status += ( + f' -> Registered IPC chan for peer actor {uid}@{chan.raddr}\n' + ) # type: ignore + + chans: list[Channel] = actor._peers[uid] + # if chans: + # # TODO: re-use channels for new connections instead + # # of always new ones? + # # => will require changing all the discovery funcs.. + + # append new channel + # TODO: can we just use list-ref directly? + chans.append(chan) + + con_status += ' -> Entering RPC msg loop..\n' + log.runtime(con_status) + + # Begin channel management - respond to remote requests and + # process received reponses. + disconnected: bool = False + last_msg: MsgType + try: + ( + disconnected, + last_msg, + ) = await _rpc.process_messages( + actor, + chan, + ) + except trio.Cancelled: + log.cancel( + 'IPC transport msg loop was cancelled\n' + f'c)>\n' + f' |_{chan}\n' + ) + raise + + finally: + local_nursery: ( + ActorNursery|None + ) = actor._actoruid2nursery.get(uid) + + # This is set in ``Portal.cancel_actor()``. So if + # the peer was cancelled we try to wait for them + # to tear down their side of the connection before + # moving on with closing our own side. + if ( + local_nursery + and ( + actor._cancel_called + or + chan._cancel_called + ) + # + # ^-TODO-^ along with this is there another condition + # that we should filter with to avoid entering this + # waiting block needlessly? + # -[ ] maybe `and local_nursery.cancelled` and/or + # only if the `._children` table is empty or has + # only `Portal`s with .chan._cancel_called == + # True` as per what we had below; the MAIN DIFF + # BEING that just bc one `Portal.cancel_actor()` + # was called, doesn't mean the whole actor-nurse + # is gonna exit any time soon right!? + # + # or + # all(chan._cancel_called for chan in chans) + + ): + log.cancel( + 'Waiting on cancel request to peer..\n' + f'c)=>\n' + f' |_{chan.uid}\n' + ) + + # XXX: this is a soft wait on the channel (and its + # underlying transport protocol) to close from the + # remote peer side since we presume that any channel + # which is mapped to a sub-actor (i.e. it's managed + # by local actor-nursery) has a message that is sent + # to the peer likely by this actor (which may be in + # a shutdown sequence due to cancellation) when the + # local runtime here is now cancelled while + # (presumably) in the middle of msg loop processing. + chan_info: str = ( + f'{chan.uid}\n' + f'|_{chan}\n' + f' |_{chan.transport}\n\n' + ) + with trio.move_on_after(0.5) as drain_cs: + drain_cs.shield = True + + # attempt to wait for the far end to close the + # channel and bail after timeout (a 2-generals + # problem on closure). + assert chan.transport + async for msg in chan.transport.drain(): + + # try to deliver any lingering msgs + # before we destroy the channel. + # This accomplishes deterministic + # ``Portal.cancel_actor()`` cancellation by + # making sure any RPC response to that call is + # delivered the local calling task. + # TODO: factor this into a helper? + log.warning( + 'Draining msg from disconnected peer\n' + f'{chan_info}' + f'{pformat(msg)}\n' + ) + # cid: str|None = msg.get('cid') + cid: str|None = msg.cid + if cid: + # deliver response to local caller/waiter + await actor._deliver_ctx_payload( + chan, + cid, + msg, + ) + if drain_cs.cancelled_caught: + log.warning( + 'Timed out waiting on IPC transport channel to drain?\n' + f'{chan_info}' + ) + + # XXX NOTE XXX when no explicit call to + # `open_root_actor()` was made by the application + # (normally we implicitly make that call inside + # the first `.open_nursery()` in root-actor + # user/app code), we can assume that either we + # are NOT the root actor or are root but the + # runtime was started manually. and thus DO have + # to wait for the nursery-enterer to exit before + # shutting down the local runtime to avoid + # clobbering any ongoing subactor + # teardown/debugging/graceful-cancel. + # + # see matching note inside `._supervise.open_nursery()` + # + # TODO: should we have a separate cs + timeout + # block here? + if ( + # XXX SO either, + # - not root OR, + # - is root but `open_root_actor()` was + # entered manually (in which case we do + # the equiv wait there using the + # `devx._debug` sub-sys APIs). + not local_nursery._implicit_runtime_started + ): + log.runtime( + 'Waiting on local actor nursery to exit..\n' + f'|_{local_nursery}\n' + ) + with trio.move_on_after(0.5) as an_exit_cs: + an_exit_cs.shield = True + await local_nursery.exited.wait() + + # TODO: currently this is always triggering for every + # sub-daemon spawned from the `piker.services._mngr`? + # -[ ] how do we ensure that the IPC is supposed to + # be long lived and isn't just a register? + # |_ in the register case how can we signal that the + # ephemeral msg loop was intentional? + if ( + # not local_nursery._implicit_runtime_started + # and + an_exit_cs.cancelled_caught + ): + report: str = ( + 'Timed out waiting on local actor-nursery to exit?\n' + f'c)>\n' + f' |_{local_nursery}\n' + ) + if children := local_nursery._children: + # indent from above local-nurse repr + report += ( + f' |_{pformat(children)}\n' + ) + + log.warning(report) + + if disconnected: + # if the transport died and this actor is still + # registered within a local nursery, we report + # that the IPC layer may have failed + # unexpectedly since it may be the cause of + # other downstream errors. + entry: tuple|None = local_nursery._children.get(uid) + if entry: + proc: trio.Process + _, proc, _ = entry + + if ( + (poll := getattr(proc, 'poll', None)) + and + poll() is None # proc still alive + ): + # 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'<=x {chan.uid}@{chan.raddr}\n' + f' |_{proc}\n' + ) + + # ``Channel`` teardown and closure sequence + # drop ref to channel so it can be gc-ed and disconnected + con_teardown_status: str = ( + f'IPC channel disconnected:\n' + f'<=x uid: {chan.uid}\n' + f' |_{pformat(chan)}\n\n' + ) + chans.remove(chan) + + # TODO: do we need to be this pedantic? + if not chans: + con_teardown_status += ( + f'-> No more channels with {chan.uid}' + ) + actor._peers.pop(uid, None) + + peers_str: str = '' + for uid, chans in actor._peers.items(): + peers_str += ( + f'uid: {uid}\n' + ) + for i, chan in enumerate(chans): + peers_str += ( + f' |_[{i}] {pformat(chan)}\n' + ) + + con_teardown_status += ( + f'-> Remaining IPC {len(actor._peers)} peers: {peers_str}\n' + ) + + # No more channels to other actors (at all) registered + # as connected. + if not actor._peers: + con_teardown_status += ( + 'Signalling no more peer channel connections' + ) + actor._no_more_peers.set() + + # NOTE: block this actor from acquiring the + # debugger-TTY-lock since we have no way to know if we + # cancelled it and further there is no way to ensure the + # lock will be released if acquired due to having no + # more active IPC channels. + if _state.is_root_process(): + pdb_lock = _debug.Lock + pdb_lock._blocked.add(uid) + + # TODO: NEEEDS TO BE TESTED! + # actually, no idea if this ever even enters.. XD + # + # XXX => YES IT DOES, when i was testing ctl-c + # from broken debug TTY locking due to + # msg-spec races on application using RunVar... + if ( + (ctx_in_debug := pdb_lock.ctx_in_debug) + and + (pdb_user_uid := ctx_in_debug.chan.uid) + and + local_nursery + ): + entry: tuple|None = local_nursery._children.get( + tuple(pdb_user_uid) + ) + if entry: + proc: trio.Process + _, proc, _ = entry + + if ( + (poll := getattr(proc, 'poll', None)) + and poll() is None + ): + log.cancel( + 'Root actor reports no-more-peers, BUT\n' + 'a DISCONNECTED child still has the debug ' + 'lock!\n\n' + # f'root uid: {actor.uid}\n' + f'last disconnected child uid: {uid}\n' + f'locking child uid: {pdb_user_uid}\n' + ) + await _debug.maybe_wait_for_debugger( + child_in_debug=True + ) + + # TODO: just bc a child's transport dropped + # doesn't mean it's not still using the pdb + # REPL! so, + # -[ ] ideally we can check out child proc + # tree to ensure that its alive (and + # actually using the REPL) before we cancel + # it's lock acquire by doing the below! + # -[ ] create a way to read the tree of each actor's + # grandchildren such that when an + # intermediary parent is cancelled but their + # child has locked the tty, the grandparent + # will not allow the parent to cancel or + # zombie reap the child! see open issue: + # - https://github.com/goodboy/tractor/issues/320 + # ------ - ------ + # if a now stale local task has the TTY lock still + # we cancel it to allow servicing other requests for + # the lock. + if ( + (db_cs := pdb_lock.get_locking_task_cs()) + and not db_cs.cancel_called + and uid == pdb_user_uid + ): + log.critical( + f'STALE DEBUG LOCK DETECTED FOR {uid}' + ) + # TODO: figure out why this breaks tests.. + db_cs.cancel() + + log.runtime(con_teardown_status) + # finally block closure + + class IPCEndpoint(Struct): ''' An instance of an IPC "bound" address where the lifetime of the @@ -379,7 +840,10 @@ async def _serve_ipc_eps( _listeners: list[SocketListener] = await listen_tn.start( partial( trio.serve_listeners, - handler=actor._stream_handler, + handler=partial( + handle_stream_from_peer, + actor=actor, + ), listeners=listeners, # NOTE: configured such that new From f67b0639b8130d72f1d7d6b88141da75d88fd316 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 11 Apr 2025 16:55:03 -0400 Subject: [PATCH 58/74] Move peer-tracking attrs from `Actor` -> `IPCServer` Namely transferring the `Actor` peer-`Channel` tracking attrs, - `._peers` which maps the uids to client channels (with duplicates apparently..) - the `._peer_connected: dict[tuple[str, str], trio.Event]` child-peer syncing table mostly used by parent actors to wait on sub's to connect back during spawn. - the `._no_more_peers = trio.Event()` level triggered state signal. Further we move over with some minor reworks, - `.wait_for_peer()` verbatim (adjusting all dependants). - factor the no-more-peers shielded wait branch-block out of the end of `async_main()` into 2 new server meths, * `.has_peers()` with optional chan-connected checking flag. * `.wait_for_no_more_peers()` which *just* does the maybe-shielded `._no_more_peers.wait()` --- tests/test_spawning.py | 6 +- tractor/_discovery.py | 5 +- tractor/_runtime.py | 62 +++++----------- tractor/_spawn.py | 8 +- tractor/_supervise.py | 10 ++- tractor/devx/_debug.py | 9 ++- tractor/ipc/_server.py | 163 ++++++++++++++++++++++++++++++++--------- 7 files changed, 176 insertions(+), 87 deletions(-) diff --git a/tests/test_spawning.py b/tests/test_spawning.py index cf373ada..30e084d5 100644 --- a/tests/test_spawning.py +++ b/tests/test_spawning.py @@ -57,7 +57,11 @@ async def spawn( ) assert len(an._children) == 1 - assert portal.channel.uid in tractor.current_actor()._peers + assert ( + portal.channel.uid + in + tractor.current_actor().ipc_server._peers + ) # get result from child subactor result = await portal.result() diff --git a/tractor/_discovery.py b/tractor/_discovery.py index aa487b26..ebaae9e8 100644 --- a/tractor/_discovery.py +++ b/tractor/_discovery.py @@ -48,6 +48,7 @@ from ._state import ( if TYPE_CHECKING: from ._runtime import Actor + from .ipc._server import IPCServer log = get_logger(__name__) @@ -79,7 +80,7 @@ async def get_registry( ) else: # TODO: try to look pre-existing connection from - # `Actor._peers` and use it instead? + # `IPCServer._peers` and use it instead? async with ( _connect_chan(addr) as chan, open_portal(chan) as regstr_ptl, @@ -111,7 +112,7 @@ def get_peer_by_name( ) -> list[Channel]|None: # at least 1 ''' Scan for an existing connection (set) to a named actor - and return any channels from `Actor._peers`. + and return any channels from `IPCServer._peers: dict`. This is an optimization method over querying the registrar for the same info. diff --git a/tractor/_runtime.py b/tractor/_runtime.py index b21ed0ee..39ac3309 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -40,9 +40,7 @@ from __future__ import annotations from contextlib import ( ExitStack, ) -from collections import defaultdict from functools import partial -from itertools import chain import importlib import importlib.util import os @@ -76,6 +74,7 @@ from tractor.msg import ( ) from .ipc import ( Channel, + # IPCServer, # causes cycles atm.. _server, ) from ._addr import ( @@ -156,7 +155,6 @@ class Actor: _root_n: Nursery|None = None _service_n: Nursery|None = None - # XXX moving to IPCServer! _ipc_server: _server.IPCServer|None = None @property @@ -246,14 +244,6 @@ class Actor: # by the user (currently called the "arbiter") self._spawn_method: str = spawn_method - self._peers: defaultdict[ - str, # uaid - list[Channel], # IPC conns from peer - ] = defaultdict(list) - self._peer_connected: dict[tuple[str, str], trio.Event] = {} - self._no_more_peers = trio.Event() - self._no_more_peers.set() - # RPC state self._ongoing_rpc_tasks = trio.Event() self._ongoing_rpc_tasks.set() @@ -338,7 +328,12 @@ class Actor: parent_uid: tuple|None = None if rent_chan := self._parent_chan: parent_uid = rent_chan.uid - peers: list[tuple] = list(self._peer_connected) + + peers: list = [] + server: _server.IPCServer = self.ipc_server + if server: + peers: list[tuple] = list(server._peer_connected) + fmtstr: str = ( f' |_id: {self.aid!r}\n' # f" aid{ds}{self.aid!r}\n" @@ -394,25 +389,6 @@ class Actor: self._reg_addrs = addrs - async def wait_for_peer( - self, - uid: tuple[str, str], - - ) -> tuple[trio.Event, Channel]: - ''' - Wait for a connection back from a (spawned sub-)actor with - a `uid` using a `trio.Event` for sync. - - ''' - log.debug(f'Waiting for peer {uid!r} to connect') - event = self._peer_connected.setdefault(uid, trio.Event()) - await event.wait() - log.debug(f'{uid!r} successfully connected back to us') - return ( - event, - self._peers[uid][-1], - ) - def load_modules( self, # debug_mode: bool = False, @@ -724,7 +700,7 @@ class Actor: ) assert isinstance(chan, Channel) - # Initial handshake: swap names. + # init handshake: swap actor-IDs. await chan._do_handshake(aid=self.aid) accept_addrs: list[UnwrappedAddress]|None = None @@ -1620,16 +1596,18 @@ async def async_main( ) # Ensure all peers (actors connected to us as clients) are finished - if not actor._no_more_peers.is_set(): - if any( - chan.connected() for chan in chain(*actor._peers.values()) - ): - teardown_report += ( - f'-> Waiting for remaining peers {actor._peers} to clear..\n' - ) - log.runtime(teardown_report) - with CancelScope(shield=True): - await actor._no_more_peers.wait() + if ( + (ipc_server := actor.ipc_server) + and + ipc_server.has_peers(check_chans=True) + ): + teardown_report += ( + f'-> Waiting for remaining peers {ipc_server._peers} to clear..\n' + ) + log.runtime(teardown_report) + await ipc_server.wait_for_no_more_peers( + shield=True, + ) teardown_report += ( '-> All peer channels are complete\n' diff --git a/tractor/_spawn.py b/tractor/_spawn.py index 5df8125a..49d14805 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -58,9 +58,11 @@ from tractor.msg.types import ( if TYPE_CHECKING: + from ipc import IPCServer from ._supervise import ActorNursery ProcessType = TypeVar('ProcessType', mp.Process, trio.Process) + log = get_logger('tractor') # placeholder for an mp start context if so using that backend @@ -481,6 +483,7 @@ async def trio_proc( cancelled_during_spawn: bool = False proc: trio.Process|None = None + ipc_server: IPCServer = actor_nursery._actor.ipc_server try: try: proc: trio.Process = await trio.lowlevel.open_process(spawn_cmd, **proc_kwargs) @@ -492,7 +495,7 @@ async def trio_proc( # wait for actor to spawn and connect back to us # channel should have handshake completed by the # local actor by the time we get a ref to it - event, chan = await actor_nursery._actor.wait_for_peer( + event, chan = await ipc_server.wait_for_peer( subactor.uid ) @@ -724,11 +727,12 @@ async def mp_proc( log.runtime(f"Started {proc}") + ipc_server: IPCServer = actor_nursery._actor.ipc_server try: # wait for actor to spawn and connect back to us # channel should have handshake completed by the # local actor by the time we get a ref to it - event, chan = await actor_nursery._actor.wait_for_peer( + event, chan = await ipc_server.wait_for_peer( subactor.uid, ) diff --git a/tractor/_supervise.py b/tractor/_supervise.py index 07ae8bb3..cdf15e19 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -53,6 +53,9 @@ from . import _spawn if TYPE_CHECKING: import multiprocessing as mp + # from .ipc._server import IPCServer + from .ipc import IPCServer + log = get_logger(__name__) @@ -315,6 +318,9 @@ class ActorNursery: children: dict = self._children child_count: int = len(children) msg: str = f'Cancelling actor nursery with {child_count} children\n' + + server: IPCServer = self._actor.ipc_server + with trio.move_on_after(3) as cs: async with trio.open_nursery( strict_exception_groups=False, @@ -337,7 +343,7 @@ class ActorNursery: else: if portal is None: # actor hasn't fully spawned yet - event = self._actor._peer_connected[subactor.uid] + event: trio.Event = server._peer_connected[subactor.uid] log.warning( f"{subactor.uid} never 't finished spawning?" ) @@ -353,7 +359,7 @@ class ActorNursery: if portal is None: # cancelled while waiting on the event # to arrive - chan = self._actor._peers[subactor.uid][-1] + chan = server._peers[subactor.uid][-1] if chan: portal = Portal(chan) else: # there's no other choice left diff --git a/tractor/devx/_debug.py b/tractor/devx/_debug.py index 0f05534b..67695feb 100644 --- a/tractor/devx/_debug.py +++ b/tractor/devx/_debug.py @@ -92,7 +92,11 @@ from tractor._state import ( if TYPE_CHECKING: from trio.lowlevel import Task from threading import Thread - from tractor.ipc import Channel + from tractor.ipc import ( + Channel, + IPCServer, + # _server, # TODO? export at top level? + ) from tractor._runtime import ( Actor, ) @@ -1434,6 +1438,7 @@ def any_connected_locker_child() -> bool: ''' actor: Actor = current_actor() + server: IPCServer = actor.ipc_server if not is_root_process(): raise InternalError('This is a root-actor only API!') @@ -1443,7 +1448,7 @@ def any_connected_locker_child() -> bool: and (uid_in_debug := ctx.chan.uid) ): - chans: list[tractor.Channel] = actor._peers.get( + chans: list[tractor.Channel] = server._peers.get( tuple(uid_in_debug) ) if chans: diff --git a/tractor/ipc/_server.py b/tractor/ipc/_server.py index 7d2ab217..e9affccd 100644 --- a/tractor/ipc/_server.py +++ b/tractor/ipc/_server.py @@ -19,10 +19,12 @@ multi-transport-protcol needs! ''' from __future__ import annotations +from collections import defaultdict from contextlib import ( asynccontextmanager as acm, ) from functools import partial +from itertools import chain import inspect from pprint import pformat from types import ( @@ -41,7 +43,7 @@ from trio import ( SocketListener, ) -from ..devx import _debug +# from ..devx import _debug from .._exceptions import ( TransportClosed, ) @@ -82,6 +84,9 @@ log = log.get_logger(__name__) # async def handle_stream_from_peer( stream: trio.SocketStream, + + *, + server: IPCServer, actor: Actor, ) -> None: @@ -99,7 +104,7 @@ async def handle_stream_from_peer( ) ''' - actor._no_more_peers = trio.Event() # unset by making new + server._no_more_peers = trio.Event() # unset by making new # TODO, debug_mode tooling for when hackin this lower layer? # with _debug.maybe_open_crash_handler( @@ -152,7 +157,7 @@ async def handle_stream_from_peer( # TODO, can we make this downstream peer tracking use the # `peer_aid` instead? familiar: str = 'new-peer' - if _pre_chan := actor._peers.get(uid): + if _pre_chan := server._peers.get(uid): familiar: str = 'pre-existing-peer' uid_short: str = f'{uid[0]}[{uid[1][-6:]}]' con_status += ( @@ -175,7 +180,7 @@ async def handle_stream_from_peer( # sub-actor there will be a spawn wait even registered # by a call to `.wait_for_peer()`. # - if a peer is connecting no such event will exit. - event: trio.Event|None = actor._peer_connected.pop( + event: trio.Event|None = server._peer_connected.pop( uid, None, ) @@ -195,7 +200,7 @@ async def handle_stream_from_peer( f' -> Registered IPC chan for peer actor {uid}@{chan.raddr}\n' ) # type: ignore - chans: list[Channel] = actor._peers[uid] + chans: list[Channel] = server._peers[uid] # if chans: # # TODO: re-use channels for new connections instead # # of always new ones? @@ -417,10 +422,10 @@ async def handle_stream_from_peer( con_teardown_status += ( f'-> No more channels with {chan.uid}' ) - actor._peers.pop(uid, None) + server._peers.pop(uid, None) peers_str: str = '' - for uid, chans in actor._peers.items(): + for uid, chans in server._peers.items(): peers_str += ( f'uid: {uid}\n' ) @@ -430,23 +435,28 @@ async def handle_stream_from_peer( ) con_teardown_status += ( - f'-> Remaining IPC {len(actor._peers)} peers: {peers_str}\n' + f'-> Remaining IPC {len(server._peers)} peers: {peers_str}\n' ) # No more channels to other actors (at all) registered # as connected. - if not actor._peers: + if not server._peers: con_teardown_status += ( 'Signalling no more peer channel connections' ) - actor._no_more_peers.set() + server._no_more_peers.set() # NOTE: block this actor from acquiring the # debugger-TTY-lock since we have no way to know if we # cancelled it and further there is no way to ensure the # lock will be released if acquired due to having no # more active IPC channels. - if _state.is_root_process(): + if ( + _state.is_root_process() + and + _state.is_debug_mode() + ): + from ..devx import _debug pdb_lock = _debug.Lock pdb_lock._blocked.add(uid) @@ -581,8 +591,23 @@ class IPCEndpoint(Struct): class IPCServer(Struct): _parent_tn: Nursery _stream_handler_tn: Nursery + # level-triggered sig for whether "no peers are currently + # connected"; field is **always** set to an instance but + # initialized with `.is_set() == True`. + _no_more_peers: trio.Event + _endpoints: list[IPCEndpoint] = [] + # connection tracking & mgmt + _peers: defaultdict[ + str, # uaid + list[Channel], # IPC conns from peer + ] = defaultdict(list) + _peer_connected: dict[ + tuple[str, str], + trio.Event, + ] = {} + # syncs for setup/teardown sequences _shutdown: trio.Event|None = None @@ -644,6 +669,65 @@ class IPCServer(Struct): f'protos: {tpt_protos!r}\n' ) + def has_peers( + self, + check_chans: bool = False, + ) -> bool: + ''' + Predicate for "are there any active peer IPC `Channel`s at the moment?" + + ''' + has_peers: bool = not self._no_more_peers.is_set() + if ( + has_peers + and + check_chans + ): + has_peers: bool = ( + any(chan.connected() + for chan in chain( + *self._peers.values() + ) + ) + and + has_peers + ) + + return has_peers + + async def wait_for_no_more_peers( + self, + shield: bool = False, + ) -> None: + with trio.CancelScope(shield=shield): + await self._no_more_peers.wait() + + async def wait_for_peer( + self, + uid: tuple[str, str], + + ) -> tuple[trio.Event, Channel]: + ''' + Wait for a connection back from a (spawned sub-)actor with + a `uid` using a `trio.Event`. + + Returns a pair of the event and the "last" registered IPC + `Channel` for the peer with `uid`. + + ''' + log.debug(f'Waiting for peer {uid!r} to connect') + event: trio.Event = self._peer_connected.setdefault( + uid, + trio.Event(), + ) + await event.wait() + log.debug(f'{uid!r} successfully connected back to us') + mru_chan: Channel = self._peers[uid][-1] + return ( + event, + mru_chan, + ) + @property def addrs(self) -> list[Address]: return [ep.addr for ep in self._endpoints] @@ -672,17 +756,27 @@ class IPCServer(Struct): return ev.is_set() def pformat(self) -> str: + eps: list[IPCEndpoint] = self._endpoints - fmtstr: str = ( - f' |_endpoints: {self._endpoints}\n' + state_repr: str = ( + f'{len(eps)!r} IPC-endpoints active' + ) + fmtstr = ( + f' |_state: {state_repr}\n' + f' no_more_peers: {self.has_peers()}\n' ) if self._shutdown is not None: shutdown_stats: EventStatistics = self._shutdown.statistics() fmtstr += ( - f'\n' - f' |_shutdown: {shutdown_stats}\n' + f' task_waiting_on_shutdown: {shutdown_stats}\n' ) + fmtstr += ( + # TODO, use the `ppfmt()` helper from `modden`! + f' |_endpoints: {pformat(self._endpoints)}\n' + f' |_peers: {len(self._peers)} connected\n' + ) + return ( f' Date: Wed, 11 Jun 2025 11:29:23 -0400 Subject: [PATCH 59/74] Log listener bind status for TCP as for UDS --- tractor/ipc/_tcp.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/tractor/ipc/_tcp.py b/tractor/ipc/_tcp.py index b534b143..4de04c25 100644 --- a/tractor/ipc/_tcp.py +++ b/tractor/ipc/_tcp.py @@ -127,6 +127,11 @@ async def start_listener( Start a TCP socket listener on the given `TCPAddress`. ''' + log.info( + f'Attempting to bind TCP socket\n' + f'>[\n' + f'|_{addr}\n' + ) # ?TODO, maybe we should just change the lower-level call this is # using internall per-listener? listeners: list[SocketListener] = await open_tcp_listeners( @@ -140,6 +145,12 @@ async def start_listener( assert len(listeners) == 1 listener = listeners[0] host, port = listener.socket.getsockname()[:2] + + log.info( + f'Listening on TCP socket\n' + f'[>\n' + f' |_{addr}\n' + ) return listener From 8538a9c591af97edd4e46ff96817ba617d27e7e7 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 11 Jun 2025 16:44:47 -0400 Subject: [PATCH 60/74] Decouple actor-state from low-level ipc-server As much as is possible given we currently do some graceful cancellation join-waiting on any connected sub-actors whenever an active `local_nursery: AcrtorNursery` in the post-rpc teardown sequence of `handle_stream_from_peer()` is detected. In such cases we try to allow the higher level inter-actor (task) context(s) to fully cancelled-ack before conducting IPC machinery shutdown. The main immediate motivation for all this is to support unit testing the `.ipc._server` APIs but in the future may be useful for anyone wanting to use our modular IPC transport layer sin-"actors". Impl deats, - drop passing an `actor: Actor` ref from as many routines in `.ipc._server` as possible instead opting to use `._state.current_actor()` where abs needed; thus the fns dropping an `actor` input param are: - `open_ipc_server()` - `IPCServer.listen_on()` - `._serve_ipc_eps()` - `.handle_stream_from_peer()` - factor the above mentioned graceful remote-cancel-ack waiting into a new `maybe_wait_on_canced_subs()` which is called from `handle_stream_from_peer()` and delivers a maybe-`local_nursery: ActorNursery` for downstream logic; it's this new fn which primarily still needs to call `current_actor()`. - in `handle_stream_from_peer()` also use `current_actor()` to check if a handshake is needed (or if it was called as part of some actor-runtime-less operation like our unit test suite!). - also don't pass an `actor` to `._rpc.process_messages()` see how-n-why below.. Surrounding ipc-server client/caller adjustments, - `._rpc.process_messages()` no longer takes an `actor` input and now calls `current_actor()` instead. - `._portal.open_portal()` is adjusted to ^. - `._runtime.async_main()` is adjusted to the `.ipc._server`'s removal of `actor` ref passing. Also, - drop some server `log.info()`s to `.runtime()` --- tractor/_portal.py | 3 +- tractor/_rpc.py | 2 +- tractor/_runtime.py | 9 +- tractor/ipc/_server.py | 427 +++++++++++++++++++++++------------------ 4 files changed, 243 insertions(+), 198 deletions(-) diff --git a/tractor/_portal.py b/tractor/_portal.py index d5dd8369..c741df7d 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -582,8 +582,7 @@ async def open_portal( msg_loop_cs = await tn.start( partial( _rpc.process_messages, - actor, - channel, + chan=channel, # if the local task is cancelled we want to keep # the msg loop running until our block ends shield=True, diff --git a/tractor/_rpc.py b/tractor/_rpc.py index 3fc11db7..c9f4312e 100644 --- a/tractor/_rpc.py +++ b/tractor/_rpc.py @@ -869,7 +869,6 @@ async def try_ship_error_to_remote( async def process_messages( - actor: Actor, chan: Channel, shield: bool = False, task_status: TaskStatus[CancelScope] = trio.TASK_STATUS_IGNORED, @@ -907,6 +906,7 @@ async def process_messages( (as utilized inside `Portal.cancel_actor()` ). ''' + actor: Actor = _state.current_actor() assert actor._service_n # runtime state sanity # TODO: once `trio` get's an "obvious way" for req/resp we diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 39ac3309..0097e224 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -1262,6 +1262,10 @@ async def async_main( the actor's "runtime" and all thus all ongoing RPC tasks. ''' + # XXX NOTE, `_state._current_actor` **must** be set prior to + # calling this core runtime entrypoint! + assert actor is _state.current_actor() + actor._task: trio.Task = trio.lowlevel.current_task() # attempt to retreive ``trio``'s sigint handler and stash it @@ -1321,7 +1325,6 @@ async def async_main( ) as service_nursery, _server.open_ipc_server( - actor=actor, parent_tn=service_nursery, stream_handler_tn=service_nursery, ) as ipc_server, @@ -1375,7 +1378,6 @@ async def async_main( 'Booting IPC server' ) eps: list = await ipc_server.listen_on( - actor=actor, accept_addrs=accept_addrs, stream_handler_nursery=service_nursery, ) @@ -1460,8 +1462,7 @@ async def async_main( await root_nursery.start( partial( _rpc.process_messages, - actor, - actor._parent_chan, + chan=actor._parent_chan, shield=True, ) ) diff --git a/tractor/ipc/_server.py b/tractor/ipc/_server.py index e9affccd..7df67aac 100644 --- a/tractor/ipc/_server.py +++ b/tractor/ipc/_server.py @@ -72,11 +72,223 @@ if TYPE_CHECKING: log = log.get_logger(__name__) +async def maybe_wait_on_canced_subs( + uid: tuple[str, str], + chan: Channel, + disconnected: bool, + + actor: Actor|None = None, + chan_drain_timeout: float = 0.5, + an_exit_timeout: float = 0.5, + +) -> ActorNursery|None: + ''' + When a process-local actor-nursery is found for the given actor + `uid` (i.e. that peer is **also** a subactor of this parent), we + attempt to (with timeouts) wait on, + + - all IPC msgs to drain on the (common) `Channel` such that all + local `Context`-parent-tasks can also gracefully collect + `ContextCancelled` msgs from their respective remote children + vs. a `chan_drain_timeout`. + + - the actor-nursery to cancel-n-join all its supervised children + (processes) *gracefully* vs. a `an_exit_timeout` and thus also + detect cases where the IPC transport connection broke but + a sub-process is detected as still alive (a case that happens + when the subactor is still in an active debugger REPL session). + + If the timeout expires in either case we ofc report with warning. + + ''' + actor = actor or _state.current_actor() + + # XXX running outside actor-runtime usage, + # - unit testing + # - possibly manual usage (eventually) ? + if not actor: + return None + + local_nursery: ( + ActorNursery|None + ) = actor._actoruid2nursery.get(uid) + + # This is set in `Portal.cancel_actor()`. So if + # the peer was cancelled we try to wait for them + # to tear down their side of the connection before + # moving on with closing our own side. + if ( + local_nursery + and ( + actor._cancel_called + or + chan._cancel_called + ) + # + # ^-TODO-^ along with this is there another condition + # that we should filter with to avoid entering this + # waiting block needlessly? + # -[ ] maybe `and local_nursery.cancelled` and/or + # only if the `._children` table is empty or has + # only `Portal`s with .chan._cancel_called == + # True` as per what we had below; the MAIN DIFF + # BEING that just bc one `Portal.cancel_actor()` + # was called, doesn't mean the whole actor-nurse + # is gonna exit any time soon right!? + # + # or + # all(chan._cancel_called for chan in chans) + + ): + log.cancel( + 'Waiting on cancel request to peer..\n' + f'c)=>\n' + f' |_{chan.uid}\n' + ) + + # XXX: this is a soft wait on the channel (and its + # underlying transport protocol) to close from the + # remote peer side since we presume that any channel + # which is mapped to a sub-actor (i.e. it's managed + # by local actor-nursery) has a message that is sent + # to the peer likely by this actor (which may be in + # a shutdown sequence due to cancellation) when the + # local runtime here is now cancelled while + # (presumably) in the middle of msg loop processing. + chan_info: str = ( + f'{chan.uid}\n' + f'|_{chan}\n' + f' |_{chan.transport}\n\n' + ) + with trio.move_on_after(chan_drain_timeout) as drain_cs: + drain_cs.shield = True + + # attempt to wait for the far end to close the + # channel and bail after timeout (a 2-generals + # problem on closure). + assert chan.transport + async for msg in chan.transport.drain(): + + # try to deliver any lingering msgs + # before we destroy the channel. + # This accomplishes deterministic + # ``Portal.cancel_actor()`` cancellation by + # making sure any RPC response to that call is + # delivered the local calling task. + # TODO: factor this into a helper? + log.warning( + 'Draining msg from disconnected peer\n' + f'{chan_info}' + f'{pformat(msg)}\n' + ) + # cid: str|None = msg.get('cid') + cid: str|None = msg.cid + if cid: + # deliver response to local caller/waiter + await actor._deliver_ctx_payload( + chan, + cid, + msg, + ) + if drain_cs.cancelled_caught: + log.warning( + 'Timed out waiting on IPC transport channel to drain?\n' + f'{chan_info}' + ) + + # XXX NOTE XXX when no explicit call to + # `open_root_actor()` was made by the application + # (normally we implicitly make that call inside + # the first `.open_nursery()` in root-actor + # user/app code), we can assume that either we + # are NOT the root actor or are root but the + # runtime was started manually. and thus DO have + # to wait for the nursery-enterer to exit before + # shutting down the local runtime to avoid + # clobbering any ongoing subactor + # teardown/debugging/graceful-cancel. + # + # see matching note inside `._supervise.open_nursery()` + # + # TODO: should we have a separate cs + timeout + # block here? + if ( + # XXX SO either, + # - not root OR, + # - is root but `open_root_actor()` was + # entered manually (in which case we do + # the equiv wait there using the + # `devx.debug` sub-sys APIs). + not local_nursery._implicit_runtime_started + ): + log.runtime( + 'Waiting on local actor nursery to exit..\n' + f'|_{local_nursery}\n' + ) + with trio.move_on_after(an_exit_timeout) as an_exit_cs: + an_exit_cs.shield = True + await local_nursery.exited.wait() + + # TODO: currently this is always triggering for every + # sub-daemon spawned from the `piker.services._mngr`? + # -[ ] how do we ensure that the IPC is supposed to + # be long lived and isn't just a register? + # |_ in the register case how can we signal that the + # ephemeral msg loop was intentional? + if ( + # not local_nursery._implicit_runtime_started + # and + an_exit_cs.cancelled_caught + ): + report: str = ( + 'Timed out waiting on local actor-nursery to exit?\n' + f'c)>\n' + f' |_{local_nursery}\n' + ) + if children := local_nursery._children: + # indent from above local-nurse repr + report += ( + f' |_{pformat(children)}\n' + ) + + log.warning(report) + + if disconnected: + # if the transport died and this actor is still + # registered within a local nursery, we report + # that the IPC layer may have failed + # unexpectedly since it may be the cause of + # other downstream errors. + entry: tuple|None = local_nursery._children.get(uid) + if entry: + proc: trio.Process + _, proc, _ = entry + + if ( + (poll := getattr(proc, 'poll', None)) + and + poll() is None # proc still alive + ): + # 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'<=x {chan.uid}@{chan.raddr}\n' + f' |_{proc}\n' + ) + + return local_nursery + # TODO multi-tpt support with per-proto peer tracking? # # -[x] maybe change to mod-func and rename for implied # multi-transport semantics? -# # -[ ] register each stream/tpt/chan with the owning `IPCEndpoint` # so that we can query per tpt all peer contact infos? # |_[ ] possibly provide a global viewing via a @@ -87,7 +299,6 @@ async def handle_stream_from_peer( *, server: IPCServer, - actor: Actor, ) -> None: ''' @@ -119,9 +330,10 @@ async def handle_stream_from_peer( # initial handshake with peer phase try: - peer_aid: msgtypes.Aid = await chan._do_handshake( - aid=actor.aid, - ) + if actor := _state.current_actor(): + peer_aid: msgtypes.Aid = await chan._do_handshake( + aid=actor.aid, + ) except ( TransportClosed, # ^XXX NOTE, the above wraps `trio` exc types raised @@ -222,8 +434,7 @@ async def handle_stream_from_peer( disconnected, last_msg, ) = await _rpc.process_messages( - actor, - chan, + chan=chan, ) except trio.Cancelled: log.cancel( @@ -234,179 +445,16 @@ async def handle_stream_from_peer( raise finally: - local_nursery: ( - ActorNursery|None - ) = actor._actoruid2nursery.get(uid) - # This is set in ``Portal.cancel_actor()``. So if - # the peer was cancelled we try to wait for them - # to tear down their side of the connection before - # moving on with closing our own side. - if ( - local_nursery - and ( - actor._cancel_called - or - chan._cancel_called - ) - # - # ^-TODO-^ along with this is there another condition - # that we should filter with to avoid entering this - # waiting block needlessly? - # -[ ] maybe `and local_nursery.cancelled` and/or - # only if the `._children` table is empty or has - # only `Portal`s with .chan._cancel_called == - # True` as per what we had below; the MAIN DIFF - # BEING that just bc one `Portal.cancel_actor()` - # was called, doesn't mean the whole actor-nurse - # is gonna exit any time soon right!? - # - # or - # all(chan._cancel_called for chan in chans) - - ): - log.cancel( - 'Waiting on cancel request to peer..\n' - f'c)=>\n' - f' |_{chan.uid}\n' - ) - - # XXX: this is a soft wait on the channel (and its - # underlying transport protocol) to close from the - # remote peer side since we presume that any channel - # which is mapped to a sub-actor (i.e. it's managed - # by local actor-nursery) has a message that is sent - # to the peer likely by this actor (which may be in - # a shutdown sequence due to cancellation) when the - # local runtime here is now cancelled while - # (presumably) in the middle of msg loop processing. - chan_info: str = ( - f'{chan.uid}\n' - f'|_{chan}\n' - f' |_{chan.transport}\n\n' - ) - with trio.move_on_after(0.5) as drain_cs: - drain_cs.shield = True - - # attempt to wait for the far end to close the - # channel and bail after timeout (a 2-generals - # problem on closure). - assert chan.transport - async for msg in chan.transport.drain(): - - # try to deliver any lingering msgs - # before we destroy the channel. - # This accomplishes deterministic - # ``Portal.cancel_actor()`` cancellation by - # making sure any RPC response to that call is - # delivered the local calling task. - # TODO: factor this into a helper? - log.warning( - 'Draining msg from disconnected peer\n' - f'{chan_info}' - f'{pformat(msg)}\n' - ) - # cid: str|None = msg.get('cid') - cid: str|None = msg.cid - if cid: - # deliver response to local caller/waiter - await actor._deliver_ctx_payload( - chan, - cid, - msg, - ) - if drain_cs.cancelled_caught: - log.warning( - 'Timed out waiting on IPC transport channel to drain?\n' - f'{chan_info}' - ) - - # XXX NOTE XXX when no explicit call to - # `open_root_actor()` was made by the application - # (normally we implicitly make that call inside - # the first `.open_nursery()` in root-actor - # user/app code), we can assume that either we - # are NOT the root actor or are root but the - # runtime was started manually. and thus DO have - # to wait for the nursery-enterer to exit before - # shutting down the local runtime to avoid - # clobbering any ongoing subactor - # teardown/debugging/graceful-cancel. - # - # see matching note inside `._supervise.open_nursery()` - # - # TODO: should we have a separate cs + timeout - # block here? - if ( - # XXX SO either, - # - not root OR, - # - is root but `open_root_actor()` was - # entered manually (in which case we do - # the equiv wait there using the - # `devx._debug` sub-sys APIs). - not local_nursery._implicit_runtime_started - ): - log.runtime( - 'Waiting on local actor nursery to exit..\n' - f'|_{local_nursery}\n' - ) - with trio.move_on_after(0.5) as an_exit_cs: - an_exit_cs.shield = True - await local_nursery.exited.wait() - - # TODO: currently this is always triggering for every - # sub-daemon spawned from the `piker.services._mngr`? - # -[ ] how do we ensure that the IPC is supposed to - # be long lived and isn't just a register? - # |_ in the register case how can we signal that the - # ephemeral msg loop was intentional? - if ( - # not local_nursery._implicit_runtime_started - # and - an_exit_cs.cancelled_caught - ): - report: str = ( - 'Timed out waiting on local actor-nursery to exit?\n' - f'c)>\n' - f' |_{local_nursery}\n' - ) - if children := local_nursery._children: - # indent from above local-nurse repr - report += ( - f' |_{pformat(children)}\n' - ) - - log.warning(report) - - if disconnected: - # if the transport died and this actor is still - # registered within a local nursery, we report - # that the IPC layer may have failed - # unexpectedly since it may be the cause of - # other downstream errors. - entry: tuple|None = local_nursery._children.get(uid) - if entry: - proc: trio.Process - _, proc, _ = entry - - if ( - (poll := getattr(proc, 'poll', None)) - and - poll() is None # proc still alive - ): - # 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'<=x {chan.uid}@{chan.raddr}\n' - f' |_{proc}\n' - ) + # check if there are subs which we should gracefully join at + # both the inter-actor-task and subprocess levels to + # gracefully remote cancel and later disconnect (particularly + # for permitting subs engaged in active debug-REPL sessions). + local_nursery: ActorNursery|None = await maybe_wait_on_canced_subs( + uid=uid, + chan=chan, + disconnected=disconnected, + ) # ``Channel`` teardown and closure sequence # drop ref to channel so it can be gc-ed and disconnected @@ -467,11 +515,11 @@ async def handle_stream_from_peer( # from broken debug TTY locking due to # msg-spec races on application using RunVar... if ( + local_nursery + and (ctx_in_debug := pdb_lock.ctx_in_debug) and (pdb_user_uid := ctx_in_debug.chan.uid) - and - local_nursery ): entry: tuple|None = local_nursery._children.get( tuple(pdb_user_uid) @@ -804,7 +852,6 @@ class IPCServer(Struct): async def listen_on( self, *, - actor: Actor, accept_addrs: list[tuple[str, int|str]]|None = None, stream_handler_nursery: Nursery|None = None, ) -> list[IPCEndpoint]: @@ -837,20 +884,19 @@ class IPCServer(Struct): f'{self}\n' ) - log.info( + log.runtime( f'Binding to endpoints for,\n' f'{accept_addrs}\n' ) eps: list[IPCEndpoint] = await self._parent_tn.start( partial( _serve_ipc_eps, - actor=actor, server=self, stream_handler_tn=stream_handler_nursery, listen_addrs=accept_addrs, ) ) - log.info( + log.runtime( f'Started IPC endpoints\n' f'{eps}\n' ) @@ -873,7 +919,6 @@ class IPCServer(Struct): async def _serve_ipc_eps( *, - actor: Actor, server: IPCServer, stream_handler_tn: Nursery, listen_addrs: list[tuple[str, int|str]], @@ -907,12 +952,13 @@ async def _serve_ipc_eps( stream_handler_tn=stream_handler_tn, ) try: - log.info( + log.runtime( f'Starting new endpoint listener\n' f'{ep}\n' ) listener: trio.abc.Listener = await ep.start_listener() assert listener is ep._listener + # actor = _state.current_actor() # if actor.is_registry: # import pdbp; pdbp.set_trace() @@ -937,7 +983,6 @@ async def _serve_ipc_eps( handler=partial( handle_stream_from_peer, server=server, - actor=actor, ), listeners=listeners, @@ -948,13 +993,13 @@ async def _serve_ipc_eps( ) ) # TODO, wow make this message better! XD - log.info( + log.runtime( 'Started server(s)\n' + '\n'.join([f'|_{addr}' for addr in listen_addrs]) ) - log.info( + log.runtime( f'Started IPC endpoints\n' f'{eps}\n' ) @@ -970,6 +1015,7 @@ async def _serve_ipc_eps( ep.close_listener() server._endpoints.remove(ep) + # actor = _state.current_actor() # if actor.is_arbiter: # import pdbp; pdbp.set_trace() @@ -980,7 +1026,6 @@ async def _serve_ipc_eps( @acm async def open_ipc_server( - actor: Actor, parent_tn: Nursery|None = None, stream_handler_tn: Nursery|None = None, From 1f3cef5ed6115b3209ef5a111111463c8ec3b0b3 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 9 Jul 2025 21:27:32 -0400 Subject: [PATCH 61/74] Fix now invalid `Actor._peers` ref.. --- tractor/_discovery.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tractor/_discovery.py b/tractor/_discovery.py index ebaae9e8..f6b3b585 100644 --- a/tractor/_discovery.py +++ b/tractor/_discovery.py @@ -48,7 +48,6 @@ from ._state import ( if TYPE_CHECKING: from ._runtime import Actor - from .ipc._server import IPCServer log = get_logger(__name__) @@ -80,7 +79,7 @@ async def get_registry( ) else: # TODO: try to look pre-existing connection from - # `IPCServer._peers` and use it instead? + # `Server._peers` and use it instead? async with ( _connect_chan(addr) as chan, open_portal(chan) as regstr_ptl, @@ -112,14 +111,14 @@ def get_peer_by_name( ) -> list[Channel]|None: # at least 1 ''' Scan for an existing connection (set) to a named actor - and return any channels from `IPCServer._peers: dict`. + and return any channels from `Server._peers: dict`. This is an optimization method over querying the registrar for the same info. ''' actor: Actor = current_actor() - to_scan: dict[tuple, list[Channel]] = actor._peers.copy() + to_scan: dict[tuple, list[Channel]] = actor.ipc_server._peers.copy() # TODO: is this ever needed? creates a duplicate channel on actor._peers # when multiple find_actor calls are made to same actor from a single ctx From 9c0de24899525e9a15c2292fc80ca473b5403bd0 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 13 May 2025 17:39:53 -0400 Subject: [PATCH 62/74] Be explicit with `SpawnSpec` processing in subs As per the outstanding TODO just above the redic `setattr()` loop in `Actor._from_parent()`!! Instead of all that risk-ay monkeying, add detailed comment-sections around each explicit assignment of each `SpawnSpec` field, including those that were already being explicitly set. Those and other deats, - ONLY enable the `.devx._debug` (CHERRY-CONFLICT: later changed to `.debug._tty_lock`) module from `Actor.__init__()` in the root actor. - ONLY enable the `.devx.debug._tty_lock` module from `Actor.__init__()` in the root actor. - add a new `get_mod_nsps2fps()` to replace the loop in init and assign the initial `.enable_modules: dict[str, str]` from it. - do `self.enable_modules.update(spawnspec.enable_modules)` instead of an overwrite and assert the table is by default empty in all subs. --- tractor/_runtime.py | 93 +++++++++++++++++++++++++++++++-------------- 1 file changed, 65 insertions(+), 28 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 0097e224..f5e363f0 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -115,6 +115,20 @@ def _get_mod_abspath(module): return os.path.abspath(module.__file__) +def get_mod_nsps2fps(mod_ns_paths: list[str]) -> dict[str, str]: + ''' + Deliver a table of py module namespace-path-`str`s mapped to + their "physical" `.py` file paths in the file-sys. + + ''' + nsp2fp: dict[str, str] = {} + for nsp in mod_ns_paths: + mod: ModuleType = importlib.import_module(nsp) + nsp2fp[nsp] = _get_mod_abspath(mod) + + return nsp2fp + + class Actor: ''' The fundamental "runtime" concurrency primitive. @@ -219,13 +233,14 @@ class Actor: # will be passed to children self._parent_main_data = _mp_fixup_main._mp_figure_out_main() + # TODO? only add this when `is_debug_mode() == True` no? # always include debugging tools module - enable_modules.append('tractor.devx._debug') + if _state.is_root_process(): + enable_modules.append('tractor.devx._debug') - self.enable_modules: dict[str, str] = {} - for name in enable_modules: - mod: ModuleType = importlib.import_module(name) - self.enable_modules[name] = _get_mod_abspath(mod) + self.enable_modules: dict[str, str] = get_mod_nsps2fps( + mod_ns_paths=enable_modules, + ) self._mods: dict[str, ModuleType] = {} self.loglevel: str = loglevel @@ -729,25 +744,33 @@ class Actor: 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.. + # ^^XXX 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 XXX^^^, can't be called here! # - # 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() + # + # => bc we haven't yet received the + # `spawnspec._runtime_vars` which contains + # `debug_mode: bool`.. + + # `SpawnSpec.bind_addrs` + # --------------------- accept_addrs: list[UnwrappedAddress] = spawnspec.bind_addrs - # TODO: another `Struct` for rtvs.. + # `SpawnSpec._runtime_vars` + # ------------------------- + # => update process-wide globals + # TODO! -[ ] another `Struct` for rtvs.. rvs: dict[str, Any] = spawnspec._runtime_vars if rvs['_debug_mode']: from .devx import ( @@ -805,18 +828,20 @@ class Actor: f'self._infected_aio = {aio_attr}\n' ) if aio_rtv: - assert trio_runtime.GLOBAL_RUN_CONTEXT.runner.is_guest - # ^TODO^ possibly add a `sniffio` or - # `trio` pub-API for `is_guest_mode()`? + assert ( + trio_runtime.GLOBAL_RUN_CONTEXT.runner.is_guest + # and + # ^TODO^ possibly add a `sniffio` or + # `trio` pub-API for `is_guest_mode()`? + ) rvs['_is_root'] = False # obvi XD - # update process-wide globals _state._runtime_vars.update(rvs) - # XXX: ``msgspec`` doesn't support serializing tuples - # so just cash manually here since it's what our - # internals expect. + # `SpawnSpec.reg_addrs` + # --------------------- + # => update parent provided registrar contact info # self.reg_addrs = [ # TODO: we don't really NEED these as tuples? @@ -827,12 +852,24 @@ class Actor: for val in spawnspec.reg_addrs ] - # TODO: better then monkey patching.. - # -[ ] maybe read the actual f#$-in `._spawn_spec` XD - for _, attr, value in pretty_struct.iter_fields( - spawnspec, - ): - setattr(self, attr, value) + # `SpawnSpec.enable_modules` + # --------------------- + # => extend RPC-python-module (capabilities) with + # those permitted by parent. + # + # NOTE, only the root actor should have + # a pre-permitted entry for `.devx.debug._tty_lock`. + assert not self.enable_modules + self.enable_modules.update( + spawnspec.enable_modules + ) + + self._parent_main_data = spawnspec._parent_main_data + # XXX QUESTION(s)^^^ + # -[ ] already set in `.__init__()` right, but how is + # it diff from this blatant parent copy? + # -[ ] do we need/want the .__init__() value in + # just the root case orr? return ( chan, From a331f6dab31b21c3ca639039a8b034b1690a969b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 14 May 2025 20:26:15 -0400 Subject: [PATCH 63/74] Return `Path` from `_get_mod_abspath()` helper fn --- tractor/_runtime.py | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/tractor/_runtime.py b/tractor/_runtime.py index f5e363f0..b25f6bc1 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -44,6 +44,7 @@ from functools import partial import importlib import importlib.util import os +from pathlib import Path from pprint import pformat import signal import sys @@ -111,8 +112,8 @@ if TYPE_CHECKING: log = get_logger('tractor') -def _get_mod_abspath(module): - return os.path.abspath(module.__file__) +def _get_mod_abspath(module: ModuleType) -> Path: + return Path(module.__file__).absolute() def get_mod_nsps2fps(mod_ns_paths: list[str]) -> dict[str, str]: @@ -124,7 +125,7 @@ def get_mod_nsps2fps(mod_ns_paths: list[str]) -> dict[str, str]: nsp2fp: dict[str, str] = {} for nsp in mod_ns_paths: mod: ModuleType = importlib.import_module(nsp) - nsp2fp[nsp] = _get_mod_abspath(mod) + nsp2fp[nsp] = str(_get_mod_abspath(mod)) return nsp2fp @@ -406,7 +407,6 @@ class Actor: def load_modules( self, - # debug_mode: bool = False, ) -> None: ''' Load explicitly enabled python modules from local fs after @@ -428,6 +428,9 @@ class Actor: parent_data['init_main_from_path']) status: str = 'Attempting to import enabled modules:\n' + + modpath: str + filepath: str for modpath, filepath in self.enable_modules.items(): # XXX append the allowed module to the python path which # should allow for relative (at least downward) imports. From ae2687b38164aa57651e5d68097c7b5c348108ac Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 11 Jun 2025 20:37:45 -0400 Subject: [PATCH 64/74] Bump lock file for new 3.13 wheels/schema Buncha either new AOTc lib whls and they added an `upload-time` field. --- uv.lock | 419 +++++++++++++++++++++++++++++--------------------------- 1 file changed, 215 insertions(+), 204 deletions(-) diff --git a/uv.lock b/uv.lock index b5d06153..168979bc 100644 --- a/uv.lock +++ b/uv.lock @@ -1,23 +1,23 @@ version = 1 -revision = 1 +revision = 2 requires-python = ">=3.11" [[package]] name = "attrs" version = "24.3.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/48/c8/6260f8ccc11f0917360fc0da435c5c9c7504e3db174d5a12a1494887b045/attrs-24.3.0.tar.gz", hash = "sha256:8f5c07333d543103541ba7be0e2ce16eeee8130cb0b3f9238ab904ce1e85baff", size = 805984 } +sdist = { url = "https://files.pythonhosted.org/packages/48/c8/6260f8ccc11f0917360fc0da435c5c9c7504e3db174d5a12a1494887b045/attrs-24.3.0.tar.gz", hash = "sha256:8f5c07333d543103541ba7be0e2ce16eeee8130cb0b3f9238ab904ce1e85baff", size = 805984, upload-time = "2024-12-16T06:59:29.899Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/89/aa/ab0f7891a01eeb2d2e338ae8fecbe57fcebea1a24dbb64d45801bfab481d/attrs-24.3.0-py3-none-any.whl", hash = "sha256:ac96cd038792094f438ad1f6ff80837353805ac950cd2aa0e0625ef19850c308", size = 63397 }, + { url = "https://files.pythonhosted.org/packages/89/aa/ab0f7891a01eeb2d2e338ae8fecbe57fcebea1a24dbb64d45801bfab481d/attrs-24.3.0-py3-none-any.whl", hash = "sha256:ac96cd038792094f438ad1f6ff80837353805ac950cd2aa0e0625ef19850c308", size = 63397, upload-time = "2024-12-16T06:59:26.977Z" }, ] [[package]] name = "bidict" version = "0.23.1" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/9a/6e/026678aa5a830e07cd9498a05d3e7e650a4f56a42f267a53d22bcda1bdc9/bidict-0.23.1.tar.gz", hash = "sha256:03069d763bc387bbd20e7d49914e75fc4132a41937fa3405417e1a5a2d006d71", size = 29093 } +sdist = { url = "https://files.pythonhosted.org/packages/9a/6e/026678aa5a830e07cd9498a05d3e7e650a4f56a42f267a53d22bcda1bdc9/bidict-0.23.1.tar.gz", hash = "sha256:03069d763bc387bbd20e7d49914e75fc4132a41937fa3405417e1a5a2d006d71", size = 29093, upload-time = "2024-02-18T19:09:05.748Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/99/37/e8730c3587a65eb5645d4aba2d27aae48e8003614d6aaf15dda67f702f1f/bidict-0.23.1-py3-none-any.whl", hash = "sha256:5dae8d4d79b552a71cbabc7deb25dfe8ce710b17ff41711e13010ead2abfc3e5", size = 32764 }, + { url = "https://files.pythonhosted.org/packages/99/37/e8730c3587a65eb5645d4aba2d27aae48e8003614d6aaf15dda67f702f1f/bidict-0.23.1-py3-none-any.whl", hash = "sha256:5dae8d4d79b552a71cbabc7deb25dfe8ce710b17ff41711e13010ead2abfc3e5", size = 32764, upload-time = "2024-02-18T19:09:04.156Z" }, ] [[package]] @@ -27,51 +27,51 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "pycparser" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/fc/97/c783634659c2920c3fc70419e3af40972dbaf758daa229a7d6ea6135c90d/cffi-1.17.1.tar.gz", hash = "sha256:1c39c6016c32bc48dd54561950ebd6836e1670f2ae46128f67cf49e789c52824", size = 516621 } +sdist = { url = "https://files.pythonhosted.org/packages/fc/97/c783634659c2920c3fc70419e3af40972dbaf758daa229a7d6ea6135c90d/cffi-1.17.1.tar.gz", hash = "sha256:1c39c6016c32bc48dd54561950ebd6836e1670f2ae46128f67cf49e789c52824", size = 516621, upload-time = "2024-09-04T20:45:21.852Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/6b/f4/927e3a8899e52a27fa57a48607ff7dc91a9ebe97399b357b85a0c7892e00/cffi-1.17.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:a45e3c6913c5b87b3ff120dcdc03f6131fa0065027d0ed7ee6190736a74cd401", size = 182264 }, - { url = "https://files.pythonhosted.org/packages/6c/f5/6c3a8efe5f503175aaddcbea6ad0d2c96dad6f5abb205750d1b3df44ef29/cffi-1.17.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:30c5e0cb5ae493c04c8b42916e52ca38079f1b235c2f8ae5f4527b963c401caf", size = 178651 }, - { url = "https://files.pythonhosted.org/packages/94/dd/a3f0118e688d1b1a57553da23b16bdade96d2f9bcda4d32e7d2838047ff7/cffi-1.17.1-cp311-cp311-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f75c7ab1f9e4aca5414ed4d8e5c0e303a34f4421f8a0d47a4d019ceff0ab6af4", size = 445259 }, - { url = "https://files.pythonhosted.org/packages/2e/ea/70ce63780f096e16ce8588efe039d3c4f91deb1dc01e9c73a287939c79a6/cffi-1.17.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a1ed2dd2972641495a3ec98445e09766f077aee98a1c896dcb4ad0d303628e41", size = 469200 }, - { url = "https://files.pythonhosted.org/packages/1c/a0/a4fa9f4f781bda074c3ddd57a572b060fa0df7655d2a4247bbe277200146/cffi-1.17.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:46bf43160c1a35f7ec506d254e5c890f3c03648a4dbac12d624e4490a7046cd1", size = 477235 }, - { url = "https://files.pythonhosted.org/packages/62/12/ce8710b5b8affbcdd5c6e367217c242524ad17a02fe5beec3ee339f69f85/cffi-1.17.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a24ed04c8ffd54b0729c07cee15a81d964e6fee0e3d4d342a27b020d22959dc6", size = 459721 }, - { url = "https://files.pythonhosted.org/packages/ff/6b/d45873c5e0242196f042d555526f92aa9e0c32355a1be1ff8c27f077fd37/cffi-1.17.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:610faea79c43e44c71e1ec53a554553fa22321b65fae24889706c0a84d4ad86d", size = 467242 }, - { url = "https://files.pythonhosted.org/packages/1a/52/d9a0e523a572fbccf2955f5abe883cfa8bcc570d7faeee06336fbd50c9fc/cffi-1.17.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:a9b15d491f3ad5d692e11f6b71f7857e7835eb677955c00cc0aefcd0669adaf6", size = 477999 }, - { url = "https://files.pythonhosted.org/packages/44/74/f2a2460684a1a2d00ca799ad880d54652841a780c4c97b87754f660c7603/cffi-1.17.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:de2ea4b5833625383e464549fec1bc395c1bdeeb5f25c4a3a82b5a8c756ec22f", size = 454242 }, - { url = "https://files.pythonhosted.org/packages/f8/4a/34599cac7dfcd888ff54e801afe06a19c17787dfd94495ab0c8d35fe99fb/cffi-1.17.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:fc48c783f9c87e60831201f2cce7f3b2e4846bf4d8728eabe54d60700b318a0b", size = 478604 }, - { url = "https://files.pythonhosted.org/packages/34/33/e1b8a1ba29025adbdcda5fb3a36f94c03d771c1b7b12f726ff7fef2ebe36/cffi-1.17.1-cp311-cp311-win32.whl", hash = "sha256:85a950a4ac9c359340d5963966e3e0a94a676bd6245a4b55bc43949eee26a655", size = 171727 }, - { url = "https://files.pythonhosted.org/packages/3d/97/50228be003bb2802627d28ec0627837ac0bf35c90cf769812056f235b2d1/cffi-1.17.1-cp311-cp311-win_amd64.whl", hash = "sha256:caaf0640ef5f5517f49bc275eca1406b0ffa6aa184892812030f04c2abf589a0", size = 181400 }, - { url = "https://files.pythonhosted.org/packages/5a/84/e94227139ee5fb4d600a7a4927f322e1d4aea6fdc50bd3fca8493caba23f/cffi-1.17.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:805b4371bf7197c329fcb3ead37e710d1bca9da5d583f5073b799d5c5bd1eee4", size = 183178 }, - { url = "https://files.pythonhosted.org/packages/da/ee/fb72c2b48656111c4ef27f0f91da355e130a923473bf5ee75c5643d00cca/cffi-1.17.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:733e99bc2df47476e3848417c5a4540522f234dfd4ef3ab7fafdf555b082ec0c", size = 178840 }, - { url = "https://files.pythonhosted.org/packages/cc/b6/db007700f67d151abadf508cbfd6a1884f57eab90b1bb985c4c8c02b0f28/cffi-1.17.1-cp312-cp312-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1257bdabf294dceb59f5e70c64a3e2f462c30c7ad68092d01bbbfb1c16b1ba36", size = 454803 }, - { url = "https://files.pythonhosted.org/packages/1a/df/f8d151540d8c200eb1c6fba8cd0dfd40904f1b0682ea705c36e6c2e97ab3/cffi-1.17.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:da95af8214998d77a98cc14e3a3bd00aa191526343078b530ceb0bd710fb48a5", size = 478850 }, - { url = "https://files.pythonhosted.org/packages/28/c0/b31116332a547fd2677ae5b78a2ef662dfc8023d67f41b2a83f7c2aa78b1/cffi-1.17.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d63afe322132c194cf832bfec0dc69a99fb9bb6bbd550f161a49e9e855cc78ff", size = 485729 }, - { url = "https://files.pythonhosted.org/packages/91/2b/9a1ddfa5c7f13cab007a2c9cc295b70fbbda7cb10a286aa6810338e60ea1/cffi-1.17.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f79fc4fc25f1c8698ff97788206bb3c2598949bfe0fef03d299eb1b5356ada99", size = 471256 }, - { url = "https://files.pythonhosted.org/packages/b2/d5/da47df7004cb17e4955df6a43d14b3b4ae77737dff8bf7f8f333196717bf/cffi-1.17.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b62ce867176a75d03a665bad002af8e6d54644fad99a3c70905c543130e39d93", size = 479424 }, - { url = "https://files.pythonhosted.org/packages/0b/ac/2a28bcf513e93a219c8a4e8e125534f4f6db03e3179ba1c45e949b76212c/cffi-1.17.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:386c8bf53c502fff58903061338ce4f4950cbdcb23e2902d86c0f722b786bbe3", size = 484568 }, - { url = "https://files.pythonhosted.org/packages/d4/38/ca8a4f639065f14ae0f1d9751e70447a261f1a30fa7547a828ae08142465/cffi-1.17.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:4ceb10419a9adf4460ea14cfd6bc43d08701f0835e979bf821052f1805850fe8", size = 488736 }, - { url = "https://files.pythonhosted.org/packages/86/c5/28b2d6f799ec0bdecf44dced2ec5ed43e0eb63097b0f58c293583b406582/cffi-1.17.1-cp312-cp312-win32.whl", hash = "sha256:a08d7e755f8ed21095a310a693525137cfe756ce62d066e53f502a83dc550f65", size = 172448 }, - { url = "https://files.pythonhosted.org/packages/50/b9/db34c4755a7bd1cb2d1603ac3863f22bcecbd1ba29e5ee841a4bc510b294/cffi-1.17.1-cp312-cp312-win_amd64.whl", hash = "sha256:51392eae71afec0d0c8fb1a53b204dbb3bcabcb3c9b807eedf3e1e6ccf2de903", size = 181976 }, - { url = "https://files.pythonhosted.org/packages/8d/f8/dd6c246b148639254dad4d6803eb6a54e8c85c6e11ec9df2cffa87571dbe/cffi-1.17.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:f3a2b4222ce6b60e2e8b337bb9596923045681d71e5a082783484d845390938e", size = 182989 }, - { url = "https://files.pythonhosted.org/packages/8b/f1/672d303ddf17c24fc83afd712316fda78dc6fce1cd53011b839483e1ecc8/cffi-1.17.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:0984a4925a435b1da406122d4d7968dd861c1385afe3b45ba82b750f229811e2", size = 178802 }, - { url = "https://files.pythonhosted.org/packages/0e/2d/eab2e858a91fdff70533cab61dcff4a1f55ec60425832ddfdc9cd36bc8af/cffi-1.17.1-cp313-cp313-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d01b12eeeb4427d3110de311e1774046ad344f5b1a7403101878976ecd7a10f3", size = 454792 }, - { url = "https://files.pythonhosted.org/packages/75/b2/fbaec7c4455c604e29388d55599b99ebcc250a60050610fadde58932b7ee/cffi-1.17.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:706510fe141c86a69c8ddc029c7910003a17353970cff3b904ff0686a5927683", size = 478893 }, - { url = "https://files.pythonhosted.org/packages/4f/b7/6e4a2162178bf1935c336d4da8a9352cccab4d3a5d7914065490f08c0690/cffi-1.17.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:de55b766c7aa2e2a3092c51e0483d700341182f08e67c63630d5b6f200bb28e5", size = 485810 }, - { url = "https://files.pythonhosted.org/packages/c7/8a/1d0e4a9c26e54746dc08c2c6c037889124d4f59dffd853a659fa545f1b40/cffi-1.17.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c59d6e989d07460165cc5ad3c61f9fd8f1b4796eacbd81cee78957842b834af4", size = 471200 }, - { url = "https://files.pythonhosted.org/packages/26/9f/1aab65a6c0db35f43c4d1b4f580e8df53914310afc10ae0397d29d697af4/cffi-1.17.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dd398dbc6773384a17fe0d3e7eeb8d1a21c2200473ee6806bb5e6a8e62bb73dd", size = 479447 }, - { url = "https://files.pythonhosted.org/packages/5f/e4/fb8b3dd8dc0e98edf1135ff067ae070bb32ef9d509d6cb0f538cd6f7483f/cffi-1.17.1-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:3edc8d958eb099c634dace3c7e16560ae474aa3803a5df240542b305d14e14ed", size = 484358 }, - { url = "https://files.pythonhosted.org/packages/f1/47/d7145bf2dc04684935d57d67dff9d6d795b2ba2796806bb109864be3a151/cffi-1.17.1-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:72e72408cad3d5419375fc87d289076ee319835bdfa2caad331e377589aebba9", size = 488469 }, - { url = "https://files.pythonhosted.org/packages/bf/ee/f94057fa6426481d663b88637a9a10e859e492c73d0384514a17d78ee205/cffi-1.17.1-cp313-cp313-win32.whl", hash = "sha256:e03eab0a8677fa80d646b5ddece1cbeaf556c313dcfac435ba11f107ba117b5d", size = 172475 }, - { url = "https://files.pythonhosted.org/packages/7c/fc/6a8cb64e5f0324877d503c854da15d76c1e50eb722e320b15345c4d0c6de/cffi-1.17.1-cp313-cp313-win_amd64.whl", hash = "sha256:f6a16c31041f09ead72d69f583767292f750d24913dadacf5756b966aacb3f1a", size = 182009 }, + { url = "https://files.pythonhosted.org/packages/6b/f4/927e3a8899e52a27fa57a48607ff7dc91a9ebe97399b357b85a0c7892e00/cffi-1.17.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:a45e3c6913c5b87b3ff120dcdc03f6131fa0065027d0ed7ee6190736a74cd401", size = 182264, upload-time = "2024-09-04T20:43:51.124Z" }, + { url = "https://files.pythonhosted.org/packages/6c/f5/6c3a8efe5f503175aaddcbea6ad0d2c96dad6f5abb205750d1b3df44ef29/cffi-1.17.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:30c5e0cb5ae493c04c8b42916e52ca38079f1b235c2f8ae5f4527b963c401caf", size = 178651, upload-time = "2024-09-04T20:43:52.872Z" }, + { url = "https://files.pythonhosted.org/packages/94/dd/a3f0118e688d1b1a57553da23b16bdade96d2f9bcda4d32e7d2838047ff7/cffi-1.17.1-cp311-cp311-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f75c7ab1f9e4aca5414ed4d8e5c0e303a34f4421f8a0d47a4d019ceff0ab6af4", size = 445259, upload-time = "2024-09-04T20:43:56.123Z" }, + { url = "https://files.pythonhosted.org/packages/2e/ea/70ce63780f096e16ce8588efe039d3c4f91deb1dc01e9c73a287939c79a6/cffi-1.17.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a1ed2dd2972641495a3ec98445e09766f077aee98a1c896dcb4ad0d303628e41", size = 469200, upload-time = "2024-09-04T20:43:57.891Z" }, + { url = "https://files.pythonhosted.org/packages/1c/a0/a4fa9f4f781bda074c3ddd57a572b060fa0df7655d2a4247bbe277200146/cffi-1.17.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:46bf43160c1a35f7ec506d254e5c890f3c03648a4dbac12d624e4490a7046cd1", size = 477235, upload-time = "2024-09-04T20:44:00.18Z" }, + { url = "https://files.pythonhosted.org/packages/62/12/ce8710b5b8affbcdd5c6e367217c242524ad17a02fe5beec3ee339f69f85/cffi-1.17.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a24ed04c8ffd54b0729c07cee15a81d964e6fee0e3d4d342a27b020d22959dc6", size = 459721, upload-time = "2024-09-04T20:44:01.585Z" }, + { url = "https://files.pythonhosted.org/packages/ff/6b/d45873c5e0242196f042d555526f92aa9e0c32355a1be1ff8c27f077fd37/cffi-1.17.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:610faea79c43e44c71e1ec53a554553fa22321b65fae24889706c0a84d4ad86d", size = 467242, upload-time = "2024-09-04T20:44:03.467Z" }, + { url = "https://files.pythonhosted.org/packages/1a/52/d9a0e523a572fbccf2955f5abe883cfa8bcc570d7faeee06336fbd50c9fc/cffi-1.17.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:a9b15d491f3ad5d692e11f6b71f7857e7835eb677955c00cc0aefcd0669adaf6", size = 477999, upload-time = "2024-09-04T20:44:05.023Z" }, + { url = "https://files.pythonhosted.org/packages/44/74/f2a2460684a1a2d00ca799ad880d54652841a780c4c97b87754f660c7603/cffi-1.17.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:de2ea4b5833625383e464549fec1bc395c1bdeeb5f25c4a3a82b5a8c756ec22f", size = 454242, upload-time = "2024-09-04T20:44:06.444Z" }, + { url = "https://files.pythonhosted.org/packages/f8/4a/34599cac7dfcd888ff54e801afe06a19c17787dfd94495ab0c8d35fe99fb/cffi-1.17.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:fc48c783f9c87e60831201f2cce7f3b2e4846bf4d8728eabe54d60700b318a0b", size = 478604, upload-time = "2024-09-04T20:44:08.206Z" }, + { url = "https://files.pythonhosted.org/packages/34/33/e1b8a1ba29025adbdcda5fb3a36f94c03d771c1b7b12f726ff7fef2ebe36/cffi-1.17.1-cp311-cp311-win32.whl", hash = "sha256:85a950a4ac9c359340d5963966e3e0a94a676bd6245a4b55bc43949eee26a655", size = 171727, upload-time = "2024-09-04T20:44:09.481Z" }, + { url = "https://files.pythonhosted.org/packages/3d/97/50228be003bb2802627d28ec0627837ac0bf35c90cf769812056f235b2d1/cffi-1.17.1-cp311-cp311-win_amd64.whl", hash = "sha256:caaf0640ef5f5517f49bc275eca1406b0ffa6aa184892812030f04c2abf589a0", size = 181400, upload-time = "2024-09-04T20:44:10.873Z" }, + { url = "https://files.pythonhosted.org/packages/5a/84/e94227139ee5fb4d600a7a4927f322e1d4aea6fdc50bd3fca8493caba23f/cffi-1.17.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:805b4371bf7197c329fcb3ead37e710d1bca9da5d583f5073b799d5c5bd1eee4", size = 183178, upload-time = "2024-09-04T20:44:12.232Z" }, + { url = "https://files.pythonhosted.org/packages/da/ee/fb72c2b48656111c4ef27f0f91da355e130a923473bf5ee75c5643d00cca/cffi-1.17.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:733e99bc2df47476e3848417c5a4540522f234dfd4ef3ab7fafdf555b082ec0c", size = 178840, upload-time = "2024-09-04T20:44:13.739Z" }, + { url = "https://files.pythonhosted.org/packages/cc/b6/db007700f67d151abadf508cbfd6a1884f57eab90b1bb985c4c8c02b0f28/cffi-1.17.1-cp312-cp312-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1257bdabf294dceb59f5e70c64a3e2f462c30c7ad68092d01bbbfb1c16b1ba36", size = 454803, upload-time = "2024-09-04T20:44:15.231Z" }, + { url = "https://files.pythonhosted.org/packages/1a/df/f8d151540d8c200eb1c6fba8cd0dfd40904f1b0682ea705c36e6c2e97ab3/cffi-1.17.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:da95af8214998d77a98cc14e3a3bd00aa191526343078b530ceb0bd710fb48a5", size = 478850, upload-time = "2024-09-04T20:44:17.188Z" }, + { url = "https://files.pythonhosted.org/packages/28/c0/b31116332a547fd2677ae5b78a2ef662dfc8023d67f41b2a83f7c2aa78b1/cffi-1.17.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d63afe322132c194cf832bfec0dc69a99fb9bb6bbd550f161a49e9e855cc78ff", size = 485729, upload-time = "2024-09-04T20:44:18.688Z" }, + { url = "https://files.pythonhosted.org/packages/91/2b/9a1ddfa5c7f13cab007a2c9cc295b70fbbda7cb10a286aa6810338e60ea1/cffi-1.17.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f79fc4fc25f1c8698ff97788206bb3c2598949bfe0fef03d299eb1b5356ada99", size = 471256, upload-time = "2024-09-04T20:44:20.248Z" }, + { url = "https://files.pythonhosted.org/packages/b2/d5/da47df7004cb17e4955df6a43d14b3b4ae77737dff8bf7f8f333196717bf/cffi-1.17.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b62ce867176a75d03a665bad002af8e6d54644fad99a3c70905c543130e39d93", size = 479424, upload-time = "2024-09-04T20:44:21.673Z" }, + { url = "https://files.pythonhosted.org/packages/0b/ac/2a28bcf513e93a219c8a4e8e125534f4f6db03e3179ba1c45e949b76212c/cffi-1.17.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:386c8bf53c502fff58903061338ce4f4950cbdcb23e2902d86c0f722b786bbe3", size = 484568, upload-time = "2024-09-04T20:44:23.245Z" }, + { url = "https://files.pythonhosted.org/packages/d4/38/ca8a4f639065f14ae0f1d9751e70447a261f1a30fa7547a828ae08142465/cffi-1.17.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:4ceb10419a9adf4460ea14cfd6bc43d08701f0835e979bf821052f1805850fe8", size = 488736, upload-time = "2024-09-04T20:44:24.757Z" }, + { url = "https://files.pythonhosted.org/packages/86/c5/28b2d6f799ec0bdecf44dced2ec5ed43e0eb63097b0f58c293583b406582/cffi-1.17.1-cp312-cp312-win32.whl", hash = "sha256:a08d7e755f8ed21095a310a693525137cfe756ce62d066e53f502a83dc550f65", size = 172448, upload-time = "2024-09-04T20:44:26.208Z" }, + { url = "https://files.pythonhosted.org/packages/50/b9/db34c4755a7bd1cb2d1603ac3863f22bcecbd1ba29e5ee841a4bc510b294/cffi-1.17.1-cp312-cp312-win_amd64.whl", hash = "sha256:51392eae71afec0d0c8fb1a53b204dbb3bcabcb3c9b807eedf3e1e6ccf2de903", size = 181976, upload-time = "2024-09-04T20:44:27.578Z" }, + { url = "https://files.pythonhosted.org/packages/8d/f8/dd6c246b148639254dad4d6803eb6a54e8c85c6e11ec9df2cffa87571dbe/cffi-1.17.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:f3a2b4222ce6b60e2e8b337bb9596923045681d71e5a082783484d845390938e", size = 182989, upload-time = "2024-09-04T20:44:28.956Z" }, + { url = "https://files.pythonhosted.org/packages/8b/f1/672d303ddf17c24fc83afd712316fda78dc6fce1cd53011b839483e1ecc8/cffi-1.17.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:0984a4925a435b1da406122d4d7968dd861c1385afe3b45ba82b750f229811e2", size = 178802, upload-time = "2024-09-04T20:44:30.289Z" }, + { url = "https://files.pythonhosted.org/packages/0e/2d/eab2e858a91fdff70533cab61dcff4a1f55ec60425832ddfdc9cd36bc8af/cffi-1.17.1-cp313-cp313-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d01b12eeeb4427d3110de311e1774046ad344f5b1a7403101878976ecd7a10f3", size = 454792, upload-time = "2024-09-04T20:44:32.01Z" }, + { url = "https://files.pythonhosted.org/packages/75/b2/fbaec7c4455c604e29388d55599b99ebcc250a60050610fadde58932b7ee/cffi-1.17.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:706510fe141c86a69c8ddc029c7910003a17353970cff3b904ff0686a5927683", size = 478893, upload-time = "2024-09-04T20:44:33.606Z" }, + { url = "https://files.pythonhosted.org/packages/4f/b7/6e4a2162178bf1935c336d4da8a9352cccab4d3a5d7914065490f08c0690/cffi-1.17.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:de55b766c7aa2e2a3092c51e0483d700341182f08e67c63630d5b6f200bb28e5", size = 485810, upload-time = "2024-09-04T20:44:35.191Z" }, + { url = "https://files.pythonhosted.org/packages/c7/8a/1d0e4a9c26e54746dc08c2c6c037889124d4f59dffd853a659fa545f1b40/cffi-1.17.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c59d6e989d07460165cc5ad3c61f9fd8f1b4796eacbd81cee78957842b834af4", size = 471200, upload-time = "2024-09-04T20:44:36.743Z" }, + { url = "https://files.pythonhosted.org/packages/26/9f/1aab65a6c0db35f43c4d1b4f580e8df53914310afc10ae0397d29d697af4/cffi-1.17.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dd398dbc6773384a17fe0d3e7eeb8d1a21c2200473ee6806bb5e6a8e62bb73dd", size = 479447, upload-time = "2024-09-04T20:44:38.492Z" }, + { url = "https://files.pythonhosted.org/packages/5f/e4/fb8b3dd8dc0e98edf1135ff067ae070bb32ef9d509d6cb0f538cd6f7483f/cffi-1.17.1-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:3edc8d958eb099c634dace3c7e16560ae474aa3803a5df240542b305d14e14ed", size = 484358, upload-time = "2024-09-04T20:44:40.046Z" }, + { url = "https://files.pythonhosted.org/packages/f1/47/d7145bf2dc04684935d57d67dff9d6d795b2ba2796806bb109864be3a151/cffi-1.17.1-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:72e72408cad3d5419375fc87d289076ee319835bdfa2caad331e377589aebba9", size = 488469, upload-time = "2024-09-04T20:44:41.616Z" }, + { url = "https://files.pythonhosted.org/packages/bf/ee/f94057fa6426481d663b88637a9a10e859e492c73d0384514a17d78ee205/cffi-1.17.1-cp313-cp313-win32.whl", hash = "sha256:e03eab0a8677fa80d646b5ddece1cbeaf556c313dcfac435ba11f107ba117b5d", size = 172475, upload-time = "2024-09-04T20:44:43.733Z" }, + { url = "https://files.pythonhosted.org/packages/7c/fc/6a8cb64e5f0324877d503c854da15d76c1e50eb722e320b15345c4d0c6de/cffi-1.17.1-cp313-cp313-win_amd64.whl", hash = "sha256:f6a16c31041f09ead72d69f583767292f750d24913dadacf5756b966aacb3f1a", size = 182009, upload-time = "2024-09-04T20:44:45.309Z" }, ] [[package]] name = "colorama" version = "0.4.6" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/d8/53/6f443c9a4a8358a93a6792e2acffb9d9d5cb0a5cfd8802644b7b1c9a02e4/colorama-0.4.6.tar.gz", hash = "sha256:08695f5cb7ed6e0531a20572697297273c47b8cae5a63ffc6d6ed5c201be6e44", size = 27697 } +sdist = { url = "https://files.pythonhosted.org/packages/d8/53/6f443c9a4a8358a93a6792e2acffb9d9d5cb0a5cfd8802644b7b1c9a02e4/colorama-0.4.6.tar.gz", hash = "sha256:08695f5cb7ed6e0531a20572697297273c47b8cae5a63ffc6d6ed5c201be6e44", size = 27697, upload-time = "2022-10-25T02:36:22.414Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/d1/d6/3965ed04c63042e047cb6a3e6ed1a63a35087b6a609aa3a15ed8ac56c221/colorama-0.4.6-py2.py3-none-any.whl", hash = "sha256:4f1d9991f5acc0ca119f9d443620b77f9d6b33703e51011c16baf57afb285fc6", size = 25335 }, + { url = "https://files.pythonhosted.org/packages/d1/d6/3965ed04c63042e047cb6a3e6ed1a63a35087b6a609aa3a15ed8ac56c221/colorama-0.4.6-py2.py3-none-any.whl", hash = "sha256:4f1d9991f5acc0ca119f9d443620b77f9d6b33703e51011c16baf57afb285fc6", size = 25335, upload-time = "2022-10-25T02:36:20.889Z" }, ] [[package]] @@ -81,9 +81,9 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "colorama", marker = "sys_platform == 'win32'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/d3/7a/359f4d5df2353f26172b3cc39ea32daa39af8de522205f512f458923e677/colorlog-6.9.0.tar.gz", hash = "sha256:bfba54a1b93b94f54e1f4fe48395725a3d92fd2a4af702f6bd70946bdc0c6ac2", size = 16624 } +sdist = { url = "https://files.pythonhosted.org/packages/d3/7a/359f4d5df2353f26172b3cc39ea32daa39af8de522205f512f458923e677/colorlog-6.9.0.tar.gz", hash = "sha256:bfba54a1b93b94f54e1f4fe48395725a3d92fd2a4af702f6bd70946bdc0c6ac2", size = 16624, upload-time = "2024-10-29T18:34:51.011Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/e3/51/9b208e85196941db2f0654ad0357ca6388ab3ed67efdbfc799f35d1f83aa/colorlog-6.9.0-py3-none-any.whl", hash = "sha256:5906e71acd67cb07a71e779c47c4bcb45fb8c2993eebe9e5adcd6a6f1b283eff", size = 11424 }, + { url = "https://files.pythonhosted.org/packages/e3/51/9b208e85196941db2f0654ad0357ca6388ab3ed67efdbfc799f35d1f83aa/colorlog-6.9.0-py3-none-any.whl", hash = "sha256:5906e71acd67cb07a71e779c47c4bcb45fb8c2993eebe9e5adcd6a6f1b283eff", size = 11424, upload-time = "2024-10-29T18:34:49.815Z" }, ] [[package]] @@ -95,98 +95,98 @@ dependencies = [ { name = "outcome" }, { name = "sniffio" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/dc/c1/ab3a42c0f3ed56df9cd33de1539b3198d98c6ccbaf88a73d6be0b72d85e0/greenback-1.2.1.tar.gz", hash = "sha256:de3ca656885c03b96dab36079f3de74bb5ba061da9bfe3bb69dccc866ef95ea3", size = 42597 } +sdist = { url = "https://files.pythonhosted.org/packages/dc/c1/ab3a42c0f3ed56df9cd33de1539b3198d98c6ccbaf88a73d6be0b72d85e0/greenback-1.2.1.tar.gz", hash = "sha256:de3ca656885c03b96dab36079f3de74bb5ba061da9bfe3bb69dccc866ef95ea3", size = 42597, upload-time = "2024-02-20T21:23:13.239Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/71/d0/b8dc79d5ecfffacad9c844b6ae76b9c6259935796d3c561deccbf8fa421d/greenback-1.2.1-py3-none-any.whl", hash = "sha256:98768edbbe4340091a9730cf64a683fcbaa3f2cb81e4ac41d7ed28d3b6f74b79", size = 28062 }, + { url = "https://files.pythonhosted.org/packages/71/d0/b8dc79d5ecfffacad9c844b6ae76b9c6259935796d3c561deccbf8fa421d/greenback-1.2.1-py3-none-any.whl", hash = "sha256:98768edbbe4340091a9730cf64a683fcbaa3f2cb81e4ac41d7ed28d3b6f74b79", size = 28062, upload-time = "2024-02-20T21:23:12.031Z" }, ] [[package]] name = "greenlet" version = "3.1.1" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/2f/ff/df5fede753cc10f6a5be0931204ea30c35fa2f2ea7a35b25bdaf4fe40e46/greenlet-3.1.1.tar.gz", hash = "sha256:4ce3ac6cdb6adf7946475d7ef31777c26d94bccc377e070a7986bd2d5c515467", size = 186022 } +sdist = { url = "https://files.pythonhosted.org/packages/2f/ff/df5fede753cc10f6a5be0931204ea30c35fa2f2ea7a35b25bdaf4fe40e46/greenlet-3.1.1.tar.gz", hash = "sha256:4ce3ac6cdb6adf7946475d7ef31777c26d94bccc377e070a7986bd2d5c515467", size = 186022, upload-time = "2024-09-20T18:21:04.506Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/28/62/1c2665558618553c42922ed47a4e6d6527e2fa3516a8256c2f431c5d0441/greenlet-3.1.1-cp311-cp311-macosx_11_0_universal2.whl", hash = "sha256:e4d333e558953648ca09d64f13e6d8f0523fa705f51cae3f03b5983489958c70", size = 272479 }, - { url = "https://files.pythonhosted.org/packages/76/9d/421e2d5f07285b6e4e3a676b016ca781f63cfe4a0cd8eaecf3fd6f7a71ae/greenlet-3.1.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:09fc016b73c94e98e29af67ab7b9a879c307c6731a2c9da0db5a7d9b7edd1159", size = 640404 }, - { url = "https://files.pythonhosted.org/packages/e5/de/6e05f5c59262a584e502dd3d261bbdd2c97ab5416cc9c0b91ea38932a901/greenlet-3.1.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d5e975ca70269d66d17dd995dafc06f1b06e8cb1ec1e9ed54c1d1e4a7c4cf26e", size = 652813 }, - { url = "https://files.pythonhosted.org/packages/49/93/d5f93c84241acdea15a8fd329362c2c71c79e1a507c3f142a5d67ea435ae/greenlet-3.1.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3b2813dc3de8c1ee3f924e4d4227999285fd335d1bcc0d2be6dc3f1f6a318ec1", size = 648517 }, - { url = "https://files.pythonhosted.org/packages/15/85/72f77fc02d00470c86a5c982b8daafdf65d38aefbbe441cebff3bf7037fc/greenlet-3.1.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e347b3bfcf985a05e8c0b7d462ba6f15b1ee1c909e2dcad795e49e91b152c383", size = 647831 }, - { url = "https://files.pythonhosted.org/packages/f7/4b/1c9695aa24f808e156c8f4813f685d975ca73c000c2a5056c514c64980f6/greenlet-3.1.1-cp311-cp311-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:9e8f8c9cb53cdac7ba9793c276acd90168f416b9ce36799b9b885790f8ad6c0a", size = 602413 }, - { url = "https://files.pythonhosted.org/packages/76/70/ad6e5b31ef330f03b12559d19fda2606a522d3849cde46b24f223d6d1619/greenlet-3.1.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:62ee94988d6b4722ce0028644418d93a52429e977d742ca2ccbe1c4f4a792511", size = 1129619 }, - { url = "https://files.pythonhosted.org/packages/f4/fb/201e1b932e584066e0f0658b538e73c459b34d44b4bd4034f682423bc801/greenlet-3.1.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:1776fd7f989fc6b8d8c8cb8da1f6b82c5814957264d1f6cf818d475ec2bf6395", size = 1155198 }, - { url = "https://files.pythonhosted.org/packages/12/da/b9ed5e310bb8b89661b80cbcd4db5a067903bbcd7fc854923f5ebb4144f0/greenlet-3.1.1-cp311-cp311-win_amd64.whl", hash = "sha256:48ca08c771c268a768087b408658e216133aecd835c0ded47ce955381105ba39", size = 298930 }, - { url = "https://files.pythonhosted.org/packages/7d/ec/bad1ac26764d26aa1353216fcbfa4670050f66d445448aafa227f8b16e80/greenlet-3.1.1-cp312-cp312-macosx_11_0_universal2.whl", hash = "sha256:4afe7ea89de619adc868e087b4d2359282058479d7cfb94970adf4b55284574d", size = 274260 }, - { url = "https://files.pythonhosted.org/packages/66/d4/c8c04958870f482459ab5956c2942c4ec35cac7fe245527f1039837c17a9/greenlet-3.1.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f406b22b7c9a9b4f8aa9d2ab13d6ae0ac3e85c9a809bd590ad53fed2bf70dc79", size = 649064 }, - { url = "https://files.pythonhosted.org/packages/51/41/467b12a8c7c1303d20abcca145db2be4e6cd50a951fa30af48b6ec607581/greenlet-3.1.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c3a701fe5a9695b238503ce5bbe8218e03c3bcccf7e204e455e7462d770268aa", size = 663420 }, - { url = "https://files.pythonhosted.org/packages/27/8f/2a93cd9b1e7107d5c7b3b7816eeadcac2ebcaf6d6513df9abaf0334777f6/greenlet-3.1.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2846930c65b47d70b9d178e89c7e1a69c95c1f68ea5aa0a58646b7a96df12441", size = 658035 }, - { url = "https://files.pythonhosted.org/packages/57/5c/7c6f50cb12be092e1dccb2599be5a942c3416dbcfb76efcf54b3f8be4d8d/greenlet-3.1.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:99cfaa2110534e2cf3ba31a7abcac9d328d1d9f1b95beede58294a60348fba36", size = 660105 }, - { url = "https://files.pythonhosted.org/packages/f1/66/033e58a50fd9ec9df00a8671c74f1f3a320564c6415a4ed82a1c651654ba/greenlet-3.1.1-cp312-cp312-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:1443279c19fca463fc33e65ef2a935a5b09bb90f978beab37729e1c3c6c25fe9", size = 613077 }, - { url = "https://files.pythonhosted.org/packages/19/c5/36384a06f748044d06bdd8776e231fadf92fc896bd12cb1c9f5a1bda9578/greenlet-3.1.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:b7cede291382a78f7bb5f04a529cb18e068dd29e0fb27376074b6d0317bf4dd0", size = 1135975 }, - { url = "https://files.pythonhosted.org/packages/38/f9/c0a0eb61bdf808d23266ecf1d63309f0e1471f284300ce6dac0ae1231881/greenlet-3.1.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:23f20bb60ae298d7d8656c6ec6db134bca379ecefadb0b19ce6f19d1f232a942", size = 1163955 }, - { url = "https://files.pythonhosted.org/packages/43/21/a5d9df1d21514883333fc86584c07c2b49ba7c602e670b174bd73cfc9c7f/greenlet-3.1.1-cp312-cp312-win_amd64.whl", hash = "sha256:7124e16b4c55d417577c2077be379514321916d5790fa287c9ed6f23bd2ffd01", size = 299655 }, - { url = "https://files.pythonhosted.org/packages/f3/57/0db4940cd7bb461365ca8d6fd53e68254c9dbbcc2b452e69d0d41f10a85e/greenlet-3.1.1-cp313-cp313-macosx_11_0_universal2.whl", hash = "sha256:05175c27cb459dcfc05d026c4232f9de8913ed006d42713cb8a5137bd49375f1", size = 272990 }, - { url = "https://files.pythonhosted.org/packages/1c/ec/423d113c9f74e5e402e175b157203e9102feeb7088cee844d735b28ef963/greenlet-3.1.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:935e943ec47c4afab8965954bf49bfa639c05d4ccf9ef6e924188f762145c0ff", size = 649175 }, - { url = "https://files.pythonhosted.org/packages/a9/46/ddbd2db9ff209186b7b7c621d1432e2f21714adc988703dbdd0e65155c77/greenlet-3.1.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:667a9706c970cb552ede35aee17339a18e8f2a87a51fba2ed39ceeeb1004798a", size = 663425 }, - { url = "https://files.pythonhosted.org/packages/bc/f9/9c82d6b2b04aa37e38e74f0c429aece5eeb02bab6e3b98e7db89b23d94c6/greenlet-3.1.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b8a678974d1f3aa55f6cc34dc480169d58f2e6d8958895d68845fa4ab566509e", size = 657736 }, - { url = "https://files.pythonhosted.org/packages/d9/42/b87bc2a81e3a62c3de2b0d550bf91a86939442b7ff85abb94eec3fc0e6aa/greenlet-3.1.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:efc0f674aa41b92da8c49e0346318c6075d734994c3c4e4430b1c3f853e498e4", size = 660347 }, - { url = "https://files.pythonhosted.org/packages/37/fa/71599c3fd06336cdc3eac52e6871cfebab4d9d70674a9a9e7a482c318e99/greenlet-3.1.1-cp313-cp313-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:0153404a4bb921f0ff1abeb5ce8a5131da56b953eda6e14b88dc6bbc04d2049e", size = 615583 }, - { url = "https://files.pythonhosted.org/packages/4e/96/e9ef85de031703ee7a4483489b40cf307f93c1824a02e903106f2ea315fe/greenlet-3.1.1-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:275f72decf9932639c1c6dd1013a1bc266438eb32710016a1c742df5da6e60a1", size = 1133039 }, - { url = "https://files.pythonhosted.org/packages/87/76/b2b6362accd69f2d1889db61a18c94bc743e961e3cab344c2effaa4b4a25/greenlet-3.1.1-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:c4aab7f6381f38a4b42f269057aee279ab0fc7bf2e929e3d4abfae97b682a12c", size = 1160716 }, - { url = "https://files.pythonhosted.org/packages/1f/1b/54336d876186920e185066d8c3024ad55f21d7cc3683c856127ddb7b13ce/greenlet-3.1.1-cp313-cp313-win_amd64.whl", hash = "sha256:b42703b1cf69f2aa1df7d1030b9d77d3e584a70755674d60e710f0af570f3761", size = 299490 }, - { url = "https://files.pythonhosted.org/packages/5f/17/bea55bf36990e1638a2af5ba10c1640273ef20f627962cf97107f1e5d637/greenlet-3.1.1-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f1695e76146579f8c06c1509c7ce4dfe0706f49c6831a817ac04eebb2fd02011", size = 643731 }, - { url = "https://files.pythonhosted.org/packages/78/d2/aa3d2157f9ab742a08e0fd8f77d4699f37c22adfbfeb0c610a186b5f75e0/greenlet-3.1.1-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:7876452af029456b3f3549b696bb36a06db7c90747740c5302f74a9e9fa14b13", size = 649304 }, - { url = "https://files.pythonhosted.org/packages/f1/8e/d0aeffe69e53ccff5a28fa86f07ad1d2d2d6537a9506229431a2a02e2f15/greenlet-3.1.1-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4ead44c85f8ab905852d3de8d86f6f8baf77109f9da589cb4fa142bd3b57b475", size = 646537 }, - { url = "https://files.pythonhosted.org/packages/05/79/e15408220bbb989469c8871062c97c6c9136770657ba779711b90870d867/greenlet-3.1.1-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8320f64b777d00dd7ccdade271eaf0cad6636343293a25074cc5566160e4de7b", size = 642506 }, - { url = "https://files.pythonhosted.org/packages/18/87/470e01a940307796f1d25f8167b551a968540fbe0551c0ebb853cb527dd6/greenlet-3.1.1-cp313-cp313t-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:6510bf84a6b643dabba74d3049ead221257603a253d0a9873f55f6a59a65f822", size = 602753 }, - { url = "https://files.pythonhosted.org/packages/e2/72/576815ba674eddc3c25028238f74d7b8068902b3968cbe456771b166455e/greenlet-3.1.1-cp313-cp313t-musllinux_1_1_aarch64.whl", hash = "sha256:04b013dc07c96f83134b1e99888e7a79979f1a247e2a9f59697fa14b5862ed01", size = 1122731 }, - { url = "https://files.pythonhosted.org/packages/ac/38/08cc303ddddc4b3d7c628c3039a61a3aae36c241ed01393d00c2fd663473/greenlet-3.1.1-cp313-cp313t-musllinux_1_1_x86_64.whl", hash = "sha256:411f015496fec93c1c8cd4e5238da364e1da7a124bcb293f085bf2860c32c6f6", size = 1142112 }, + { url = "https://files.pythonhosted.org/packages/28/62/1c2665558618553c42922ed47a4e6d6527e2fa3516a8256c2f431c5d0441/greenlet-3.1.1-cp311-cp311-macosx_11_0_universal2.whl", hash = "sha256:e4d333e558953648ca09d64f13e6d8f0523fa705f51cae3f03b5983489958c70", size = 272479, upload-time = "2024-09-20T17:07:22.332Z" }, + { url = "https://files.pythonhosted.org/packages/76/9d/421e2d5f07285b6e4e3a676b016ca781f63cfe4a0cd8eaecf3fd6f7a71ae/greenlet-3.1.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:09fc016b73c94e98e29af67ab7b9a879c307c6731a2c9da0db5a7d9b7edd1159", size = 640404, upload-time = "2024-09-20T17:36:45.588Z" }, + { url = "https://files.pythonhosted.org/packages/e5/de/6e05f5c59262a584e502dd3d261bbdd2c97ab5416cc9c0b91ea38932a901/greenlet-3.1.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d5e975ca70269d66d17dd995dafc06f1b06e8cb1ec1e9ed54c1d1e4a7c4cf26e", size = 652813, upload-time = "2024-09-20T17:39:19.052Z" }, + { url = "https://files.pythonhosted.org/packages/49/93/d5f93c84241acdea15a8fd329362c2c71c79e1a507c3f142a5d67ea435ae/greenlet-3.1.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3b2813dc3de8c1ee3f924e4d4227999285fd335d1bcc0d2be6dc3f1f6a318ec1", size = 648517, upload-time = "2024-09-20T17:44:24.101Z" }, + { url = "https://files.pythonhosted.org/packages/15/85/72f77fc02d00470c86a5c982b8daafdf65d38aefbbe441cebff3bf7037fc/greenlet-3.1.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e347b3bfcf985a05e8c0b7d462ba6f15b1ee1c909e2dcad795e49e91b152c383", size = 647831, upload-time = "2024-09-20T17:08:40.577Z" }, + { url = "https://files.pythonhosted.org/packages/f7/4b/1c9695aa24f808e156c8f4813f685d975ca73c000c2a5056c514c64980f6/greenlet-3.1.1-cp311-cp311-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:9e8f8c9cb53cdac7ba9793c276acd90168f416b9ce36799b9b885790f8ad6c0a", size = 602413, upload-time = "2024-09-20T17:08:31.728Z" }, + { url = "https://files.pythonhosted.org/packages/76/70/ad6e5b31ef330f03b12559d19fda2606a522d3849cde46b24f223d6d1619/greenlet-3.1.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:62ee94988d6b4722ce0028644418d93a52429e977d742ca2ccbe1c4f4a792511", size = 1129619, upload-time = "2024-09-20T17:44:14.222Z" }, + { url = "https://files.pythonhosted.org/packages/f4/fb/201e1b932e584066e0f0658b538e73c459b34d44b4bd4034f682423bc801/greenlet-3.1.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:1776fd7f989fc6b8d8c8cb8da1f6b82c5814957264d1f6cf818d475ec2bf6395", size = 1155198, upload-time = "2024-09-20T17:09:23.903Z" }, + { url = "https://files.pythonhosted.org/packages/12/da/b9ed5e310bb8b89661b80cbcd4db5a067903bbcd7fc854923f5ebb4144f0/greenlet-3.1.1-cp311-cp311-win_amd64.whl", hash = "sha256:48ca08c771c268a768087b408658e216133aecd835c0ded47ce955381105ba39", size = 298930, upload-time = "2024-09-20T17:25:18.656Z" }, + { url = "https://files.pythonhosted.org/packages/7d/ec/bad1ac26764d26aa1353216fcbfa4670050f66d445448aafa227f8b16e80/greenlet-3.1.1-cp312-cp312-macosx_11_0_universal2.whl", hash = "sha256:4afe7ea89de619adc868e087b4d2359282058479d7cfb94970adf4b55284574d", size = 274260, upload-time = "2024-09-20T17:08:07.301Z" }, + { url = "https://files.pythonhosted.org/packages/66/d4/c8c04958870f482459ab5956c2942c4ec35cac7fe245527f1039837c17a9/greenlet-3.1.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f406b22b7c9a9b4f8aa9d2ab13d6ae0ac3e85c9a809bd590ad53fed2bf70dc79", size = 649064, upload-time = "2024-09-20T17:36:47.628Z" }, + { url = "https://files.pythonhosted.org/packages/51/41/467b12a8c7c1303d20abcca145db2be4e6cd50a951fa30af48b6ec607581/greenlet-3.1.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c3a701fe5a9695b238503ce5bbe8218e03c3bcccf7e204e455e7462d770268aa", size = 663420, upload-time = "2024-09-20T17:39:21.258Z" }, + { url = "https://files.pythonhosted.org/packages/27/8f/2a93cd9b1e7107d5c7b3b7816eeadcac2ebcaf6d6513df9abaf0334777f6/greenlet-3.1.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2846930c65b47d70b9d178e89c7e1a69c95c1f68ea5aa0a58646b7a96df12441", size = 658035, upload-time = "2024-09-20T17:44:26.501Z" }, + { url = "https://files.pythonhosted.org/packages/57/5c/7c6f50cb12be092e1dccb2599be5a942c3416dbcfb76efcf54b3f8be4d8d/greenlet-3.1.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:99cfaa2110534e2cf3ba31a7abcac9d328d1d9f1b95beede58294a60348fba36", size = 660105, upload-time = "2024-09-20T17:08:42.048Z" }, + { url = "https://files.pythonhosted.org/packages/f1/66/033e58a50fd9ec9df00a8671c74f1f3a320564c6415a4ed82a1c651654ba/greenlet-3.1.1-cp312-cp312-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:1443279c19fca463fc33e65ef2a935a5b09bb90f978beab37729e1c3c6c25fe9", size = 613077, upload-time = "2024-09-20T17:08:33.707Z" }, + { url = "https://files.pythonhosted.org/packages/19/c5/36384a06f748044d06bdd8776e231fadf92fc896bd12cb1c9f5a1bda9578/greenlet-3.1.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:b7cede291382a78f7bb5f04a529cb18e068dd29e0fb27376074b6d0317bf4dd0", size = 1135975, upload-time = "2024-09-20T17:44:15.989Z" }, + { url = "https://files.pythonhosted.org/packages/38/f9/c0a0eb61bdf808d23266ecf1d63309f0e1471f284300ce6dac0ae1231881/greenlet-3.1.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:23f20bb60ae298d7d8656c6ec6db134bca379ecefadb0b19ce6f19d1f232a942", size = 1163955, upload-time = "2024-09-20T17:09:25.539Z" }, + { url = "https://files.pythonhosted.org/packages/43/21/a5d9df1d21514883333fc86584c07c2b49ba7c602e670b174bd73cfc9c7f/greenlet-3.1.1-cp312-cp312-win_amd64.whl", hash = "sha256:7124e16b4c55d417577c2077be379514321916d5790fa287c9ed6f23bd2ffd01", size = 299655, upload-time = "2024-09-20T17:21:22.427Z" }, + { url = "https://files.pythonhosted.org/packages/f3/57/0db4940cd7bb461365ca8d6fd53e68254c9dbbcc2b452e69d0d41f10a85e/greenlet-3.1.1-cp313-cp313-macosx_11_0_universal2.whl", hash = "sha256:05175c27cb459dcfc05d026c4232f9de8913ed006d42713cb8a5137bd49375f1", size = 272990, upload-time = "2024-09-20T17:08:26.312Z" }, + { url = "https://files.pythonhosted.org/packages/1c/ec/423d113c9f74e5e402e175b157203e9102feeb7088cee844d735b28ef963/greenlet-3.1.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:935e943ec47c4afab8965954bf49bfa639c05d4ccf9ef6e924188f762145c0ff", size = 649175, upload-time = "2024-09-20T17:36:48.983Z" }, + { url = "https://files.pythonhosted.org/packages/a9/46/ddbd2db9ff209186b7b7c621d1432e2f21714adc988703dbdd0e65155c77/greenlet-3.1.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:667a9706c970cb552ede35aee17339a18e8f2a87a51fba2ed39ceeeb1004798a", size = 663425, upload-time = "2024-09-20T17:39:22.705Z" }, + { url = "https://files.pythonhosted.org/packages/bc/f9/9c82d6b2b04aa37e38e74f0c429aece5eeb02bab6e3b98e7db89b23d94c6/greenlet-3.1.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b8a678974d1f3aa55f6cc34dc480169d58f2e6d8958895d68845fa4ab566509e", size = 657736, upload-time = "2024-09-20T17:44:28.544Z" }, + { url = "https://files.pythonhosted.org/packages/d9/42/b87bc2a81e3a62c3de2b0d550bf91a86939442b7ff85abb94eec3fc0e6aa/greenlet-3.1.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:efc0f674aa41b92da8c49e0346318c6075d734994c3c4e4430b1c3f853e498e4", size = 660347, upload-time = "2024-09-20T17:08:45.56Z" }, + { url = "https://files.pythonhosted.org/packages/37/fa/71599c3fd06336cdc3eac52e6871cfebab4d9d70674a9a9e7a482c318e99/greenlet-3.1.1-cp313-cp313-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:0153404a4bb921f0ff1abeb5ce8a5131da56b953eda6e14b88dc6bbc04d2049e", size = 615583, upload-time = "2024-09-20T17:08:36.85Z" }, + { url = "https://files.pythonhosted.org/packages/4e/96/e9ef85de031703ee7a4483489b40cf307f93c1824a02e903106f2ea315fe/greenlet-3.1.1-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:275f72decf9932639c1c6dd1013a1bc266438eb32710016a1c742df5da6e60a1", size = 1133039, upload-time = "2024-09-20T17:44:18.287Z" }, + { url = "https://files.pythonhosted.org/packages/87/76/b2b6362accd69f2d1889db61a18c94bc743e961e3cab344c2effaa4b4a25/greenlet-3.1.1-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:c4aab7f6381f38a4b42f269057aee279ab0fc7bf2e929e3d4abfae97b682a12c", size = 1160716, upload-time = "2024-09-20T17:09:27.112Z" }, + { url = "https://files.pythonhosted.org/packages/1f/1b/54336d876186920e185066d8c3024ad55f21d7cc3683c856127ddb7b13ce/greenlet-3.1.1-cp313-cp313-win_amd64.whl", hash = "sha256:b42703b1cf69f2aa1df7d1030b9d77d3e584a70755674d60e710f0af570f3761", size = 299490, upload-time = "2024-09-20T17:17:09.501Z" }, + { url = "https://files.pythonhosted.org/packages/5f/17/bea55bf36990e1638a2af5ba10c1640273ef20f627962cf97107f1e5d637/greenlet-3.1.1-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f1695e76146579f8c06c1509c7ce4dfe0706f49c6831a817ac04eebb2fd02011", size = 643731, upload-time = "2024-09-20T17:36:50.376Z" }, + { url = "https://files.pythonhosted.org/packages/78/d2/aa3d2157f9ab742a08e0fd8f77d4699f37c22adfbfeb0c610a186b5f75e0/greenlet-3.1.1-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:7876452af029456b3f3549b696bb36a06db7c90747740c5302f74a9e9fa14b13", size = 649304, upload-time = "2024-09-20T17:39:24.55Z" }, + { url = "https://files.pythonhosted.org/packages/f1/8e/d0aeffe69e53ccff5a28fa86f07ad1d2d2d6537a9506229431a2a02e2f15/greenlet-3.1.1-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4ead44c85f8ab905852d3de8d86f6f8baf77109f9da589cb4fa142bd3b57b475", size = 646537, upload-time = "2024-09-20T17:44:31.102Z" }, + { url = "https://files.pythonhosted.org/packages/05/79/e15408220bbb989469c8871062c97c6c9136770657ba779711b90870d867/greenlet-3.1.1-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8320f64b777d00dd7ccdade271eaf0cad6636343293a25074cc5566160e4de7b", size = 642506, upload-time = "2024-09-20T17:08:47.852Z" }, + { url = "https://files.pythonhosted.org/packages/18/87/470e01a940307796f1d25f8167b551a968540fbe0551c0ebb853cb527dd6/greenlet-3.1.1-cp313-cp313t-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:6510bf84a6b643dabba74d3049ead221257603a253d0a9873f55f6a59a65f822", size = 602753, upload-time = "2024-09-20T17:08:38.079Z" }, + { url = "https://files.pythonhosted.org/packages/e2/72/576815ba674eddc3c25028238f74d7b8068902b3968cbe456771b166455e/greenlet-3.1.1-cp313-cp313t-musllinux_1_1_aarch64.whl", hash = "sha256:04b013dc07c96f83134b1e99888e7a79979f1a247e2a9f59697fa14b5862ed01", size = 1122731, upload-time = "2024-09-20T17:44:20.556Z" }, + { url = "https://files.pythonhosted.org/packages/ac/38/08cc303ddddc4b3d7c628c3039a61a3aae36c241ed01393d00c2fd663473/greenlet-3.1.1-cp313-cp313t-musllinux_1_1_x86_64.whl", hash = "sha256:411f015496fec93c1c8cd4e5238da364e1da7a124bcb293f085bf2860c32c6f6", size = 1142112, upload-time = "2024-09-20T17:09:28.753Z" }, ] [[package]] name = "idna" version = "3.10" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/f1/70/7703c29685631f5a7590aa73f1f1d3fa9a380e654b86af429e0934a32f7d/idna-3.10.tar.gz", hash = "sha256:12f65c9b470abda6dc35cf8e63cc574b1c52b11df2c86030af0ac09b01b13ea9", size = 190490 } +sdist = { url = "https://files.pythonhosted.org/packages/f1/70/7703c29685631f5a7590aa73f1f1d3fa9a380e654b86af429e0934a32f7d/idna-3.10.tar.gz", hash = "sha256:12f65c9b470abda6dc35cf8e63cc574b1c52b11df2c86030af0ac09b01b13ea9", size = 190490, upload-time = "2024-09-15T18:07:39.745Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/76/c6/c88e154df9c4e1a2a66ccf0005a88dfb2650c1dffb6f5ce603dfbd452ce3/idna-3.10-py3-none-any.whl", hash = "sha256:946d195a0d259cbba61165e88e65941f16e9b36ea6ddb97f00452bae8b1287d3", size = 70442 }, + { url = "https://files.pythonhosted.org/packages/76/c6/c88e154df9c4e1a2a66ccf0005a88dfb2650c1dffb6f5ce603dfbd452ce3/idna-3.10-py3-none-any.whl", hash = "sha256:946d195a0d259cbba61165e88e65941f16e9b36ea6ddb97f00452bae8b1287d3", size = 70442, upload-time = "2024-09-15T18:07:37.964Z" }, ] [[package]] name = "iniconfig" version = "2.0.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/d7/4b/cbd8e699e64a6f16ca3a8220661b5f83792b3017d0f79807cb8708d33913/iniconfig-2.0.0.tar.gz", hash = "sha256:2d91e135bf72d31a410b17c16da610a82cb55f6b0477d1a902134b24a455b8b3", size = 4646 } +sdist = { url = "https://files.pythonhosted.org/packages/d7/4b/cbd8e699e64a6f16ca3a8220661b5f83792b3017d0f79807cb8708d33913/iniconfig-2.0.0.tar.gz", hash = "sha256:2d91e135bf72d31a410b17c16da610a82cb55f6b0477d1a902134b24a455b8b3", size = 4646, upload-time = "2023-01-07T11:08:11.254Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/ef/a6/62565a6e1cf69e10f5727360368e451d4b7f58beeac6173dc9db836a5b46/iniconfig-2.0.0-py3-none-any.whl", hash = "sha256:b6a85871a79d2e3b22d2d1b94ac2824226a63c6b741c88f7ae975f18b6778374", size = 5892 }, + { url = "https://files.pythonhosted.org/packages/ef/a6/62565a6e1cf69e10f5727360368e451d4b7f58beeac6173dc9db836a5b46/iniconfig-2.0.0-py3-none-any.whl", hash = "sha256:b6a85871a79d2e3b22d2d1b94ac2824226a63c6b741c88f7ae975f18b6778374", size = 5892, upload-time = "2023-01-07T11:08:09.864Z" }, ] [[package]] name = "msgspec" version = "0.19.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/cf/9b/95d8ce458462b8b71b8a70fa94563b2498b89933689f3a7b8911edfae3d7/msgspec-0.19.0.tar.gz", hash = "sha256:604037e7cd475345848116e89c553aa9a233259733ab51986ac924ab1b976f8e", size = 216934 } +sdist = { url = "https://files.pythonhosted.org/packages/cf/9b/95d8ce458462b8b71b8a70fa94563b2498b89933689f3a7b8911edfae3d7/msgspec-0.19.0.tar.gz", hash = "sha256:604037e7cd475345848116e89c553aa9a233259733ab51986ac924ab1b976f8e", size = 216934, upload-time = "2024-12-27T17:40:28.597Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/24/d4/2ec2567ac30dab072cce3e91fb17803c52f0a37aab6b0c24375d2b20a581/msgspec-0.19.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:aa77046904db764b0462036bc63ef71f02b75b8f72e9c9dd4c447d6da1ed8f8e", size = 187939 }, - { url = "https://files.pythonhosted.org/packages/2b/c0/18226e4328897f4f19875cb62bb9259fe47e901eade9d9376ab5f251a929/msgspec-0.19.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:047cfa8675eb3bad68722cfe95c60e7afabf84d1bd8938979dd2b92e9e4a9551", size = 182202 }, - { url = "https://files.pythonhosted.org/packages/81/25/3a4b24d468203d8af90d1d351b77ea3cffb96b29492855cf83078f16bfe4/msgspec-0.19.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e78f46ff39a427e10b4a61614a2777ad69559cc8d603a7c05681f5a595ea98f7", size = 209029 }, - { url = "https://files.pythonhosted.org/packages/85/2e/db7e189b57901955239f7689b5dcd6ae9458637a9c66747326726c650523/msgspec-0.19.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6c7adf191e4bd3be0e9231c3b6dc20cf1199ada2af523885efc2ed218eafd011", size = 210682 }, - { url = "https://files.pythonhosted.org/packages/03/97/7c8895c9074a97052d7e4a1cc1230b7b6e2ca2486714eb12c3f08bb9d284/msgspec-0.19.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:f04cad4385e20be7c7176bb8ae3dca54a08e9756cfc97bcdb4f18560c3042063", size = 214003 }, - { url = "https://files.pythonhosted.org/packages/61/61/e892997bcaa289559b4d5869f066a8021b79f4bf8e955f831b095f47a4cd/msgspec-0.19.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:45c8fb410670b3b7eb884d44a75589377c341ec1392b778311acdbfa55187716", size = 216833 }, - { url = "https://files.pythonhosted.org/packages/ce/3d/71b2dffd3a1c743ffe13296ff701ee503feaebc3f04d0e75613b6563c374/msgspec-0.19.0-cp311-cp311-win_amd64.whl", hash = "sha256:70eaef4934b87193a27d802534dc466778ad8d536e296ae2f9334e182ac27b6c", size = 186184 }, - { url = "https://files.pythonhosted.org/packages/b2/5f/a70c24f075e3e7af2fae5414c7048b0e11389685b7f717bb55ba282a34a7/msgspec-0.19.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:f98bd8962ad549c27d63845b50af3f53ec468b6318400c9f1adfe8b092d7b62f", size = 190485 }, - { url = "https://files.pythonhosted.org/packages/89/b0/1b9763938cfae12acf14b682fcf05c92855974d921a5a985ecc197d1c672/msgspec-0.19.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:43bbb237feab761b815ed9df43b266114203f53596f9b6e6f00ebd79d178cdf2", size = 183910 }, - { url = "https://files.pythonhosted.org/packages/87/81/0c8c93f0b92c97e326b279795f9c5b956c5a97af28ca0fbb9fd86c83737a/msgspec-0.19.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4cfc033c02c3e0aec52b71710d7f84cb3ca5eb407ab2ad23d75631153fdb1f12", size = 210633 }, - { url = "https://files.pythonhosted.org/packages/d0/ef/c5422ce8af73928d194a6606f8ae36e93a52fd5e8df5abd366903a5ca8da/msgspec-0.19.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d911c442571605e17658ca2b416fd8579c5050ac9adc5e00c2cb3126c97f73bc", size = 213594 }, - { url = "https://files.pythonhosted.org/packages/19/2b/4137bc2ed45660444842d042be2cf5b18aa06efd2cda107cff18253b9653/msgspec-0.19.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:757b501fa57e24896cf40a831442b19a864f56d253679f34f260dcb002524a6c", size = 214053 }, - { url = "https://files.pythonhosted.org/packages/9d/e6/8ad51bdc806aac1dc501e8fe43f759f9ed7284043d722b53323ea421c360/msgspec-0.19.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:5f0f65f29b45e2816d8bded36e6b837a4bf5fb60ec4bc3c625fa2c6da4124537", size = 219081 }, - { url = "https://files.pythonhosted.org/packages/b1/ef/27dd35a7049c9a4f4211c6cd6a8c9db0a50647546f003a5867827ec45391/msgspec-0.19.0-cp312-cp312-win_amd64.whl", hash = "sha256:067f0de1c33cfa0b6a8206562efdf6be5985b988b53dd244a8e06f993f27c8c0", size = 187467 }, - { url = "https://files.pythonhosted.org/packages/3c/cb/2842c312bbe618d8fefc8b9cedce37f773cdc8fa453306546dba2c21fd98/msgspec-0.19.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:f12d30dd6266557aaaf0aa0f9580a9a8fbeadfa83699c487713e355ec5f0bd86", size = 190498 }, - { url = "https://files.pythonhosted.org/packages/58/95/c40b01b93465e1a5f3b6c7d91b10fb574818163740cc3acbe722d1e0e7e4/msgspec-0.19.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:82b2c42c1b9ebc89e822e7e13bbe9d17ede0c23c187469fdd9505afd5a481314", size = 183950 }, - { url = "https://files.pythonhosted.org/packages/e8/f0/5b764e066ce9aba4b70d1db8b087ea66098c7c27d59b9dd8a3532774d48f/msgspec-0.19.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:19746b50be214a54239aab822964f2ac81e38b0055cca94808359d779338c10e", size = 210647 }, - { url = "https://files.pythonhosted.org/packages/9d/87/bc14f49bc95c4cb0dd0a8c56028a67c014ee7e6818ccdce74a4862af259b/msgspec-0.19.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:60ef4bdb0ec8e4ad62e5a1f95230c08efb1f64f32e6e8dd2ced685bcc73858b5", size = 213563 }, - { url = "https://files.pythonhosted.org/packages/53/2f/2b1c2b056894fbaa975f68f81e3014bb447516a8b010f1bed3fb0e016ed7/msgspec-0.19.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:ac7f7c377c122b649f7545810c6cd1b47586e3aa3059126ce3516ac7ccc6a6a9", size = 213996 }, - { url = "https://files.pythonhosted.org/packages/aa/5a/4cd408d90d1417e8d2ce6a22b98a6853c1b4d7cb7669153e4424d60087f6/msgspec-0.19.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:a5bc1472223a643f5ffb5bf46ccdede7f9795078194f14edd69e3aab7020d327", size = 219087 }, - { url = "https://files.pythonhosted.org/packages/23/d8/f15b40611c2d5753d1abb0ca0da0c75348daf1252220e5dda2867bd81062/msgspec-0.19.0-cp313-cp313-win_amd64.whl", hash = "sha256:317050bc0f7739cb30d257ff09152ca309bf5a369854bbf1e57dffc310c1f20f", size = 187432 }, + { url = "https://files.pythonhosted.org/packages/24/d4/2ec2567ac30dab072cce3e91fb17803c52f0a37aab6b0c24375d2b20a581/msgspec-0.19.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:aa77046904db764b0462036bc63ef71f02b75b8f72e9c9dd4c447d6da1ed8f8e", size = 187939, upload-time = "2024-12-27T17:39:32.347Z" }, + { url = "https://files.pythonhosted.org/packages/2b/c0/18226e4328897f4f19875cb62bb9259fe47e901eade9d9376ab5f251a929/msgspec-0.19.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:047cfa8675eb3bad68722cfe95c60e7afabf84d1bd8938979dd2b92e9e4a9551", size = 182202, upload-time = "2024-12-27T17:39:33.633Z" }, + { url = "https://files.pythonhosted.org/packages/81/25/3a4b24d468203d8af90d1d351b77ea3cffb96b29492855cf83078f16bfe4/msgspec-0.19.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e78f46ff39a427e10b4a61614a2777ad69559cc8d603a7c05681f5a595ea98f7", size = 209029, upload-time = "2024-12-27T17:39:35.023Z" }, + { url = "https://files.pythonhosted.org/packages/85/2e/db7e189b57901955239f7689b5dcd6ae9458637a9c66747326726c650523/msgspec-0.19.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6c7adf191e4bd3be0e9231c3b6dc20cf1199ada2af523885efc2ed218eafd011", size = 210682, upload-time = "2024-12-27T17:39:36.384Z" }, + { url = "https://files.pythonhosted.org/packages/03/97/7c8895c9074a97052d7e4a1cc1230b7b6e2ca2486714eb12c3f08bb9d284/msgspec-0.19.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:f04cad4385e20be7c7176bb8ae3dca54a08e9756cfc97bcdb4f18560c3042063", size = 214003, upload-time = "2024-12-27T17:39:39.097Z" }, + { url = "https://files.pythonhosted.org/packages/61/61/e892997bcaa289559b4d5869f066a8021b79f4bf8e955f831b095f47a4cd/msgspec-0.19.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:45c8fb410670b3b7eb884d44a75589377c341ec1392b778311acdbfa55187716", size = 216833, upload-time = "2024-12-27T17:39:41.203Z" }, + { url = "https://files.pythonhosted.org/packages/ce/3d/71b2dffd3a1c743ffe13296ff701ee503feaebc3f04d0e75613b6563c374/msgspec-0.19.0-cp311-cp311-win_amd64.whl", hash = "sha256:70eaef4934b87193a27d802534dc466778ad8d536e296ae2f9334e182ac27b6c", size = 186184, upload-time = "2024-12-27T17:39:43.702Z" }, + { url = "https://files.pythonhosted.org/packages/b2/5f/a70c24f075e3e7af2fae5414c7048b0e11389685b7f717bb55ba282a34a7/msgspec-0.19.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:f98bd8962ad549c27d63845b50af3f53ec468b6318400c9f1adfe8b092d7b62f", size = 190485, upload-time = "2024-12-27T17:39:44.974Z" }, + { url = "https://files.pythonhosted.org/packages/89/b0/1b9763938cfae12acf14b682fcf05c92855974d921a5a985ecc197d1c672/msgspec-0.19.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:43bbb237feab761b815ed9df43b266114203f53596f9b6e6f00ebd79d178cdf2", size = 183910, upload-time = "2024-12-27T17:39:46.401Z" }, + { url = "https://files.pythonhosted.org/packages/87/81/0c8c93f0b92c97e326b279795f9c5b956c5a97af28ca0fbb9fd86c83737a/msgspec-0.19.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4cfc033c02c3e0aec52b71710d7f84cb3ca5eb407ab2ad23d75631153fdb1f12", size = 210633, upload-time = "2024-12-27T17:39:49.099Z" }, + { url = "https://files.pythonhosted.org/packages/d0/ef/c5422ce8af73928d194a6606f8ae36e93a52fd5e8df5abd366903a5ca8da/msgspec-0.19.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d911c442571605e17658ca2b416fd8579c5050ac9adc5e00c2cb3126c97f73bc", size = 213594, upload-time = "2024-12-27T17:39:51.204Z" }, + { url = "https://files.pythonhosted.org/packages/19/2b/4137bc2ed45660444842d042be2cf5b18aa06efd2cda107cff18253b9653/msgspec-0.19.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:757b501fa57e24896cf40a831442b19a864f56d253679f34f260dcb002524a6c", size = 214053, upload-time = "2024-12-27T17:39:52.866Z" }, + { url = "https://files.pythonhosted.org/packages/9d/e6/8ad51bdc806aac1dc501e8fe43f759f9ed7284043d722b53323ea421c360/msgspec-0.19.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:5f0f65f29b45e2816d8bded36e6b837a4bf5fb60ec4bc3c625fa2c6da4124537", size = 219081, upload-time = "2024-12-27T17:39:55.142Z" }, + { url = "https://files.pythonhosted.org/packages/b1/ef/27dd35a7049c9a4f4211c6cd6a8c9db0a50647546f003a5867827ec45391/msgspec-0.19.0-cp312-cp312-win_amd64.whl", hash = "sha256:067f0de1c33cfa0b6a8206562efdf6be5985b988b53dd244a8e06f993f27c8c0", size = 187467, upload-time = "2024-12-27T17:39:56.531Z" }, + { url = "https://files.pythonhosted.org/packages/3c/cb/2842c312bbe618d8fefc8b9cedce37f773cdc8fa453306546dba2c21fd98/msgspec-0.19.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:f12d30dd6266557aaaf0aa0f9580a9a8fbeadfa83699c487713e355ec5f0bd86", size = 190498, upload-time = "2024-12-27T17:40:00.427Z" }, + { url = "https://files.pythonhosted.org/packages/58/95/c40b01b93465e1a5f3b6c7d91b10fb574818163740cc3acbe722d1e0e7e4/msgspec-0.19.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:82b2c42c1b9ebc89e822e7e13bbe9d17ede0c23c187469fdd9505afd5a481314", size = 183950, upload-time = "2024-12-27T17:40:04.219Z" }, + { url = "https://files.pythonhosted.org/packages/e8/f0/5b764e066ce9aba4b70d1db8b087ea66098c7c27d59b9dd8a3532774d48f/msgspec-0.19.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:19746b50be214a54239aab822964f2ac81e38b0055cca94808359d779338c10e", size = 210647, upload-time = "2024-12-27T17:40:05.606Z" }, + { url = "https://files.pythonhosted.org/packages/9d/87/bc14f49bc95c4cb0dd0a8c56028a67c014ee7e6818ccdce74a4862af259b/msgspec-0.19.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:60ef4bdb0ec8e4ad62e5a1f95230c08efb1f64f32e6e8dd2ced685bcc73858b5", size = 213563, upload-time = "2024-12-27T17:40:10.516Z" }, + { url = "https://files.pythonhosted.org/packages/53/2f/2b1c2b056894fbaa975f68f81e3014bb447516a8b010f1bed3fb0e016ed7/msgspec-0.19.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:ac7f7c377c122b649f7545810c6cd1b47586e3aa3059126ce3516ac7ccc6a6a9", size = 213996, upload-time = "2024-12-27T17:40:12.244Z" }, + { url = "https://files.pythonhosted.org/packages/aa/5a/4cd408d90d1417e8d2ce6a22b98a6853c1b4d7cb7669153e4424d60087f6/msgspec-0.19.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:a5bc1472223a643f5ffb5bf46ccdede7f9795078194f14edd69e3aab7020d327", size = 219087, upload-time = "2024-12-27T17:40:14.881Z" }, + { url = "https://files.pythonhosted.org/packages/23/d8/f15b40611c2d5753d1abb0ca0da0c75348daf1252220e5dda2867bd81062/msgspec-0.19.0-cp313-cp313-win_amd64.whl", hash = "sha256:317050bc0f7739cb30d257ff09152ca309bf5a369854bbf1e57dffc310c1f20f", size = 187432, upload-time = "2024-12-27T17:40:16.256Z" }, ] [[package]] @@ -196,18 +196,18 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "attrs" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/98/df/77698abfac98571e65ffeb0c1fba8ffd692ab8458d617a0eed7d9a8d38f2/outcome-1.3.0.post0.tar.gz", hash = "sha256:9dcf02e65f2971b80047b377468e72a268e15c0af3cf1238e6ff14f7f91143b8", size = 21060 } +sdist = { url = "https://files.pythonhosted.org/packages/98/df/77698abfac98571e65ffeb0c1fba8ffd692ab8458d617a0eed7d9a8d38f2/outcome-1.3.0.post0.tar.gz", hash = "sha256:9dcf02e65f2971b80047b377468e72a268e15c0af3cf1238e6ff14f7f91143b8", size = 21060, upload-time = "2023-10-26T04:26:04.361Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/55/8b/5ab7257531a5d830fc8000c476e63c935488d74609b50f9384a643ec0a62/outcome-1.3.0.post0-py2.py3-none-any.whl", hash = "sha256:e771c5ce06d1415e356078d3bdd68523f284b4ce5419828922b6871e65eda82b", size = 10692 }, + { url = "https://files.pythonhosted.org/packages/55/8b/5ab7257531a5d830fc8000c476e63c935488d74609b50f9384a643ec0a62/outcome-1.3.0.post0-py2.py3-none-any.whl", hash = "sha256:e771c5ce06d1415e356078d3bdd68523f284b4ce5419828922b6871e65eda82b", size = 10692, upload-time = "2023-10-26T04:26:02.532Z" }, ] [[package]] name = "packaging" version = "24.2" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/d0/63/68dbb6eb2de9cb10ee4c9c14a0148804425e13c4fb20d61cce69f53106da/packaging-24.2.tar.gz", hash = "sha256:c228a6dc5e932d346bc5739379109d49e8853dd8223571c7c5b55260edc0b97f", size = 163950 } +sdist = { url = "https://files.pythonhosted.org/packages/d0/63/68dbb6eb2de9cb10ee4c9c14a0148804425e13c4fb20d61cce69f53106da/packaging-24.2.tar.gz", hash = "sha256:c228a6dc5e932d346bc5739379109d49e8853dd8223571c7c5b55260edc0b97f", size = 163950, upload-time = "2024-11-08T09:47:47.202Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/88/ef/eb23f262cca3c0c4eb7ab1933c3b1f03d021f2c48f54763065b6f0e321be/packaging-24.2-py3-none-any.whl", hash = "sha256:09abb1bccd265c01f4a3aa3f7a7db064b36514d2cba19a2f694fe6150451a759", size = 65451 }, + { url = "https://files.pythonhosted.org/packages/88/ef/eb23f262cca3c0c4eb7ab1933c3b1f03d021f2c48f54763065b6f0e321be/packaging-24.2-py3-none-any.whl", hash = "sha256:09abb1bccd265c01f4a3aa3f7a7db064b36514d2cba19a2f694fe6150451a759", size = 65451, upload-time = "2024-11-08T09:47:44.722Z" }, ] [[package]] @@ -219,9 +219,9 @@ dependencies = [ { name = "pygments" }, { name = "tabcompleter" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/69/13/80da03638f62facbee76312ca9ee5941c017b080f2e4c6919fd4e87e16e3/pdbp-1.6.1.tar.gz", hash = "sha256:f4041642952a05df89664e166d5bd379607a0866ddd753c06874f65552bdf40b", size = 25322 } +sdist = { url = "https://files.pythonhosted.org/packages/69/13/80da03638f62facbee76312ca9ee5941c017b080f2e4c6919fd4e87e16e3/pdbp-1.6.1.tar.gz", hash = "sha256:f4041642952a05df89664e166d5bd379607a0866ddd753c06874f65552bdf40b", size = 25322, upload-time = "2024-11-07T15:36:43.062Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/29/93/d56fb9ba5569dc29d8263c72e46d21a2fd38741339ebf03f54cf7561828c/pdbp-1.6.1-py3-none-any.whl", hash = "sha256:f10bad2ee044c0e5c168cb0825abfdbdc01c50013e9755df5261b060bdd35c22", size = 21495 }, + { url = "https://files.pythonhosted.org/packages/29/93/d56fb9ba5569dc29d8263c72e46d21a2fd38741339ebf03f54cf7561828c/pdbp-1.6.1-py3-none-any.whl", hash = "sha256:f10bad2ee044c0e5c168cb0825abfdbdc01c50013e9755df5261b060bdd35c22", size = 21495, upload-time = "2024-11-07T15:36:41.061Z" }, ] [[package]] @@ -231,18 +231,18 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "ptyprocess" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/42/92/cc564bf6381ff43ce1f4d06852fc19a2f11d180f23dc32d9588bee2f149d/pexpect-4.9.0.tar.gz", hash = "sha256:ee7d41123f3c9911050ea2c2dac107568dc43b2d3b0c7557a33212c398ead30f", size = 166450 } +sdist = { url = "https://files.pythonhosted.org/packages/42/92/cc564bf6381ff43ce1f4d06852fc19a2f11d180f23dc32d9588bee2f149d/pexpect-4.9.0.tar.gz", hash = "sha256:ee7d41123f3c9911050ea2c2dac107568dc43b2d3b0c7557a33212c398ead30f", size = 166450, upload-time = "2023-11-25T09:07:26.339Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/9e/c3/059298687310d527a58bb01f3b1965787ee3b40dce76752eda8b44e9a2c5/pexpect-4.9.0-py2.py3-none-any.whl", hash = "sha256:7236d1e080e4936be2dc3e326cec0af72acf9212a7e1d060210e70a47e253523", size = 63772 }, + { url = "https://files.pythonhosted.org/packages/9e/c3/059298687310d527a58bb01f3b1965787ee3b40dce76752eda8b44e9a2c5/pexpect-4.9.0-py2.py3-none-any.whl", hash = "sha256:7236d1e080e4936be2dc3e326cec0af72acf9212a7e1d060210e70a47e253523", size = 63772, upload-time = "2023-11-25T06:56:14.81Z" }, ] [[package]] name = "pluggy" version = "1.5.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/96/2d/02d4312c973c6050a18b314a5ad0b3210edb65a906f868e31c111dede4a6/pluggy-1.5.0.tar.gz", hash = "sha256:2cffa88e94fdc978c4c574f15f9e59b7f4201d439195c3715ca9e2486f1d0cf1", size = 67955 } +sdist = { url = "https://files.pythonhosted.org/packages/96/2d/02d4312c973c6050a18b314a5ad0b3210edb65a906f868e31c111dede4a6/pluggy-1.5.0.tar.gz", hash = "sha256:2cffa88e94fdc978c4c574f15f9e59b7f4201d439195c3715ca9e2486f1d0cf1", size = 67955, upload-time = "2024-04-20T21:34:42.531Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/88/5f/e351af9a41f866ac3f1fac4ca0613908d9a41741cfcf2228f4ad853b697d/pluggy-1.5.0-py3-none-any.whl", hash = "sha256:44e1ad92c8ca002de6377e165f3e0f1be63266ab4d554740532335b9d75ea669", size = 20556 }, + { url = "https://files.pythonhosted.org/packages/88/5f/e351af9a41f866ac3f1fac4ca0613908d9a41741cfcf2228f4ad853b697d/pluggy-1.5.0-py3-none-any.whl", hash = "sha256:44e1ad92c8ca002de6377e165f3e0f1be63266ab4d554740532335b9d75ea669", size = 20556, upload-time = "2024-04-20T21:34:40.434Z" }, ] [[package]] @@ -252,66 +252,66 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "wcwidth" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/a1/e1/bd15cb8ffdcfeeb2bdc215de3c3cffca11408d829e4b8416dcfe71ba8854/prompt_toolkit-3.0.50.tar.gz", hash = "sha256:544748f3860a2623ca5cd6d2795e7a14f3d0e1c3c9728359013f79877fc89bab", size = 429087 } +sdist = { url = "https://files.pythonhosted.org/packages/a1/e1/bd15cb8ffdcfeeb2bdc215de3c3cffca11408d829e4b8416dcfe71ba8854/prompt_toolkit-3.0.50.tar.gz", hash = "sha256:544748f3860a2623ca5cd6d2795e7a14f3d0e1c3c9728359013f79877fc89bab", size = 429087, upload-time = "2025-01-20T15:55:35.072Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/e4/ea/d836f008d33151c7a1f62caf3d8dd782e4d15f6a43897f64480c2b8de2ad/prompt_toolkit-3.0.50-py3-none-any.whl", hash = "sha256:9b6427eb19e479d98acff65196a307c555eb567989e6d88ebbb1b509d9779198", size = 387816 }, + { url = "https://files.pythonhosted.org/packages/e4/ea/d836f008d33151c7a1f62caf3d8dd782e4d15f6a43897f64480c2b8de2ad/prompt_toolkit-3.0.50-py3-none-any.whl", hash = "sha256:9b6427eb19e479d98acff65196a307c555eb567989e6d88ebbb1b509d9779198", size = 387816, upload-time = "2025-01-20T15:55:29.98Z" }, ] [[package]] name = "psutil" version = "7.0.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/2a/80/336820c1ad9286a4ded7e845b2eccfcb27851ab8ac6abece774a6ff4d3de/psutil-7.0.0.tar.gz", hash = "sha256:7be9c3eba38beccb6495ea33afd982a44074b78f28c434a1f51cc07fd315c456", size = 497003 } +sdist = { url = "https://files.pythonhosted.org/packages/2a/80/336820c1ad9286a4ded7e845b2eccfcb27851ab8ac6abece774a6ff4d3de/psutil-7.0.0.tar.gz", hash = "sha256:7be9c3eba38beccb6495ea33afd982a44074b78f28c434a1f51cc07fd315c456", size = 497003, upload-time = "2025-02-13T21:54:07.946Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/ed/e6/2d26234410f8b8abdbf891c9da62bee396583f713fb9f3325a4760875d22/psutil-7.0.0-cp36-abi3-macosx_10_9_x86_64.whl", hash = "sha256:101d71dc322e3cffd7cea0650b09b3d08b8e7c4109dd6809fe452dfd00e58b25", size = 238051 }, - { url = "https://files.pythonhosted.org/packages/04/8b/30f930733afe425e3cbfc0e1468a30a18942350c1a8816acfade80c005c4/psutil-7.0.0-cp36-abi3-macosx_11_0_arm64.whl", hash = "sha256:39db632f6bb862eeccf56660871433e111b6ea58f2caea825571951d4b6aa3da", size = 239535 }, - { url = "https://files.pythonhosted.org/packages/2a/ed/d362e84620dd22876b55389248e522338ed1bf134a5edd3b8231d7207f6d/psutil-7.0.0-cp36-abi3-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1fcee592b4c6f146991ca55919ea3d1f8926497a713ed7faaf8225e174581e91", size = 275004 }, - { url = "https://files.pythonhosted.org/packages/bf/b9/b0eb3f3cbcb734d930fdf839431606844a825b23eaf9a6ab371edac8162c/psutil-7.0.0-cp36-abi3-manylinux_2_12_x86_64.manylinux2010_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4b1388a4f6875d7e2aff5c4ca1cc16c545ed41dd8bb596cefea80111db353a34", size = 277986 }, - { url = "https://files.pythonhosted.org/packages/eb/a2/709e0fe2f093556c17fbafda93ac032257242cabcc7ff3369e2cb76a97aa/psutil-7.0.0-cp36-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a5f098451abc2828f7dc6b58d44b532b22f2088f4999a937557b603ce72b1993", size = 279544 }, - { url = "https://files.pythonhosted.org/packages/50/e6/eecf58810b9d12e6427369784efe814a1eec0f492084ce8eb8f4d89d6d61/psutil-7.0.0-cp37-abi3-win32.whl", hash = "sha256:ba3fcef7523064a6c9da440fc4d6bd07da93ac726b5733c29027d7dc95b39d99", size = 241053 }, - { url = "https://files.pythonhosted.org/packages/50/1b/6921afe68c74868b4c9fa424dad3be35b095e16687989ebbb50ce4fceb7c/psutil-7.0.0-cp37-abi3-win_amd64.whl", hash = "sha256:4cf3d4eb1aa9b348dec30105c55cd9b7d4629285735a102beb4441e38db90553", size = 244885 }, + { url = "https://files.pythonhosted.org/packages/ed/e6/2d26234410f8b8abdbf891c9da62bee396583f713fb9f3325a4760875d22/psutil-7.0.0-cp36-abi3-macosx_10_9_x86_64.whl", hash = "sha256:101d71dc322e3cffd7cea0650b09b3d08b8e7c4109dd6809fe452dfd00e58b25", size = 238051, upload-time = "2025-02-13T21:54:12.36Z" }, + { url = "https://files.pythonhosted.org/packages/04/8b/30f930733afe425e3cbfc0e1468a30a18942350c1a8816acfade80c005c4/psutil-7.0.0-cp36-abi3-macosx_11_0_arm64.whl", hash = "sha256:39db632f6bb862eeccf56660871433e111b6ea58f2caea825571951d4b6aa3da", size = 239535, upload-time = "2025-02-13T21:54:16.07Z" }, + { url = "https://files.pythonhosted.org/packages/2a/ed/d362e84620dd22876b55389248e522338ed1bf134a5edd3b8231d7207f6d/psutil-7.0.0-cp36-abi3-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1fcee592b4c6f146991ca55919ea3d1f8926497a713ed7faaf8225e174581e91", size = 275004, upload-time = "2025-02-13T21:54:18.662Z" }, + { url = "https://files.pythonhosted.org/packages/bf/b9/b0eb3f3cbcb734d930fdf839431606844a825b23eaf9a6ab371edac8162c/psutil-7.0.0-cp36-abi3-manylinux_2_12_x86_64.manylinux2010_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4b1388a4f6875d7e2aff5c4ca1cc16c545ed41dd8bb596cefea80111db353a34", size = 277986, upload-time = "2025-02-13T21:54:21.811Z" }, + { url = "https://files.pythonhosted.org/packages/eb/a2/709e0fe2f093556c17fbafda93ac032257242cabcc7ff3369e2cb76a97aa/psutil-7.0.0-cp36-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a5f098451abc2828f7dc6b58d44b532b22f2088f4999a937557b603ce72b1993", size = 279544, upload-time = "2025-02-13T21:54:24.68Z" }, + { url = "https://files.pythonhosted.org/packages/50/e6/eecf58810b9d12e6427369784efe814a1eec0f492084ce8eb8f4d89d6d61/psutil-7.0.0-cp37-abi3-win32.whl", hash = "sha256:ba3fcef7523064a6c9da440fc4d6bd07da93ac726b5733c29027d7dc95b39d99", size = 241053, upload-time = "2025-02-13T21:54:34.31Z" }, + { url = "https://files.pythonhosted.org/packages/50/1b/6921afe68c74868b4c9fa424dad3be35b095e16687989ebbb50ce4fceb7c/psutil-7.0.0-cp37-abi3-win_amd64.whl", hash = "sha256:4cf3d4eb1aa9b348dec30105c55cd9b7d4629285735a102beb4441e38db90553", size = 244885, upload-time = "2025-02-13T21:54:37.486Z" }, ] [[package]] name = "ptyprocess" version = "0.7.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/20/e5/16ff212c1e452235a90aeb09066144d0c5a6a8c0834397e03f5224495c4e/ptyprocess-0.7.0.tar.gz", hash = "sha256:5c5d0a3b48ceee0b48485e0c26037c0acd7d29765ca3fbb5cb3831d347423220", size = 70762 } +sdist = { url = "https://files.pythonhosted.org/packages/20/e5/16ff212c1e452235a90aeb09066144d0c5a6a8c0834397e03f5224495c4e/ptyprocess-0.7.0.tar.gz", hash = "sha256:5c5d0a3b48ceee0b48485e0c26037c0acd7d29765ca3fbb5cb3831d347423220", size = 70762, upload-time = "2020-12-28T15:15:30.155Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/22/a6/858897256d0deac81a172289110f31629fc4cee19b6f01283303e18c8db3/ptyprocess-0.7.0-py2.py3-none-any.whl", hash = "sha256:4b41f3967fce3af57cc7e94b888626c18bf37a083e3651ca8feeb66d492fef35", size = 13993 }, + { url = "https://files.pythonhosted.org/packages/22/a6/858897256d0deac81a172289110f31629fc4cee19b6f01283303e18c8db3/ptyprocess-0.7.0-py2.py3-none-any.whl", hash = "sha256:4b41f3967fce3af57cc7e94b888626c18bf37a083e3651ca8feeb66d492fef35", size = 13993, upload-time = "2020-12-28T15:15:28.35Z" }, ] [[package]] name = "pycparser" version = "2.22" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/1d/b2/31537cf4b1ca988837256c910a668b553fceb8f069bedc4b1c826024b52c/pycparser-2.22.tar.gz", hash = "sha256:491c8be9c040f5390f5bf44a5b07752bd07f56edf992381b05c701439eec10f6", size = 172736 } +sdist = { url = "https://files.pythonhosted.org/packages/1d/b2/31537cf4b1ca988837256c910a668b553fceb8f069bedc4b1c826024b52c/pycparser-2.22.tar.gz", hash = "sha256:491c8be9c040f5390f5bf44a5b07752bd07f56edf992381b05c701439eec10f6", size = 172736, upload-time = "2024-03-30T13:22:22.564Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/13/a3/a812df4e2dd5696d1f351d58b8fe16a405b234ad2886a0dab9183fb78109/pycparser-2.22-py3-none-any.whl", hash = "sha256:c3702b6d3dd8c7abc1afa565d7e63d53a1d0bd86cdc24edd75470f4de499cfcc", size = 117552 }, + { url = "https://files.pythonhosted.org/packages/13/a3/a812df4e2dd5696d1f351d58b8fe16a405b234ad2886a0dab9183fb78109/pycparser-2.22-py3-none-any.whl", hash = "sha256:c3702b6d3dd8c7abc1afa565d7e63d53a1d0bd86cdc24edd75470f4de499cfcc", size = 117552, upload-time = "2024-03-30T13:22:20.476Z" }, ] [[package]] name = "pygments" version = "2.19.1" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/7c/2d/c3338d48ea6cc0feb8446d8e6937e1408088a72a39937982cc6111d17f84/pygments-2.19.1.tar.gz", hash = "sha256:61c16d2a8576dc0649d9f39e089b5f02bcd27fba10d8fb4dcc28173f7a45151f", size = 4968581 } +sdist = { url = "https://files.pythonhosted.org/packages/7c/2d/c3338d48ea6cc0feb8446d8e6937e1408088a72a39937982cc6111d17f84/pygments-2.19.1.tar.gz", hash = "sha256:61c16d2a8576dc0649d9f39e089b5f02bcd27fba10d8fb4dcc28173f7a45151f", size = 4968581, upload-time = "2025-01-06T17:26:30.443Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/8a/0b/9fcc47d19c48b59121088dd6da2488a49d5f72dacf8262e2790a1d2c7d15/pygments-2.19.1-py3-none-any.whl", hash = "sha256:9ea1544ad55cecf4b8242fab6dd35a93bbce657034b0611ee383099054ab6d8c", size = 1225293 }, + { url = "https://files.pythonhosted.org/packages/8a/0b/9fcc47d19c48b59121088dd6da2488a49d5f72dacf8262e2790a1d2c7d15/pygments-2.19.1-py3-none-any.whl", hash = "sha256:9ea1544ad55cecf4b8242fab6dd35a93bbce657034b0611ee383099054ab6d8c", size = 1225293, upload-time = "2025-01-06T17:26:25.553Z" }, ] [[package]] name = "pyperclip" version = "1.9.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/30/23/2f0a3efc4d6a32f3b63cdff36cd398d9701d26cda58e3ab97ac79fb5e60d/pyperclip-1.9.0.tar.gz", hash = "sha256:b7de0142ddc81bfc5c7507eea19da920b92252b548b96186caf94a5e2527d310", size = 20961 } +sdist = { url = "https://files.pythonhosted.org/packages/30/23/2f0a3efc4d6a32f3b63cdff36cd398d9701d26cda58e3ab97ac79fb5e60d/pyperclip-1.9.0.tar.gz", hash = "sha256:b7de0142ddc81bfc5c7507eea19da920b92252b548b96186caf94a5e2527d310", size = 20961, upload-time = "2024-06-18T20:38:48.401Z" } [[package]] name = "pyreadline3" version = "3.5.4" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/0f/49/4cea918a08f02817aabae639e3d0ac046fef9f9180518a3ad394e22da148/pyreadline3-3.5.4.tar.gz", hash = "sha256:8d57d53039a1c75adba8e50dd3d992b28143480816187ea5efbd5c78e6c885b7", size = 99839 } +sdist = { url = "https://files.pythonhosted.org/packages/0f/49/4cea918a08f02817aabae639e3d0ac046fef9f9180518a3ad394e22da148/pyreadline3-3.5.4.tar.gz", hash = "sha256:8d57d53039a1c75adba8e50dd3d992b28143480816187ea5efbd5c78e6c885b7", size = 99839, upload-time = "2024-09-19T02:40:10.062Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/5a/dc/491b7661614ab97483abf2056be1deee4dc2490ecbf7bff9ab5cdbac86e1/pyreadline3-3.5.4-py3-none-any.whl", hash = "sha256:eaf8e6cc3c49bcccf145fc6067ba8643d1df34d604a1ec0eccbf7a18e6d3fae6", size = 83178 }, + { url = "https://files.pythonhosted.org/packages/5a/dc/491b7661614ab97483abf2056be1deee4dc2490ecbf7bff9ab5cdbac86e1/pyreadline3-3.5.4-py3-none-any.whl", hash = "sha256:eaf8e6cc3c49bcccf145fc6067ba8643d1df34d604a1ec0eccbf7a18e6d3fae6", size = 83178, upload-time = "2024-09-19T02:40:08.598Z" }, ] [[package]] @@ -324,36 +324,36 @@ dependencies = [ { name = "packaging" }, { name = "pluggy" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/ae/3c/c9d525a414d506893f0cd8a8d0de7706446213181570cdbd766691164e40/pytest-8.3.5.tar.gz", hash = "sha256:f4efe70cc14e511565ac476b57c279e12a855b11f48f212af1080ef2263d3845", size = 1450891 } +sdist = { url = "https://files.pythonhosted.org/packages/ae/3c/c9d525a414d506893f0cd8a8d0de7706446213181570cdbd766691164e40/pytest-8.3.5.tar.gz", hash = "sha256:f4efe70cc14e511565ac476b57c279e12a855b11f48f212af1080ef2263d3845", size = 1450891, upload-time = "2025-03-02T12:54:54.503Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/30/3d/64ad57c803f1fa1e963a7946b6e0fea4a70df53c1a7fed304586539c2bac/pytest-8.3.5-py3-none-any.whl", hash = "sha256:c69214aa47deac29fad6c2a4f590b9c4a9fdb16a403176fe154b79c0b4d4d820", size = 343634 }, + { url = "https://files.pythonhosted.org/packages/30/3d/64ad57c803f1fa1e963a7946b6e0fea4a70df53c1a7fed304586539c2bac/pytest-8.3.5-py3-none-any.whl", hash = "sha256:c69214aa47deac29fad6c2a4f590b9c4a9fdb16a403176fe154b79c0b4d4d820", size = 343634, upload-time = "2025-03-02T12:54:52.069Z" }, ] [[package]] name = "sniffio" version = "1.3.1" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/a2/87/a6771e1546d97e7e041b6ae58d80074f81b7d5121207425c964ddf5cfdbd/sniffio-1.3.1.tar.gz", hash = "sha256:f4324edc670a0f49750a81b895f35c3adb843cca46f0530f79fc1babb23789dc", size = 20372 } +sdist = { url = "https://files.pythonhosted.org/packages/a2/87/a6771e1546d97e7e041b6ae58d80074f81b7d5121207425c964ddf5cfdbd/sniffio-1.3.1.tar.gz", hash = "sha256:f4324edc670a0f49750a81b895f35c3adb843cca46f0530f79fc1babb23789dc", size = 20372, upload-time = "2024-02-25T23:20:04.057Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/e9/44/75a9c9421471a6c4805dbf2356f7c181a29c1879239abab1ea2cc8f38b40/sniffio-1.3.1-py3-none-any.whl", hash = "sha256:2f6da418d1f1e0fddd844478f41680e794e6051915791a034ff65e5f100525a2", size = 10235 }, + { url = "https://files.pythonhosted.org/packages/e9/44/75a9c9421471a6c4805dbf2356f7c181a29c1879239abab1ea2cc8f38b40/sniffio-1.3.1-py3-none-any.whl", hash = "sha256:2f6da418d1f1e0fddd844478f41680e794e6051915791a034ff65e5f100525a2", size = 10235, upload-time = "2024-02-25T23:20:01.196Z" }, ] [[package]] name = "sortedcontainers" version = "2.4.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/e8/c4/ba2f8066cceb6f23394729afe52f3bf7adec04bf9ed2c820b39e19299111/sortedcontainers-2.4.0.tar.gz", hash = "sha256:25caa5a06cc30b6b83d11423433f65d1f9d76c4c6a0c90e3379eaa43b9bfdb88", size = 30594 } +sdist = { url = "https://files.pythonhosted.org/packages/e8/c4/ba2f8066cceb6f23394729afe52f3bf7adec04bf9ed2c820b39e19299111/sortedcontainers-2.4.0.tar.gz", hash = "sha256:25caa5a06cc30b6b83d11423433f65d1f9d76c4c6a0c90e3379eaa43b9bfdb88", size = 30594, upload-time = "2021-05-16T22:03:42.897Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/32/46/9cb0e58b2deb7f82b84065f37f3bffeb12413f947f9388e4cac22c4621ce/sortedcontainers-2.4.0-py2.py3-none-any.whl", hash = "sha256:a163dcaede0f1c021485e957a39245190e74249897e2ae4b2aa38595db237ee0", size = 29575 }, + { url = "https://files.pythonhosted.org/packages/32/46/9cb0e58b2deb7f82b84065f37f3bffeb12413f947f9388e4cac22c4621ce/sortedcontainers-2.4.0-py2.py3-none-any.whl", hash = "sha256:a163dcaede0f1c021485e957a39245190e74249897e2ae4b2aa38595db237ee0", size = 29575, upload-time = "2021-05-16T22:03:41.177Z" }, ] [[package]] name = "stackscope" version = "0.2.2" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/4a/fc/20dbb993353f31230138f3c63f3f0c881d1853e70d7a30cd68d2ba4cf1e2/stackscope-0.2.2.tar.gz", hash = "sha256:f508c93eb4861ada466dd3ff613ca203962ceb7587ad013759f15394e6a4e619", size = 90479 } +sdist = { url = "https://files.pythonhosted.org/packages/4a/fc/20dbb993353f31230138f3c63f3f0c881d1853e70d7a30cd68d2ba4cf1e2/stackscope-0.2.2.tar.gz", hash = "sha256:f508c93eb4861ada466dd3ff613ca203962ceb7587ad013759f15394e6a4e619", size = 90479, upload-time = "2024-02-27T22:02:15.831Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/f1/5f/0a674fcafa03528089badb46419413f342537b5b57d2fefc9900fb8ee4e4/stackscope-0.2.2-py3-none-any.whl", hash = "sha256:c199b0cda738d39c993ee04eb01961b06b7e9aeb43ebf9fd6226cdd72ea9faf6", size = 80807 }, + { url = "https://files.pythonhosted.org/packages/f1/5f/0a674fcafa03528089badb46419413f342537b5b57d2fefc9900fb8ee4e4/stackscope-0.2.2-py3-none-any.whl", hash = "sha256:c199b0cda738d39c993ee04eb01961b06b7e9aeb43ebf9fd6226cdd72ea9faf6", size = 80807, upload-time = "2024-02-27T22:02:13.692Z" }, ] [[package]] @@ -363,9 +363,9 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "pyreadline3", marker = "sys_platform == 'win32'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/73/1a/ed3544579628c5709bae6fae2255e94c6982a9ff77d42d8ba59fd2f3b21a/tabcompleter-1.4.0.tar.gz", hash = "sha256:7562a9938e62f8e7c3be612c3ac4e14c5ec4307b58ba9031c148260e866e8814", size = 10431 } +sdist = { url = "https://files.pythonhosted.org/packages/73/1a/ed3544579628c5709bae6fae2255e94c6982a9ff77d42d8ba59fd2f3b21a/tabcompleter-1.4.0.tar.gz", hash = "sha256:7562a9938e62f8e7c3be612c3ac4e14c5ec4307b58ba9031c148260e866e8814", size = 10431, upload-time = "2024-10-28T00:44:52.665Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/65/44/bb509c3d2c0b5a87e7a5af1d5917a402a32ff026f777a6d7cb6990746cbb/tabcompleter-1.4.0-py3-none-any.whl", hash = "sha256:d744aa735b49c0a6cc2fb8fcd40077fec47425e4388301010b14e6ce3311368b", size = 6725 }, + { url = "https://files.pythonhosted.org/packages/65/44/bb509c3d2c0b5a87e7a5af1d5917a402a32ff026f777a6d7cb6990746cbb/tabcompleter-1.4.0-py3-none-any.whl", hash = "sha256:d744aa735b49c0a6cc2fb8fcd40077fec47425e4388301010b14e6ce3311368b", size = 6725, upload-time = "2024-10-28T00:44:51.267Z" }, ] [[package]] @@ -392,6 +392,7 @@ dev = [ { name = "pyperclip" }, { name = "pytest" }, { name = "stackscope" }, + { name = "typing-extensions" }, { name = "xonsh" }, ] @@ -416,6 +417,7 @@ dev = [ { name = "pyperclip", specifier = ">=1.9.0" }, { name = "pytest", specifier = ">=8.3.5" }, { name = "stackscope", specifier = ">=0.2.2,<0.3" }, + { name = "typing-extensions", specifier = ">=4.13.2" }, { name = "xonsh", specifier = ">=0.19.2" }, ] @@ -426,9 +428,9 @@ source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "trio" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/f8/8e/fdd7bc467b40eedd0a5f2ed36b0d692c6e6f2473be00c8160e2e9f53adc1/tricycle-0.4.1.tar.gz", hash = "sha256:f56edb4b3e1bed3e2552b1b499b24a2dab47741e92e9b4d806acc5c35c9e6066", size = 41551 } +sdist = { url = "https://files.pythonhosted.org/packages/f8/8e/fdd7bc467b40eedd0a5f2ed36b0d692c6e6f2473be00c8160e2e9f53adc1/tricycle-0.4.1.tar.gz", hash = "sha256:f56edb4b3e1bed3e2552b1b499b24a2dab47741e92e9b4d806acc5c35c9e6066", size = 41551, upload-time = "2024-02-02T20:41:15.298Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/d7/c6/7cc05d60e21c683df99167db071ce5d848f5063c2a63971a8443466f603e/tricycle-0.4.1-py3-none-any.whl", hash = "sha256:67900995a73e7445e2c70250cdca04a778d9c3923dd960a97ad4569085e0fb3f", size = 35316 }, + { url = "https://files.pythonhosted.org/packages/d7/c6/7cc05d60e21c683df99167db071ce5d848f5063c2a63971a8443466f603e/tricycle-0.4.1-py3-none-any.whl", hash = "sha256:67900995a73e7445e2c70250cdca04a778d9c3923dd960a97ad4569085e0fb3f", size = 35316, upload-time = "2024-02-02T20:41:14.108Z" }, ] [[package]] @@ -443,82 +445,91 @@ dependencies = [ { name = "sniffio" }, { name = "sortedcontainers" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/a1/47/f62e62a1a6f37909aed0bf8f5d5411e06fa03846cfcb64540cd1180ccc9f/trio-0.29.0.tar.gz", hash = "sha256:ea0d3967159fc130acb6939a0be0e558e364fee26b5deeecc893a6b08c361bdf", size = 588952 } +sdist = { url = "https://files.pythonhosted.org/packages/a1/47/f62e62a1a6f37909aed0bf8f5d5411e06fa03846cfcb64540cd1180ccc9f/trio-0.29.0.tar.gz", hash = "sha256:ea0d3967159fc130acb6939a0be0e558e364fee26b5deeecc893a6b08c361bdf", size = 588952, upload-time = "2025-02-14T07:13:50.724Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/c9/55/c4d9bea8b3d7937901958f65124123512419ab0eb73695e5f382521abbfb/trio-0.29.0-py3-none-any.whl", hash = "sha256:d8c463f1a9cc776ff63e331aba44c125f423a5a13c684307e828d930e625ba66", size = 492920 }, + { url = "https://files.pythonhosted.org/packages/c9/55/c4d9bea8b3d7937901958f65124123512419ab0eb73695e5f382521abbfb/trio-0.29.0-py3-none-any.whl", hash = "sha256:d8c463f1a9cc776ff63e331aba44c125f423a5a13c684307e828d930e625ba66", size = 492920, upload-time = "2025-02-14T07:13:48.696Z" }, +] + +[[package]] +name = "typing-extensions" +version = "4.13.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/f6/37/23083fcd6e35492953e8d2aaaa68b860eb422b34627b13f2ce3eb6106061/typing_extensions-4.13.2.tar.gz", hash = "sha256:e6c81219bd689f51865d9e372991c540bda33a0379d5573cddb9a3a23f7caaef", size = 106967, upload-time = "2025-04-10T14:19:05.416Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/8b/54/b1ae86c0973cc6f0210b53d508ca3641fb6d0c56823f288d108bc7ab3cc8/typing_extensions-4.13.2-py3-none-any.whl", hash = "sha256:a439e7c04b49fec3e5d3e2beaa21755cadbbdc391694e28ccdd36ca4a1408f8c", size = 45806, upload-time = "2025-04-10T14:19:03.967Z" }, ] [[package]] name = "wcwidth" version = "0.2.13" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/6c/63/53559446a878410fc5a5974feb13d31d78d752eb18aeba59c7fef1af7598/wcwidth-0.2.13.tar.gz", hash = "sha256:72ea0c06399eb286d978fdedb6923a9eb47e1c486ce63e9b4e64fc18303972b5", size = 101301 } +sdist = { url = "https://files.pythonhosted.org/packages/6c/63/53559446a878410fc5a5974feb13d31d78d752eb18aeba59c7fef1af7598/wcwidth-0.2.13.tar.gz", hash = "sha256:72ea0c06399eb286d978fdedb6923a9eb47e1c486ce63e9b4e64fc18303972b5", size = 101301, upload-time = "2024-01-06T02:10:57.829Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/fd/84/fd2ba7aafacbad3c4201d395674fc6348826569da3c0937e75505ead3528/wcwidth-0.2.13-py2.py3-none-any.whl", hash = "sha256:3da69048e4540d84af32131829ff948f1e022c1c6bdb8d6102117aac784f6859", size = 34166 }, + { url = "https://files.pythonhosted.org/packages/fd/84/fd2ba7aafacbad3c4201d395674fc6348826569da3c0937e75505ead3528/wcwidth-0.2.13-py2.py3-none-any.whl", hash = "sha256:3da69048e4540d84af32131829ff948f1e022c1c6bdb8d6102117aac784f6859", size = 34166, upload-time = "2024-01-06T02:10:55.763Z" }, ] [[package]] name = "wrapt" version = "1.17.2" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/c3/fc/e91cc220803d7bc4db93fb02facd8461c37364151b8494762cc88b0fbcef/wrapt-1.17.2.tar.gz", hash = "sha256:41388e9d4d1522446fe79d3213196bd9e3b301a336965b9e27ca2788ebd122f3", size = 55531 } +sdist = { url = "https://files.pythonhosted.org/packages/c3/fc/e91cc220803d7bc4db93fb02facd8461c37364151b8494762cc88b0fbcef/wrapt-1.17.2.tar.gz", hash = "sha256:41388e9d4d1522446fe79d3213196bd9e3b301a336965b9e27ca2788ebd122f3", size = 55531, upload-time = "2025-01-14T10:35:45.465Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/cd/f7/a2aab2cbc7a665efab072344a8949a71081eed1d2f451f7f7d2b966594a2/wrapt-1.17.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:ff04ef6eec3eee8a5efef2401495967a916feaa353643defcc03fc74fe213b58", size = 53308 }, - { url = "https://files.pythonhosted.org/packages/50/ff/149aba8365fdacef52b31a258c4dc1c57c79759c335eff0b3316a2664a64/wrapt-1.17.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:4db983e7bca53819efdbd64590ee96c9213894272c776966ca6306b73e4affda", size = 38488 }, - { url = "https://files.pythonhosted.org/packages/65/46/5a917ce85b5c3b490d35c02bf71aedaa9f2f63f2d15d9949cc4ba56e8ba9/wrapt-1.17.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:9abc77a4ce4c6f2a3168ff34b1da9b0f311a8f1cfd694ec96b0603dff1c79438", size = 38776 }, - { url = "https://files.pythonhosted.org/packages/ca/74/336c918d2915a4943501c77566db41d1bd6e9f4dbc317f356b9a244dfe83/wrapt-1.17.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0b929ac182f5ace000d459c59c2c9c33047e20e935f8e39371fa6e3b85d56f4a", size = 83776 }, - { url = "https://files.pythonhosted.org/packages/09/99/c0c844a5ccde0fe5761d4305485297f91d67cf2a1a824c5f282e661ec7ff/wrapt-1.17.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f09b286faeff3c750a879d336fb6d8713206fc97af3adc14def0cdd349df6000", size = 75420 }, - { url = "https://files.pythonhosted.org/packages/b4/b0/9fc566b0fe08b282c850063591a756057c3247b2362b9286429ec5bf1721/wrapt-1.17.2-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1a7ed2d9d039bd41e889f6fb9364554052ca21ce823580f6a07c4ec245c1f5d6", size = 83199 }, - { url = "https://files.pythonhosted.org/packages/9d/4b/71996e62d543b0a0bd95dda485219856def3347e3e9380cc0d6cf10cfb2f/wrapt-1.17.2-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:129a150f5c445165ff941fc02ee27df65940fcb8a22a61828b1853c98763a64b", size = 82307 }, - { url = "https://files.pythonhosted.org/packages/39/35/0282c0d8789c0dc9bcc738911776c762a701f95cfe113fb8f0b40e45c2b9/wrapt-1.17.2-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:1fb5699e4464afe5c7e65fa51d4f99e0b2eadcc176e4aa33600a3df7801d6662", size = 75025 }, - { url = "https://files.pythonhosted.org/packages/4f/6d/90c9fd2c3c6fee181feecb620d95105370198b6b98a0770cba090441a828/wrapt-1.17.2-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:9a2bce789a5ea90e51a02dfcc39e31b7f1e662bc3317979aa7e5538e3a034f72", size = 81879 }, - { url = "https://files.pythonhosted.org/packages/8f/fa/9fb6e594f2ce03ef03eddbdb5f4f90acb1452221a5351116c7c4708ac865/wrapt-1.17.2-cp311-cp311-win32.whl", hash = "sha256:4afd5814270fdf6380616b321fd31435a462019d834f83c8611a0ce7484c7317", size = 36419 }, - { url = "https://files.pythonhosted.org/packages/47/f8/fb1773491a253cbc123c5d5dc15c86041f746ed30416535f2a8df1f4a392/wrapt-1.17.2-cp311-cp311-win_amd64.whl", hash = "sha256:acc130bc0375999da18e3d19e5a86403667ac0c4042a094fefb7eec8ebac7cf3", size = 38773 }, - { url = "https://files.pythonhosted.org/packages/a1/bd/ab55f849fd1f9a58ed7ea47f5559ff09741b25f00c191231f9f059c83949/wrapt-1.17.2-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:d5e2439eecc762cd85e7bd37161d4714aa03a33c5ba884e26c81559817ca0925", size = 53799 }, - { url = "https://files.pythonhosted.org/packages/53/18/75ddc64c3f63988f5a1d7e10fb204ffe5762bc663f8023f18ecaf31a332e/wrapt-1.17.2-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:3fc7cb4c1c744f8c05cd5f9438a3caa6ab94ce8344e952d7c45a8ed59dd88392", size = 38821 }, - { url = "https://files.pythonhosted.org/packages/48/2a/97928387d6ed1c1ebbfd4efc4133a0633546bec8481a2dd5ec961313a1c7/wrapt-1.17.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:8fdbdb757d5390f7c675e558fd3186d590973244fab0c5fe63d373ade3e99d40", size = 38919 }, - { url = "https://files.pythonhosted.org/packages/73/54/3bfe5a1febbbccb7a2f77de47b989c0b85ed3a6a41614b104204a788c20e/wrapt-1.17.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5bb1d0dbf99411f3d871deb6faa9aabb9d4e744d67dcaaa05399af89d847a91d", size = 88721 }, - { url = "https://files.pythonhosted.org/packages/25/cb/7262bc1b0300b4b64af50c2720ef958c2c1917525238d661c3e9a2b71b7b/wrapt-1.17.2-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d18a4865f46b8579d44e4fe1e2bcbc6472ad83d98e22a26c963d46e4c125ef0b", size = 80899 }, - { url = "https://files.pythonhosted.org/packages/2a/5a/04cde32b07a7431d4ed0553a76fdb7a61270e78c5fd5a603e190ac389f14/wrapt-1.17.2-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bc570b5f14a79734437cb7b0500376b6b791153314986074486e0b0fa8d71d98", size = 89222 }, - { url = "https://files.pythonhosted.org/packages/09/28/2e45a4f4771fcfb109e244d5dbe54259e970362a311b67a965555ba65026/wrapt-1.17.2-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:6d9187b01bebc3875bac9b087948a2bccefe464a7d8f627cf6e48b1bbae30f82", size = 86707 }, - { url = "https://files.pythonhosted.org/packages/c6/d2/dcb56bf5f32fcd4bd9aacc77b50a539abdd5b6536872413fd3f428b21bed/wrapt-1.17.2-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:9e8659775f1adf02eb1e6f109751268e493c73716ca5761f8acb695e52a756ae", size = 79685 }, - { url = "https://files.pythonhosted.org/packages/80/4e/eb8b353e36711347893f502ce91c770b0b0929f8f0bed2670a6856e667a9/wrapt-1.17.2-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:e8b2816ebef96d83657b56306152a93909a83f23994f4b30ad4573b00bd11bb9", size = 87567 }, - { url = "https://files.pythonhosted.org/packages/17/27/4fe749a54e7fae6e7146f1c7d914d28ef599dacd4416566c055564080fe2/wrapt-1.17.2-cp312-cp312-win32.whl", hash = "sha256:468090021f391fe0056ad3e807e3d9034e0fd01adcd3bdfba977b6fdf4213ea9", size = 36672 }, - { url = "https://files.pythonhosted.org/packages/15/06/1dbf478ea45c03e78a6a8c4be4fdc3c3bddea5c8de8a93bc971415e47f0f/wrapt-1.17.2-cp312-cp312-win_amd64.whl", hash = "sha256:ec89ed91f2fa8e3f52ae53cd3cf640d6feff92ba90d62236a81e4e563ac0e991", size = 38865 }, - { url = "https://files.pythonhosted.org/packages/ce/b9/0ffd557a92f3b11d4c5d5e0c5e4ad057bd9eb8586615cdaf901409920b14/wrapt-1.17.2-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:6ed6ffac43aecfe6d86ec5b74b06a5be33d5bb9243d055141e8cabb12aa08125", size = 53800 }, - { url = "https://files.pythonhosted.org/packages/c0/ef/8be90a0b7e73c32e550c73cfb2fa09db62234227ece47b0e80a05073b375/wrapt-1.17.2-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:35621ae4c00e056adb0009f8e86e28eb4a41a4bfa8f9bfa9fca7d343fe94f998", size = 38824 }, - { url = "https://files.pythonhosted.org/packages/36/89/0aae34c10fe524cce30fe5fc433210376bce94cf74d05b0d68344c8ba46e/wrapt-1.17.2-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:a604bf7a053f8362d27eb9fefd2097f82600b856d5abe996d623babd067b1ab5", size = 38920 }, - { url = "https://files.pythonhosted.org/packages/3b/24/11c4510de906d77e0cfb5197f1b1445d4fec42c9a39ea853d482698ac681/wrapt-1.17.2-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5cbabee4f083b6b4cd282f5b817a867cf0b1028c54d445b7ec7cfe6505057cf8", size = 88690 }, - { url = "https://files.pythonhosted.org/packages/71/d7/cfcf842291267bf455b3e266c0c29dcb675b5540ee8b50ba1699abf3af45/wrapt-1.17.2-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:49703ce2ddc220df165bd2962f8e03b84c89fee2d65e1c24a7defff6f988f4d6", size = 80861 }, - { url = "https://files.pythonhosted.org/packages/d5/66/5d973e9f3e7370fd686fb47a9af3319418ed925c27d72ce16b791231576d/wrapt-1.17.2-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8112e52c5822fc4253f3901b676c55ddf288614dc7011634e2719718eaa187dc", size = 89174 }, - { url = "https://files.pythonhosted.org/packages/a7/d3/8e17bb70f6ae25dabc1aaf990f86824e4fd98ee9cadf197054e068500d27/wrapt-1.17.2-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:9fee687dce376205d9a494e9c121e27183b2a3df18037f89d69bd7b35bcf59e2", size = 86721 }, - { url = "https://files.pythonhosted.org/packages/6f/54/f170dfb278fe1c30d0ff864513cff526d624ab8de3254b20abb9cffedc24/wrapt-1.17.2-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:18983c537e04d11cf027fbb60a1e8dfd5190e2b60cc27bc0808e653e7b218d1b", size = 79763 }, - { url = "https://files.pythonhosted.org/packages/4a/98/de07243751f1c4a9b15c76019250210dd3486ce098c3d80d5f729cba029c/wrapt-1.17.2-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:703919b1633412ab54bcf920ab388735832fdcb9f9a00ae49387f0fe67dad504", size = 87585 }, - { url = "https://files.pythonhosted.org/packages/f9/f0/13925f4bd6548013038cdeb11ee2cbd4e37c30f8bfd5db9e5a2a370d6e20/wrapt-1.17.2-cp313-cp313-win32.whl", hash = "sha256:abbb9e76177c35d4e8568e58650aa6926040d6a9f6f03435b7a522bf1c487f9a", size = 36676 }, - { url = "https://files.pythonhosted.org/packages/bf/ae/743f16ef8c2e3628df3ddfd652b7d4c555d12c84b53f3d8218498f4ade9b/wrapt-1.17.2-cp313-cp313-win_amd64.whl", hash = "sha256:69606d7bb691b50a4240ce6b22ebb319c1cfb164e5f6569835058196e0f3a845", size = 38871 }, - { url = "https://files.pythonhosted.org/packages/3d/bc/30f903f891a82d402ffb5fda27ec1d621cc97cb74c16fea0b6141f1d4e87/wrapt-1.17.2-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:4a721d3c943dae44f8e243b380cb645a709ba5bd35d3ad27bc2ed947e9c68192", size = 56312 }, - { url = "https://files.pythonhosted.org/packages/8a/04/c97273eb491b5f1c918857cd26f314b74fc9b29224521f5b83f872253725/wrapt-1.17.2-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:766d8bbefcb9e00c3ac3b000d9acc51f1b399513f44d77dfe0eb026ad7c9a19b", size = 40062 }, - { url = "https://files.pythonhosted.org/packages/4e/ca/3b7afa1eae3a9e7fefe499db9b96813f41828b9fdb016ee836c4c379dadb/wrapt-1.17.2-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:e496a8ce2c256da1eb98bd15803a79bee00fc351f5dfb9ea82594a3f058309e0", size = 40155 }, - { url = "https://files.pythonhosted.org/packages/89/be/7c1baed43290775cb9030c774bc53c860db140397047cc49aedaf0a15477/wrapt-1.17.2-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:40d615e4fe22f4ad3528448c193b218e077656ca9ccb22ce2cb20db730f8d306", size = 113471 }, - { url = "https://files.pythonhosted.org/packages/32/98/4ed894cf012b6d6aae5f5cc974006bdeb92f0241775addad3f8cd6ab71c8/wrapt-1.17.2-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a5aaeff38654462bc4b09023918b7f21790efb807f54c000a39d41d69cf552cb", size = 101208 }, - { url = "https://files.pythonhosted.org/packages/ea/fd/0c30f2301ca94e655e5e057012e83284ce8c545df7661a78d8bfca2fac7a/wrapt-1.17.2-cp313-cp313t-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9a7d15bbd2bc99e92e39f49a04653062ee6085c0e18b3b7512a4f2fe91f2d681", size = 109339 }, - { url = "https://files.pythonhosted.org/packages/75/56/05d000de894c4cfcb84bcd6b1df6214297b8089a7bd324c21a4765e49b14/wrapt-1.17.2-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:e3890b508a23299083e065f435a492b5435eba6e304a7114d2f919d400888cc6", size = 110232 }, - { url = "https://files.pythonhosted.org/packages/53/f8/c3f6b2cf9b9277fb0813418e1503e68414cd036b3b099c823379c9575e6d/wrapt-1.17.2-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:8c8b293cd65ad716d13d8dd3624e42e5a19cc2a2f1acc74b30c2c13f15cb61a6", size = 100476 }, - { url = "https://files.pythonhosted.org/packages/a7/b1/0bb11e29aa5139d90b770ebbfa167267b1fc548d2302c30c8f7572851738/wrapt-1.17.2-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:4c82b8785d98cdd9fed4cac84d765d234ed3251bd6afe34cb7ac523cb93e8b4f", size = 106377 }, - { url = "https://files.pythonhosted.org/packages/6a/e1/0122853035b40b3f333bbb25f1939fc1045e21dd518f7f0922b60c156f7c/wrapt-1.17.2-cp313-cp313t-win32.whl", hash = "sha256:13e6afb7fe71fe7485a4550a8844cc9ffbe263c0f1a1eea569bc7091d4898555", size = 37986 }, - { url = "https://files.pythonhosted.org/packages/09/5e/1655cf481e079c1f22d0cabdd4e51733679932718dc23bf2db175f329b76/wrapt-1.17.2-cp313-cp313t-win_amd64.whl", hash = "sha256:eaf675418ed6b3b31c7a989fd007fa7c3be66ce14e5c3b27336383604c9da85c", size = 40750 }, - { url = "https://files.pythonhosted.org/packages/2d/82/f56956041adef78f849db6b289b282e72b55ab8045a75abad81898c28d19/wrapt-1.17.2-py3-none-any.whl", hash = "sha256:b18f2d1533a71f069c7f82d524a52599053d4c7166e9dd374ae2136b7f40f7c8", size = 23594 }, + { url = "https://files.pythonhosted.org/packages/cd/f7/a2aab2cbc7a665efab072344a8949a71081eed1d2f451f7f7d2b966594a2/wrapt-1.17.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:ff04ef6eec3eee8a5efef2401495967a916feaa353643defcc03fc74fe213b58", size = 53308, upload-time = "2025-01-14T10:33:33.992Z" }, + { url = "https://files.pythonhosted.org/packages/50/ff/149aba8365fdacef52b31a258c4dc1c57c79759c335eff0b3316a2664a64/wrapt-1.17.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:4db983e7bca53819efdbd64590ee96c9213894272c776966ca6306b73e4affda", size = 38488, upload-time = "2025-01-14T10:33:35.264Z" }, + { url = "https://files.pythonhosted.org/packages/65/46/5a917ce85b5c3b490d35c02bf71aedaa9f2f63f2d15d9949cc4ba56e8ba9/wrapt-1.17.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:9abc77a4ce4c6f2a3168ff34b1da9b0f311a8f1cfd694ec96b0603dff1c79438", size = 38776, upload-time = "2025-01-14T10:33:38.28Z" }, + { url = "https://files.pythonhosted.org/packages/ca/74/336c918d2915a4943501c77566db41d1bd6e9f4dbc317f356b9a244dfe83/wrapt-1.17.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0b929ac182f5ace000d459c59c2c9c33047e20e935f8e39371fa6e3b85d56f4a", size = 83776, upload-time = "2025-01-14T10:33:40.678Z" }, + { url = "https://files.pythonhosted.org/packages/09/99/c0c844a5ccde0fe5761d4305485297f91d67cf2a1a824c5f282e661ec7ff/wrapt-1.17.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f09b286faeff3c750a879d336fb6d8713206fc97af3adc14def0cdd349df6000", size = 75420, upload-time = "2025-01-14T10:33:41.868Z" }, + { url = "https://files.pythonhosted.org/packages/b4/b0/9fc566b0fe08b282c850063591a756057c3247b2362b9286429ec5bf1721/wrapt-1.17.2-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1a7ed2d9d039bd41e889f6fb9364554052ca21ce823580f6a07c4ec245c1f5d6", size = 83199, upload-time = "2025-01-14T10:33:43.598Z" }, + { url = "https://files.pythonhosted.org/packages/9d/4b/71996e62d543b0a0bd95dda485219856def3347e3e9380cc0d6cf10cfb2f/wrapt-1.17.2-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:129a150f5c445165ff941fc02ee27df65940fcb8a22a61828b1853c98763a64b", size = 82307, upload-time = "2025-01-14T10:33:48.499Z" }, + { url = "https://files.pythonhosted.org/packages/39/35/0282c0d8789c0dc9bcc738911776c762a701f95cfe113fb8f0b40e45c2b9/wrapt-1.17.2-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:1fb5699e4464afe5c7e65fa51d4f99e0b2eadcc176e4aa33600a3df7801d6662", size = 75025, upload-time = "2025-01-14T10:33:51.191Z" }, + { url = "https://files.pythonhosted.org/packages/4f/6d/90c9fd2c3c6fee181feecb620d95105370198b6b98a0770cba090441a828/wrapt-1.17.2-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:9a2bce789a5ea90e51a02dfcc39e31b7f1e662bc3317979aa7e5538e3a034f72", size = 81879, upload-time = "2025-01-14T10:33:52.328Z" }, + { url = "https://files.pythonhosted.org/packages/8f/fa/9fb6e594f2ce03ef03eddbdb5f4f90acb1452221a5351116c7c4708ac865/wrapt-1.17.2-cp311-cp311-win32.whl", hash = "sha256:4afd5814270fdf6380616b321fd31435a462019d834f83c8611a0ce7484c7317", size = 36419, upload-time = "2025-01-14T10:33:53.551Z" }, + { url = "https://files.pythonhosted.org/packages/47/f8/fb1773491a253cbc123c5d5dc15c86041f746ed30416535f2a8df1f4a392/wrapt-1.17.2-cp311-cp311-win_amd64.whl", hash = "sha256:acc130bc0375999da18e3d19e5a86403667ac0c4042a094fefb7eec8ebac7cf3", size = 38773, upload-time = "2025-01-14T10:33:56.323Z" }, + { url = "https://files.pythonhosted.org/packages/a1/bd/ab55f849fd1f9a58ed7ea47f5559ff09741b25f00c191231f9f059c83949/wrapt-1.17.2-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:d5e2439eecc762cd85e7bd37161d4714aa03a33c5ba884e26c81559817ca0925", size = 53799, upload-time = "2025-01-14T10:33:57.4Z" }, + { url = "https://files.pythonhosted.org/packages/53/18/75ddc64c3f63988f5a1d7e10fb204ffe5762bc663f8023f18ecaf31a332e/wrapt-1.17.2-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:3fc7cb4c1c744f8c05cd5f9438a3caa6ab94ce8344e952d7c45a8ed59dd88392", size = 38821, upload-time = "2025-01-14T10:33:59.334Z" }, + { url = "https://files.pythonhosted.org/packages/48/2a/97928387d6ed1c1ebbfd4efc4133a0633546bec8481a2dd5ec961313a1c7/wrapt-1.17.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:8fdbdb757d5390f7c675e558fd3186d590973244fab0c5fe63d373ade3e99d40", size = 38919, upload-time = "2025-01-14T10:34:04.093Z" }, + { url = "https://files.pythonhosted.org/packages/73/54/3bfe5a1febbbccb7a2f77de47b989c0b85ed3a6a41614b104204a788c20e/wrapt-1.17.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5bb1d0dbf99411f3d871deb6faa9aabb9d4e744d67dcaaa05399af89d847a91d", size = 88721, upload-time = "2025-01-14T10:34:07.163Z" }, + { url = "https://files.pythonhosted.org/packages/25/cb/7262bc1b0300b4b64af50c2720ef958c2c1917525238d661c3e9a2b71b7b/wrapt-1.17.2-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d18a4865f46b8579d44e4fe1e2bcbc6472ad83d98e22a26c963d46e4c125ef0b", size = 80899, upload-time = "2025-01-14T10:34:09.82Z" }, + { url = "https://files.pythonhosted.org/packages/2a/5a/04cde32b07a7431d4ed0553a76fdb7a61270e78c5fd5a603e190ac389f14/wrapt-1.17.2-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bc570b5f14a79734437cb7b0500376b6b791153314986074486e0b0fa8d71d98", size = 89222, upload-time = "2025-01-14T10:34:11.258Z" }, + { url = "https://files.pythonhosted.org/packages/09/28/2e45a4f4771fcfb109e244d5dbe54259e970362a311b67a965555ba65026/wrapt-1.17.2-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:6d9187b01bebc3875bac9b087948a2bccefe464a7d8f627cf6e48b1bbae30f82", size = 86707, upload-time = "2025-01-14T10:34:12.49Z" }, + { url = "https://files.pythonhosted.org/packages/c6/d2/dcb56bf5f32fcd4bd9aacc77b50a539abdd5b6536872413fd3f428b21bed/wrapt-1.17.2-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:9e8659775f1adf02eb1e6f109751268e493c73716ca5761f8acb695e52a756ae", size = 79685, upload-time = "2025-01-14T10:34:15.043Z" }, + { url = "https://files.pythonhosted.org/packages/80/4e/eb8b353e36711347893f502ce91c770b0b0929f8f0bed2670a6856e667a9/wrapt-1.17.2-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:e8b2816ebef96d83657b56306152a93909a83f23994f4b30ad4573b00bd11bb9", size = 87567, upload-time = "2025-01-14T10:34:16.563Z" }, + { url = "https://files.pythonhosted.org/packages/17/27/4fe749a54e7fae6e7146f1c7d914d28ef599dacd4416566c055564080fe2/wrapt-1.17.2-cp312-cp312-win32.whl", hash = "sha256:468090021f391fe0056ad3e807e3d9034e0fd01adcd3bdfba977b6fdf4213ea9", size = 36672, upload-time = "2025-01-14T10:34:17.727Z" }, + { url = "https://files.pythonhosted.org/packages/15/06/1dbf478ea45c03e78a6a8c4be4fdc3c3bddea5c8de8a93bc971415e47f0f/wrapt-1.17.2-cp312-cp312-win_amd64.whl", hash = "sha256:ec89ed91f2fa8e3f52ae53cd3cf640d6feff92ba90d62236a81e4e563ac0e991", size = 38865, upload-time = "2025-01-14T10:34:19.577Z" }, + { url = "https://files.pythonhosted.org/packages/ce/b9/0ffd557a92f3b11d4c5d5e0c5e4ad057bd9eb8586615cdaf901409920b14/wrapt-1.17.2-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:6ed6ffac43aecfe6d86ec5b74b06a5be33d5bb9243d055141e8cabb12aa08125", size = 53800, upload-time = "2025-01-14T10:34:21.571Z" }, + { url = "https://files.pythonhosted.org/packages/c0/ef/8be90a0b7e73c32e550c73cfb2fa09db62234227ece47b0e80a05073b375/wrapt-1.17.2-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:35621ae4c00e056adb0009f8e86e28eb4a41a4bfa8f9bfa9fca7d343fe94f998", size = 38824, upload-time = "2025-01-14T10:34:22.999Z" }, + { url = "https://files.pythonhosted.org/packages/36/89/0aae34c10fe524cce30fe5fc433210376bce94cf74d05b0d68344c8ba46e/wrapt-1.17.2-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:a604bf7a053f8362d27eb9fefd2097f82600b856d5abe996d623babd067b1ab5", size = 38920, upload-time = "2025-01-14T10:34:25.386Z" }, + { url = "https://files.pythonhosted.org/packages/3b/24/11c4510de906d77e0cfb5197f1b1445d4fec42c9a39ea853d482698ac681/wrapt-1.17.2-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5cbabee4f083b6b4cd282f5b817a867cf0b1028c54d445b7ec7cfe6505057cf8", size = 88690, upload-time = "2025-01-14T10:34:28.058Z" }, + { url = "https://files.pythonhosted.org/packages/71/d7/cfcf842291267bf455b3e266c0c29dcb675b5540ee8b50ba1699abf3af45/wrapt-1.17.2-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:49703ce2ddc220df165bd2962f8e03b84c89fee2d65e1c24a7defff6f988f4d6", size = 80861, upload-time = "2025-01-14T10:34:29.167Z" }, + { url = "https://files.pythonhosted.org/packages/d5/66/5d973e9f3e7370fd686fb47a9af3319418ed925c27d72ce16b791231576d/wrapt-1.17.2-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8112e52c5822fc4253f3901b676c55ddf288614dc7011634e2719718eaa187dc", size = 89174, upload-time = "2025-01-14T10:34:31.702Z" }, + { url = "https://files.pythonhosted.org/packages/a7/d3/8e17bb70f6ae25dabc1aaf990f86824e4fd98ee9cadf197054e068500d27/wrapt-1.17.2-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:9fee687dce376205d9a494e9c121e27183b2a3df18037f89d69bd7b35bcf59e2", size = 86721, upload-time = "2025-01-14T10:34:32.91Z" }, + { url = "https://files.pythonhosted.org/packages/6f/54/f170dfb278fe1c30d0ff864513cff526d624ab8de3254b20abb9cffedc24/wrapt-1.17.2-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:18983c537e04d11cf027fbb60a1e8dfd5190e2b60cc27bc0808e653e7b218d1b", size = 79763, upload-time = "2025-01-14T10:34:34.903Z" }, + { url = "https://files.pythonhosted.org/packages/4a/98/de07243751f1c4a9b15c76019250210dd3486ce098c3d80d5f729cba029c/wrapt-1.17.2-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:703919b1633412ab54bcf920ab388735832fdcb9f9a00ae49387f0fe67dad504", size = 87585, upload-time = "2025-01-14T10:34:36.13Z" }, + { url = "https://files.pythonhosted.org/packages/f9/f0/13925f4bd6548013038cdeb11ee2cbd4e37c30f8bfd5db9e5a2a370d6e20/wrapt-1.17.2-cp313-cp313-win32.whl", hash = "sha256:abbb9e76177c35d4e8568e58650aa6926040d6a9f6f03435b7a522bf1c487f9a", size = 36676, upload-time = "2025-01-14T10:34:37.962Z" }, + { url = "https://files.pythonhosted.org/packages/bf/ae/743f16ef8c2e3628df3ddfd652b7d4c555d12c84b53f3d8218498f4ade9b/wrapt-1.17.2-cp313-cp313-win_amd64.whl", hash = "sha256:69606d7bb691b50a4240ce6b22ebb319c1cfb164e5f6569835058196e0f3a845", size = 38871, upload-time = "2025-01-14T10:34:39.13Z" }, + { url = "https://files.pythonhosted.org/packages/3d/bc/30f903f891a82d402ffb5fda27ec1d621cc97cb74c16fea0b6141f1d4e87/wrapt-1.17.2-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:4a721d3c943dae44f8e243b380cb645a709ba5bd35d3ad27bc2ed947e9c68192", size = 56312, upload-time = "2025-01-14T10:34:40.604Z" }, + { url = "https://files.pythonhosted.org/packages/8a/04/c97273eb491b5f1c918857cd26f314b74fc9b29224521f5b83f872253725/wrapt-1.17.2-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:766d8bbefcb9e00c3ac3b000d9acc51f1b399513f44d77dfe0eb026ad7c9a19b", size = 40062, upload-time = "2025-01-14T10:34:45.011Z" }, + { url = "https://files.pythonhosted.org/packages/4e/ca/3b7afa1eae3a9e7fefe499db9b96813f41828b9fdb016ee836c4c379dadb/wrapt-1.17.2-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:e496a8ce2c256da1eb98bd15803a79bee00fc351f5dfb9ea82594a3f058309e0", size = 40155, upload-time = "2025-01-14T10:34:47.25Z" }, + { url = "https://files.pythonhosted.org/packages/89/be/7c1baed43290775cb9030c774bc53c860db140397047cc49aedaf0a15477/wrapt-1.17.2-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:40d615e4fe22f4ad3528448c193b218e077656ca9ccb22ce2cb20db730f8d306", size = 113471, upload-time = "2025-01-14T10:34:50.934Z" }, + { url = "https://files.pythonhosted.org/packages/32/98/4ed894cf012b6d6aae5f5cc974006bdeb92f0241775addad3f8cd6ab71c8/wrapt-1.17.2-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a5aaeff38654462bc4b09023918b7f21790efb807f54c000a39d41d69cf552cb", size = 101208, upload-time = "2025-01-14T10:34:52.297Z" }, + { url = "https://files.pythonhosted.org/packages/ea/fd/0c30f2301ca94e655e5e057012e83284ce8c545df7661a78d8bfca2fac7a/wrapt-1.17.2-cp313-cp313t-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9a7d15bbd2bc99e92e39f49a04653062ee6085c0e18b3b7512a4f2fe91f2d681", size = 109339, upload-time = "2025-01-14T10:34:53.489Z" }, + { url = "https://files.pythonhosted.org/packages/75/56/05d000de894c4cfcb84bcd6b1df6214297b8089a7bd324c21a4765e49b14/wrapt-1.17.2-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:e3890b508a23299083e065f435a492b5435eba6e304a7114d2f919d400888cc6", size = 110232, upload-time = "2025-01-14T10:34:55.327Z" }, + { url = "https://files.pythonhosted.org/packages/53/f8/c3f6b2cf9b9277fb0813418e1503e68414cd036b3b099c823379c9575e6d/wrapt-1.17.2-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:8c8b293cd65ad716d13d8dd3624e42e5a19cc2a2f1acc74b30c2c13f15cb61a6", size = 100476, upload-time = "2025-01-14T10:34:58.055Z" }, + { url = "https://files.pythonhosted.org/packages/a7/b1/0bb11e29aa5139d90b770ebbfa167267b1fc548d2302c30c8f7572851738/wrapt-1.17.2-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:4c82b8785d98cdd9fed4cac84d765d234ed3251bd6afe34cb7ac523cb93e8b4f", size = 106377, upload-time = "2025-01-14T10:34:59.3Z" }, + { url = "https://files.pythonhosted.org/packages/6a/e1/0122853035b40b3f333bbb25f1939fc1045e21dd518f7f0922b60c156f7c/wrapt-1.17.2-cp313-cp313t-win32.whl", hash = "sha256:13e6afb7fe71fe7485a4550a8844cc9ffbe263c0f1a1eea569bc7091d4898555", size = 37986, upload-time = "2025-01-14T10:35:00.498Z" }, + { url = "https://files.pythonhosted.org/packages/09/5e/1655cf481e079c1f22d0cabdd4e51733679932718dc23bf2db175f329b76/wrapt-1.17.2-cp313-cp313t-win_amd64.whl", hash = "sha256:eaf675418ed6b3b31c7a989fd007fa7c3be66ce14e5c3b27336383604c9da85c", size = 40750, upload-time = "2025-01-14T10:35:03.378Z" }, + { url = "https://files.pythonhosted.org/packages/2d/82/f56956041adef78f849db6b289b282e72b55ab8045a75abad81898c28d19/wrapt-1.17.2-py3-none-any.whl", hash = "sha256:b18f2d1533a71f069c7f82d524a52599053d4c7166e9dd374ae2136b7f40f7c8", size = 23594, upload-time = "2025-01-14T10:35:44.018Z" }, ] [[package]] name = "xonsh" version = "0.19.2" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/68/4e/56e95a5e607eb3b0da37396f87cde70588efc8ef819ab16f02d5b8378dc4/xonsh-0.19.2.tar.gz", hash = "sha256:cfdd0680d954a2c3aefd6caddcc7143a3d06aa417ed18365a08219bb71b960b0", size = 799960 } +sdist = { url = "https://files.pythonhosted.org/packages/68/4e/56e95a5e607eb3b0da37396f87cde70588efc8ef819ab16f02d5b8378dc4/xonsh-0.19.2.tar.gz", hash = "sha256:cfdd0680d954a2c3aefd6caddcc7143a3d06aa417ed18365a08219bb71b960b0", size = 799960, upload-time = "2025-02-11T17:10:43.563Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/6c/13/281094759df87b23b3c02dc4a16603ab08ea54d7f6acfeb69f3341137c7a/xonsh-0.19.2-py310-none-any.whl", hash = "sha256:ec7f163fd3a4943782aa34069d4e72793328c916a5975949dbec8536cbfc089b", size = 642301 }, - { url = "https://files.pythonhosted.org/packages/29/41/a51e4c3918fe9a293b150cb949b1b8c6d45eb17dfed480dcb76ea43df4e7/xonsh-0.19.2-py311-none-any.whl", hash = "sha256:53c45f7a767901f2f518f9b8dd60fc653e0498e56e89825e1710bb0859985049", size = 642286 }, - { url = "https://files.pythonhosted.org/packages/0a/93/9a77b731f492fac27c577dea2afb5a2bcc2a6a1c79be0c86c95498060270/xonsh-0.19.2-py312-none-any.whl", hash = "sha256:b24c619aa52b59eae4d35c4195dba9b19a2c548fb5c42c6f85f2b8ccb96807b5", size = 642386 }, - { url = "https://files.pythonhosted.org/packages/be/75/070324769c1ff88d971ce040f4f486339be98e0a365c8dd9991eb654265b/xonsh-0.19.2-py313-none-any.whl", hash = "sha256:c53ef6c19f781fbc399ed1b382b5c2aac2125010679a3b61d643978273c27df0", size = 642873 }, - { url = "https://files.pythonhosted.org/packages/fa/cb/2c7ccec54f5b0e73fdf7650e8336582ff0347d9001c5ef8271dc00c034fe/xonsh-0.19.2-py39-none-any.whl", hash = "sha256:bcc0225dc3847f1ed2f175dac6122fbcc54cea67d9c2dc2753d9615e2a5ff284", size = 634602 }, + { url = "https://files.pythonhosted.org/packages/6c/13/281094759df87b23b3c02dc4a16603ab08ea54d7f6acfeb69f3341137c7a/xonsh-0.19.2-py310-none-any.whl", hash = "sha256:ec7f163fd3a4943782aa34069d4e72793328c916a5975949dbec8536cbfc089b", size = 642301, upload-time = "2025-02-11T17:10:39.244Z" }, + { url = "https://files.pythonhosted.org/packages/29/41/a51e4c3918fe9a293b150cb949b1b8c6d45eb17dfed480dcb76ea43df4e7/xonsh-0.19.2-py311-none-any.whl", hash = "sha256:53c45f7a767901f2f518f9b8dd60fc653e0498e56e89825e1710bb0859985049", size = 642286, upload-time = "2025-02-11T17:10:41.678Z" }, + { url = "https://files.pythonhosted.org/packages/0a/93/9a77b731f492fac27c577dea2afb5a2bcc2a6a1c79be0c86c95498060270/xonsh-0.19.2-py312-none-any.whl", hash = "sha256:b24c619aa52b59eae4d35c4195dba9b19a2c548fb5c42c6f85f2b8ccb96807b5", size = 642386, upload-time = "2025-02-11T17:10:43.688Z" }, + { url = "https://files.pythonhosted.org/packages/be/75/070324769c1ff88d971ce040f4f486339be98e0a365c8dd9991eb654265b/xonsh-0.19.2-py313-none-any.whl", hash = "sha256:c53ef6c19f781fbc399ed1b382b5c2aac2125010679a3b61d643978273c27df0", size = 642873, upload-time = "2025-02-11T17:10:39.297Z" }, + { url = "https://files.pythonhosted.org/packages/fa/cb/2c7ccec54f5b0e73fdf7650e8336582ff0347d9001c5ef8271dc00c034fe/xonsh-0.19.2-py39-none-any.whl", hash = "sha256:bcc0225dc3847f1ed2f175dac6122fbcc54cea67d9c2dc2753d9615e2a5ff284", size = 634602, upload-time = "2025-02-11T17:10:37.004Z" }, ] From 6d2f4d108d9d427b7441fcef1c505ce4f0863b50 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 12 Jun 2025 23:22:16 -0400 Subject: [PATCH 65/74] Detail the docs on `Context._maybe_raise_remote_err()` --- tractor/_context.py | 45 +++++++++++++++++++++++++++++++-------------- 1 file changed, 31 insertions(+), 14 deletions(-) diff --git a/tractor/_context.py b/tractor/_context.py index e5cce1ec..6130d004 100644 --- a/tractor/_context.py +++ b/tractor/_context.py @@ -1069,9 +1069,25 @@ class Context: |RemoteActorError # stream overrun caused and ignored by us ): ''' - Maybe raise a remote error depending on the type of error - and *who* (i.e. which task from which actor) requested - a cancellation (if any). + Maybe raise a remote error depending on the type of error and + *who*, i.e. which side of the task pair across actors, + requested a cancellation (if any). + + Depending on the input config-params suppress raising + certain remote excs: + + - if `remote_error: ContextCancelled` (ctxc) AND this side's + task is the "requester", it at somem point called + `Context.cancel()`, then the peer's ctxc is treated + as a "cancel ack". + + |_ this behaves exactly like how `trio.Nursery.cancel_scope` + absorbs any `BaseExceptionGroup[trio.Cancelled]` wherein the + owning parent task never will raise a `trio.Cancelled` + if `CancelScope.cancel_called == True`. + + - `remote_error: StreamOverrrun` (overrun) AND + `raise_overrun_from_self` is set. ''' __tracebackhide__: bool = hide_tb @@ -1113,18 +1129,19 @@ class Context: # for this ^, NO right? ) or ( - # NOTE: whenever this context is the cause of an - # overrun on the remote side (aka we sent msgs too - # fast that the remote task was overrun according - # to `MsgStream` buffer settings) AND the caller - # has requested to not raise overruns this side - # caused, we also silently absorb any remotely - # boxed `StreamOverrun`. This is mostly useful for - # supressing such faults during - # cancellation/error/final-result handling inside - # `msg._ops.drain_to_final_msg()` such that we do not - # raise such errors particularly in the case where + # NOTE: whenever this side is the cause of an + # overrun on the peer side, i.e. we sent msgs too + # fast and the peer task was overrun according + # to `MsgStream` buffer settings, AND this was + # called with `raise_overrun_from_self=True` (the + # default), silently absorb any `StreamOverrun`. + # + # XXX, this is namely useful for supressing such faults + # during cancellation/error/final-result handling inside + # `.msg._ops.drain_to_final_msg()` such that we do not + # raise during a cancellation-request, i.e. when # `._cancel_called == True`. + # not raise_overrun_from_self and isinstance(remote_error, RemoteActorError) and remote_error.boxed_type is StreamOverrun From 05df634d62775ed59f75299a85aea793f459af93 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 15 Jun 2025 22:05:51 -0400 Subject: [PATCH 66/74] Use `Channel.aid: Aid` throughout `.ipc._server` --- tractor/ipc/_server.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tractor/ipc/_server.py b/tractor/ipc/_server.py index 7df67aac..e06e747c 100644 --- a/tractor/ipc/_server.py +++ b/tractor/ipc/_server.py @@ -143,7 +143,7 @@ async def maybe_wait_on_canced_subs( log.cancel( 'Waiting on cancel request to peer..\n' f'c)=>\n' - f' |_{chan.uid}\n' + f' |_{chan.aid}\n' ) # XXX: this is a soft wait on the channel (and its @@ -156,7 +156,7 @@ async def maybe_wait_on_canced_subs( # local runtime here is now cancelled while # (presumably) in the middle of msg loop processing. chan_info: str = ( - f'{chan.uid}\n' + f'{chan.aid}\n' f'|_{chan}\n' f' |_{chan.transport}\n\n' ) @@ -279,7 +279,7 @@ async def maybe_wait_on_canced_subs( log.runtime( f'Peer IPC broke but subproc is alive?\n\n' - f'<=x {chan.uid}@{chan.raddr}\n' + f'<=x {chan.aid}@{chan.raddr}\n' f' |_{proc}\n' ) @@ -460,7 +460,7 @@ async def handle_stream_from_peer( # drop ref to channel so it can be gc-ed and disconnected con_teardown_status: str = ( f'IPC channel disconnected:\n' - f'<=x uid: {chan.uid}\n' + f'<=x uid: {chan.aid}\n' f' |_{pformat(chan)}\n\n' ) chans.remove(chan) @@ -468,7 +468,7 @@ async def handle_stream_from_peer( # TODO: do we need to be this pedantic? if not chans: con_teardown_status += ( - f'-> No more channels with {chan.uid}' + f'-> No more channels with {chan.aid}' ) server._peers.pop(uid, None) @@ -519,7 +519,7 @@ async def handle_stream_from_peer( and (ctx_in_debug := pdb_lock.ctx_in_debug) and - (pdb_user_uid := ctx_in_debug.chan.uid) + (pdb_user_uid := ctx_in_debug.chan.aid) ): entry: tuple|None = local_nursery._children.get( tuple(pdb_user_uid) From 28e32b8f852cc41e3fb017fb51407b196bac9dcf Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 17 Jun 2025 11:33:36 -0400 Subject: [PATCH 67/74] Use `enable_transports: list[str]` parameter Actually applying the input it in the root as well as all sub-actors by passing it down to sub-actors through runtime-vars as delivered by the initial `SpawnSpec` msg during child runtime init. Impl deats, - add a new `_state._runtime_vars['_enable_tpts']: list[str]` field set by the input param (if provided) to `.open_root_actor()`. - mk `current_ipc_protos()` return the runtime-var entry with instead the default in the `_runtime_vars: dict` set to `[_def_tpt_proto]`. - in `.open_root_actor()`, still error on this being a >1 `list[str]` until we have more testing infra/suites to audit multi-protos per actor. - return the new value (as 3rd element) from `Actor._from_parent()` as per the todo note; means `_runtime.async_main()` will allocate `accept_addrs` as tpt-specific `Address` entries and pass them to `IPCServer.listen_on()`. Also, - also add a new `_state._runtime_vars['_root_addrs']: list = []` field with the intent of fully replacing the `'_root_mailbox'` field since, * it will need to be a collection to support multi-tpt, * it's a more cohesive field name alongside `_registry_addrs`, * the root actor of every tree needs to have a dedicated addr set (separate from any host-singleton registry actor) so that all its subs can contact it for capabilities mgmt including debugger access/locking. - in the root, populate the field in `._runtime.async_main()` and for now just set '_root_mailbox' to the first entry in that list in anticipation of future multi-homing/transport support. --- tractor/_root.py | 13 +++++++++---- tractor/_runtime.py | 12 ++++++------ tractor/_state.py | 28 +++++++++++++++++----------- tractor/msg/types.py | 1 + 4 files changed, 33 insertions(+), 21 deletions(-) diff --git a/tractor/_root.py b/tractor/_root.py index 1bcfa00b..81ff884b 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -217,11 +217,16 @@ async def open_root_actor( ): if enable_transports is None: enable_transports: list[str] = _state.current_ipc_protos() + else: + _state._runtime_vars['_enable_tpts'] = enable_transports - # TODO! support multi-tpts per actor! Bo - assert ( - len(enable_transports) == 1 - ), 'No multi-tpt support yet!' + # TODO! support multi-tpts per actor! + # Bo + if not len(enable_transports) == 1: + raise RuntimeError( + f'No multi-tpt support yet!\n' + f'enable_transports={enable_transports!r}\n' + ) _debug.hide_runtime_frames() __tracebackhide__: bool = hide_tb diff --git a/tractor/_runtime.py b/tractor/_runtime.py index b25f6bc1..015c235c 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -877,9 +877,7 @@ class Actor: return ( chan, accept_addrs, - None, - # ^TODO, preferred tpts list from rent! - # -[ ] need to extend the `SpawnSpec` tho! + _state._runtime_vars['_enable_tpts'] ) # failed to connect back? @@ -1453,10 +1451,12 @@ async def async_main( # all sub-actors should be able to speak to # their root actor over that channel. if _state._runtime_vars['_is_root']: + raddrs: list[Address] = _state._runtime_vars['_root_addrs'] for addr in accept_addrs: - waddr = wrap_address(addr) - if waddr == waddr.get_root(): - _state._runtime_vars['_root_mailbox'] = addr + waddr: Address = wrap_address(addr) + raddrs.append(addr) + else: + _state._runtime_vars['_root_mailbox'] = raddrs[0] # Register with the arbiter if we're told its addr log.runtime( diff --git a/tractor/_state.py b/tractor/_state.py index bc12d0de..4eba74f0 100644 --- a/tractor/_state.py +++ b/tractor/_state.py @@ -37,14 +37,28 @@ if TYPE_CHECKING: from ._context import Context +# default IPC transport protocol settings +TransportProtocolKey = Literal[ + 'tcp', + 'uds', +] +_def_tpt_proto: TransportProtocolKey = 'tcp' + _current_actor: Actor|None = None # type: ignore # noqa _last_actor_terminated: Actor|None = None # TODO: mk this a `msgspec.Struct`! _runtime_vars: dict[str, Any] = { '_debug_mode': False, - '_is_root': False, - '_root_mailbox': (None, None), + # root of actor-process tree info + '_is_root': False, # bool + '_root_mailbox': (None, None), # tuple[str|None, str|None] + '_root_addrs': [], # tuple[str|None, str|None] + + # parent->chld ipc protocol caps + '_enable_tpts': [_def_tpt_proto], + + # registrar info '_registry_addrs': [], '_is_infected_aio': False, @@ -172,14 +186,6 @@ def get_rt_dir( return rtdir -# default IPC transport protocol settings -TransportProtocolKey = Literal[ - 'tcp', - 'uds', -] -_def_tpt_proto: TransportProtocolKey = 'tcp' - - def current_ipc_protos() -> list[str]: ''' Return the list of IPC transport protocol keys currently @@ -189,4 +195,4 @@ def current_ipc_protos() -> list[str]: concrete-backend sub-types defined throughout `tractor.ipc`. ''' - return [_def_tpt_proto] + return _runtime_vars['_enable_tpts'] diff --git a/tractor/msg/types.py b/tractor/msg/types.py index 86752aba..aaf8d137 100644 --- a/tractor/msg/types.py +++ b/tractor/msg/types.py @@ -170,6 +170,7 @@ class SpawnSpec( # a hard `Struct` def for all of these fields! _parent_main_data: dict _runtime_vars: dict[str, Any] + # ^NOTE see `._state._runtime_vars: dict` # module import capability enable_modules: dict[str, str] From 27e6ad18eea5e4d93466c4dd17769e844901730e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 8 Jul 2025 12:42:32 -0400 Subject: [PATCH 68/74] Mk `.ipc._tcp.TCPAddress` validate with `ipaddress` Both via a post-init method to validate the original input `._host: str` and in `.is_valid` to ensure the host-part isn't something, esoteric.. --- tractor/ipc/_tcp.py | 35 ++++++++++++++++++++++++++++++++++- 1 file changed, 34 insertions(+), 1 deletion(-) diff --git a/tractor/ipc/_tcp.py b/tractor/ipc/_tcp.py index 4de04c25..e945cdfb 100644 --- a/tractor/ipc/_tcp.py +++ b/tractor/ipc/_tcp.py @@ -18,6 +18,7 @@ TCP implementation of tractor.ipc._transport.MsgTransport protocol ''' from __future__ import annotations +import ipaddress from typing import ( ClassVar, ) @@ -50,13 +51,45 @@ class TCPAddress( _host: str _port: int + def __post_init__(self): + try: + ipaddress.ip_address(self._host) + except ValueError as valerr: + raise ValueError( + 'Invalid {type(self).__name__}._host = {self._host!r}\n' + ) from valerr + proto_key: ClassVar[str] = 'tcp' unwrapped_type: ClassVar[type] = tuple[str, int] def_bindspace: ClassVar[str] = '127.0.0.1' + # ?TODO, actually validate ipv4/6 with stdlib's `ipaddress` @property def is_valid(self) -> bool: - return self._port != 0 + ''' + Predicate to ensure a valid socket-address pair. + + ''' + return ( + self._port != 0 + and + (ipaddr := ipaddress.ip_address(self._host)) + and not ( + ipaddr.is_reserved + or + ipaddr.is_unspecified + or + ipaddr.is_link_local + or + ipaddr.is_link_local + or + ipaddr.is_multicast + or + ipaddr.is_global + ) + ) + # ^XXX^ see various properties of invalid addrs here, + # https://docs.python.org/3/library/ipaddress.html#ipaddress.IPv4Address @property def bindspace(self) -> str: From c0eef3bac31e7472bd2fdb3399b9ae8771a6d764 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 13 Jul 2025 13:20:40 -0400 Subject: [PATCH 69/74] Bump GH CI to use `uv` throughout! See docs: https://docs.astral.sh/uv/guides/integration/github/ Summary, - drop `mypy` job for now since I'd like to move to trying `ty`. - convert sdist built to `uv build` - just run test suite on py3.13 for now, not sure if 3.12 will break due to the eg stuff or not? --- .github/workflows/ci.yml | 121 +++++++++++++++++++++++++-------------- 1 file changed, 79 insertions(+), 42 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 571bd1db..581bd4f7 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -8,46 +8,70 @@ on: workflow_dispatch: jobs: - - mypy: - name: 'MyPy' - runs-on: ubuntu-latest - - steps: - - name: Checkout - uses: actions/checkout@v2 - - - name: Setup python - uses: actions/setup-python@v2 - with: - python-version: '3.11' - - - name: Install dependencies - run: pip install -U . --upgrade-strategy eager -r requirements-test.txt - - - name: Run MyPy check - run: mypy tractor/ --ignore-missing-imports --show-traceback - + # ------ sdist ------ # test that we can generate a software distribution and install it # thus avoid missing file issues after packaging. + # + # -[x] produce sdist with uv + # ------ - ------ sdist-linux: name: 'sdist' runs-on: ubuntu-latest steps: - name: Checkout - uses: actions/checkout@v2 + uses: actions/checkout@v4 - - name: Setup python - uses: actions/setup-python@v2 - with: - python-version: '3.11' + - name: Install latest uv + uses: astral-sh/setup-uv@v6 - - name: Build sdist - run: python setup.py sdist --formats=zip + - name: Build sdist as tar.gz + run: uv build --sdist --python=3.13 - - name: Install sdist from .zips - run: python -m pip install dist/*.zip + - name: Install sdist from .tar.gz + run: python -m pip install dist/*.tar.gz + + # ------ type-check ------ + # mypy: + # name: 'MyPy' + # runs-on: ubuntu-latest + + # steps: + # - name: Checkout + # uses: actions/checkout@v4 + + # - name: Install latest uv + # uses: astral-sh/setup-uv@v6 + + # # faster due to server caching? + # # https://docs.astral.sh/uv/guides/integration/github/#setting-up-python + # - name: "Set up Python" + # uses: actions/setup-python@v6 + # with: + # python-version-file: "pyproject.toml" + + # # w uv + # # - name: Set up Python + # # run: uv python install + + # - name: Setup uv venv + # run: uv venv .venv --python=3.13 + + # - name: Install + # run: uv sync --dev + + # # TODO, ty cmd over repo + # # - name: type check with ty + # # run: ty ./tractor/ + + # # - uses: actions/cache@v3 + # # name: Cache uv virtenv as default .venv + # # with: + # # path: ./.venv + # # key: venv-${{ hashFiles('uv.lock') }} + + # - name: Run MyPy check + # run: mypy tractor/ --ignore-missing-imports --show-traceback testing-linux: @@ -59,32 +83,45 @@ jobs: fail-fast: false matrix: os: [ubuntu-latest] - python: ['3.11'] + python-version: ['3.13'] spawn_backend: [ 'trio', - 'mp_spawn', - 'mp_forkserver', + # 'mp_spawn', + # 'mp_forkserver', ] steps: - - name: Checkout - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - - name: Setup python - uses: actions/setup-python@v2 + - name: 'Install uv + py-${{ matrix.python-version }}' + uses: astral-sh/setup-uv@v6 with: - python-version: '${{ matrix.python }}' + python-version: ${{ matrix.python-version }} - - name: Install dependencies - run: pip install -U . -r requirements-test.txt -r requirements-docs.txt --upgrade-strategy eager + # GH way.. faster? + # - name: setup-python@v6 + # uses: actions/setup-python@v6 + # with: + # python-version: '${{ matrix.python-version }}' - - name: List dependencies - run: pip list + # consider caching for speedups? + # https://docs.astral.sh/uv/guides/integration/github/#caching + + - name: Install the project w uv + run: uv sync --locked --all-extras --dev + + # - name: Install dependencies + # run: pip install -U . -r requirements-test.txt -r requirements-docs.txt --upgrade-strategy eager + + - name: List deps tree + run: uv tree - name: Run tests - run: pytest tests/ --spawn-backend=${{ matrix.spawn_backend }} -rsx + run: uv run pytest tests/ --spawn-backend=${{ matrix.spawn_backend }} -rsx + # XXX legacy NOTE XXX + # # We skip 3.10 on windows for now due to not having any collabs to # debug the CI failures. Anyone wanting to hack and solve them is very # welcome, but our primary user base is not using that OS. From cbfb0d014406b6739c1981556a8878a5758e6958 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 13 Jul 2025 13:26:22 -0400 Subject: [PATCH 70/74] Don't use `uv sync --locked` for now --- .github/workflows/ci.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 581bd4f7..be5cb272 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -109,7 +109,7 @@ jobs: # https://docs.astral.sh/uv/guides/integration/github/#caching - name: Install the project w uv - run: uv sync --locked --all-extras --dev + run: uv sync --all-extras --dev # - name: Install dependencies # run: pip install -U . -r requirements-test.txt -r requirements-docs.txt --upgrade-strategy eager From 309360daa261883ac0205865fd7781dfa1b54202 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 13 Jul 2025 13:43:48 -0400 Subject: [PATCH 71/74] Add latest `typing-extension`, needed by `stackscope` --- pyproject.toml | 3 +++ uv.lock | 8 ++++---- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/pyproject.toml b/pyproject.toml index da3105ff..a0491598 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -61,6 +61,9 @@ dev = [ # `tractor.devx` tooling "greenback>=1.2.1,<2", "stackscope>=0.2.2,<0.3", + # ^ requires this? + "typing-extensions>=4.14.1", + "pyperclip>=1.9.0", "prompt-toolkit>=3.0.50", "xonsh>=0.19.2", diff --git a/uv.lock b/uv.lock index 168979bc..3c05dc2f 100644 --- a/uv.lock +++ b/uv.lock @@ -417,7 +417,7 @@ dev = [ { name = "pyperclip", specifier = ">=1.9.0" }, { name = "pytest", specifier = ">=8.3.5" }, { name = "stackscope", specifier = ">=0.2.2,<0.3" }, - { name = "typing-extensions", specifier = ">=4.13.2" }, + { name = "typing-extensions", specifier = ">=4.14.1" }, { name = "xonsh", specifier = ">=0.19.2" }, ] @@ -452,11 +452,11 @@ wheels = [ [[package]] name = "typing-extensions" -version = "4.13.2" +version = "4.14.1" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/f6/37/23083fcd6e35492953e8d2aaaa68b860eb422b34627b13f2ce3eb6106061/typing_extensions-4.13.2.tar.gz", hash = "sha256:e6c81219bd689f51865d9e372991c540bda33a0379d5573cddb9a3a23f7caaef", size = 106967, upload-time = "2025-04-10T14:19:05.416Z" } +sdist = { url = "https://files.pythonhosted.org/packages/98/5a/da40306b885cc8c09109dc2e1abd358d5684b1425678151cdaed4731c822/typing_extensions-4.14.1.tar.gz", hash = "sha256:38b39f4aeeab64884ce9f74c94263ef78f3c22467c8724005483154c26648d36", size = 107673, upload-time = "2025-07-04T13:28:34.16Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/8b/54/b1ae86c0973cc6f0210b53d508ca3641fb6d0c56823f288d108bc7ab3cc8/typing_extensions-4.13.2-py3-none-any.whl", hash = "sha256:a439e7c04b49fec3e5d3e2beaa21755cadbbdc391694e28ccdd36ca4a1408f8c", size = 45806, upload-time = "2025-04-10T14:19:03.967Z" }, + { url = "https://files.pythonhosted.org/packages/b5/00/d631e67a838026495268c2f6884f3711a15a9a2a96cd244fdaea53b823fb/typing_extensions-4.14.1-py3-none-any.whl", hash = "sha256:d1e1e3b58374dc93031d6eda2420a48ea44a36c2b4766a4fdeb3710755731d76", size = 43906, upload-time = "2025-07-04T13:28:32.743Z" }, ] [[package]] From c2e7dc74071ee32e6395ee900721403715311a00 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 15 May 2025 15:08:21 -0400 Subject: [PATCH 72/74] Avoid silent `stackscope`-test fail due to dep Oddly my env was borked bc a missing sub-dep (`typing-extensions` apparently not added by `uv` for `stackscope`?) and then `stackscope` was silently failing import and caused the shield-pause test to also fail (since it couldn't match the expected `log.devx()` on console). The import failure is not very explanatory due to the `log.warning()`; change it to `.error()` level. Also, explicitly import `_sync_pause_from_builtin` in `examples/debugging/restore_builtin_breakpoint.py` to ensure the ref is exported properly from `.devx.debug` (which it wasn't during dev of the prior commit Bp). --- examples/debugging/restore_builtin_breakpoint.py | 6 ++++++ pyproject.toml | 1 - tractor/devx/_stackscope.py | 6 +++--- 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/examples/debugging/restore_builtin_breakpoint.py b/examples/debugging/restore_builtin_breakpoint.py index b591b0f7..1d045a6b 100644 --- a/examples/debugging/restore_builtin_breakpoint.py +++ b/examples/debugging/restore_builtin_breakpoint.py @@ -4,6 +4,11 @@ import sys import trio import tractor +# ensure mod-path is correct! +from tractor.devx.debug import ( + _sync_pause_from_builtin as _sync_pause_from_builtin, +) + async def main() -> None: @@ -13,6 +18,7 @@ async def main() -> None: async with tractor.open_nursery( debug_mode=True, + loglevel='devx', ) as an: assert an assert ( diff --git a/pyproject.toml b/pyproject.toml index a0491598..44ed7d68 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -63,7 +63,6 @@ dev = [ "stackscope>=0.2.2,<0.3", # ^ requires this? "typing-extensions>=4.14.1", - "pyperclip>=1.9.0", "prompt-toolkit>=3.0.50", "xonsh>=0.19.2", diff --git a/tractor/devx/_stackscope.py b/tractor/devx/_stackscope.py index ccc46534..b64be324 100644 --- a/tractor/devx/_stackscope.py +++ b/tractor/devx/_stackscope.py @@ -237,7 +237,7 @@ def enable_stack_on_sig( try: import stackscope except ImportError: - log.warning( + log.error( '`stackscope` not installed for use in debug mode!' ) return None @@ -255,8 +255,8 @@ def enable_stack_on_sig( dump_tree_on_sig, ) log.devx( - 'Enabling trace-trees on `SIGUSR1` ' - 'since `stackscope` is installed @ \n' + f'Enabling trace-trees on `SIGUSR1` ' + f'since `stackscope` is installed @ \n' f'{stackscope!r}\n\n' f'With `SIGUSR1` handler\n' f'|_{dump_tree_on_sig}\n' From 161884fbf1b3835a44ef27367dc86d78e049a486 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 13 Jul 2025 13:56:37 -0400 Subject: [PATCH 73/74] Adjust back `.devx._debug` import Bc this history is pre `.devx.debug` subpkg creation.. --- examples/debugging/restore_builtin_breakpoint.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/debugging/restore_builtin_breakpoint.py b/examples/debugging/restore_builtin_breakpoint.py index 1d045a6b..872dd648 100644 --- a/examples/debugging/restore_builtin_breakpoint.py +++ b/examples/debugging/restore_builtin_breakpoint.py @@ -5,7 +5,7 @@ import trio import tractor # ensure mod-path is correct! -from tractor.devx.debug import ( +from tractor.devx._debug import ( _sync_pause_from_builtin as _sync_pause_from_builtin, ) From ad9833a73ab0cc98949e4e5e2a90dbc94f8a254f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 13 Jul 2025 14:56:57 -0400 Subject: [PATCH 74/74] Update actions badge links in readme --- docs/README.rst | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/docs/README.rst b/docs/README.rst index e3bd9f84..630f5398 100644 --- a/docs/README.rst +++ b/docs/README.rst @@ -689,9 +689,11 @@ channel`_! .. _msgspec: https://jcristharif.com/msgspec/ .. _guest: https://trio.readthedocs.io/en/stable/reference-lowlevel.html?highlight=guest%20mode#using-guest-mode-to-run-trio-on-top-of-other-event-loops - -.. |gh_actions| image:: https://img.shields.io/endpoint.svg?url=https%3A%2F%2Factions-badge.atrox.dev%2Fgoodboy%2Ftractor%2Fbadge&style=popout-square - :target: https://actions-badge.atrox.dev/goodboy/tractor/goto +.. + NOTE, on generating badge links from the UI + https://docs.github.com/en/actions/how-tos/monitoring-and-troubleshooting-workflows/monitoring-workflows/adding-a-workflow-status-badge?ref=gitguardian-blog-automated-secrets-detection#using-the-ui +.. |gh_actions| image:: https://github.com/goodboy/tractor/actions/workflows/ci.yml/badge.svg?branch=main + :target: https://github.com/goodboy/tractor/actions/workflows/ci.yml .. |docs| image:: https://readthedocs.org/projects/tractor/badge/?version=latest :target: https://tractor.readthedocs.io/en/latest/?badge=latest