From dd1c0fa51d9d572d4fe5f3839b7d75f6f201b838 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sun, 16 Mar 2025 17:50:13 -0300 Subject: [PATCH 01/41] Better encapsulate RingBuff ctx managment methods and support non ipc usage Add trio.StrictFIFOLock on sender.send_all Support max_bytes argument on receive_some, keep track of write_ptr on receiver Add max_bytes receive test test_ringbuf_max_bytes Add docstrings to all ringbuf tests Remove EFD_NONBLOCK support, not necesary anymore since we can use abandon_on_cancel=True on trio.to_thread.run_sync Close eventfd's after usage on open_ringbuf --- tests/test_ringbuf.py | 54 ++++++++++++ tractor/ipc/_ringbuf.py | 180 ++++++++++++++++++++++------------------ 2 files changed, 153 insertions(+), 81 deletions(-) diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index 0d3b420b..7f2419d3 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -64,6 +64,8 @@ async def child_write_shm( for msg in msgs: await sender.send_all(msg) + print('writer exit') + @pytest.mark.parametrize( 'msg_amount,rand_min,rand_max,buf_size', @@ -89,6 +91,15 @@ def test_ringbuf( rand_max: int, buf_size: int ): + ''' + - Open a new ring buf on root actor + - Create a sender subactor and generate {msg_amount} messages + optionally with a random amount of bytes at the end of each, + return total_bytes on `ctx.started`, then send all messages + - Create a receiver subactor and receive until total_bytes are + read, print simple perf stats. + + ''' async def main(): with open_ringbuf( 'test_ringbuf', @@ -146,6 +157,11 @@ async def child_blocked_receiver( def test_ring_reader_cancel(): + ''' + Test that a receiver blocked on eventfd(2) read responds to + cancellation. + + ''' async def main(): with open_ringbuf('test_ring_cancel_reader') as token: async with ( @@ -184,6 +200,11 @@ async def child_blocked_sender( def test_ring_sender_cancel(): + ''' + Test that a sender blocked on eventfd(2) read responds to + cancellation. + + ''' async def main(): with open_ringbuf( 'test_ring_cancel_sender', @@ -209,3 +230,36 @@ def test_ring_sender_cancel(): with pytest.raises(tractor._exceptions.ContextCancelled): trio.run(main) + + +def test_ringbuf_max_bytes(): + ''' + Test that RingBuffReceiver.receive_some's max_bytes optional + argument works correctly, send a msg of size 100, then + force receive of messages with max_bytes == 1, wait until + 100 of these messages are received, then compare join of + msgs with original message + + ''' + msg = b''.join(str(i % 10).encode() for i in range(100)) + msgs = [] + + async def main(): + with open_ringbuf( + 'test_ringbuf_max_bytes', + buf_size=10 + ) as token: + async with ( + trio.open_nursery() as n, + RingBuffSender(token, is_ipc=False) as sender, + RingBuffReceiver(token, is_ipc=False) as receiver + ): + n.start_soon(sender.send_all, msg) + while len(msgs) < len(msg): + msg_part = await receiver.receive_some(max_bytes=1) + msg_part = bytes(msg_part) + assert len(msg_part) == 1 + msgs.append(msg_part) + + trio.run(main) + assert msg == b''.join(msgs) diff --git a/tractor/ipc/_ringbuf.py b/tractor/ipc/_ringbuf.py index 6337eea1..304454ed 100644 --- a/tractor/ipc/_ringbuf.py +++ b/tractor/ipc/_ringbuf.py @@ -28,11 +28,15 @@ from msgspec import ( ) from ._linux import ( - EFD_NONBLOCK, open_eventfd, + close_eventfd, EventFD ) from ._mp_bs import disable_mantracker +from tractor.log import get_logger + + +log = get_logger(__name__) disable_mantracker() @@ -64,8 +68,6 @@ class RBToken(Struct, frozen=True): 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, @@ -75,16 +77,21 @@ def open_ringbuf( try: token = RBToken( shm_name=shm_name, - write_eventfd=open_eventfd(flags=write_efd_flags), - wrap_eventfd=open_eventfd(flags=wrap_efd_flags), + write_eventfd=open_eventfd(), + wrap_eventfd=open_eventfd(), buf_size=buf_size ) yield token + close_eventfd(token.write_eventfd) + close_eventfd(token.wrap_eventfd) finally: shm.unlink() +Buffer = bytes | bytearray | memoryview + + class RingBuffSender(trio.abc.SendStream): ''' IPC Reliable Ring Buffer sender side implementation @@ -97,24 +104,26 @@ class RingBuffSender(trio.abc.SendStream): self, token: RBToken, start_ptr: int = 0, + is_ipc: bool = True ): - token = RBToken.from_msg(token) - self._shm = SharedMemory( - name=token.shm_name, - size=token.buf_size, - create=False - ) - self._write_event = EventFD(token.write_eventfd, 'w') - self._wrap_event = EventFD(token.wrap_eventfd, 'r') + self._token = RBToken.from_msg(token) + self._shm: SharedMemory | None = None + self._write_event = EventFD(self._token.write_eventfd, 'w') + self._wrap_event = EventFD(self._token.wrap_eventfd, 'r') self._ptr = start_ptr + self._is_ipc = is_ipc + self._send_lock = trio.StrictFIFOLock() + @property - def key(self) -> str: + def name(self) -> str: + if not self._shm: + raise ValueError('shared memory not initialized yet!') return self._shm.name @property def size(self) -> int: - return self._shm.size + return self._token.buf_size @property def ptr(self) -> int: @@ -128,38 +137,48 @@ class RingBuffSender(trio.abc.SendStream): 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() + async def send_all(self, data: Buffer): + async with self._send_lock: + # 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) + # 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 + # 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() - self._shm.close() - - async def __aenter__(self): + def open(self): + self._shm = SharedMemory( + name=self._token.shm_name, + size=self._token.buf_size, + create=False + ) self._write_event.open() self._wrap_event.open() + + async def aclose(self): + if self._is_ipc: + self._write_event.close() + self._wrap_event.close() + self._shm.close() + + async def __aenter__(self): + self.open() return self @@ -175,26 +194,25 @@ class RingBuffReceiver(trio.abc.ReceiveStream): self, token: RBToken, start_ptr: int = 0, - flags: int = 0 + is_ipc: bool = True ): - token = RBToken.from_msg(token) - self._shm = SharedMemory( - name=token.shm_name, - size=token.buf_size, - create=False - ) - self._write_event = EventFD(token.write_eventfd, 'w') - self._wrap_event = EventFD(token.wrap_eventfd, 'r') + self._token = RBToken.from_msg(token) + self._shm: SharedMemory | None = None + self._write_event = EventFD(self._token.write_eventfd, 'w') + self._wrap_event = EventFD(self._token.wrap_eventfd, 'r') self._ptr = start_ptr - self._flags = flags + self._write_ptr = start_ptr + self._is_ipc = is_ipc @property - def key(self) -> str: + def name(self) -> str: + if not self._shm: + raise ValueError('shared memory not initialized yet!') return self._shm.name @property def size(self) -> int: - return self._shm.size + return self._token.buf_size @property def ptr(self) -> int: @@ -208,46 +226,46 @@ class RingBuffReceiver(trio.abc.ReceiveStream): 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: + async def receive_some(self, max_bytes: int | None = None) -> memoryview: + delta = self._write_ptr - self._ptr + if delta == 0: delta = await self._write_event.read() + self._write_ptr += delta + + if isinstance(max_bytes, int): + if max_bytes == 0: + raise ValueError('if set, max_bytes must be > 0') + delta = min(delta, max_bytes) + + target_ptr = self._ptr + delta # fetch next segment and advance ptr - next_ptr = self._ptr + delta - segment = self._shm.buf[self._ptr:next_ptr] - self._ptr = next_ptr + segment = self._shm.buf[self._ptr:target_ptr] + self._ptr = target_ptr - if self.ptr == self.size: + if self._ptr == self.size: # reached the end, signal wrap around self._ptr = 0 + self._write_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): + def open(self): + self._shm = SharedMemory( + name=self._token.shm_name, + size=self._token.buf_size, + create=False + ) self._write_event.open() self._wrap_event.open() + + async def aclose(self): + if self._is_ipc: + self._write_event.close() + self._wrap_event.close() + self._shm.close() + + async def __aenter__(self): + self.open() return self -- 2.34.1 From a7df2132fa53fbd5de91b1712b2e9a66d1bca994 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sun, 16 Mar 2025 23:57:26 -0300 Subject: [PATCH 02/41] Switch `tractor.ipc.MsgTransport.stream` type to `trio.abc.Stream` Add EOF signaling mechanism Support proper `receive_some` end of stream semantics Add StapledStream non-ipc test Create MsgpackRBStream similar to MsgpackTCPStream for buffered whole-msg reads Add EventFD.read cancellation on EventFD.close mechanism using cancel scope Add test for eventfd cancellation Improve and add docstrings --- tests/test_eventfd.py | 32 +++ tests/test_ringbuf.py | 195 ++++++++++++--- tractor/_testing/samples.py | 4 + tractor/ipc/__init__.py | 28 +++ tractor/ipc/_linux.py | 36 ++- tractor/ipc/_ringbuf.py | 463 +++++++++++++++++++++++++++++++++--- tractor/ipc/_transport.py | 2 +- 7 files changed, 688 insertions(+), 72 deletions(-) create mode 100644 tests/test_eventfd.py diff --git a/tests/test_eventfd.py b/tests/test_eventfd.py new file mode 100644 index 00000000..3d757169 --- /dev/null +++ b/tests/test_eventfd.py @@ -0,0 +1,32 @@ +import trio +import pytest +from tractor.ipc import ( + open_eventfd, + EFDReadCancelled, + EventFD +) + + +def test_eventfd_read_cancellation(): + ''' + Ensure EventFD.read raises EFDReadCancelled if EventFD.close() + is called. + + ''' + fd = open_eventfd() + + async def _read(event: EventFD): + with pytest.raises(EFDReadCancelled): + await event.read() + + async def main(): + async with trio.open_nursery() as n: + with ( + EventFD(fd, 'w') as event, + trio.fail_after(3) + ): + n.start_soon(_read, event) + await trio.sleep(0.2) + event.close() + + trio.run(main) diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index 7f2419d3..102b3f81 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -6,12 +6,15 @@ import pytest import tractor from tractor.ipc._ringbuf import ( open_ringbuf, + attach_to_ringbuf_receiver, + attach_to_ringbuf_sender, + attach_to_ringbuf_pair, + attach_to_ringbuf_stream, RBToken, - RingBuffSender, - RingBuffReceiver ) from tractor._testing.samples import ( - generate_sample_messages, + generate_single_byte_msgs, + generate_sample_messages ) # in case you don't want to melt your cores, uncomment dis! @@ -23,20 +26,14 @@ async def child_read_shm( ctx: tractor.Context, msg_amount: int, token: RBToken, - total_bytes: int, ) -> None: recvd_bytes = 0 await ctx.started() start_ts = time.time() - async with RingBuffReceiver(token) as receiver: - while recvd_bytes < total_bytes: - msg = await receiver.receive_some() + async with attach_to_ringbuf_receiver(token) as receiver: + async for msg in receiver: 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) @@ -44,6 +41,7 @@ async def child_read_shm( print(f'\n\telapsed ms: {elapsed_ms}') print(f'\tmsg/sec: {int(msg_amount / elapsed):,}') print(f'\tbytes/sec: {int(recvd_bytes / elapsed):,}') + print(f'\treceived bytes: {recvd_bytes}') @tractor.context @@ -60,7 +58,7 @@ async def child_write_shm( rand_max=rand_max, ) await ctx.started(total_bytes) - async with RingBuffSender(token) as sender: + async with attach_to_ringbuf_sender(token, cleanup=False) as sender: for msg in msgs: await sender.send_all(msg) @@ -105,14 +103,8 @@ def test_ringbuf( 'test_ringbuf', buf_size=buf_size ) as token: - proc_kwargs = { - 'pass_fds': (token.write_eventfd, token.wrap_eventfd) - } + proc_kwargs = {'pass_fds': token.fds} - common_kwargs = { - 'msg_amount': msg_amount, - 'token': token, - } async with tractor.open_nursery() as an: send_p = await an.start_actor( 'ring_sender', @@ -127,14 +119,15 @@ def test_ringbuf( async with ( send_p.open_context( child_write_shm, + token=token, + msg_amount=msg_amount, 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, + token=token, + msg_amount=msg_amount ) as (sctx, _sent), ): await recv_p.result() @@ -151,7 +144,7 @@ async def child_blocked_receiver( ctx: tractor.Context, token: RBToken ): - async with RingBuffReceiver(token) as receiver: + async with attach_to_ringbuf_receiver(token) as receiver: await ctx.started() await receiver.receive_some() @@ -166,13 +159,13 @@ def test_ring_reader_cancel(): with open_ringbuf('test_ring_cancel_reader') as token: async with ( tractor.open_nursery() as an, - RingBuffSender(token) as _sender, + attach_to_ringbuf_sender(token) as _sender, ): recv_p = await an.start_actor( 'ring_blocked_receiver', enable_modules=[__name__], proc_kwargs={ - 'pass_fds': (token.write_eventfd, token.wrap_eventfd) + 'pass_fds': token.fds } ) async with ( @@ -194,7 +187,7 @@ async def child_blocked_sender( ctx: tractor.Context, token: RBToken ): - async with RingBuffSender(token) as sender: + async with attach_to_ringbuf_sender(token) as sender: await ctx.started() await sender.send_all(b'this will wrap') @@ -215,7 +208,7 @@ def test_ring_sender_cancel(): 'ring_blocked_sender', enable_modules=[__name__], proc_kwargs={ - 'pass_fds': (token.write_eventfd, token.wrap_eventfd) + 'pass_fds': token.fds } ) async with ( @@ -241,7 +234,7 @@ def test_ringbuf_max_bytes(): msgs with original message ''' - msg = b''.join(str(i % 10).encode() for i in range(100)) + msg = generate_single_byte_msgs(100) msgs = [] async def main(): @@ -251,15 +244,153 @@ def test_ringbuf_max_bytes(): ) as token: async with ( trio.open_nursery() as n, - RingBuffSender(token, is_ipc=False) as sender, - RingBuffReceiver(token, is_ipc=False) as receiver + attach_to_ringbuf_sender(token, cleanup=False) as sender, + attach_to_ringbuf_receiver(token, cleanup=False) as receiver ): - n.start_soon(sender.send_all, msg) + async def _send_and_close(): + await sender.send_all(msg) + await sender.aclose() + + n.start_soon(_send_and_close) while len(msgs) < len(msg): msg_part = await receiver.receive_some(max_bytes=1) - msg_part = bytes(msg_part) assert len(msg_part) == 1 msgs.append(msg_part) trio.run(main) assert msg == b''.join(msgs) + + +def test_stapled_ringbuf(): + ''' + Open two ringbufs and give tokens to tasks (swap them such that in/out tokens + are inversed on each task) which will open the streams and use trio.StapledStream + to have a single bidirectional stream. + + Then take turns to send and receive messages. + + ''' + msg = generate_single_byte_msgs(100) + pair_0_msgs = [] + pair_1_msgs = [] + + pair_0_done = trio.Event() + pair_1_done = trio.Event() + + async def pair_0(token_in: RBToken, token_out: RBToken): + async with attach_to_ringbuf_pair( + token_in, + token_out, + cleanup_in=False, + cleanup_out=False + ) as stream: + # first turn to send + await stream.send_all(msg) + + # second turn to receive + while len(pair_0_msgs) != len(msg): + _msg = await stream.receive_some(max_bytes=1) + pair_0_msgs.append(_msg) + + pair_0_done.set() + await pair_1_done.wait() + + + async def pair_1(token_in: RBToken, token_out: RBToken): + async with attach_to_ringbuf_pair( + token_in, + token_out, + cleanup_in=False, + cleanup_out=False + ) as stream: + # first turn to receive + while len(pair_1_msgs) != len(msg): + _msg = await stream.receive_some(max_bytes=1) + pair_1_msgs.append(_msg) + + # second turn to send + await stream.send_all(msg) + + pair_1_done.set() + await pair_0_done.wait() + + + async def main(): + with tractor.ipc.open_ringbuf_pair( + 'test_stapled_ringbuf' + ) as (token_0, token_1): + async with trio.open_nursery() as n: + n.start_soon(pair_0, token_0, token_1) + n.start_soon(pair_1, token_1, token_0) + + + trio.run(main) + + assert msg == b''.join(pair_0_msgs) + assert msg == b''.join(pair_1_msgs) + + +@tractor.context +async def child_transport_sender( + ctx: tractor.Context, + msg_amount_min: int, + msg_amount_max: int, + token_in: RBToken, + token_out: RBToken +): + import random + msgs, _total_bytes = generate_sample_messages( + random.randint(msg_amount_min, msg_amount_max), + rand_min=256, + rand_max=1024, + ) + async with attach_to_ringbuf_stream( + token_in, + token_out + ) as transport: + await ctx.started(msgs) + + for msg in msgs: + await transport.send(msg) + + await transport.recv() + + +def test_ringbuf_transport(): + + msg_amount_min = 100 + msg_amount_max = 1000 + + async def main(): + with tractor.ipc.open_ringbuf_pair( + 'test_ringbuf_transport' + ) as (token_0, token_1): + async with ( + attach_to_ringbuf_stream(token_0, token_1) as transport, + tractor.open_nursery() as an + ): + recv_p = await an.start_actor( + 'test_ringbuf_transport_sender', + enable_modules=[__name__], + proc_kwargs={ + 'pass_fds': token_0.fds + token_1.fds + } + ) + async with ( + recv_p.open_context( + child_transport_sender, + msg_amount_min=msg_amount_min, + msg_amount_max=msg_amount_max, + token_in=token_1, + token_out=token_0 + ) as (ctx, msgs), + ): + recv_msgs = [] + while len(recv_msgs) < len(msgs): + recv_msgs.append(await transport.recv()) + + await transport.send(b'end') + await recv_p.cancel_actor() + assert recv_msgs == msgs + + trio.run(main) diff --git a/tractor/_testing/samples.py b/tractor/_testing/samples.py index a87a22c4..1454ee3d 100644 --- a/tractor/_testing/samples.py +++ b/tractor/_testing/samples.py @@ -2,6 +2,10 @@ import os import random +def generate_single_byte_msgs(amount: int) -> bytes: + return b''.join(str(i % 10).encode() for i in range(amount)) + + def generate_sample_messages( amount: int, rand_min: int = 0, diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index 2c6c3b5d..a988f9a6 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -18,7 +18,35 @@ A modular IPC layer supporting the power of cross-process SC! ''' +import platform + 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, + EFDReadCancelled as EFDReadCancelled, + EventFD as EventFD, + ) + + from ._ringbuf import ( + RBToken as RBToken, + open_ringbuf as open_ringbuf, + RingBuffSender as RingBuffSender, + RingBuffReceiver as RingBuffReceiver, + open_ringbuf_pair as open_ringbuf_pair, + attach_to_ringbuf_receiver as attach_to_ringbuf_receiver, + attach_to_ringbuf_sender as attach_to_ringbuf_sender, + attach_to_ringbuf_pair as attach_to_ringbuf_pair, + attach_to_ringbuf_stream as attach_to_ringbuf_stream, + MsgpackRBStream as MsgpackRBStream + ) diff --git a/tractor/ipc/_linux.py b/tractor/ipc/_linux.py index 88d80d1c..afce6bff 100644 --- a/tractor/ipc/_linux.py +++ b/tractor/ipc/_linux.py @@ -108,6 +108,10 @@ def close_eventfd(fd: int) -> int: raise OSError(errno.errorcode[ffi.errno], 'close failed') +class EFDReadCancelled(Exception): + ... + + class EventFD: ''' Use a previously opened eventfd(2), meant to be used in @@ -124,6 +128,7 @@ class EventFD: self._fd: int = fd self._omode: str = omode self._fobj = None + self._cscope: trio.CancelScope | None = None @property def fd(self) -> int | None: @@ -133,17 +138,38 @@ class EventFD: return write_eventfd(self._fd, value) async def read(self) -> int: - return await trio.to_thread.run_sync( - read_eventfd, self._fd, - abandon_on_cancel=True - ) + ''' + Async wrapper for `read_eventfd(self.fd)` + + `trio.to_thread.run_sync` is used, need to use a `trio.CancelScope` + in order to make it cancellable when `self.close()` is called. + + ''' + self._cscope = trio.CancelScope() + with self._cscope: + return await trio.to_thread.run_sync( + read_eventfd, self._fd, + abandon_on_cancel=True + ) + + if self._cscope.cancelled_caught: + raise EFDReadCancelled + + self._cscope = None def open(self): self._fobj = os.fdopen(self._fd, self._omode) def close(self): if self._fobj: - self._fobj.close() + try: + self._fobj.close() + + except OSError: + ... + + if self._cscope: + self._cscope.cancel() def __enter__(self): self.open() diff --git a/tractor/ipc/_ringbuf.py b/tractor/ipc/_ringbuf.py index 304454ed..7529c942 100644 --- a/tractor/ipc/_ringbuf.py +++ b/tractor/ipc/_ringbuf.py @@ -18,10 +18,22 @@ IPC Reliable RingBuffer implementation ''' from __future__ import annotations -from contextlib import contextmanager as cm +import struct +from collections.abc import ( + AsyncGenerator, + AsyncIterator +) +from contextlib import ( + contextmanager as cm, + asynccontextmanager as acm +) +from typing import ( + Any +) from multiprocessing.shared_memory import SharedMemory import trio +from tricycle import BufferedReceiveStream from msgspec import ( Struct, to_builtins @@ -30,10 +42,16 @@ from msgspec import ( from ._linux import ( open_eventfd, close_eventfd, + EFDReadCancelled, EventFD ) from ._mp_bs import disable_mantracker from tractor.log import get_logger +from tractor._exceptions import ( + TransportClosed, + InternalError +) +from tractor.ipc import MsgTransport log = get_logger(__name__) @@ -41,16 +59,21 @@ log = get_logger(__name__) disable_mantracker() +_DEFAULT_RB_SIZE = 10 * 1024 + class RBToken(Struct, frozen=True): ''' - RingBuffer token contains necesary info to open the two + RingBuffer token contains necesary info to open the three eventfds and the shared memory ''' shm_name: str - write_eventfd: int - wrap_eventfd: int + + write_eventfd: int # used to signal writer ptr advance + wrap_eventfd: int # used to signal reader ready after wrap around + eof_eventfd: int # used to signal writer closed + buf_size: int def as_msg(self): @@ -63,12 +86,29 @@ class RBToken(Struct, frozen=True): return RBToken(**msg) + @property + def fds(self) -> tuple[int, int, int]: + ''' + Useful for `pass_fds` params + + ''' + return ( + self.write_eventfd, + self.wrap_eventfd, + self.eof_eventfd + ) + @cm def open_ringbuf( shm_name: str, - buf_size: int = 10 * 1024, + buf_size: int = _DEFAULT_RB_SIZE, ) -> RBToken: + ''' + Handle resources for a ringbuf (shm, eventfd), yield `RBToken` to + be used with `attach_to_ringbuf_sender` and `attach_to_ringbuf_receiver` + + ''' shm = SharedMemory( name=shm_name, size=buf_size, @@ -79,11 +119,27 @@ def open_ringbuf( shm_name=shm_name, write_eventfd=open_eventfd(), wrap_eventfd=open_eventfd(), + eof_eventfd=open_eventfd(), buf_size=buf_size ) yield token - close_eventfd(token.write_eventfd) - close_eventfd(token.wrap_eventfd) + try: + close_eventfd(token.write_eventfd) + + except OSError: + ... + + try: + close_eventfd(token.wrap_eventfd) + + except OSError: + ... + + try: + close_eventfd(token.eof_eventfd) + + except OSError: + ... finally: shm.unlink() @@ -91,28 +147,36 @@ def open_ringbuf( Buffer = bytes | bytearray | memoryview +''' +IPC Reliable Ring Buffer + +`eventfd(2)` is used for wrap around sync, to signal writes to +the reader and end of stream. + +''' + class RingBuffSender(trio.abc.SendStream): ''' - IPC Reliable Ring Buffer sender side implementation + Ring Buffer sender side implementation - `eventfd(2)` is used for wrap around sync, and also to signal - writes to the reader. + Do not use directly! manage with `attach_to_ringbuf_sender` + after having opened a ringbuf context with `open_ringbuf`. ''' def __init__( self, token: RBToken, - start_ptr: int = 0, - is_ipc: bool = True + cleanup: bool = False ): self._token = RBToken.from_msg(token) self._shm: SharedMemory | None = None self._write_event = EventFD(self._token.write_eventfd, 'w') self._wrap_event = EventFD(self._token.wrap_eventfd, 'r') - self._ptr = start_ptr + self._eof_event = EventFD(self._token.eof_eventfd, 'w') + self._ptr = 0 - self._is_ipc = is_ipc + self._cleanup = cleanup self._send_lock = trio.StrictFIFOLock() @property @@ -170,13 +234,21 @@ class RingBuffSender(trio.abc.SendStream): ) self._write_event.open() self._wrap_event.open() + self._eof_event.open() - async def aclose(self): - if self._is_ipc: + def close(self): + self._eof_event.write( + self._ptr if self._ptr > 0 else self.size + ) + if self._cleanup: self._write_event.close() self._wrap_event.close() + self._eof_event.close() self._shm.close() + async def aclose(self): + self.close() + async def __aenter__(self): self.open() return self @@ -184,25 +256,27 @@ class RingBuffSender(trio.abc.SendStream): class RingBuffReceiver(trio.abc.ReceiveStream): ''' - IPC Reliable Ring Buffer receiver side implementation + Ring Buffer receiver side implementation - `eventfd(2)` is used for wrap around sync, and also to signal - writes to the reader. + Do not use directly! manage with `attach_to_ringbuf_receiver` + after having opened a ringbuf context with `open_ringbuf`. ''' def __init__( self, token: RBToken, - start_ptr: int = 0, - is_ipc: bool = True + cleanup: bool = True, ): self._token = RBToken.from_msg(token) self._shm: SharedMemory | None = None self._write_event = EventFD(self._token.write_eventfd, 'w') self._wrap_event = EventFD(self._token.wrap_eventfd, 'r') - self._ptr = start_ptr - self._write_ptr = start_ptr - self._is_ipc = is_ipc + self._eof_event = EventFD(self._token.eof_eventfd, 'r') + self._ptr: int = 0 + self._write_ptr: int = 0 + self._end_ptr: int = -1 + + self._cleanup: bool = cleanup @property def name(self) -> str: @@ -226,21 +300,71 @@ class RingBuffReceiver(trio.abc.ReceiveStream): def wrap_fd(self) -> int: return self._wrap_event.fd - async def receive_some(self, max_bytes: int | None = None) -> memoryview: + async def _eof_monitor_task(self): + ''' + Long running EOF event monitor, automatically run in bg by + `attach_to_ringbuf_receiver` context manager, if EOF event + is set its value will be the end pointer (highest valid + index to be read from buf, after setting the `self._end_ptr` + we close the write event which should cancel any blocked + `self._write_event.read()`s on it. + + ''' + try: + self._end_ptr = await self._eof_event.read() + self._write_event.close() + + except EFDReadCancelled: + ... + + async def receive_some(self, max_bytes: int | None = None) -> bytes: + ''' + Receive up to `max_bytes`, if no `max_bytes` is provided + a reasonable default is used. + + ''' + if max_bytes is None: + max_bytes: int = _DEFAULT_RB_SIZE + + if max_bytes < 1: + raise ValueError("max_bytes must be >= 1") + + # delta is remaining bytes we havent read delta = self._write_ptr - self._ptr if delta == 0: - delta = await self._write_event.read() - self._write_ptr += delta + # we have read all we can, see if new data is available + if self._end_ptr < 0: + # if we havent been signaled about EOF yet + try: + delta = await self._write_event.read() + self._write_ptr += delta - if isinstance(max_bytes, int): - if max_bytes == 0: - raise ValueError('if set, max_bytes must be > 0') - delta = min(delta, max_bytes) + except EFDReadCancelled: + # while waiting for new data `self._write_event` was closed + # this means writer signaled EOF + if self._end_ptr > 0: + # final self._write_ptr modification and recalculate delta + self._write_ptr = self._end_ptr + delta = self._end_ptr - self._ptr + + else: + # shouldnt happen cause self._eof_monitor_task always sets + # self._end_ptr before closing self._write_event + raise InternalError( + 'self._write_event.read cancelled but self._end_ptr is not set' + ) + + else: + # no more bytes to read and self._end_ptr set, EOF reached + return b'' + + # dont overflow caller + delta = min(delta, max_bytes) target_ptr = self._ptr + delta # fetch next segment and advance ptr - segment = self._shm.buf[self._ptr:target_ptr] + segment = bytes(self._shm.buf[self._ptr:target_ptr]) self._ptr = target_ptr if self._ptr == self.size: @@ -259,13 +383,284 @@ class RingBuffReceiver(trio.abc.ReceiveStream): ) self._write_event.open() self._wrap_event.open() + self._eof_event.open() - async def aclose(self): - if self._is_ipc: + def close(self): + if self._cleanup: self._write_event.close() self._wrap_event.close() + self._eof_event.close() self._shm.close() + async def aclose(self): + self.close() + async def __aenter__(self): self.open() return self + + +@acm +async def attach_to_ringbuf_receiver( + token: RBToken, + cleanup: bool = True +): + ''' + Instantiate a RingBuffReceiver from a previously opened + RBToken. + + Launches `receiver._eof_monitor_task` in a `trio.Nursery`. + ''' + async with ( + trio.open_nursery() as n, + RingBuffReceiver( + token, + cleanup=cleanup + ) as receiver + ): + n.start_soon(receiver._eof_monitor_task) + yield receiver + +@acm +async def attach_to_ringbuf_sender( + token: RBToken, + cleanup: bool = True +): + ''' + Instantiate a RingBuffSender from a previously opened + RBToken. + + ''' + async with RingBuffSender( + token, + cleanup=cleanup + ) as sender: + yield sender + + +@cm +def open_ringbuf_pair( + name: str, + buf_size: int = _DEFAULT_RB_SIZE +): + ''' + Handle resources for a ringbuf pair to be used for + bidirectional messaging. + + ''' + with ( + open_ringbuf( + name + '.pair0', + buf_size=buf_size + ) as token_0, + + open_ringbuf( + name + '.pair1', + buf_size=buf_size + ) as token_1 + ): + yield token_0, token_1 + + +@acm +async def attach_to_ringbuf_pair( + token_in: RBToken, + token_out: RBToken, + cleanup_in: bool = True, + cleanup_out: bool = True +): + ''' + Instantiate a trio.StapledStream from a previously opened + ringbuf pair. + + ''' + async with ( + attach_to_ringbuf_receiver( + token_in, + cleanup=cleanup_in + ) as receiver, + attach_to_ringbuf_sender( + token_out, + cleanup=cleanup_out + ) as sender, + ): + yield trio.StapledStream(sender, receiver) + + +class MsgpackRBStream(MsgTransport): + + def __init__( + self, + stream: trio.StapledStream + ): + self.stream = stream + + # create read loop intance + self._aiter_pkts = self._iter_packets() + self._send_lock = trio.StrictFIFOLock() + + self.drained: list[dict] = [] + + self.recv_stream = BufferedReceiveStream( + transport_stream=stream + ) + + 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. + + ''' + + 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 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. + + ''' + async with self._send_lock: + size: bytes = struct.pack(" 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 + + +@acm +async def attach_to_ringbuf_stream( + token_in: RBToken, + token_out: RBToken, + cleanup_in: bool = True, + cleanup_out: bool = True +): + ''' + Wrap a ringbuf trio.StapledStream in a MsgpackRBStream + + ''' + async with attach_to_ringbuf_pair( + token_in, + token_out, + cleanup_in=cleanup_in, + cleanup_out=cleanup_out + ) as stream: + yield MsgpackRBStream(stream) diff --git a/tractor/ipc/_transport.py b/tractor/ipc/_transport.py index 6bfa5f6a..eb8ff3c9 100644 --- a/tractor/ipc/_transport.py +++ b/tractor/ipc/_transport.py @@ -78,7 +78,7 @@ class MsgTransport(Protocol): # eventual msg definition/types? # - https://docs.python.org/3/library/typing.html#typing.Protocol - stream: trio.SocketStream + stream: trio.abc.Stream drained: list[MsgType] address_type: ClassVar[Type[Address]] -- 2.34.1 From 69ceee09f2483d1103156a778ccbe6ac302d3cbd Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Tue, 18 Mar 2025 13:19:40 -0300 Subject: [PATCH 03/41] Improve test_ringbuf test, drop MsgTransport ring buf impl for now in favour of a trio.abc.Channel[bytes] impl, add docstrings --- tests/test_ringbuf.py | 87 ++++++++---- tractor/_testing/samples.py | 46 +++++- tractor/ipc/__init__.py | 6 +- tractor/ipc/_ringbuf.py | 269 ++++++++++++++---------------------- 4 files changed, 210 insertions(+), 198 deletions(-) diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index 102b3f81..bde46fa8 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -1,4 +1,5 @@ import time +import hashlib import trio import pytest @@ -8,8 +9,8 @@ from tractor.ipc._ringbuf import ( open_ringbuf, attach_to_ringbuf_receiver, attach_to_ringbuf_sender, - attach_to_ringbuf_pair, attach_to_ringbuf_stream, + attach_to_ringbuf_channel, RBToken, ) from tractor._testing.samples import ( @@ -26,12 +27,26 @@ async def child_read_shm( ctx: tractor.Context, msg_amount: int, token: RBToken, -) -> None: - recvd_bytes = 0 +) -> str: + ''' + Sub-actor used in `test_ringbuf`. + + Attach to a ringbuf and receive all messages until end of stream. + Keep track of how many bytes received and also calculate + sha256 of the whole byte stream. + + Calculate and print performance stats, finally return calculated + hash. + + ''' await ctx.started() + print('reader started') + recvd_bytes = 0 + recvd_hash = hashlib.sha256() start_ts = time.time() async with attach_to_ringbuf_receiver(token) as receiver: async for msg in receiver: + recvd_hash.update(msg) recvd_bytes += len(msg) end_ts = time.time() @@ -41,7 +56,9 @@ async def child_read_shm( print(f'\n\telapsed ms: {elapsed_ms}') print(f'\tmsg/sec: {int(msg_amount / elapsed):,}') print(f'\tbytes/sec: {int(recvd_bytes / elapsed):,}') - print(f'\treceived bytes: {recvd_bytes}') + print(f'\treceived bytes: {recvd_bytes:,}') + + return recvd_hash.hexdigest() @tractor.context @@ -52,12 +69,26 @@ async def child_write_shm( rand_max: int, token: RBToken, ) -> None: - msgs, total_bytes = generate_sample_messages( + ''' + Sub-actor used in `test_ringbuf` + + Generate `msg_amount` payloads with + `random.randint(rand_min, rand_max)` random bytes at the end, + Calculate sha256 hash and send it to parent on `ctx.started`. + + Attach to ringbuf and send all generated messages. + + ''' + msgs, _total_bytes = generate_sample_messages( msg_amount, rand_min=rand_min, rand_max=rand_max, ) - await ctx.started(total_bytes) + print('writer hashing payload...') + sent_hash = hashlib.sha256(b''.join(msgs)).hexdigest() + print('writer done hashing.') + await ctx.started(sent_hash) + print('writer started') async with attach_to_ringbuf_sender(token, cleanup=False) as sender: for msg in msgs: await sender.send_all(msg) @@ -91,11 +122,12 @@ def test_ringbuf( ): ''' - Open a new ring buf on root actor - - Create a sender subactor and generate {msg_amount} messages - optionally with a random amount of bytes at the end of each, - return total_bytes on `ctx.started`, then send all messages - - Create a receiver subactor and receive until total_bytes are - read, print simple perf stats. + - Open `child_write_shm` ctx in sub-actor which will generate a + random payload and send its hash on `ctx.started`, finally sending + the payload through the stream. + - Open `child_read_shm` ctx in sub-actor which will receive the + payload, calculate perf stats and return the hash. + - Compare both hashes ''' async def main(): @@ -123,14 +155,16 @@ def test_ringbuf( msg_amount=msg_amount, rand_min=rand_min, rand_max=rand_max, - ) as (sctx, total_bytes), + ) as (_sctx, sent_hash), recv_p.open_context( child_read_shm, token=token, msg_amount=msg_amount - ) as (sctx, _sent), + ) as (rctx, _sent), ): - await recv_p.result() + recvd_hash = await rctx.result() + + assert sent_hash == recvd_hash await send_p.cancel_actor() await recv_p.cancel_actor() @@ -278,7 +312,7 @@ def test_stapled_ringbuf(): pair_1_done = trio.Event() async def pair_0(token_in: RBToken, token_out: RBToken): - async with attach_to_ringbuf_pair( + async with attach_to_ringbuf_stream( token_in, token_out, cleanup_in=False, @@ -297,7 +331,7 @@ def test_stapled_ringbuf(): async def pair_1(token_in: RBToken, token_out: RBToken): - async with attach_to_ringbuf_pair( + async with attach_to_ringbuf_stream( token_in, token_out, cleanup_in=False, @@ -331,7 +365,7 @@ def test_stapled_ringbuf(): @tractor.context -async def child_transport_sender( +async def child_channel_sender( ctx: tractor.Context, msg_amount_min: int, msg_amount_max: int, @@ -344,19 +378,17 @@ async def child_transport_sender( rand_min=256, rand_max=1024, ) - async with attach_to_ringbuf_stream( + async with attach_to_ringbuf_channel( token_in, token_out - ) as transport: + ) as chan: await ctx.started(msgs) for msg in msgs: - await transport.send(msg) - - await transport.recv() + await chan.send(msg) -def test_ringbuf_transport(): +def test_ringbuf_channel(): msg_amount_min = 100 msg_amount_max = 1000 @@ -366,7 +398,7 @@ def test_ringbuf_transport(): 'test_ringbuf_transport' ) as (token_0, token_1): async with ( - attach_to_ringbuf_stream(token_0, token_1) as transport, + attach_to_ringbuf_channel(token_0, token_1) as chan, tractor.open_nursery() as an ): recv_p = await an.start_actor( @@ -378,7 +410,7 @@ def test_ringbuf_transport(): ) async with ( recv_p.open_context( - child_transport_sender, + child_channel_sender, msg_amount_min=msg_amount_min, msg_amount_max=msg_amount_max, token_in=token_1, @@ -386,10 +418,9 @@ def test_ringbuf_transport(): ) as (ctx, msgs), ): recv_msgs = [] - while len(recv_msgs) < len(msgs): - recv_msgs.append(await transport.recv()) + async for msg in chan: + recv_msgs.append(msg) - await transport.send(b'end') await recv_p.cancel_actor() assert recv_msgs == msgs diff --git a/tractor/_testing/samples.py b/tractor/_testing/samples.py index 1454ee3d..4249bae9 100644 --- a/tractor/_testing/samples.py +++ b/tractor/_testing/samples.py @@ -3,6 +3,18 @@ import random def generate_single_byte_msgs(amount: int) -> bytes: + ''' + Generate a byte instance of len `amount` with: + + ``` + byte_at_index(i) = (i % 10).encode() + ``` + + this results in constantly repeating sequences of: + + b'0123456789' + + ''' return b''.join(str(i % 10).encode() for i in range(amount)) @@ -10,15 +22,39 @@ def generate_sample_messages( amount: int, rand_min: int = 0, rand_max: int = 0, - silent: bool = False + silent: bool = False, ) -> tuple[list[bytes], int]: + ''' + Generate bytes msgs for tests. + Messages will have the following format: + + ``` + b'[{i:08}]' + os.urandom(random.randint(rand_min, rand_max)) + ``` + + so for message index 25: + + b'[00000025]' + random_bytes + + ''' msgs = [] size = 0 + log_interval = None if not silent: print(f'\ngenerating {amount} messages...') + # calculate an apropiate log interval based on + # max message size + max_msg_size = 10 + rand_max + + if max_msg_size <= 32 * 1024: + log_interval = 10_000 + + else: + log_interval = 1000 + for i in range(amount): msg = f'[{i:08}]'.encode('utf-8') @@ -30,7 +66,13 @@ def generate_sample_messages( msgs.append(msg) - if not silent and i and i % 10_000 == 0: + if ( + not silent + and + i > 0 + and + i % log_interval == 0 + ): print(f'{i} generated') if not silent: diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index a988f9a6..052876e4 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -46,7 +46,9 @@ if platform.system() == 'Linux': open_ringbuf_pair as open_ringbuf_pair, attach_to_ringbuf_receiver as attach_to_ringbuf_receiver, attach_to_ringbuf_sender as attach_to_ringbuf_sender, - attach_to_ringbuf_pair as attach_to_ringbuf_pair, attach_to_ringbuf_stream as attach_to_ringbuf_stream, - MsgpackRBStream as MsgpackRBStream + RingBuffBytesSender as RingBuffBytesSender, + RingBuffBytesReceiver as RingBuffBytesReceiver, + RingBuffChannel as RingBuffChannel, + attach_to_ringbuf_channel as attach_to_ringbuf_channel ) diff --git a/tractor/ipc/_ringbuf.py b/tractor/ipc/_ringbuf.py index 7529c942..038d9e73 100644 --- a/tractor/ipc/_ringbuf.py +++ b/tractor/ipc/_ringbuf.py @@ -19,17 +19,10 @@ IPC Reliable RingBuffer implementation ''' from __future__ import annotations import struct -from collections.abc import ( - AsyncGenerator, - AsyncIterator -) from contextlib import ( contextmanager as cm, asynccontextmanager as acm ) -from typing import ( - Any -) from multiprocessing.shared_memory import SharedMemory import trio @@ -48,10 +41,8 @@ from ._linux import ( from ._mp_bs import disable_mantracker from tractor.log import get_logger from tractor._exceptions import ( - TransportClosed, InternalError ) -from tractor.ipc import MsgTransport log = get_logger(__name__) @@ -147,6 +138,7 @@ def open_ringbuf( Buffer = bytes | bytearray | memoryview + ''' IPC Reliable Ring Buffer @@ -406,7 +398,7 @@ async def attach_to_ringbuf_receiver( cleanup: bool = True ): ''' - Instantiate a RingBuffReceiver from a previously opened + Attach a RingBuffReceiver from a previously opened RBToken. Launches `receiver._eof_monitor_task` in a `trio.Nursery`. @@ -421,13 +413,14 @@ async def attach_to_ringbuf_receiver( n.start_soon(receiver._eof_monitor_task) yield receiver + @acm async def attach_to_ringbuf_sender( token: RBToken, cleanup: bool = True ): ''' - Instantiate a RingBuffSender from a previously opened + Attach a RingBuffSender from a previously opened RBToken. ''' @@ -463,14 +456,14 @@ def open_ringbuf_pair( @acm -async def attach_to_ringbuf_pair( +async def attach_to_ringbuf_stream( token_in: RBToken, token_out: RBToken, cleanup_in: bool = True, cleanup_out: bool = True ): ''' - Instantiate a trio.StapledStream from a previously opened + Attach a trio.StapledStream from a previously opened ringbuf pair. ''' @@ -487,180 +480,124 @@ async def attach_to_ringbuf_pair( yield trio.StapledStream(sender, receiver) -class MsgpackRBStream(MsgTransport): +class RingBuffBytesSender(trio.abc.SendChannel[bytes]): + ''' + In order to guarantee full messages are received, all bytes + sent by `RingBuffBytesSender` are preceded with a 4 byte header + which decodes into a uint32 indicating the actual size of the + next payload. + + ''' def __init__( self, - stream: trio.StapledStream + sender: RingBuffSender ): - self.stream = stream - - # create read loop intance - self._aiter_pkts = self._iter_packets() + self._sender = sender self._send_lock = trio.StrictFIFOLock() - self.drained: list[dict] = [] - - self.recv_stream = BufferedReceiveStream( - transport_stream=stream - ) - - 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. - - ''' - - 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 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. - - ''' + async def send(self, value: bytes) -> None: async with self._send_lock: - size: bytes = struct.pack(" Any: - return await self._aiter_pkts.asend(None) + async def aclose(self) -> None: + async with self._send_lock: + await self._sender.aclose() - async def drain(self) -> AsyncIterator[dict]: + +class RingBuffBytesReceiver(trio.abc.ReceiveChannel[bytes]): + ''' + See `RingBuffBytesSender` docstring. + + A `tricycle.BufferedReceiveStream` is used for the + `receive_exactly` API. + ''' + def __init__( + self, + receiver: RingBuffReceiver + ): + self._receiver = receiver + + async def _receive_exactly(self, num_bytes: int) -> bytes: ''' - Drain the stream's remaining messages sent from - the far end until the connection is closed by - the peer. + Fetch bytes from receiver until we read exactly `num_bytes` + or end of stream is signaled. ''' - try: - async for msg in self._iter_packets(): - self.drained.append(msg) - except TransportClosed: - for msg in self.drained: - yield msg + payload = b'' + while len(payload) < num_bytes: + remaining = num_bytes - len(payload) - def __aiter__(self): - return self._aiter_pkts + new_bytes = await self._receiver.receive_some( + max_bytes=remaining + ) + + if new_bytes == b'': + raise trio.EndOfChannel + + payload += new_bytes + + return payload + + async def receive(self) -> bytes: + header: bytes = await self._receive_exactly(4) + size: int + size, = struct.unpack(" None: + await self._receiver.aclose() + + +class RingBuffChannel(trio.abc.Channel[bytes]): + ''' + Combine `RingBuffBytesSender` and `RingBuffBytesReceiver` + in order to expose the bidirectional `trio.abc.Channel` API. + + ''' + def __init__( + self, + sender: RingBuffBytesSender, + receiver: RingBuffBytesReceiver + ): + self._sender = sender + self._receiver = receiver + + async def send(self, value: bytes): + await self._sender.send(value) + + async def receive(self) -> bytes: + return await self._receiver.receive() + + async def aclose(self): + await self._receiver.aclose() + await self._sender.aclose() @acm -async def attach_to_ringbuf_stream( +async def attach_to_ringbuf_channel( token_in: RBToken, token_out: RBToken, cleanup_in: bool = True, cleanup_out: bool = True ): ''' - Wrap a ringbuf trio.StapledStream in a MsgpackRBStream + Attach to an already opened ringbuf pair and return + a `RingBuffChannel`. ''' - async with attach_to_ringbuf_pair( - token_in, - token_out, - cleanup_in=cleanup_in, - cleanup_out=cleanup_out - ) as stream: - yield MsgpackRBStream(stream) + async with ( + attach_to_ringbuf_receiver( + token_in, + cleanup=cleanup_in + ) as receiver, + attach_to_ringbuf_sender( + token_out, + cleanup=cleanup_out + ) as sender, + ): + yield RingBuffChannel( + RingBuffBytesSender(sender), + RingBuffBytesReceiver(receiver) + ) -- 2.34.1 From 9f788e07d47b9d04f84b4260830e518b78e578fd Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Tue, 18 Mar 2025 13:47:41 -0300 Subject: [PATCH 04/41] Add direct ctx managers for RB channels --- tractor/ipc/__init__.py | 4 +++- tractor/ipc/_ringbuf.py | 39 +++++++++++++++++++++++++++++++++------ 2 files changed, 36 insertions(+), 7 deletions(-) diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index 052876e4..b4931869 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -50,5 +50,7 @@ if platform.system() == 'Linux': RingBuffBytesSender as RingBuffBytesSender, RingBuffBytesReceiver as RingBuffBytesReceiver, RingBuffChannel as RingBuffChannel, - attach_to_ringbuf_channel as attach_to_ringbuf_channel + attach_to_ringbuf_schannel as attach_to_ringbuf_schannel, + attach_to_ringbuf_rchannel as attach_to_ringbuf_rchannel, + attach_to_ringbuf_channel as attach_to_ringbuf_channel, ) diff --git a/tractor/ipc/_ringbuf.py b/tractor/ipc/_ringbuf.py index 038d9e73..42403937 100644 --- a/tractor/ipc/_ringbuf.py +++ b/tractor/ipc/_ringbuf.py @@ -550,6 +550,36 @@ class RingBuffBytesReceiver(trio.abc.ReceiveChannel[bytes]): await self._receiver.aclose() +@acm +async def attach_to_ringbuf_rchannel( + token: RBToken, + cleanup: bool = True +): + ''' + Attach a RingBuffBytesReceiver from a previously opened + RBToken. + ''' + async with attach_to_ringbuf_receiver( + token, cleanup=cleanup + ) as receiver: + yield RingBuffBytesReceiver(receiver) + + +@acm +async def attach_to_ringbuf_schannel( + token: RBToken, + cleanup: bool = True +): + ''' + Attach a RingBuffBytesSender from a previously opened + RBToken. + ''' + async with attach_to_ringbuf_sender( + token, cleanup=cleanup + ) as sender: + yield RingBuffBytesSender(sender) + + class RingBuffChannel(trio.abc.Channel[bytes]): ''' Combine `RingBuffBytesSender` and `RingBuffBytesReceiver` @@ -588,16 +618,13 @@ async def attach_to_ringbuf_channel( ''' async with ( - attach_to_ringbuf_receiver( + attach_to_ringbuf_rchannel( token_in, cleanup=cleanup_in ) as receiver, - attach_to_ringbuf_sender( + attach_to_ringbuf_schannel( token_out, cleanup=cleanup_out ) as sender, ): - yield RingBuffChannel( - RingBuffBytesSender(sender), - RingBuffBytesReceiver(receiver) - ) + yield RingBuffChannel(sender, receiver) -- 2.34.1 From ce09c70a74d68687d4062f5ccfe0dd264bb291ed Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Tue, 18 Mar 2025 22:48:12 -0300 Subject: [PATCH 05/41] Add direct read method on EventFD Type hint all ctx managers in _ringbuf.py Remove unnecesary send lock on ring chan sender Handle EOF on ring chan receiver Rename ringbuf tests to make it less redundant --- tests/test_ringbuf.py | 8 ++--- tractor/ipc/_linux.py | 8 +++++ tractor/ipc/_ringbuf.py | 75 ++++++++++++++++++----------------------- 3 files changed, 45 insertions(+), 46 deletions(-) diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index bde46fa8..fc8fd3bd 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -183,7 +183,7 @@ async def child_blocked_receiver( await receiver.receive_some() -def test_ring_reader_cancel(): +def test_reader_cancel(): ''' Test that a receiver blocked on eventfd(2) read responds to cancellation. @@ -226,7 +226,7 @@ async def child_blocked_sender( await sender.send_all(b'this will wrap') -def test_ring_sender_cancel(): +def test_sender_cancel(): ''' Test that a sender blocked on eventfd(2) read responds to cancellation. @@ -259,7 +259,7 @@ def test_ring_sender_cancel(): trio.run(main) -def test_ringbuf_max_bytes(): +def test_receiver_max_bytes(): ''' Test that RingBuffReceiver.receive_some's max_bytes optional argument works correctly, send a msg of size 100, then @@ -388,7 +388,7 @@ async def child_channel_sender( await chan.send(msg) -def test_ringbuf_channel(): +def test_channel(): msg_amount_min = 100 msg_amount_max = 1000 diff --git a/tractor/ipc/_linux.py b/tractor/ipc/_linux.py index afce6bff..0c05260e 100644 --- a/tractor/ipc/_linux.py +++ b/tractor/ipc/_linux.py @@ -157,6 +157,14 @@ class EventFD: self._cscope = None + def read_direct(self) -> int: + ''' + Direct call to `read_eventfd(self.fd)`, unless `eventfd` was + opened with `EFD_NONBLOCK` its gonna block the thread. + + ''' + return read_eventfd(self._fd) + def open(self): self._fobj = os.fdopen(self._fd, self._omode) diff --git a/tractor/ipc/_ringbuf.py b/tractor/ipc/_ringbuf.py index 42403937..09c955ac 100644 --- a/tractor/ipc/_ringbuf.py +++ b/tractor/ipc/_ringbuf.py @@ -19,6 +19,10 @@ IPC Reliable RingBuffer implementation ''' from __future__ import annotations import struct +from typing import ( + ContextManager, + AsyncContextManager +) from contextlib import ( contextmanager as cm, asynccontextmanager as acm @@ -26,7 +30,6 @@ from contextlib import ( from multiprocessing.shared_memory import SharedMemory import trio -from tricycle import BufferedReceiveStream from msgspec import ( Struct, to_builtins @@ -34,7 +37,6 @@ from msgspec import ( from ._linux import ( open_eventfd, - close_eventfd, EFDReadCancelled, EventFD ) @@ -94,7 +96,7 @@ class RBToken(Struct, frozen=True): def open_ringbuf( shm_name: str, buf_size: int = _DEFAULT_RB_SIZE, -) -> RBToken: +) -> ContextManager[RBToken]: ''' Handle resources for a ringbuf (shm, eventfd), yield `RBToken` to be used with `attach_to_ringbuf_sender` and `attach_to_ringbuf_receiver` @@ -106,31 +108,19 @@ def open_ringbuf( create=True ) try: - token = RBToken( - shm_name=shm_name, - write_eventfd=open_eventfd(), - wrap_eventfd=open_eventfd(), - eof_eventfd=open_eventfd(), - buf_size=buf_size - ) - yield token - try: - close_eventfd(token.write_eventfd) - - except OSError: - ... - - try: - close_eventfd(token.wrap_eventfd) - - except OSError: - ... - - try: - close_eventfd(token.eof_eventfd) - - except OSError: - ... + with ( + EventFD(open_eventfd(), 'r') as write_event, + EventFD(open_eventfd(), 'r') as wrap_event, + EventFD(open_eventfd(), 'r') as eof_event, + ): + token = RBToken( + shm_name=shm_name, + write_eventfd=write_event.fd, + wrap_eventfd=wrap_event.fd, + eof_eventfd=eof_event.fd, + buf_size=buf_size + ) + yield token finally: shm.unlink() @@ -232,6 +222,7 @@ class RingBuffSender(trio.abc.SendStream): self._eof_event.write( self._ptr if self._ptr > 0 else self.size ) + if self._cleanup: self._write_event.close() self._wrap_event.close() @@ -239,7 +230,8 @@ class RingBuffSender(trio.abc.SendStream): self._shm.close() async def aclose(self): - self.close() + async with self._send_lock: + self.close() async def __aenter__(self): self.open() @@ -396,7 +388,7 @@ class RingBuffReceiver(trio.abc.ReceiveStream): async def attach_to_ringbuf_receiver( token: RBToken, cleanup: bool = True -): +) -> AsyncContextManager[RingBuffReceiver]: ''' Attach a RingBuffReceiver from a previously opened RBToken. @@ -418,7 +410,7 @@ async def attach_to_ringbuf_receiver( async def attach_to_ringbuf_sender( token: RBToken, cleanup: bool = True -): +) -> AsyncContextManager[RingBuffSender]: ''' Attach a RingBuffSender from a previously opened RBToken. @@ -435,7 +427,7 @@ async def attach_to_ringbuf_sender( def open_ringbuf_pair( name: str, buf_size: int = _DEFAULT_RB_SIZE -): +) -> ContextManager[tuple(RBToken, RBToken)]: ''' Handle resources for a ringbuf pair to be used for bidirectional messaging. @@ -461,7 +453,7 @@ async def attach_to_ringbuf_stream( token_out: RBToken, cleanup_in: bool = True, cleanup_out: bool = True -): +) -> AsyncContextManager[trio.StapledStream]: ''' Attach a trio.StapledStream from a previously opened ringbuf pair. @@ -494,16 +486,13 @@ class RingBuffBytesSender(trio.abc.SendChannel[bytes]): sender: RingBuffSender ): self._sender = sender - self._send_lock = trio.StrictFIFOLock() async def send(self, value: bytes) -> None: - async with self._send_lock: - size: bytes = struct.pack(" None: - async with self._send_lock: - await self._sender.aclose() + await self._sender.aclose() class RingBuffBytesReceiver(trio.abc.ReceiveChannel[bytes]): @@ -544,6 +533,8 @@ class RingBuffBytesReceiver(trio.abc.ReceiveChannel[bytes]): header: bytes = await self._receive_exactly(4) size: int size, = struct.unpack(" None: @@ -554,7 +545,7 @@ class RingBuffBytesReceiver(trio.abc.ReceiveChannel[bytes]): async def attach_to_ringbuf_rchannel( token: RBToken, cleanup: bool = True -): +) -> AsyncContextManager[RingBuffBytesReceiver]: ''' Attach a RingBuffBytesReceiver from a previously opened RBToken. @@ -569,7 +560,7 @@ async def attach_to_ringbuf_rchannel( async def attach_to_ringbuf_schannel( token: RBToken, cleanup: bool = True -): +) -> AsyncContextManager[RingBuffBytesSender]: ''' Attach a RingBuffBytesSender from a previously opened RBToken. @@ -611,7 +602,7 @@ async def attach_to_ringbuf_channel( token_out: RBToken, cleanup_in: bool = True, cleanup_out: bool = True -): +) -> AsyncContextManager[RingBuffChannel]: ''' Attach to an already opened ringbuf pair and return a `RingBuffChannel`. -- 2.34.1 From 3b5ade71182dcca8c59c1fcaed5347c02fe8efa5 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 20 Mar 2025 21:12:06 -0300 Subject: [PATCH 06/41] Catch trio cancellation on RingBuffReceiver bg eof listener task, add batched mode to RingBuffBytesSender --- tractor/ipc/_ringbuf.py | 39 ++++++++++++++++++++++++++++++++++----- 1 file changed, 34 insertions(+), 5 deletions(-) diff --git a/tractor/ipc/_ringbuf.py b/tractor/ipc/_ringbuf.py index 09c955ac..7d96eeda 100644 --- a/tractor/ipc/_ringbuf.py +++ b/tractor/ipc/_ringbuf.py @@ -301,6 +301,9 @@ class RingBuffReceiver(trio.abc.ReceiveStream): except EFDReadCancelled: ... + except trio.Cancelled: + ... + async def receive_some(self, max_bytes: int | None = None) -> bytes: ''' Receive up to `max_bytes`, if no `max_bytes` is provided @@ -480,16 +483,41 @@ class RingBuffBytesSender(trio.abc.SendChannel[bytes]): which decodes into a uint32 indicating the actual size of the next payload. + Optional batch mode: + + If `batch_size` > 1 messages wont get sent immediately but will be + stored until `batch_size` messages are pending, then it will send + them all at once. + + `batch_size` can be changed dynamically but always call, `flush()` + right before. + ''' def __init__( self, - sender: RingBuffSender + sender: RingBuffSender, + batch_size: int = 1 ): self._sender = sender + self.batch_size = batch_size + self._batch_msg_len = 0 + self._batch: bytes = b'' + + async def flush(self) -> None: + await self._sender.send_all(self._batch) + self._batch = b'' + self._batch_msg_len = 0 async def send(self, value: bytes) -> None: - size: bytes = struct.pack(" None: await self._sender.aclose() @@ -559,7 +587,8 @@ async def attach_to_ringbuf_rchannel( @acm async def attach_to_ringbuf_schannel( token: RBToken, - cleanup: bool = True + cleanup: bool = True, + batch_size: int = 1, ) -> AsyncContextManager[RingBuffBytesSender]: ''' Attach a RingBuffBytesSender from a previously opened @@ -568,7 +597,7 @@ async def attach_to_ringbuf_schannel( async with attach_to_ringbuf_sender( token, cleanup=cleanup ) as sender: - yield RingBuffBytesSender(sender) + yield RingBuffBytesSender(sender, batch_size=batch_size) class RingBuffChannel(trio.abc.Channel[bytes]): -- 2.34.1 From f0af419ab22d4e5a7ebd9d7d2f4d3f7c93cd167f Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sat, 22 Mar 2025 16:54:00 -0300 Subject: [PATCH 07/41] Important RingBuffBytesSender fix on non batched mode! & downgrade nix-shell python to lowest supported --- tractor/ipc/_ringbuf.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tractor/ipc/_ringbuf.py b/tractor/ipc/_ringbuf.py index 7d96eeda..10975b7a 100644 --- a/tractor/ipc/_ringbuf.py +++ b/tractor/ipc/_ringbuf.py @@ -512,6 +512,7 @@ class RingBuffBytesSender(trio.abc.SendChannel[bytes]): msg: bytes = struct.pack(" Date: Wed, 2 Apr 2025 14:39:02 -0300 Subject: [PATCH 08/41] Linux top-level submodule and ipc._ringbuf submodule Added ringd actor to broker ring buf resources dynamically Added ring pubsub based on ringd Created tractor.linux submodule and moved eventfd stuff there Implemented linux file descriptor ipc share async helpers --- tractor/ipc/__init__.py | 12 - .../ipc/{_ringbuf.py => _ringbuf/__init__.py} | 134 ++++++++--- tractor/ipc/_ringbuf/_pubsub.py | 219 ++++++++++++++++++ tractor/ipc/_ringbuf/_ringd.py | 172 ++++++++++++++ tractor/linux/__init__.py | 4 + tractor/linux/_fdshare.py | 81 +++++++ tractor/{ipc/_linux.py => linux/eventfd.py} | 0 7 files changed, 574 insertions(+), 48 deletions(-) rename tractor/ipc/{_ringbuf.py => _ringbuf/__init__.py} (84%) create mode 100644 tractor/ipc/_ringbuf/_pubsub.py create mode 100644 tractor/ipc/_ringbuf/_ringd.py create mode 100644 tractor/linux/__init__.py create mode 100644 tractor/linux/_fdshare.py rename tractor/{ipc/_linux.py => linux/eventfd.py} (100%) diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index b4931869..c2ddfe48 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -26,18 +26,6 @@ from ._chan import ( ) 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, - EFDReadCancelled as EFDReadCancelled, - EventFD as EventFD, - ) - from ._ringbuf import ( RBToken as RBToken, open_ringbuf as open_ringbuf, diff --git a/tractor/ipc/_ringbuf.py b/tractor/ipc/_ringbuf/__init__.py similarity index 84% rename from tractor/ipc/_ringbuf.py rename to tractor/ipc/_ringbuf/__init__.py index 10975b7a..f9b770a1 100644 --- a/tractor/ipc/_ringbuf.py +++ b/tractor/ipc/_ringbuf/__init__.py @@ -35,16 +35,16 @@ from msgspec import ( to_builtins ) -from ._linux import ( +from ...log import get_logger +from ..._exceptions import ( + InternalError +) +from .._mp_bs import disable_mantracker +from ...linux.eventfd import ( open_eventfd, EFDReadCancelled, EventFD ) -from ._mp_bs import disable_mantracker -from tractor.log import get_logger -from tractor._exceptions import ( - InternalError -) log = get_logger(__name__) @@ -183,6 +183,9 @@ class RingBuffSender(trio.abc.SendStream): def wrap_fd(self) -> int: return self._wrap_event.fd + async def _wait_wrap(self): + await self._wrap_event.read() + async def send_all(self, data: Buffer): async with self._send_lock: # while data is larger than the remaining buf @@ -193,7 +196,7 @@ class RingBuffSender(trio.abc.SendStream): 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() + await self._wait_wrap() # wrap around and trim already written bytes self._ptr = 0 @@ -209,14 +212,19 @@ class RingBuffSender(trio.abc.SendStream): raise NotImplementedError def open(self): - self._shm = SharedMemory( - name=self._token.shm_name, - size=self._token.buf_size, - create=False - ) - self._write_event.open() - self._wrap_event.open() - self._eof_event.open() + try: + self._shm = SharedMemory( + name=self._token.shm_name, + size=self._token.buf_size, + create=False + ) + self._write_event.open() + self._wrap_event.open() + self._eof_event.open() + + except Exception as e: + e.add_note(f'while opening sender for {self._token.as_msg()}') + raise e def close(self): self._eof_event.write( @@ -363,14 +371,19 @@ class RingBuffReceiver(trio.abc.ReceiveStream): return segment def open(self): - self._shm = SharedMemory( - name=self._token.shm_name, - size=self._token.buf_size, - create=False - ) - self._write_event.open() - self._wrap_event.open() - self._eof_event.open() + try: + self._shm = SharedMemory( + name=self._token.shm_name, + size=self._token.buf_size, + create=False + ) + self._write_event.open() + self._wrap_event.open() + self._eof_event.open() + + except Exception as e: + e.add_note(f'while opening receiver for {self._token.as_msg()}') + raise e def close(self): if self._cleanup: @@ -502,26 +515,52 @@ class RingBuffBytesSender(trio.abc.SendChannel[bytes]): self.batch_size = batch_size self._batch_msg_len = 0 self._batch: bytes = b'' + self._send_lock = trio.StrictFIFOLock() - async def flush(self) -> None: + @property + def pending_msgs(self) -> int: + return self._batch_msg_len + + @property + def must_flush(self) -> bool: + return self._batch_msg_len >= self.batch_size + + async def _flush( + self, + new_batch_size: int | None = None + ) -> None: await self._sender.send_all(self._batch) self._batch = b'' self._batch_msg_len = 0 + if new_batch_size: + self.batch_size = new_batch_size + + async def flush( + self, + new_batch_size: int | None = None + ) -> None: + async with self._send_lock: + await self._flush(new_batch_size=new_batch_size) async def send(self, value: bytes) -> None: - msg: bytes = struct.pack(" None: + await self.flush(new_batch_size=1) + await self.send(b'') async def aclose(self) -> None: - await self._sender.aclose() + async with self._send_lock: + await self._sender.aclose() class RingBuffBytesReceiver(trio.abc.ReceiveChannel[bytes]): @@ -615,9 +654,30 @@ class RingBuffChannel(trio.abc.Channel[bytes]): self._sender = sender self._receiver = receiver - async def send(self, value: bytes): + @property + def batch_size(self) -> int: + return self._sender.batch_size + + @batch_size.setter + def batch_size(self, value: int) -> None: + self._sender.batch_size = value + + @property + def pending_msgs(self) -> int: + return self._sender.pending_msgs + + async def flush( + self, + new_batch_size: int | None = None + ) -> None: + await self._sender.flush(new_batch_size=new_batch_size) + + async def send(self, value: bytes) -> None: await self._sender.send(value) + async def send_eof(self) -> None: + await self._sender.send_eof() + async def receive(self) -> bytes: return await self._receiver.receive() @@ -631,7 +691,8 @@ async def attach_to_ringbuf_channel( token_in: RBToken, token_out: RBToken, cleanup_in: bool = True, - cleanup_out: bool = True + cleanup_out: bool = True, + batch_size: int = 1 ) -> AsyncContextManager[RingBuffChannel]: ''' Attach to an already opened ringbuf pair and return @@ -645,7 +706,8 @@ async def attach_to_ringbuf_channel( ) as receiver, attach_to_ringbuf_schannel( token_out, - cleanup=cleanup_out + cleanup=cleanup_out, + batch_size=batch_size ) as sender, ): yield RingBuffChannel(sender, receiver) diff --git a/tractor/ipc/_ringbuf/_pubsub.py b/tractor/ipc/_ringbuf/_pubsub.py new file mode 100644 index 00000000..a41a83dd --- /dev/null +++ b/tractor/ipc/_ringbuf/_pubsub.py @@ -0,0 +1,219 @@ +import time +from abc import ( + ABC, + abstractmethod +) +from contextlib import asynccontextmanager as acm +from dataclasses import dataclass + +import trio +import tractor + +from tractor.ipc import ( + RingBuffBytesSender, + attach_to_ringbuf_schannel, + attach_to_ringbuf_rchannel +) + +import tractor.ipc._ringbuf._ringd as ringd + + +log = tractor.log.get_logger(__name__) + + +@dataclass +class ChannelInfo: + connect_time: float + name: str + channel: RingBuffBytesSender + cancel_scope: trio.CancelScope + + +class ChannelManager(ABC): + + def __init__( + self, + n: trio.Nursery, + ): + self._n = n + self._channels: list[ChannelInfo] = [] + + @abstractmethod + async def _channel_handler_task(self, name: str): + ... + + def find_channel(self, name: str) -> tuple[int, ChannelInfo] | None: + for entry in enumerate(self._channels): + i, info = entry + if info.name == name: + return entry + + return None + + def _maybe_destroy_channel(self, name: str): + maybe_entry = self.find_channel(name) + if maybe_entry: + i, info = maybe_entry + info.cancel_scope.cancel() + del self._channels[i] + + def add_channel(self, name: str): + self._n.start_soon( + self._channel_handler_task, + name + ) + + def remove_channel(self, name: str): + self._maybe_destroy_channel(name) + + def __len__(self) -> int: + return len(self._channels) + + async def aclose(self) -> None: + for chan in self._channels: + self._maybe_destroy_channel(chan.name) + + +class RingBuffPublisher(ChannelManager, trio.abc.SendChannel[bytes]): + + def __init__( + self, + n: trio.Nursery, + buf_size: int = 10 * 1024, + batch_size: int = 1 + ): + super().__init__(n) + self._connect_event = trio.Event() + self._next_turn: int = 0 + + self._batch_size: int = batch_size + + async def _channel_handler_task( + self, + name: str + ): + async with ( + ringd.open_ringbuf( + name=name, + must_exist=True, + ) as token, + attach_to_ringbuf_schannel(token) as schan + ): + with trio.CancelScope() as cancel_scope: + self._channels.append(ChannelInfo( + connect_time=time.time(), + name=name, + channel=schan, + cancel_scope=cancel_scope + )) + self._connect_event.set() + await trio.sleep_forever() + + self._maybe_destroy_channel(name) + + async def send(self, msg: bytes): + # wait at least one decoder connected + if len(self) == 0: + await self._connect_event.wait() + self._connect_event = trio.Event() + + if self._next_turn >= len(self): + self._next_turn = 0 + + turn = self._next_turn + self._next_turn += 1 + + output = self._channels[turn] + await output.channel.send(msg) + + @property + def batch_size(self) -> int: + return self._batch_size + + @batch_size.setter + def set_batch_size(self, value: int) -> None: + for output in self._channels: + output.channel.batch_size = value + + async def flush( + self, + new_batch_size: int | None = None + ): + for output in self._channels: + await output.channel.flush( + new_batch_size=new_batch_size + ) + + async def send_eof(self): + for output in self._channels: + await output.channel.send_eof() + + +@acm +async def open_ringbuf_publisher( + buf_size: int = 10 * 1024, + batch_size: int = 1 +): + async with ( + trio.open_nursery() as n, + RingBuffPublisher( + n, + buf_size=buf_size, + batch_size=batch_size + ) as outputs + ): + yield outputs + await outputs.aclose() + + + +class RingBuffSubscriber(ChannelManager, trio.abc.ReceiveChannel[bytes]): + def __init__( + self, + n: trio.Nursery, + ): + super().__init__(n) + self._send_chan, self._recv_chan = trio.open_memory_channel(0) + + async def _channel_handler_task( + self, + name: str + ): + async with ( + ringd.open_ringbuf( + name=name, + must_exist=True + ) as token, + + attach_to_ringbuf_rchannel(token) as rchan + ): + with trio.CancelScope() as cancel_scope: + self._channels.append(ChannelInfo( + connect_time=time.time(), + name=name, + channel=rchan, + cancel_scope=cancel_scope + )) + send_chan = self._send_chan.clone() + try: + async for msg in rchan: + await send_chan.send(msg) + + except tractor._exceptions.InternalError: + ... + + self._maybe_destroy_channel(name) + + async def receive(self) -> bytes: + return await self._recv_chan.receive() + + +@acm +async def open_ringbuf_subscriber(): + async with ( + trio.open_nursery() as n, + RingBuffSubscriber(n) as inputs + ): + yield inputs + await inputs.aclose() + diff --git a/tractor/ipc/_ringbuf/_ringd.py b/tractor/ipc/_ringbuf/_ringd.py new file mode 100644 index 00000000..fb255979 --- /dev/null +++ b/tractor/ipc/_ringbuf/_ringd.py @@ -0,0 +1,172 @@ +import os +import tempfile +from pathlib import Path +from contextlib import ( + asynccontextmanager as acm +) + +import trio +import tractor +from tractor.linux import send_fds, recv_fds + +from . import ( + RBToken, + open_ringbuf as ipc_open_ringbuf +) + + +log = tractor.log.get_logger(__name__) +# log = tractor.log.get_console_log(level='info') + + +_ringd_actor_name = 'ringd' +_root_key = _ringd_actor_name + f'-{os.getpid()}' +_rings: dict[str, RBToken] = {} + + +async def _attach_to_ring( + ring_name: str +) -> RBToken: + actor = tractor.current_actor() + + fd_amount = 3 + sock_path = str( + Path(tempfile.gettempdir()) + / + f'{os.getpid()}-pass-ring-fds-{ring_name}-to-{actor.name}.sock' + ) + + log.info(f'trying to attach to ring {ring_name}...') + + async with ( + tractor.find_actor(_ringd_actor_name) as ringd, + ringd.open_context( + _pass_fds, + name=ring_name, + sock_path=sock_path + ) as (ctx, token), + recv_fds(sock_path, fd_amount) as fds, + ): + log.info( + f'received fds: {fds}' + ) + + token = RBToken.from_msg(token) + + write, wrap, eof = fds + + return RBToken( + shm_name=token.shm_name, + write_eventfd=write, + wrap_eventfd=wrap, + eof_eventfd=eof, + buf_size=token.buf_size + ) + + +@tractor.context +async def _pass_fds( + ctx: tractor.Context, + name: str, + sock_path: str +): + global _rings + + token = _rings[name] + + async with send_fds(token.fds, sock_path): + log.info(f'connected to {sock_path} for fd passing') + await ctx.started(token) + + log.info(f'fds {token.fds} sent') + + return token + + +@tractor.context +async def _open_ringbuf( + ctx: tractor.Context, + name: str, + must_exist: bool = False, + buf_size: int = 10 * 1024 +): + global _root_key, _rings + + teardown = trio.Event() + async def _teardown_listener(task_status=trio.TASK_STATUS_IGNORED): + async with ctx.open_stream() as stream: + task_status.started() + await stream.receive() + teardown.set() + + log.info(f'maybe open ring {name}, must_exist = {must_exist}') + + token = _rings.get(name, None) + + async with trio.open_nursery() as n: + if token: + log.info(f'ring {name} exists') + await ctx.started() + await n.start(_teardown_listener) + await teardown.wait() + return + + if must_exist: + raise FileNotFoundError( + f'Tried to open_ringbuf but it doesn\'t exist: {name}' + ) + + with ipc_open_ringbuf( + _root_key + name, + buf_size=buf_size + ) as token: + _rings[name] = token + log.info(f'ring {name} created') + await ctx.started() + await n.start(_teardown_listener) + await teardown.wait() + del _rings[name] + + log.info(f'ring {name} destroyed') + + +@acm +async def open_ringd(**kwargs) -> tractor.Portal: + async with tractor.open_nursery(**kwargs) as an: + portal = await an.start_actor( + _ringd_actor_name, + enable_modules=[__name__] + ) + yield portal + await an.cancel() + + +@acm +async def wait_for_ringd() -> tractor.Portal: + async with tractor.wait_for_actor( + _ringd_actor_name + ) as portal: + yield portal + + +@acm +async def open_ringbuf( + name: str, + must_exist: bool = False, + buf_size: int = 10 * 1024 +) -> RBToken: + async with ( + wait_for_ringd() as ringd, + ringd.open_context( + _open_ringbuf, + name=name, + must_exist=must_exist, + buf_size=buf_size + ) as (rd_ctx, _), + rd_ctx.open_stream() as stream, + ): + token = await _attach_to_ring(name) + log.info(f'attached to {token}') + yield token + await stream.send(b'bye') + diff --git a/tractor/linux/__init__.py b/tractor/linux/__init__.py new file mode 100644 index 00000000..dce926c8 --- /dev/null +++ b/tractor/linux/__init__.py @@ -0,0 +1,4 @@ +from ._fdshare import ( + send_fds as send_fds, + recv_fds as recv_fds +) diff --git a/tractor/linux/_fdshare.py b/tractor/linux/_fdshare.py new file mode 100644 index 00000000..a1ddceec --- /dev/null +++ b/tractor/linux/_fdshare.py @@ -0,0 +1,81 @@ +''' +Re-Impl of multiprocessing.reduction.sendfds & recvfds, +using acms and trio +''' +import array +from contextlib import asynccontextmanager as acm + +import trio +from trio import socket + + +@acm +async def send_fds(fds: list[int], sock_path: str): + sock = socket.socket(socket.AF_UNIX, socket.SOCK_STREAM) + await sock.bind(sock_path) + sock.listen(1) + fds = array.array('i', fds) + # first byte of msg will be len of fds to send % 256 + msg = bytes([len(fds) % 256]) + yield + conn, _ = await sock.accept() + await conn.sendmsg( + [msg], + [(socket.SOL_SOCKET, socket.SCM_RIGHTS, fds)] + ) + # wait ack + if await conn.recv(1) != b'A': + raise RuntimeError('did not receive acknowledgement of fd') + + conn.close() + sock.close() + + +@acm +async def recv_fds(sock_path: str, amount: int) -> tuple: + stream = await trio.open_unix_socket(sock_path) + sock = stream.socket + a = array.array('i') + bytes_size = a.itemsize * amount + msg, ancdata, flags, addr = await sock.recvmsg( + 1, socket.CMSG_SPACE(bytes_size) + ) + if not msg and not ancdata: + raise EOFError + try: + await sock.send(b'A') # Ack + + if len(ancdata) != 1: + raise RuntimeError( + f'received {len(ancdata)} items of ancdata' + ) + + cmsg_level, cmsg_type, cmsg_data = ancdata[0] + # check proper msg type + if ( + cmsg_level == socket.SOL_SOCKET + and + cmsg_type == socket.SCM_RIGHTS + ): + # check proper data alignment + if len(cmsg_data) % a.itemsize != 0: + raise ValueError + + # attempt to cast as int array + a.frombytes(cmsg_data) + + # check first byte of message is amount % 256 + if len(a) % 256 != msg[0]: + raise AssertionError( + 'Len is {0:n} but msg[0] is {1!r}'.format( + len(a), msg[0] + ) + ) + + yield tuple(a) + return + + except (ValueError, IndexError): + pass + + raise RuntimeError('Invalid data received') diff --git a/tractor/ipc/_linux.py b/tractor/linux/eventfd.py similarity index 100% rename from tractor/ipc/_linux.py rename to tractor/linux/eventfd.py -- 2.34.1 From 1bb9918e2df97e9a380f9d519cf8a6cbcd38effa Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Wed, 2 Apr 2025 14:45:21 -0300 Subject: [PATCH 09/41] Add ringd test, which also tests fd share --- tests/test_ringd.py | 81 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 81 insertions(+) create mode 100644 tests/test_ringd.py diff --git a/tests/test_ringd.py b/tests/test_ringd.py new file mode 100644 index 00000000..6ce71333 --- /dev/null +++ b/tests/test_ringd.py @@ -0,0 +1,81 @@ +import trio +import tractor + +from tractor.ipc import ( + attach_to_ringbuf_rchannel, + attach_to_ringbuf_schannel +) + +import tractor.ipc._ringbuf._ringd as ringd + + +log = tractor.log.get_console_log(level='info') + + +@tractor.context +async def recv_child( + ctx: tractor.Context, + ring_name: str +): + async with ( + ringd.open_ringbuf(ring_name) as token, + + attach_to_ringbuf_rchannel(token) as chan, + ): + await ctx.started() + async for msg in chan: + log.info(f'received {int.from_bytes(msg)}') + + +@tractor.context +async def send_child( + ctx: tractor.Context, + ring_name: str +): + async with ( + ringd.open_ringbuf(ring_name) as token, + + attach_to_ringbuf_schannel(token) as chan, + ): + await ctx.started() + for i in range(100): + await chan.send(i.to_bytes(4)) + log.info(f'sent {i}') + + + +def test_ringd(): + async def main(): + async with ( + tractor.open_nursery() as an, + + ringd.open_ringd( + loglevel='info' + ) + ): + recv_portal = await an.start_actor( + 'recv', + enable_modules=[__name__] + ) + send_portal = await an.start_actor( + 'send', + enable_modules=[__name__] + ) + + async with ( + recv_portal.open_context( + recv_child, + ring_name='ring' + ) as (rctx, _), + + send_portal.open_context( + send_child, + ring_name='ring' + ) as (sctx, _), + ): + await rctx.wait_for_result() + await sctx.wait_for_result() + + await an.cancel() + + trio.run(main) -- 2.34.1 From 6646deb7f405be16dd5e4519279eab055b49761d Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Wed, 2 Apr 2025 14:53:15 -0300 Subject: [PATCH 10/41] Add LICENSE headers and module docstring on new files --- tractor/ipc/_ringbuf/_pubsub.py | 19 +++++++++++++++++++ tractor/ipc/_ringbuf/_ringd.py | 25 +++++++++++++++++++++++++ tractor/linux/_fdshare.py | 15 +++++++++++++++ tractor/linux/eventfd.py | 2 +- 4 files changed, 60 insertions(+), 1 deletion(-) diff --git a/tractor/ipc/_ringbuf/_pubsub.py b/tractor/ipc/_ringbuf/_pubsub.py index a41a83dd..94ce7460 100644 --- a/tractor/ipc/_ringbuf/_pubsub.py +++ b/tractor/ipc/_ringbuf/_pubsub.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 . +''' +Ring buffer ipc publish-subscribe mechanism brokered by ringd +can dynamically add new outputs (publisher) or inputs (subscriber) +''' import time from abc import ( ABC, diff --git a/tractor/ipc/_ringbuf/_ringd.py b/tractor/ipc/_ringbuf/_ringd.py index fb255979..a35de84b 100644 --- a/tractor/ipc/_ringbuf/_ringd.py +++ b/tractor/ipc/_ringbuf/_ringd.py @@ -1,3 +1,28 @@ +# 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 . +''' +Actor to broker ringbuf resources, creates and allocates +the resources, then automatically does fd passing. + +call open_ringd in your root actor + +then on actors that need a ringbuf resource use + +open_ringbuf acm, will automatically contact ringd. +''' import os import tempfile from pathlib import Path diff --git a/tractor/linux/_fdshare.py b/tractor/linux/_fdshare.py index a1ddceec..82dd0e69 100644 --- a/tractor/linux/_fdshare.py +++ b/tractor/linux/_fdshare.py @@ -1,3 +1,18 @@ +# 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 . ''' Re-Impl of multiprocessing.reduction.sendfds & recvfds, using acms and trio diff --git a/tractor/linux/eventfd.py b/tractor/linux/eventfd.py index 0c05260e..f262c051 100644 --- a/tractor/linux/eventfd.py +++ b/tractor/linux/eventfd.py @@ -14,7 +14,7 @@ # 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 +Expose libc eventfd APIs ''' import os -- 2.34.1 From e34b6519c7056708f78c91224a8efeb2590210ed Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Wed, 2 Apr 2025 15:00:40 -0300 Subject: [PATCH 11/41] recv_fds doesnt need to be an acm --- tractor/ipc/_ringbuf/_ringd.py | 2 +- tractor/linux/_fdshare.py | 6 ++---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/tractor/ipc/_ringbuf/_ringd.py b/tractor/ipc/_ringbuf/_ringd.py index a35de84b..0568771c 100644 --- a/tractor/ipc/_ringbuf/_ringd.py +++ b/tractor/ipc/_ringbuf/_ringd.py @@ -70,8 +70,8 @@ async def _attach_to_ring( name=ring_name, sock_path=sock_path ) as (ctx, token), - recv_fds(sock_path, fd_amount) as fds, ): + fds = await recv_fds(sock_path, fd_amount) log.info( f'received fds: {fds}' ) diff --git a/tractor/linux/_fdshare.py b/tractor/linux/_fdshare.py index 82dd0e69..e3817c34 100644 --- a/tractor/linux/_fdshare.py +++ b/tractor/linux/_fdshare.py @@ -29,10 +29,10 @@ async def send_fds(fds: list[int], sock_path: str): sock = socket.socket(socket.AF_UNIX, socket.SOCK_STREAM) await sock.bind(sock_path) sock.listen(1) + yield fds = array.array('i', fds) # first byte of msg will be len of fds to send % 256 msg = bytes([len(fds) % 256]) - yield conn, _ = await sock.accept() await conn.sendmsg( [msg], @@ -46,7 +46,6 @@ async def send_fds(fds: list[int], sock_path: str): sock.close() -@acm async def recv_fds(sock_path: str, amount: int) -> tuple: stream = await trio.open_unix_socket(sock_path) sock = stream.socket @@ -87,8 +86,7 @@ async def recv_fds(sock_path: str, amount: int) -> tuple: ) ) - yield tuple(a) - return + return tuple(a) except (ValueError, IndexError): pass -- 2.34.1 From 28b86cb880c6222e10ace95165aa1afedcfb955f Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Wed, 2 Apr 2025 15:17:56 -0300 Subject: [PATCH 12/41] Dont use relative import on ringd --- tractor/ipc/_ringbuf/_ringd.py | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/tractor/ipc/_ringbuf/_ringd.py b/tractor/ipc/_ringbuf/_ringd.py index 0568771c..16314c72 100644 --- a/tractor/ipc/_ringbuf/_ringd.py +++ b/tractor/ipc/_ringbuf/_ringd.py @@ -34,10 +34,8 @@ import trio import tractor from tractor.linux import send_fds, recv_fds -from . import ( - RBToken, - open_ringbuf as ipc_open_ringbuf -) +import tractor.ipc._ringbuf as ringbuf +from tractor.ipc._ringbuf import RBToken log = tractor.log.get_logger(__name__) @@ -141,7 +139,7 @@ async def _open_ringbuf( f'Tried to open_ringbuf but it doesn\'t exist: {name}' ) - with ipc_open_ringbuf( + with ringbuf.open_ringbuf( _root_key + name, buf_size=buf_size ) as token: -- 2.34.1 From 4b9d6b9276c7f016539d30bdab52b29d75988f03 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 3 Apr 2025 11:48:07 -0300 Subject: [PATCH 13/41] Improve error handling in fdshare functions, add comments --- tractor/linux/_fdshare.py | 141 +++++++++++++++++++++++++++----------- 1 file changed, 102 insertions(+), 39 deletions(-) diff --git a/tractor/linux/_fdshare.py b/tractor/linux/_fdshare.py index e3817c34..c632f532 100644 --- a/tractor/linux/_fdshare.py +++ b/tractor/linux/_fdshare.py @@ -14,81 +14,144 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . ''' -Re-Impl of multiprocessing.reduction.sendfds & recvfds, -using acms and trio +Reimplementation of multiprocessing.reduction.sendfds & recvfds, using acms and trio. + +cpython impl: +https://github.com/python/cpython/blob/275056a7fdcbe36aaac494b4183ae59943a338eb/Lib/multiprocessing/reduction.py#L138 ''' import array +from typing import AsyncContextManager from contextlib import asynccontextmanager as acm import trio from trio import socket +class FDSharingError(Exception): + ... + + @acm -async def send_fds(fds: list[int], sock_path: str): +async def send_fds(fds: list[int], sock_path: str) -> AsyncContextManager[None]: + ''' + Async trio reimplementation of `multiprocessing.reduction.sendfds` + + https://github.com/python/cpython/blob/275056a7fdcbe36aaac494b4183ae59943a338eb/Lib/multiprocessing/reduction.py#L142 + + It's implemented using an async context manager in order to simplyfy usage + with `tractor.context`s, we can open a context in a remote actor that uses + this acm inside of it, and uses `ctx.started()` to signal the original + caller actor to perform the `recv_fds` call. + + See `tractor.ipc._ringbuf._ringd._attach_to_ring` for an example. + ''' sock = socket.socket(socket.AF_UNIX, socket.SOCK_STREAM) await sock.bind(sock_path) sock.listen(1) - yield - fds = array.array('i', fds) - # first byte of msg will be len of fds to send % 256 - msg = bytes([len(fds) % 256]) + + yield # socket is setup, ready for receiver connect + + # wait until receiver connects conn, _ = await sock.accept() + + # setup int array for fds + fds = array.array('i', fds) + + # first byte of msg will be len of fds to send % 256, acting as a fd amount + # verification on `recv_fds` we refer to it as `check_byte` + msg = bytes([len(fds) % 256]) + + # send msg with custom SCM_RIGHTS type await conn.sendmsg( [msg], [(socket.SOL_SOCKET, socket.SCM_RIGHTS, fds)] ) - # wait ack + + # finally wait receiver ack if await conn.recv(1) != b'A': - raise RuntimeError('did not receive acknowledgement of fd') + raise FDSharingError('did not receive acknowledgement of fd') conn.close() sock.close() async def recv_fds(sock_path: str, amount: int) -> tuple: + ''' + Async trio reimplementation of `multiprocessing.reduction.recvfds` + + https://github.com/python/cpython/blob/275056a7fdcbe36aaac494b4183ae59943a338eb/Lib/multiprocessing/reduction.py#L150 + + It's equivalent to std just using `trio.open_unix_socket` for connecting and + changes on error handling. + + See `tractor.ipc._ringbuf._ringd._attach_to_ring` for an example. + ''' stream = await trio.open_unix_socket(sock_path) sock = stream.socket + + # prepare int array for fds a = array.array('i') bytes_size = a.itemsize * amount + + # receive 1 byte + space necesary for SCM_RIGHTS msg for {amount} fds msg, ancdata, flags, addr = await sock.recvmsg( 1, socket.CMSG_SPACE(bytes_size) ) + + # maybe failed to receive msg? if not msg and not ancdata: - raise EOFError - try: - await sock.send(b'A') # Ack + raise FDSharingError(f'Expected to receive {amount} fds from {sock_path}, but got EOF') - if len(ancdata) != 1: - raise RuntimeError( - f'received {len(ancdata)} items of ancdata' - ) + # send ack, std comment mentions this ack pattern was to get around an + # old macosx bug, but they are not sure if its necesary any more, in + # any case its not a bad pattern to keep + await sock.send(b'A') # Ack - cmsg_level, cmsg_type, cmsg_data = ancdata[0] - # check proper msg type - if ( - cmsg_level == socket.SOL_SOCKET - and - cmsg_type == socket.SCM_RIGHTS - ): - # check proper data alignment - if len(cmsg_data) % a.itemsize != 0: - raise ValueError + # expect to receive only one `ancdata` item + if len(ancdata) != 1: + raise FDSharingError( + f'Expected to receive exactly one \"ancdata\" but got {len(ancdata)}: {ancdata}' + ) - # attempt to cast as int array - a.frombytes(cmsg_data) + # unpack SCM_RIGHTS msg + cmsg_level, cmsg_type, cmsg_data = ancdata[0] - # check first byte of message is amount % 256 - if len(a) % 256 != msg[0]: - raise AssertionError( - 'Len is {0:n} but msg[0] is {1!r}'.format( - len(a), msg[0] - ) - ) + # check proper msg type + if cmsg_level != socket.SOL_SOCKET: + raise FDSharingError( + f'Expected CMSG level to be SOL_SOCKET({socket.SOL_SOCKET}) but got {cmsg_level}' + ) - return tuple(a) + if cmsg_type != socket.SCM_RIGHTS: + raise FDSharingError( + f'Expected CMSG type to be SCM_RIGHTS({socket.SCM_RIGHTS}) but got {cmsg_type}' + ) - except (ValueError, IndexError): - pass + # check proper data alignment + length = len(cmsg_data) + if length % a.itemsize != 0: + raise FDSharingError( + f'CMSG data alignment error: len of {length} is not divisible by int size {a.itemsize}' + ) - raise RuntimeError('Invalid data received') + # attempt to cast as int array + a.frombytes(cmsg_data) + + # validate length check byte + valid_check_byte = amount % 256 # check byte acording to `recv_fds` caller + recvd_check_byte = msg[0] # actual received check byte + payload_check_byte = len(a) % 256 # check byte acording to received fd int array + + if recvd_check_byte != payload_check_byte: + raise FDSharingError( + 'Validation failed: received check byte ' + f'({recvd_check_byte}) does not match fd int array len % 256 ({payload_check_byte})' + ) + + if valid_check_byte != recvd_check_byte: + raise FDSharingError( + 'Validation failed: received check byte ' + f'({recvd_check_byte}) does not match expected fd amount % 256 ({valid_check_byte})' + ) + + return tuple(a) -- 2.34.1 From b1e1187a19256f22d06c63c45e2b82b1af093884 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 3 Apr 2025 12:34:40 -0300 Subject: [PATCH 14/41] Switch to using typing.Protocl instead of abc.ABC on ChannelManager, improve abstraction and add comments --- tractor/ipc/_ringbuf/_pubsub.py | 164 +++++++++++++++++++++++--------- 1 file changed, 120 insertions(+), 44 deletions(-) diff --git a/tractor/ipc/_ringbuf/_pubsub.py b/tractor/ipc/_ringbuf/_pubsub.py index 94ce7460..50c48366 100644 --- a/tractor/ipc/_ringbuf/_pubsub.py +++ b/tractor/ipc/_ringbuf/_pubsub.py @@ -18,9 +18,12 @@ Ring buffer ipc publish-subscribe mechanism brokered by ringd can dynamically add new outputs (publisher) or inputs (subscriber) ''' import time -from abc import ( - ABC, - abstractmethod +from typing import ( + runtime_checkable, + Protocol, + TypeVar, + Self, + AsyncContextManager ) from contextlib import asynccontextmanager as acm from dataclasses import dataclass @@ -30,6 +33,7 @@ import tractor from tractor.ipc import ( RingBuffBytesSender, + RingBuffBytesReceiver, attach_to_ringbuf_schannel, attach_to_ringbuf_rchannel ) @@ -40,28 +44,72 @@ import tractor.ipc._ringbuf._ringd as ringd log = tractor.log.get_logger(__name__) +ChannelType = TypeVar('ChannelType') + + @dataclass class ChannelInfo: connect_time: float name: str - channel: RingBuffBytesSender + channel: ChannelType cancel_scope: trio.CancelScope -class ChannelManager(ABC): +# TODO: maybe move this abstraction to another module or standalone? +# its not ring buf specific and allows fan out and fan in an a dynamic +# amount of channels +@runtime_checkable +class ChannelManager(Protocol[ChannelType]): + ''' + Common data structures and methods pubsub classes use to manage channels & + their related handler background tasks, as well as cancellation of them. + + ''' def __init__( self, n: trio.Nursery, ): self._n = n - self._channels: list[ChannelInfo] = [] + self._channels: list[Self.ChannelInfo] = [] - @abstractmethod - async def _channel_handler_task(self, name: str): + async def _open_channel( + self, + name: str + ) -> AsyncContextManager[ChannelType]: + ''' + Used to instantiate channel resources given a name + + ''' ... + async def _channel_task(self, info: ChannelInfo) -> None: + ''' + Long running task that manages the channel + + ''' + ... + + async def _channel_handler_task(self, name: str): + async with self._open_channel(name) as chan: + with trio.CancelScope() as cancel_scope: + info = Self.ChannelInfo( + connect_time=time.time(), + name=name, + channel=chan, + cancel_scope=cancel_scope + ) + self._channels.append(info) + await self._channel_task(info) + + self._maybe_destroy_channel(name) + def find_channel(self, name: str) -> tuple[int, ChannelInfo] | None: + ''' + Given a channel name maybe return its index and value from + internal _channels list. + + ''' for entry in enumerate(self._channels): i, info = entry if info.name == name: @@ -70,6 +118,11 @@ class ChannelManager(ABC): return None def _maybe_destroy_channel(self, name: str): + ''' + If channel exists cancel its scope and remove from internal + _channels list. + + ''' maybe_entry = self.find_channel(name) if maybe_entry: i, info = maybe_entry @@ -77,12 +130,20 @@ class ChannelManager(ABC): del self._channels[i] def add_channel(self, name: str): + ''' + Add a new channel to be handled + + ''' self._n.start_soon( self._channel_handler_task, name ) def remove_channel(self, name: str): + ''' + Remove a channel and stop its handling + + ''' self._maybe_destroy_channel(name) def __len__(self) -> int: @@ -92,8 +153,24 @@ class ChannelManager(ABC): for chan in self._channels: self._maybe_destroy_channel(chan.name) + async def __aenter__(self): + return self -class RingBuffPublisher(ChannelManager, trio.abc.SendChannel[bytes]): + async def __aexit__(self, exc_type, exc_val, exc_tb): + await self.aclose() + + +class RingBuffPublisher( + ChannelManager[RingBuffBytesSender] +): + ''' + Implement ChannelManager protocol + trio.abc.SendChannel[bytes] + using ring buffers as transport. + + - use a `trio.Event` to make sure `send` blocks until at least one channel + available. + + ''' def __init__( self, @@ -107,28 +184,23 @@ class RingBuffPublisher(ChannelManager, trio.abc.SendChannel[bytes]): self._batch_size: int = batch_size - async def _channel_handler_task( + @acm + async def _open_channel( self, name: str - ): + ) -> AsyncContextManager[RingBuffBytesSender]: async with ( ringd.open_ringbuf( name=name, must_exist=True, ) as token, - attach_to_ringbuf_schannel(token) as schan + attach_to_ringbuf_schannel(token) as chan ): - with trio.CancelScope() as cancel_scope: - self._channels.append(ChannelInfo( - connect_time=time.time(), - name=name, - channel=schan, - cancel_scope=cancel_scope - )) - self._connect_event.set() - await trio.sleep_forever() + yield chan - self._maybe_destroy_channel(name) + async def _channel_task(self, info: Self.ChannelInfo) -> None: + self._connect_event.set() + await trio.sleep_forever() async def send(self, msg: bytes): # wait at least one decoder connected @@ -182,11 +254,21 @@ async def open_ringbuf_publisher( ) as outputs ): yield outputs - await outputs.aclose() -class RingBuffSubscriber(ChannelManager, trio.abc.ReceiveChannel[bytes]): +class RingBuffSubscriber( + ChannelManager[RingBuffBytesReceiver] +): + ''' + Implement ChannelManager protocol + trio.abc.ReceiveChannel[bytes] + using ring buffers as transport. + + - use a trio memory channel pair to multiplex all received messages into a + single `trio.MemoryReceiveChannel`, give a sender channel clone to each + _channel_task. + + ''' def __init__( self, n: trio.Nursery, @@ -194,34 +276,29 @@ class RingBuffSubscriber(ChannelManager, trio.abc.ReceiveChannel[bytes]): super().__init__(n) self._send_chan, self._recv_chan = trio.open_memory_channel(0) - async def _channel_handler_task( + @acm + async def _open_channel( self, name: str - ): + ) -> AsyncContextManager[RingBuffBytesReceiver]: async with ( ringd.open_ringbuf( name=name, - must_exist=True + must_exist=True, ) as token, - - attach_to_ringbuf_rchannel(token) as rchan + attach_to_ringbuf_rchannel(token) as chan ): - with trio.CancelScope() as cancel_scope: - self._channels.append(ChannelInfo( - connect_time=time.time(), - name=name, - channel=rchan, - cancel_scope=cancel_scope - )) - send_chan = self._send_chan.clone() - try: - async for msg in rchan: - await send_chan.send(msg) + yield chan - except tractor._exceptions.InternalError: - ... + async def _channel_task(self, info: ChannelInfo) -> None: + send_chan = self._send_chan.clone() + try: + async for msg in info.channel: + await send_chan.send(msg) - self._maybe_destroy_channel(name) + except tractor._exceptions.InternalError: + # TODO: cleaner cancellation! + ... async def receive(self) -> bytes: return await self._recv_chan.receive() @@ -234,5 +311,4 @@ async def open_ringbuf_subscriber(): RingBuffSubscriber(n) as inputs ): yield inputs - await inputs.aclose() -- 2.34.1 From 4385d38bc445199e389911bee989bc6f79b37364 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 3 Apr 2025 12:35:09 -0300 Subject: [PATCH 15/41] Add header and fix white lines --- tractor/ipc/__init__.py | 1 - tractor/linux/__init__.py | 15 +++++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index c2ddfe48..0a57e969 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -13,7 +13,6 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . - ''' A modular IPC layer supporting the power of cross-process SC! diff --git a/tractor/linux/__init__.py b/tractor/linux/__init__.py index dce926c8..211a0040 100644 --- a/tractor/linux/__init__.py +++ b/tractor/linux/__init__.py @@ -1,3 +1,18 @@ +# 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 ._fdshare import ( send_fds as send_fds, recv_fds as recv_fds -- 2.34.1 From 95ea4647cc786860aa6c075532c72662c060cedf Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 3 Apr 2025 13:11:00 -0300 Subject: [PATCH 16/41] Woops fix old typing Self stuff --- tractor/ipc/_ringbuf/_pubsub.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tractor/ipc/_ringbuf/_pubsub.py b/tractor/ipc/_ringbuf/_pubsub.py index 50c48366..fe8b5b5b 100644 --- a/tractor/ipc/_ringbuf/_pubsub.py +++ b/tractor/ipc/_ringbuf/_pubsub.py @@ -22,7 +22,6 @@ from typing import ( runtime_checkable, Protocol, TypeVar, - Self, AsyncContextManager ) from contextlib import asynccontextmanager as acm @@ -71,7 +70,7 @@ class ChannelManager(Protocol[ChannelType]): n: trio.Nursery, ): self._n = n - self._channels: list[Self.ChannelInfo] = [] + self._channels: list[ChannelInfo] = [] async def _open_channel( self, @@ -93,7 +92,7 @@ class ChannelManager(Protocol[ChannelType]): async def _channel_handler_task(self, name: str): async with self._open_channel(name) as chan: with trio.CancelScope() as cancel_scope: - info = Self.ChannelInfo( + info = ChannelInfo( connect_time=time.time(), name=name, channel=chan, @@ -198,7 +197,7 @@ class RingBuffPublisher( ): yield chan - async def _channel_task(self, info: Self.ChannelInfo) -> None: + async def _channel_task(self, info: ChannelInfo) -> None: self._connect_event.set() await trio.sleep_forever() -- 2.34.1 From 3568ba5d5d76cff72fa587103f111157c3042d61 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Fri, 4 Apr 2025 02:36:59 -0300 Subject: [PATCH 17/41] Rename RingBuff -> RingBuffer Combine RingBuffer stream and channel apis Implement RingBufferReceiveChannel.receive_nowait Make msg generator calculate hash --- tests/test_ringbuf.py | 94 +------ tractor/_testing/samples.py | 7 +- tractor/ipc/__init__.py | 17 +- tractor/ipc/_ringbuf/__init__.py | 445 ++++++++++++++----------------- 4 files changed, 221 insertions(+), 342 deletions(-) diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index fc8fd3bd..fc177976 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -9,7 +9,6 @@ from tractor.ipc._ringbuf import ( open_ringbuf, attach_to_ringbuf_receiver, attach_to_ringbuf_sender, - attach_to_ringbuf_stream, attach_to_ringbuf_channel, RBToken, ) @@ -25,7 +24,6 @@ pytestmark = pytest.mark.skip @tractor.context async def child_read_shm( ctx: tractor.Context, - msg_amount: int, token: RBToken, ) -> str: ''' @@ -41,11 +39,13 @@ async def child_read_shm( ''' await ctx.started() print('reader started') + msg_amount = 0 recvd_bytes = 0 recvd_hash = hashlib.sha256() start_ts = time.time() async with attach_to_ringbuf_receiver(token) as receiver: async for msg in receiver: + msg_amount += 1 recvd_hash.update(msg) recvd_bytes += len(msg) @@ -79,19 +79,16 @@ async def child_write_shm( Attach to ringbuf and send all generated messages. ''' - msgs, _total_bytes = generate_sample_messages( + sent_hash, msgs, _total_bytes = generate_sample_messages( msg_amount, rand_min=rand_min, rand_max=rand_max, ) - print('writer hashing payload...') - sent_hash = hashlib.sha256(b''.join(msgs)).hexdigest() - print('writer done hashing.') await ctx.started(sent_hash) print('writer started') async with attach_to_ringbuf_sender(token, cleanup=False) as sender: for msg in msgs: - await sender.send_all(msg) + await sender.send(msg) print('writer exit') @@ -159,7 +156,6 @@ def test_ringbuf( recv_p.open_context( child_read_shm, token=token, - msg_amount=msg_amount ) as (rctx, _sent), ): recvd_hash = await rctx.result() @@ -295,75 +291,6 @@ def test_receiver_max_bytes(): assert msg == b''.join(msgs) -def test_stapled_ringbuf(): - ''' - Open two ringbufs and give tokens to tasks (swap them such that in/out tokens - are inversed on each task) which will open the streams and use trio.StapledStream - to have a single bidirectional stream. - - Then take turns to send and receive messages. - - ''' - msg = generate_single_byte_msgs(100) - pair_0_msgs = [] - pair_1_msgs = [] - - pair_0_done = trio.Event() - pair_1_done = trio.Event() - - async def pair_0(token_in: RBToken, token_out: RBToken): - async with attach_to_ringbuf_stream( - token_in, - token_out, - cleanup_in=False, - cleanup_out=False - ) as stream: - # first turn to send - await stream.send_all(msg) - - # second turn to receive - while len(pair_0_msgs) != len(msg): - _msg = await stream.receive_some(max_bytes=1) - pair_0_msgs.append(_msg) - - pair_0_done.set() - await pair_1_done.wait() - - - async def pair_1(token_in: RBToken, token_out: RBToken): - async with attach_to_ringbuf_stream( - token_in, - token_out, - cleanup_in=False, - cleanup_out=False - ) as stream: - # first turn to receive - while len(pair_1_msgs) != len(msg): - _msg = await stream.receive_some(max_bytes=1) - pair_1_msgs.append(_msg) - - # second turn to send - await stream.send_all(msg) - - pair_1_done.set() - await pair_0_done.wait() - - - async def main(): - with tractor.ipc.open_ringbuf_pair( - 'test_stapled_ringbuf' - ) as (token_0, token_1): - async with trio.open_nursery() as n: - n.start_soon(pair_0, token_0, token_1) - n.start_soon(pair_1, token_1, token_0) - - - trio.run(main) - - assert msg == b''.join(pair_0_msgs) - assert msg == b''.join(pair_1_msgs) - - @tractor.context async def child_channel_sender( ctx: tractor.Context, @@ -373,7 +300,7 @@ async def child_channel_sender( token_out: RBToken ): import random - msgs, _total_bytes = generate_sample_messages( + _hash, msgs, _total_bytes = generate_sample_messages( random.randint(msg_amount_min, msg_amount_max), rand_min=256, rand_max=1024, @@ -383,7 +310,6 @@ async def child_channel_sender( token_out ) as chan: await ctx.started(msgs) - for msg in msgs: await chan.send(msg) @@ -396,16 +322,16 @@ def test_channel(): async def main(): with tractor.ipc.open_ringbuf_pair( 'test_ringbuf_transport' - ) as (token_0, token_1): + ) as (send_token, recv_token): async with ( - attach_to_ringbuf_channel(token_0, token_1) as chan, + attach_to_ringbuf_channel(send_token, recv_token) as chan, tractor.open_nursery() as an ): recv_p = await an.start_actor( 'test_ringbuf_transport_sender', enable_modules=[__name__], proc_kwargs={ - 'pass_fds': token_0.fds + token_1.fds + 'pass_fds': send_token.fds + recv_token.fds } ) async with ( @@ -413,8 +339,8 @@ def test_channel(): child_channel_sender, msg_amount_min=msg_amount_min, msg_amount_max=msg_amount_max, - token_in=token_1, - token_out=token_0 + token_in=recv_token, + token_out=send_token ) as (ctx, msgs), ): recv_msgs = [] diff --git a/tractor/_testing/samples.py b/tractor/_testing/samples.py index 4249bae9..f8671332 100644 --- a/tractor/_testing/samples.py +++ b/tractor/_testing/samples.py @@ -1,5 +1,6 @@ import os import random +import hashlib def generate_single_byte_msgs(amount: int) -> bytes: @@ -23,7 +24,7 @@ def generate_sample_messages( rand_min: int = 0, rand_max: int = 0, silent: bool = False, -) -> tuple[list[bytes], int]: +) -> tuple[str, list[bytes], int]: ''' Generate bytes msgs for tests. @@ -55,6 +56,7 @@ def generate_sample_messages( else: log_interval = 1000 + payload_hash = hashlib.sha256() for i in range(amount): msg = f'[{i:08}]'.encode('utf-8') @@ -64,6 +66,7 @@ def generate_sample_messages( size += len(msg) + payload_hash.update(msg) msgs.append(msg) if ( @@ -78,4 +81,4 @@ def generate_sample_messages( if not silent: print(f'done, {size:,} bytes in total') - return msgs, size + return payload_hash.hexdigest(), msgs, size diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index 0a57e969..f2f42839 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -27,17 +27,16 @@ from ._chan import ( if platform.system() == 'Linux': from ._ringbuf import ( RBToken as RBToken, + open_ringbuf as open_ringbuf, - RingBuffSender as RingBuffSender, - RingBuffReceiver as RingBuffReceiver, open_ringbuf_pair as open_ringbuf_pair, - attach_to_ringbuf_receiver as attach_to_ringbuf_receiver, + + RingBufferSendChannel as RingBufferSendChannel, attach_to_ringbuf_sender as attach_to_ringbuf_sender, - attach_to_ringbuf_stream as attach_to_ringbuf_stream, - RingBuffBytesSender as RingBuffBytesSender, - RingBuffBytesReceiver as RingBuffBytesReceiver, - RingBuffChannel as RingBuffChannel, - attach_to_ringbuf_schannel as attach_to_ringbuf_schannel, - attach_to_ringbuf_rchannel as attach_to_ringbuf_rchannel, + + RingBufferReceiveChannel as RingBufferReceiveChannel, + attach_to_ringbuf_receiver as attach_to_ringbuf_receiver, + + RingBufferChannel as RingBufferChannel, attach_to_ringbuf_channel as attach_to_ringbuf_channel, ) diff --git a/tractor/ipc/_ringbuf/__init__.py b/tractor/ipc/_ringbuf/__init__.py index f9b770a1..58d90dab 100644 --- a/tractor/ipc/_ringbuf/__init__.py +++ b/tractor/ipc/_ringbuf/__init__.py @@ -126,6 +126,30 @@ def open_ringbuf( shm.unlink() +@cm +def open_ringbuf_pair( + name: str, + buf_size: int = _DEFAULT_RB_SIZE +) -> ContextManager[tuple(RBToken, RBToken)]: + ''' + Handle resources for a ringbuf pair to be used for + bidirectional messaging. + + ''' + with ( + open_ringbuf( + name + '.send', + buf_size=buf_size + ) as send_token, + + open_ringbuf( + name + '.recv', + buf_size=buf_size + ) as recv_token + ): + yield send_token, recv_token + + Buffer = bytes | bytearray | memoryview @@ -135,32 +159,65 @@ IPC Reliable Ring Buffer `eventfd(2)` is used for wrap around sync, to signal writes to the reader and end of stream. +In order to guarantee full messages are received, all bytes +sent by `RingBufferSendChannel` are preceded with a 4 byte header +which decodes into a uint32 indicating the actual size of the +next full payload. + ''' -class RingBuffSender(trio.abc.SendStream): +class RingBufferSendChannel(trio.abc.SendChannel[bytes]): ''' Ring Buffer sender side implementation Do not use directly! manage with `attach_to_ringbuf_sender` after having opened a ringbuf context with `open_ringbuf`. + Optional batch mode: + + If `batch_size` > 1 messages wont get sent immediately but will be + stored until `batch_size` messages are pending, then it will send + them all at once. + + `batch_size` can be changed dynamically but always call, `flush()` + right before. + ''' def __init__( self, token: RBToken, + batch_size: int = 1, cleanup: bool = False ): self._token = RBToken.from_msg(token) + self.batch_size = batch_size + + # ringbuf os resources self._shm: SharedMemory | None = None self._write_event = EventFD(self._token.write_eventfd, 'w') self._wrap_event = EventFD(self._token.wrap_eventfd, 'r') self._eof_event = EventFD(self._token.eof_eventfd, 'w') + + # current write pointer self._ptr = 0 + # when `batch_size` > 1 store messages on `self._batch` and write them + # all, once `len(self._batch) == `batch_size` + self._batch: list[bytes] = [] + self._cleanup = cleanup self._send_lock = trio.StrictFIFOLock() + @acm + async def _maybe_lock(self) -> AsyncContextManager[None]: + if self._send_lock.locked(): + yield + return + + async with self._send_lock: + yield + @property def name(self) -> str: if not self._shm: @@ -183,11 +240,19 @@ class RingBuffSender(trio.abc.SendStream): def wrap_fd(self) -> int: return self._wrap_event.fd + @property + def pending_msgs(self) -> int: + return len(self._batch) + + @property + def must_flush(self) -> bool: + return self.pending_msgs >= self.batch_size + async def _wait_wrap(self): await self._wrap_event.read() async def send_all(self, data: Buffer): - async with self._send_lock: + async with self._maybe_lock(): # while data is larger than the remaining buf target_ptr = self.ptr + len(data) while target_ptr > self.size: @@ -211,6 +276,34 @@ class RingBuffSender(trio.abc.SendStream): async def wait_send_all_might_not_block(self): raise NotImplementedError + async def flush( + self, + new_batch_size: int | None = None + ) -> None: + async with self._maybe_lock(): + for msg in self._batch: + await self.send_all(msg) + + self._batch = [] + if new_batch_size: + self.batch_size = new_batch_size + + async def send(self, value: bytes) -> None: + async with self._maybe_lock(): + msg: bytes = struct.pack(" None: + async with self._send_lock: + await self.flush(new_batch_size=1) + await self.send(b'') + def open(self): try: self._shm = SharedMemory( @@ -238,15 +331,14 @@ class RingBuffSender(trio.abc.SendStream): self._shm.close() async def aclose(self): - async with self._send_lock: - self.close() + self.close() async def __aenter__(self): self.open() return self -class RingBuffReceiver(trio.abc.ReceiveStream): +class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): ''' Ring Buffer receiver side implementation @@ -312,21 +404,48 @@ class RingBuffReceiver(trio.abc.ReceiveStream): except trio.Cancelled: ... - async def receive_some(self, max_bytes: int | None = None) -> bytes: + def receive_nowait(self, max_bytes: int = _DEFAULT_RB_SIZE) -> bytes: + ''' + Try to receive any bytes we can without blocking or raise + `trio.WouldBlock`. + + ''' + if max_bytes < 1: + raise ValueError("max_bytes must be >= 1") + + delta = self._write_ptr - self._ptr + if delta == 0: + raise trio.WouldBlock + + # dont overflow caller + delta = min(delta, max_bytes) + + target_ptr = self._ptr + delta + + # fetch next segment and advance ptr + segment = bytes(self._shm.buf[self._ptr:target_ptr]) + self._ptr = target_ptr + + if self._ptr == self.size: + # reached the end, signal wrap around + self._ptr = 0 + self._write_ptr = 0 + self._wrap_event.write(1) + + return segment + + async def receive_some(self, max_bytes: int = _DEFAULT_RB_SIZE) -> bytes: ''' Receive up to `max_bytes`, if no `max_bytes` is provided a reasonable default is used. + Can return < max_bytes. + ''' - if max_bytes is None: - max_bytes: int = _DEFAULT_RB_SIZE + try: + return self.receive_nowait(max_bytes=max_bytes) - if max_bytes < 1: - raise ValueError("max_bytes must be >= 1") - - # delta is remaining bytes we havent read - delta = self._write_ptr - self._ptr - if delta == 0: + except trio.WouldBlock: # we have read all we can, see if new data is available if self._end_ptr < 0: # if we havent been signaled about EOF yet @@ -353,22 +472,39 @@ class RingBuffReceiver(trio.abc.ReceiveStream): # no more bytes to read and self._end_ptr set, EOF reached return b'' - # dont overflow caller - delta = min(delta, max_bytes) + return await self.receive_some(max_bytes=max_bytes) - target_ptr = self._ptr + delta + async def receive_exactly(self, num_bytes: int) -> bytes: + ''' + Fetch bytes until we read exactly `num_bytes` or EOF. - # fetch next segment and advance ptr - segment = bytes(self._shm.buf[self._ptr:target_ptr]) - self._ptr = target_ptr + ''' + payload = b'' + while len(payload) < num_bytes: + remaining = num_bytes - len(payload) - if self._ptr == self.size: - # reached the end, signal wrap around - self._ptr = 0 - self._write_ptr = 0 - self._wrap_event.write(1) + new_bytes = await self.receive_some( + max_bytes=remaining + ) - return segment + if new_bytes == b'': + raise trio.EndOfChannel + + payload += new_bytes + + return payload + + async def receive(self) -> bytes: + ''' + Receive a complete payload + + ''' + header: bytes = await self.receive_exactly(4) + size: int + size, = struct.unpack(" AsyncContextManager[RingBuffReceiver]: + +) -> AsyncContextManager[RingBufferReceiveChannel]: ''' - Attach a RingBuffReceiver from a previously opened + Attach a RingBufferReceiveChannel from a previously opened RBToken. Launches `receiver._eof_monitor_task` in a `trio.Nursery`. ''' async with ( trio.open_nursery() as n, - RingBuffReceiver( + RingBufferReceiveChannel( token, cleanup=cleanup ) as receiver @@ -424,232 +562,33 @@ async def attach_to_ringbuf_receiver( @acm async def attach_to_ringbuf_sender( + token: RBToken, cleanup: bool = True -) -> AsyncContextManager[RingBuffSender]: + +) -> AsyncContextManager[RingBufferSendChannel]: ''' - Attach a RingBuffSender from a previously opened + Attach a RingBufferSendChannel from a previously opened RBToken. ''' - async with RingBuffSender( + async with RingBufferSendChannel( token, cleanup=cleanup ) as sender: yield sender -@cm -def open_ringbuf_pair( - name: str, - buf_size: int = _DEFAULT_RB_SIZE -) -> ContextManager[tuple(RBToken, RBToken)]: +class RingBufferChannel(trio.abc.Channel[bytes]): ''' - Handle resources for a ringbuf pair to be used for - bidirectional messaging. - - ''' - with ( - open_ringbuf( - name + '.pair0', - buf_size=buf_size - ) as token_0, - - open_ringbuf( - name + '.pair1', - buf_size=buf_size - ) as token_1 - ): - yield token_0, token_1 - - -@acm -async def attach_to_ringbuf_stream( - token_in: RBToken, - token_out: RBToken, - cleanup_in: bool = True, - cleanup_out: bool = True -) -> AsyncContextManager[trio.StapledStream]: - ''' - Attach a trio.StapledStream from a previously opened - ringbuf pair. - - ''' - async with ( - attach_to_ringbuf_receiver( - token_in, - cleanup=cleanup_in - ) as receiver, - attach_to_ringbuf_sender( - token_out, - cleanup=cleanup_out - ) as sender, - ): - yield trio.StapledStream(sender, receiver) - - - -class RingBuffBytesSender(trio.abc.SendChannel[bytes]): - ''' - In order to guarantee full messages are received, all bytes - sent by `RingBuffBytesSender` are preceded with a 4 byte header - which decodes into a uint32 indicating the actual size of the - next payload. - - Optional batch mode: - - If `batch_size` > 1 messages wont get sent immediately but will be - stored until `batch_size` messages are pending, then it will send - them all at once. - - `batch_size` can be changed dynamically but always call, `flush()` - right before. - - ''' - def __init__( - self, - sender: RingBuffSender, - batch_size: int = 1 - ): - self._sender = sender - self.batch_size = batch_size - self._batch_msg_len = 0 - self._batch: bytes = b'' - self._send_lock = trio.StrictFIFOLock() - - @property - def pending_msgs(self) -> int: - return self._batch_msg_len - - @property - def must_flush(self) -> bool: - return self._batch_msg_len >= self.batch_size - - async def _flush( - self, - new_batch_size: int | None = None - ) -> None: - await self._sender.send_all(self._batch) - self._batch = b'' - self._batch_msg_len = 0 - if new_batch_size: - self.batch_size = new_batch_size - - async def flush( - self, - new_batch_size: int | None = None - ) -> None: - async with self._send_lock: - await self._flush(new_batch_size=new_batch_size) - - async def send(self, value: bytes) -> None: - async with self._send_lock: - msg: bytes = struct.pack(" None: - await self.flush(new_batch_size=1) - await self.send(b'') - - async def aclose(self) -> None: - async with self._send_lock: - await self._sender.aclose() - - -class RingBuffBytesReceiver(trio.abc.ReceiveChannel[bytes]): - ''' - See `RingBuffBytesSender` docstring. - - A `tricycle.BufferedReceiveStream` is used for the - `receive_exactly` API. - ''' - def __init__( - self, - receiver: RingBuffReceiver - ): - self._receiver = receiver - - async def _receive_exactly(self, num_bytes: int) -> bytes: - ''' - Fetch bytes from receiver until we read exactly `num_bytes` - or end of stream is signaled. - - ''' - payload = b'' - while len(payload) < num_bytes: - remaining = num_bytes - len(payload) - - new_bytes = await self._receiver.receive_some( - max_bytes=remaining - ) - - if new_bytes == b'': - raise trio.EndOfChannel - - payload += new_bytes - - return payload - - async def receive(self) -> bytes: - header: bytes = await self._receive_exactly(4) - size: int - size, = struct.unpack(" None: - await self._receiver.aclose() - - -@acm -async def attach_to_ringbuf_rchannel( - token: RBToken, - cleanup: bool = True -) -> AsyncContextManager[RingBuffBytesReceiver]: - ''' - Attach a RingBuffBytesReceiver from a previously opened - RBToken. - ''' - async with attach_to_ringbuf_receiver( - token, cleanup=cleanup - ) as receiver: - yield RingBuffBytesReceiver(receiver) - - -@acm -async def attach_to_ringbuf_schannel( - token: RBToken, - cleanup: bool = True, - batch_size: int = 1, -) -> AsyncContextManager[RingBuffBytesSender]: - ''' - Attach a RingBuffBytesSender from a previously opened - RBToken. - ''' - async with attach_to_ringbuf_sender( - token, cleanup=cleanup - ) as sender: - yield RingBuffBytesSender(sender, batch_size=batch_size) - - -class RingBuffChannel(trio.abc.Channel[bytes]): - ''' - Combine `RingBuffBytesSender` and `RingBuffBytesReceiver` + Combine `RingBufferSendChannel` and `RingBufferReceiveChannel` in order to expose the bidirectional `trio.abc.Channel` API. ''' def __init__( self, - sender: RingBuffBytesSender, - receiver: RingBuffBytesReceiver + sender: RingBufferSendChannel, + receiver: RingBufferReceiveChannel ): self._sender = sender self._receiver = receiver @@ -666,6 +605,12 @@ class RingBuffChannel(trio.abc.Channel[bytes]): def pending_msgs(self) -> int: return self._sender.pending_msgs + async def send_all(self, value: bytes) -> None: + await self._sender.send_all(value) + + async def wait_send_all_might_not_block(self): + await self._sender.wait_send_all_might_not_block() + async def flush( self, new_batch_size: int | None = None @@ -678,6 +623,15 @@ class RingBuffChannel(trio.abc.Channel[bytes]): async def send_eof(self) -> None: await self._sender.send_eof() + def receive_nowait(self, max_bytes: int = _DEFAULT_RB_SIZE) -> bytes: + return self._receiver.receive_nowait(max_bytes=max_bytes) + + async def receive_some(self, max_bytes: int = _DEFAULT_RB_SIZE) -> bytes: + return await self._receiver.receive_some(max_bytes=max_bytes) + + async def receive_exactly(self, num_bytes: int) -> bytes: + return await self._receiver.receive_exactly(num_bytes) + async def receive(self) -> bytes: return await self._receiver.receive() @@ -691,23 +645,20 @@ async def attach_to_ringbuf_channel( token_in: RBToken, token_out: RBToken, cleanup_in: bool = True, - cleanup_out: bool = True, - batch_size: int = 1 -) -> AsyncContextManager[RingBuffChannel]: + cleanup_out: bool = True +) -> AsyncContextManager[trio.StapledStream]: ''' - Attach to an already opened ringbuf pair and return - a `RingBuffChannel`. + Attach to two previously opened `RBToken`s and return a `RingBufferChannel` ''' async with ( - attach_to_ringbuf_rchannel( + attach_to_ringbuf_receiver( token_in, cleanup=cleanup_in ) as receiver, - attach_to_ringbuf_schannel( + attach_to_ringbuf_sender( token_out, - cleanup=cleanup_out, - batch_size=batch_size + cleanup=cleanup_out ) as sender, ): - yield RingBuffChannel(sender, receiver) + yield RingBufferChannel(sender, receiver) -- 2.34.1 From bebd32702359b7e56896d5680868c0d7c67a6062 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Fri, 4 Apr 2025 02:41:50 -0300 Subject: [PATCH 18/41] Improve ringd ringbuf lifecycle Unlink sock after use in fdshare --- tests/test_ringd.py | 209 ++++++++++++++++++++++++++++++++- tractor/ipc/_ringbuf/_ringd.py | 126 ++++++++++++++------ tractor/linux/_fdshare.py | 2 + 3 files changed, 293 insertions(+), 44 deletions(-) diff --git a/tests/test_ringd.py b/tests/test_ringd.py index 6ce71333..40040a43 100644 --- a/tests/test_ringd.py +++ b/tests/test_ringd.py @@ -1,9 +1,14 @@ import trio import tractor +import msgspec from tractor.ipc import ( - attach_to_ringbuf_rchannel, - attach_to_ringbuf_schannel + attach_to_ringbuf_receiver, + attach_to_ringbuf_sender +) +from tractor.ipc._ringbuf._pubsub import ( + open_ringbuf_publisher, + open_ringbuf_subscriber ) import tractor.ipc._ringbuf._ringd as ringd @@ -20,7 +25,7 @@ async def recv_child( async with ( ringd.open_ringbuf(ring_name) as token, - attach_to_ringbuf_rchannel(token) as chan, + attach_to_ringbuf_receiver(token) as chan, ): await ctx.started() async for msg in chan: @@ -35,7 +40,7 @@ async def send_child( async with ( ringd.open_ringbuf(ring_name) as token, - attach_to_ringbuf_schannel(token) as chan, + attach_to_ringbuf_sender(token) as chan, ): await ctx.started() for i in range(100): @@ -45,6 +50,13 @@ async def send_child( def test_ringd(): + ''' + Spawn ringd actor and two childs that access same ringbuf through ringd. + + Both will use `ringd.open_ringbuf` to allocate the ringbuf, then attach to + them as sender and receiver. + + ''' async def main(): async with ( tractor.open_nursery() as an, @@ -73,9 +85,194 @@ def test_ringd(): ring_name='ring' ) as (sctx, _), ): - await rctx.wait_for_result() - await sctx.wait_for_result() + ... await an.cancel() trio.run(main) + + +# class Struct(msgspec.Struct): +# +# def encode(self) -> bytes: +# return msgspec.msgpack.encode(self) +# +# +# class AddChannelMsg(Struct, frozen=True, tag=True): +# name: str +# +# +# class RemoveChannelMsg(Struct, frozen=True, tag=True): +# name: str +# +# +# class RangeMsg(Struct, frozen=True, tag=True): +# start: int +# end: int +# +# +# ControlMessages = AddChannelMsg | RemoveChannelMsg | RangeMsg +# +# +# @tractor.context +# async def subscriber_child(ctx: tractor.Context): +# await ctx.started() +# async with ( +# open_ringbuf_subscriber(guarantee_order=True) as subs, +# trio.open_nursery() as n, +# ctx.open_stream() as stream +# ): +# range_msg = None +# range_event = trio.Event() +# range_scope = trio.CancelScope() +# +# async def _control_listen_task(): +# nonlocal range_msg, range_event +# async for msg in stream: +# msg = msgspec.msgpack.decode(msg, type=ControlMessages) +# match msg: +# case AddChannelMsg(): +# await subs.add_channel(msg.name, must_exist=False) +# +# case RemoveChannelMsg(): +# await subs.remove_channel(msg.name) +# +# case RangeMsg(): +# range_msg = msg +# range_event.set() +# +# await stream.send(b'ack') +# +# range_scope.cancel() +# +# n.start_soon(_control_listen_task) +# +# with range_scope: +# while True: +# await range_event.wait() +# range_event = trio.Event() +# for i in range(range_msg.start, range_msg.end): +# recv = int.from_bytes(await subs.receive()) +# # if recv != i: +# # raise AssertionError( +# # f'received: {recv} expected: {i}' +# # ) +# +# log.info(f'received: {recv} expected: {i}') +# +# await stream.send(b'valid range') +# log.info('FINISHED RANGE') +# +# log.info('subscriber exit') +# +# +# @tractor.context +# async def publisher_child(ctx: tractor.Context): +# await ctx.started() +# async with ( +# open_ringbuf_publisher(batch_size=1, guarantee_order=True) as pub, +# ctx.open_stream() as stream +# ): +# abs_index = 0 +# async for msg in stream: +# msg = msgspec.msgpack.decode(msg, type=ControlMessages) +# match msg: +# case AddChannelMsg(): +# await pub.add_channel(msg.name, must_exist=True) +# +# case RemoveChannelMsg(): +# await pub.remove_channel(msg.name) +# +# case RangeMsg(): +# for i in range(msg.start, msg.end): +# await pub.send(i.to_bytes(4)) +# log.info(f'sent {i}, index: {abs_index}') +# abs_index += 1 +# +# await stream.send(b'ack') +# +# log.info('publisher exit') +# +# +# +# def test_pubsub(): +# ''' +# Spawn ringd actor and two childs that access same ringbuf through ringd. +# +# Both will use `ringd.open_ringbuf` to allocate the ringbuf, then attach to +# them as sender and receiver. +# +# ''' +# async def main(): +# async with ( +# tractor.open_nursery( +# loglevel='info', +# # debug_mode=True, +# # enable_stack_on_sig=True +# ) as an, +# +# ringd.open_ringd() +# ): +# recv_portal = await an.start_actor( +# 'recv', +# enable_modules=[__name__] +# ) +# send_portal = await an.start_actor( +# 'send', +# enable_modules=[__name__] +# ) +# +# async with ( +# recv_portal.open_context(subscriber_child) as (rctx, _), +# rctx.open_stream() as recv_stream, +# send_portal.open_context(publisher_child) as (sctx, _), +# sctx.open_stream() as send_stream, +# ): +# async def send_wait_ack(msg: bytes): +# await recv_stream.send(msg) +# ack = await recv_stream.receive() +# assert ack == b'ack' +# +# await send_stream.send(msg) +# ack = await send_stream.receive() +# assert ack == b'ack' +# +# async def add_channel(name: str): +# await send_wait_ack(AddChannelMsg(name=name).encode()) +# +# async def remove_channel(name: str): +# await send_wait_ack(RemoveChannelMsg(name=name).encode()) +# +# async def send_range(start: int, end: int): +# await send_wait_ack(RangeMsg(start=start, end=end).encode()) +# range_ack = await recv_stream.receive() +# assert range_ack == b'valid range' +# +# # simple test, open one channel and send 0..100 range +# ring_name = 'ring-first' +# await add_channel(ring_name) +# await send_range(0, 100) +# await remove_channel(ring_name) +# +# # redo +# ring_name = 'ring-redo' +# await add_channel(ring_name) +# await send_range(0, 100) +# await remove_channel(ring_name) +# +# # multi chan test +# ring_names = [] +# for i in range(3): +# ring_names.append(f'multi-ring-{i}') +# +# for name in ring_names: +# await add_channel(name) +# +# await send_range(0, 300) +# +# for name in ring_names: +# await remove_channel(name) +# +# await an.cancel() +# +# trio.run(main) diff --git a/tractor/ipc/_ringbuf/_ringd.py b/tractor/ipc/_ringbuf/_ringd.py index 16314c72..24c3e530 100644 --- a/tractor/ipc/_ringbuf/_ringd.py +++ b/tractor/ipc/_ringbuf/_ringd.py @@ -29,6 +29,7 @@ from pathlib import Path from contextlib import ( asynccontextmanager as acm ) +from dataclasses import dataclass import trio import tractor @@ -42,12 +43,41 @@ log = tractor.log.get_logger(__name__) # log = tractor.log.get_console_log(level='info') +class RingNotFound(Exception): + ... + + _ringd_actor_name = 'ringd' _root_key = _ringd_actor_name + f'-{os.getpid()}' -_rings: dict[str, RBToken] = {} + + +@dataclass +class RingInfo: + token: RBToken + creator: str + unlink: trio.Event() + + +_rings: dict[str, RingInfo] = {} + + +def _maybe_get_ring(name: str) -> RingInfo | None: + if name in _rings: + return _rings[name] + + return None + + +def _insert_ring(name: str, info: RingInfo): + _rings[name] = info + + +def _destroy_ring(name: str): + del _rings[name] async def _attach_to_ring( + ringd_pid: int, ring_name: str ) -> RBToken: actor = tractor.current_actor() @@ -56,7 +86,7 @@ async def _attach_to_ring( sock_path = str( Path(tempfile.gettempdir()) / - f'{os.getpid()}-pass-ring-fds-{ring_name}-to-{actor.name}.sock' + f'ringd-{ringd_pid}-{ring_name}-to-{actor.name}.sock' ) log.info(f'trying to attach to ring {ring_name}...') @@ -94,8 +124,12 @@ async def _pass_fds( sock_path: str ): global _rings + info = _maybe_get_ring(name) - token = _rings[name] + if not info: + raise RingNotFound(f'Ring \"{name}\" not found!') + + token = info.token async with send_fds(token.fds, sock_path): log.info(f'connected to {sock_path} for fd passing') @@ -109,48 +143,58 @@ async def _pass_fds( @tractor.context async def _open_ringbuf( ctx: tractor.Context, + caller: str, name: str, + buf_size: int = 10 * 1024, must_exist: bool = False, - buf_size: int = 10 * 1024 ): global _root_key, _rings + log.info(f'maybe open ring {name} from {caller}, must_exist = {must_exist}') + + info = _maybe_get_ring(name) + + if info: + log.info(f'ring {name} exists, {caller} attached') + + await ctx.started(os.getpid()) - teardown = trio.Event() - async def _teardown_listener(task_status=trio.TASK_STATUS_IGNORED): async with ctx.open_stream() as stream: - task_status.started() await stream.receive() - teardown.set() - log.info(f'maybe open ring {name}, must_exist = {must_exist}') + info.unlink.set() - token = _rings.get(name, None) + log.info(f'{caller} detached from ring {name}') - async with trio.open_nursery() as n: - if token: - log.info(f'ring {name} exists') - await ctx.started() - await n.start(_teardown_listener) - await teardown.wait() - return + return - if must_exist: - raise FileNotFoundError( - f'Tried to open_ringbuf but it doesn\'t exist: {name}' + if must_exist: + raise RingNotFound( + f'Tried to open_ringbuf but it doesn\'t exist: {name}' + ) + + with ringbuf.open_ringbuf( + _root_key + name, + buf_size=buf_size + ) as token: + unlink_event = trio.Event() + _insert_ring( + name, + RingInfo( + token=token, + creator=caller, + unlink=unlink_event, ) + ) + log.info(f'ring {name} created by {caller}') + await ctx.started(os.getpid()) - with ringbuf.open_ringbuf( - _root_key + name, - buf_size=buf_size - ) as token: - _rings[name] = token - log.info(f'ring {name} created') - await ctx.started() - await n.start(_teardown_listener) - await teardown.wait() - del _rings[name] + async with ctx.open_stream() as stream: + await stream.receive() - log.info(f'ring {name} destroyed') + await unlink_event.wait() + _destroy_ring(name) + + log.info(f'ring {name} destroyed by {caller}') @acm @@ -174,22 +218,28 @@ async def wait_for_ringd() -> tractor.Portal: @acm async def open_ringbuf( + name: str, + buf_size: int = 10 * 1024, + must_exist: bool = False, - buf_size: int = 10 * 1024 + ) -> RBToken: + actor = tractor.current_actor() async with ( wait_for_ringd() as ringd, + ringd.open_context( _open_ringbuf, + caller=actor.name, name=name, - must_exist=must_exist, - buf_size=buf_size - ) as (rd_ctx, _), - rd_ctx.open_stream() as stream, + buf_size=buf_size, + must_exist=must_exist + ) as (rd_ctx, ringd_pid), + + rd_ctx.open_stream() as _stream, ): - token = await _attach_to_ring(name) + token = await _attach_to_ring(ringd_pid, name) log.info(f'attached to {token}') yield token - await stream.send(b'bye') diff --git a/tractor/linux/_fdshare.py b/tractor/linux/_fdshare.py index c632f532..16a748b8 100644 --- a/tractor/linux/_fdshare.py +++ b/tractor/linux/_fdshare.py @@ -19,6 +19,7 @@ Reimplementation of multiprocessing.reduction.sendfds & recvfds, using acms and cpython impl: https://github.com/python/cpython/blob/275056a7fdcbe36aaac494b4183ae59943a338eb/Lib/multiprocessing/reduction.py#L138 ''' +import os import array from typing import AsyncContextManager from contextlib import asynccontextmanager as acm @@ -73,6 +74,7 @@ async def send_fds(fds: list[int], sock_path: str) -> AsyncContextManager[None]: conn.close() sock.close() + os.unlink(sock_path) async def recv_fds(sock_path: str, amount: int) -> tuple: -- 2.34.1 From 1dfc639e54de24aaf856aef6ea91c60239ce8ca6 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Fri, 4 Apr 2025 02:44:45 -0300 Subject: [PATCH 19/41] Fully test and fix bugs on _ringbuf._pubsub Add generic channel orderer --- tests/test_ringd.py | 368 +++++++++++------------ tractor/ipc/_ringbuf/_pubsub.py | 511 +++++++++++++++++++++++--------- tractor/trionics/__init__.py | 5 + tractor/trionics/_ordering.py | 89 ++++++ 4 files changed, 648 insertions(+), 325 deletions(-) create mode 100644 tractor/trionics/_ordering.py diff --git a/tests/test_ringd.py b/tests/test_ringd.py index 40040a43..3eda428a 100644 --- a/tests/test_ringd.py +++ b/tests/test_ringd.py @@ -92,187 +92,187 @@ def test_ringd(): trio.run(main) -# class Struct(msgspec.Struct): -# -# def encode(self) -> bytes: -# return msgspec.msgpack.encode(self) -# -# -# class AddChannelMsg(Struct, frozen=True, tag=True): -# name: str -# -# -# class RemoveChannelMsg(Struct, frozen=True, tag=True): -# name: str -# -# -# class RangeMsg(Struct, frozen=True, tag=True): -# start: int -# end: int -# -# -# ControlMessages = AddChannelMsg | RemoveChannelMsg | RangeMsg -# -# -# @tractor.context -# async def subscriber_child(ctx: tractor.Context): -# await ctx.started() -# async with ( -# open_ringbuf_subscriber(guarantee_order=True) as subs, -# trio.open_nursery() as n, -# ctx.open_stream() as stream -# ): -# range_msg = None -# range_event = trio.Event() -# range_scope = trio.CancelScope() -# -# async def _control_listen_task(): -# nonlocal range_msg, range_event -# async for msg in stream: -# msg = msgspec.msgpack.decode(msg, type=ControlMessages) -# match msg: -# case AddChannelMsg(): -# await subs.add_channel(msg.name, must_exist=False) -# -# case RemoveChannelMsg(): -# await subs.remove_channel(msg.name) -# -# case RangeMsg(): -# range_msg = msg -# range_event.set() -# -# await stream.send(b'ack') -# -# range_scope.cancel() -# -# n.start_soon(_control_listen_task) -# -# with range_scope: -# while True: -# await range_event.wait() -# range_event = trio.Event() -# for i in range(range_msg.start, range_msg.end): -# recv = int.from_bytes(await subs.receive()) -# # if recv != i: -# # raise AssertionError( -# # f'received: {recv} expected: {i}' -# # ) -# -# log.info(f'received: {recv} expected: {i}') -# -# await stream.send(b'valid range') -# log.info('FINISHED RANGE') -# -# log.info('subscriber exit') -# -# -# @tractor.context -# async def publisher_child(ctx: tractor.Context): -# await ctx.started() -# async with ( -# open_ringbuf_publisher(batch_size=1, guarantee_order=True) as pub, -# ctx.open_stream() as stream -# ): -# abs_index = 0 -# async for msg in stream: -# msg = msgspec.msgpack.decode(msg, type=ControlMessages) -# match msg: -# case AddChannelMsg(): -# await pub.add_channel(msg.name, must_exist=True) -# -# case RemoveChannelMsg(): -# await pub.remove_channel(msg.name) -# -# case RangeMsg(): -# for i in range(msg.start, msg.end): -# await pub.send(i.to_bytes(4)) -# log.info(f'sent {i}, index: {abs_index}') -# abs_index += 1 -# -# await stream.send(b'ack') -# -# log.info('publisher exit') -# -# -# -# def test_pubsub(): -# ''' -# Spawn ringd actor and two childs that access same ringbuf through ringd. -# -# Both will use `ringd.open_ringbuf` to allocate the ringbuf, then attach to -# them as sender and receiver. -# -# ''' -# async def main(): -# async with ( -# tractor.open_nursery( -# loglevel='info', -# # debug_mode=True, -# # enable_stack_on_sig=True -# ) as an, -# -# ringd.open_ringd() -# ): -# recv_portal = await an.start_actor( -# 'recv', -# enable_modules=[__name__] -# ) -# send_portal = await an.start_actor( -# 'send', -# enable_modules=[__name__] -# ) -# -# async with ( -# recv_portal.open_context(subscriber_child) as (rctx, _), -# rctx.open_stream() as recv_stream, -# send_portal.open_context(publisher_child) as (sctx, _), -# sctx.open_stream() as send_stream, -# ): -# async def send_wait_ack(msg: bytes): -# await recv_stream.send(msg) -# ack = await recv_stream.receive() -# assert ack == b'ack' -# -# await send_stream.send(msg) -# ack = await send_stream.receive() -# assert ack == b'ack' -# -# async def add_channel(name: str): -# await send_wait_ack(AddChannelMsg(name=name).encode()) -# -# async def remove_channel(name: str): -# await send_wait_ack(RemoveChannelMsg(name=name).encode()) -# -# async def send_range(start: int, end: int): -# await send_wait_ack(RangeMsg(start=start, end=end).encode()) -# range_ack = await recv_stream.receive() -# assert range_ack == b'valid range' -# -# # simple test, open one channel and send 0..100 range -# ring_name = 'ring-first' -# await add_channel(ring_name) -# await send_range(0, 100) -# await remove_channel(ring_name) -# -# # redo -# ring_name = 'ring-redo' -# await add_channel(ring_name) -# await send_range(0, 100) -# await remove_channel(ring_name) -# -# # multi chan test -# ring_names = [] -# for i in range(3): -# ring_names.append(f'multi-ring-{i}') -# -# for name in ring_names: -# await add_channel(name) -# -# await send_range(0, 300) -# -# for name in ring_names: -# await remove_channel(name) -# -# await an.cancel() -# -# trio.run(main) +class Struct(msgspec.Struct): + + def encode(self) -> bytes: + return msgspec.msgpack.encode(self) + + +class AddChannelMsg(Struct, frozen=True, tag=True): + name: str + + +class RemoveChannelMsg(Struct, frozen=True, tag=True): + name: str + + +class RangeMsg(Struct, frozen=True, tag=True): + start: int + end: int + + +ControlMessages = AddChannelMsg | RemoveChannelMsg | RangeMsg + + +@tractor.context +async def subscriber_child(ctx: tractor.Context): + await ctx.started() + async with ( + open_ringbuf_subscriber(guarantee_order=True) as subs, + trio.open_nursery() as n, + ctx.open_stream() as stream + ): + range_msg = None + range_event = trio.Event() + range_scope = trio.CancelScope() + + async def _control_listen_task(): + nonlocal range_msg, range_event + async for msg in stream: + msg = msgspec.msgpack.decode(msg, type=ControlMessages) + match msg: + case AddChannelMsg(): + await subs.add_channel(msg.name, must_exist=False) + + case RemoveChannelMsg(): + await subs.remove_channel(msg.name) + + case RangeMsg(): + range_msg = msg + range_event.set() + + await stream.send(b'ack') + + range_scope.cancel() + + n.start_soon(_control_listen_task) + + with range_scope: + while True: + await range_event.wait() + range_event = trio.Event() + for i in range(range_msg.start, range_msg.end): + recv = int.from_bytes(await subs.receive()) + # if recv != i: + # raise AssertionError( + # f'received: {recv} expected: {i}' + # ) + + log.info(f'received: {recv} expected: {i}') + + await stream.send(b'valid range') + log.info('FINISHED RANGE') + + log.info('subscriber exit') + + +@tractor.context +async def publisher_child(ctx: tractor.Context): + await ctx.started() + async with ( + open_ringbuf_publisher(batch_size=1, guarantee_order=True) as pub, + ctx.open_stream() as stream + ): + abs_index = 0 + async for msg in stream: + msg = msgspec.msgpack.decode(msg, type=ControlMessages) + match msg: + case AddChannelMsg(): + await pub.add_channel(msg.name, must_exist=True) + + case RemoveChannelMsg(): + await pub.remove_channel(msg.name) + + case RangeMsg(): + for i in range(msg.start, msg.end): + await pub.send(i.to_bytes(4)) + log.info(f'sent {i}, index: {abs_index}') + abs_index += 1 + + await stream.send(b'ack') + + log.info('publisher exit') + + + +def test_pubsub(): + ''' + Spawn ringd actor and two childs that access same ringbuf through ringd. + + Both will use `ringd.open_ringbuf` to allocate the ringbuf, then attach to + them as sender and receiver. + + ''' + async def main(): + async with ( + tractor.open_nursery( + loglevel='info', + # debug_mode=True, + # enable_stack_on_sig=True + ) as an, + + ringd.open_ringd() + ): + recv_portal = await an.start_actor( + 'recv', + enable_modules=[__name__] + ) + send_portal = await an.start_actor( + 'send', + enable_modules=[__name__] + ) + + async with ( + recv_portal.open_context(subscriber_child) as (rctx, _), + rctx.open_stream() as recv_stream, + send_portal.open_context(publisher_child) as (sctx, _), + sctx.open_stream() as send_stream, + ): + async def send_wait_ack(msg: bytes): + await recv_stream.send(msg) + ack = await recv_stream.receive() + assert ack == b'ack' + + await send_stream.send(msg) + ack = await send_stream.receive() + assert ack == b'ack' + + async def add_channel(name: str): + await send_wait_ack(AddChannelMsg(name=name).encode()) + + async def remove_channel(name: str): + await send_wait_ack(RemoveChannelMsg(name=name).encode()) + + async def send_range(start: int, end: int): + await send_wait_ack(RangeMsg(start=start, end=end).encode()) + range_ack = await recv_stream.receive() + assert range_ack == b'valid range' + + # simple test, open one channel and send 0..100 range + ring_name = 'ring-first' + await add_channel(ring_name) + await send_range(0, 100) + await remove_channel(ring_name) + + # redo + ring_name = 'ring-redo' + await add_channel(ring_name) + await send_range(0, 100) + await remove_channel(ring_name) + + # multi chan test + ring_names = [] + for i in range(3): + ring_names.append(f'multi-ring-{i}') + + for name in ring_names: + await add_channel(name) + + await send_range(0, 300) + + for name in ring_names: + await remove_channel(name) + + await an.cancel() + + trio.run(main) diff --git a/tractor/ipc/_ringbuf/_pubsub.py b/tractor/ipc/_ringbuf/_pubsub.py index fe8b5b5b..4d5e0d20 100644 --- a/tractor/ipc/_ringbuf/_pubsub.py +++ b/tractor/ipc/_ringbuf/_pubsub.py @@ -17,13 +17,14 @@ Ring buffer ipc publish-subscribe mechanism brokered by ringd can dynamically add new outputs (publisher) or inputs (subscriber) ''' -import time from typing import ( - runtime_checkable, - Protocol, TypeVar, + Generic, + Callable, + Awaitable, AsyncContextManager ) +from functools import partial from contextlib import asynccontextmanager as acm from dataclasses import dataclass @@ -31,12 +32,16 @@ import trio import tractor from tractor.ipc import ( - RingBuffBytesSender, - RingBuffBytesReceiver, - attach_to_ringbuf_schannel, - attach_to_ringbuf_rchannel + RingBufferSendChannel, + RingBufferReceiveChannel, + attach_to_ringbuf_sender, + attach_to_ringbuf_receiver ) +from tractor.trionics import ( + order_send_channel, + order_receive_channel +) import tractor.ipc._ringbuf._ringd as ringd @@ -48,66 +53,100 @@ ChannelType = TypeVar('ChannelType') @dataclass class ChannelInfo: - connect_time: float name: str channel: ChannelType cancel_scope: trio.CancelScope -# TODO: maybe move this abstraction to another module or standalone? -# its not ring buf specific and allows fan out and fan in an a dynamic -# amount of channels -@runtime_checkable -class ChannelManager(Protocol[ChannelType]): +class ChannelManager(trio.abc.AsyncResource, Generic[ChannelType]): ''' - Common data structures and methods pubsub classes use to manage channels & - their related handler background tasks, as well as cancellation of them. + Helper for managing channel resources and their handler tasks with + cancellation, add or remove channels dynamically! ''' def __init__( self, + # nursery used to spawn channel handler tasks n: trio.Nursery, + + # acm will be used for setup & teardown of channel resources + open_channel_acm: Callable[..., AsyncContextManager[ChannelType]], + + # long running bg task to handle channel + channel_task: Callable[..., Awaitable[None]] ): self._n = n + self._open_channel = open_channel_acm + self._channel_task = channel_task + + # signal when a new channel conects and we previously had none + self._connect_event = trio.Event() + + # store channel runtime variables self._channels: list[ChannelInfo] = [] - async def _open_channel( + # methods that modify self._channels should be ordered by FIFO + self._lock = trio.StrictFIFOLock() + + @acm + async def maybe_lock(self): + ''' + If lock is not held, acquire + + ''' + if self._lock.locked(): + yield + return + + async with self._lock: + yield + + @property + def channels(self) -> list[ChannelInfo]: + return self._channels + + async def _channel_handler_task( self, - name: str - ) -> AsyncContextManager[ChannelType]: + name: str, + task_status: trio.TASK_STATUS_IGNORED, + **kwargs + ): ''' - Used to instantiate channel resources given a name + Open channel resources, add to internal data structures, signal channel + connect through trio.Event, and run `channel_task` with cancel scope, + and finally, maybe remove channel from internal data structures. + Spawned by `add_channel` function, lock is held from begining of fn + until `task_status.started()` call. + + kwargs are proxied to `self._open_channel` acm. ''' - ... + async with self._open_channel(name, **kwargs) as chan: + cancel_scope = trio.CancelScope() + info = ChannelInfo( + name=name, + channel=chan, + cancel_scope=cancel_scope + ) + self._channels.append(info) - async def _channel_task(self, info: ChannelInfo) -> None: - ''' - Long running task that manages the channel + if len(self) == 1: + self._connect_event.set() - ''' - ... + task_status.started() - async def _channel_handler_task(self, name: str): - async with self._open_channel(name) as chan: - with trio.CancelScope() as cancel_scope: - info = ChannelInfo( - connect_time=time.time(), - name=name, - channel=chan, - cancel_scope=cancel_scope - ) - self._channels.append(info) + with cancel_scope: await self._channel_task(info) - self._maybe_destroy_channel(name) + await self._maybe_destroy_channel(name) - def find_channel(self, name: str) -> tuple[int, ChannelInfo] | None: + def _find_channel(self, name: str) -> tuple[int, ChannelInfo] | None: ''' Given a channel name maybe return its index and value from internal _channels list. + Only use after acquiring lock. ''' for entry in enumerate(self._channels): i, info = entry @@ -116,105 +155,114 @@ class ChannelManager(Protocol[ChannelType]): return None - def _maybe_destroy_channel(self, name: str): + + async def _maybe_destroy_channel(self, name: str): ''' If channel exists cancel its scope and remove from internal _channels list. ''' - maybe_entry = self.find_channel(name) - if maybe_entry: - i, info = maybe_entry - info.cancel_scope.cancel() - del self._channels[i] + async with self.maybe_lock(): + maybe_entry = self._find_channel(name) + if maybe_entry: + i, info = maybe_entry + info.cancel_scope.cancel() + del self._channels[i] - def add_channel(self, name: str): + async def add_channel(self, name: str, **kwargs): ''' Add a new channel to be handled ''' - self._n.start_soon( - self._channel_handler_task, - name - ) + async with self.maybe_lock(): + await self._n.start(partial( + self._channel_handler_task, + name, + **kwargs + )) - def remove_channel(self, name: str): + async def remove_channel(self, name: str): ''' Remove a channel and stop its handling ''' - self._maybe_destroy_channel(name) + async with self.maybe_lock(): + await self._maybe_destroy_channel(name) + + # if that was last channel reset connect event + if len(self) == 0: + self._connect_event = trio.Event() + + async def wait_for_channel(self): + ''' + Wait until at least one channel added + + ''' + await self._connect_event.wait() + self._connect_event = trio.Event() def __len__(self) -> int: return len(self._channels) + def __getitem__(self, name: str): + maybe_entry = self._find_channel(name) + if maybe_entry: + _, info = maybe_entry + return info + + raise KeyError(f'Channel {name} not found!') + async def aclose(self) -> None: - for chan in self._channels: - self._maybe_destroy_channel(chan.name) - - async def __aenter__(self): - return self - - async def __aexit__(self, exc_type, exc_val, exc_tb): - await self.aclose() + async with self.maybe_lock(): + for info in self._channels: + await self.remove_channel(info.name) -class RingBuffPublisher( - ChannelManager[RingBuffBytesSender] -): +''' +Ring buffer publisher & subscribe pattern mediated by `ringd` actor. + +''' + +@dataclass +class PublisherChannels: + ring: RingBufferSendChannel + schan: trio.MemorySendChannel + rchan: trio.MemoryReceiveChannel + + +class RingBufferPublisher(trio.abc.SendChannel[bytes]): ''' - Implement ChannelManager protocol + trio.abc.SendChannel[bytes] - using ring buffers as transport. + Use ChannelManager to create a multi ringbuf round robin sender that can + dynamically add or remove more outputs. - - use a `trio.Event` to make sure `send` blocks until at least one channel - available. + Don't instantiate directly, use `open_ringbuf_publisher` acm to manage its + lifecycle. ''' - def __init__( self, n: trio.Nursery, + + # new ringbufs created will have this buf_size buf_size: int = 10 * 1024, + + # global batch size for all channels batch_size: int = 1 ): - super().__init__(n) - self._connect_event = trio.Event() - self._next_turn: int = 0 - + self._buf_size = buf_size self._batch_size: int = batch_size - @acm - async def _open_channel( - self, - name: str - ) -> AsyncContextManager[RingBuffBytesSender]: - async with ( - ringd.open_ringbuf( - name=name, - must_exist=True, - ) as token, - attach_to_ringbuf_schannel(token) as chan - ): - yield chan + self._chanmngr = ChannelManager[PublisherChannels]( + n, + self._open_channel, + self._channel_task + ) - async def _channel_task(self, info: ChannelInfo) -> None: - self._connect_event.set() - await trio.sleep_forever() + # methods that send data over the channels need to be acquire send lock + # in order to guarantee order of operations + self._send_lock = trio.StrictFIFOLock() - async def send(self, msg: bytes): - # wait at least one decoder connected - if len(self) == 0: - await self._connect_event.wait() - self._connect_event = trio.Event() - - if self._next_turn >= len(self): - self._next_turn = 0 - - turn = self._next_turn - self._next_turn += 1 - - output = self._channels[turn] - await output.channel.send(msg) + self._next_turn: int = 0 @property def batch_size(self) -> int: @@ -222,92 +270,273 @@ class RingBuffPublisher( @batch_size.setter def set_batch_size(self, value: int) -> None: - for output in self._channels: - output.channel.batch_size = value + for info in self.channels: + info.channel.ring.batch_size = value - async def flush( + @property + def channels(self) -> list[ChannelInfo]: + return self._chanmngr.channels + + def get_channel(self, name: str) -> ChannelInfo: + ''' + Get underlying ChannelInfo from name + + ''' + return self._chanmngr[name] + + async def add_channel( self, - new_batch_size: int | None = None + name: str, + must_exist: bool = False ): - for output in self._channels: - await output.channel.flush( - new_batch_size=new_batch_size + ''' + Store additional runtime info for channel and add channel to underlying + ChannelManager + + ''' + await self._chanmngr.add_channel(name, must_exist=must_exist) + + async def remove_channel(self, name: str): + ''' + Send EOF to channel (does `channel.flush` also) then remove from + `ChannelManager` acquire both `self._send_lock` and + `self._chanmngr.maybe_lock()` in order to ensure no channel + modifications or sends happen concurrenty + ''' + async with self._chanmngr.maybe_lock(): + # ensure all pending messages are sent + info = self.get_channel(name) + + try: + while True: + msg = info.channel.rchan.receive_nowait() + await info.channel.ring.send(msg) + + except trio.WouldBlock: + await info.channel.ring.flush() + + await info.channel.schan.aclose() + + # finally remove from ChannelManager + await self._chanmngr.remove_channel(name) + + @acm + async def _open_channel( + + self, + name: str, + must_exist: bool = False + + ) -> AsyncContextManager[PublisherChannels]: + ''' + Open a ringbuf through `ringd` and attach as send side + ''' + async with ( + ringd.open_ringbuf( + name=name, + buf_size=self._buf_size, + must_exist=must_exist, + ) as token, + attach_to_ringbuf_sender(token) as ring, + ): + schan, rchan = trio.open_memory_channel(0) + yield PublisherChannels( + ring=ring, + schan=schan, + rchan=rchan ) - async def send_eof(self): - for output in self._channels: - await output.channel.send_eof() + async def _channel_task(self, info: ChannelInfo) -> None: + ''' + Forever get current runtime info for channel, wait on its next pending + payloads update event then drain all into send channel. + + ''' + async for msg in info.channel.rchan: + await info.channel.ring.send(msg) + + async def send(self, msg: bytes): + ''' + If no output channels connected, wait until one, then fetch the next + channel based on turn, add the indexed payload and update + `self._next_turn` & `self._next_index`. + + Needs to acquire `self._send_lock` to make sure updates to turn & index + variables dont happen out of order. + + ''' + async with self._send_lock: + # wait at least one decoder connected + if len(self.channels) == 0: + await self._chanmngr.wait_for_channel() + + if self._next_turn >= len(self.channels): + self._next_turn = 0 + + info = self.channels[self._next_turn] + await info.channel.schan.send(msg) + + self._next_turn += 1 + + async def aclose(self) -> None: + await self._chanmngr.aclose() @acm async def open_ringbuf_publisher( + buf_size: int = 10 * 1024, - batch_size: int = 1 -): + batch_size: int = 1, + guarantee_order: bool = False, + force_cancel: bool = False + +) -> AsyncContextManager[RingBufferPublisher]: + ''' + Open a new ringbuf publisher + + ''' async with ( trio.open_nursery() as n, - RingBuffPublisher( + RingBufferPublisher( n, buf_size=buf_size, batch_size=batch_size - ) as outputs + ) as publisher ): - yield outputs + if guarantee_order: + order_send_channel(publisher) + + yield publisher + + if force_cancel: + # implicitly cancel any running channel handler task + n.cancel_scope.cancel() - -class RingBuffSubscriber( - ChannelManager[RingBuffBytesReceiver] -): +class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): ''' - Implement ChannelManager protocol + trio.abc.ReceiveChannel[bytes] - using ring buffers as transport. + Use ChannelManager to create a multi ringbuf receiver that can + dynamically add or remove more inputs and combine all into a single output. - - use a trio memory channel pair to multiplex all received messages into a - single `trio.MemoryReceiveChannel`, give a sender channel clone to each - _channel_task. + In order for `self.receive` messages to be returned in order, publisher + will send all payloads as `OrderedPayload` msgpack encoded msgs, this + allows our channel handler tasks to just stash the out of order payloads + inside `self._pending_payloads` and if a in order payload is available + signal through `self._new_payload_event`. + + On `self.receive` we wait until at least one channel is connected, then if + an in order payload is pending, we pop and return it, in case no in order + payload is available wait until next `self._new_payload_event.set()`. ''' def __init__( self, n: trio.Nursery, + + # if connecting to a publisher that has already sent messages set + # to the next expected payload index this subscriber will receive + start_index: int = 0 ): - super().__init__(n) - self._send_chan, self._recv_chan = trio.open_memory_channel(0) + self._chanmngr = ChannelManager[RingBufferReceiveChannel]( + n, + self._open_channel, + self._channel_task + ) + + self._schan, self._rchan = trio.open_memory_channel(0) + + @property + def channels(self) -> list[ChannelInfo]: + return self._chanmngr.channels + + def get_channel(self, name: str): + return self._chanmngr[name] + + async def add_channel(self, name: str, must_exist: bool = False): + ''' + Add new input channel by name + + ''' + await self._chanmngr.add_channel(name, must_exist=must_exist) + + async def remove_channel(self, name: str): + ''' + Remove an input channel by name + + ''' + await self._chanmngr.remove_channel(name) @acm async def _open_channel( + self, - name: str - ) -> AsyncContextManager[RingBuffBytesReceiver]: + name: str, + must_exist: bool = False + + ) -> AsyncContextManager[RingBufferReceiveChannel]: + ''' + Open a ringbuf through `ringd` and attach as receiver side + ''' async with ( ringd.open_ringbuf( name=name, - must_exist=True, + must_exist=must_exist, ) as token, - attach_to_ringbuf_rchannel(token) as chan + attach_to_ringbuf_receiver(token) as chan ): yield chan async def _channel_task(self, info: ChannelInfo) -> None: - send_chan = self._send_chan.clone() - try: - async for msg in info.channel: - await send_chan.send(msg) + ''' + Iterate over receive channel messages, decode them as `OrderedPayload`s + and stash them in `self._pending_payloads`, in case we can pop next in + order payload, signal through setting `self._new_payload_event`. - except tractor._exceptions.InternalError: - # TODO: cleaner cancellation! - ... + ''' + while True: + try: + msg = await info.channel.receive() + await self._schan.send(msg) + + except tractor.linux.eventfd.EFDReadCancelled as e: + # when channel gets removed while we are doing a receive + log.exception(e) + break + + except trio.EndOfChannel: + break async def receive(self) -> bytes: - return await self._recv_chan.receive() + ''' + Receive next in order msg + ''' + return await self._rchan.receive() + async def aclose(self) -> None: + await self._chanmngr.aclose() @acm -async def open_ringbuf_subscriber(): +async def open_ringbuf_subscriber( + + guarantee_order: bool = False, + force_cancel: bool = False + +) -> AsyncContextManager[RingBufferPublisher]: + ''' + Open a new ringbuf subscriber + + ''' async with ( trio.open_nursery() as n, - RingBuffSubscriber(n) as inputs + RingBufferSubscriber( + n, + ) as subscriber ): - yield inputs + if guarantee_order: + order_receive_channel(subscriber) + yield subscriber + + if force_cancel: + # implicitly cancel any running channel handler task + n.cancel_scope.cancel() diff --git a/tractor/trionics/__init__.py b/tractor/trionics/__init__.py index df9b6f26..97d03da7 100644 --- a/tractor/trionics/__init__.py +++ b/tractor/trionics/__init__.py @@ -32,3 +32,8 @@ from ._broadcast import ( from ._beg import ( collapse_eg as collapse_eg, ) + +from ._ordering import ( + order_send_channel as order_send_channel, + order_receive_channel as order_receive_channel +) diff --git a/tractor/trionics/_ordering.py b/tractor/trionics/_ordering.py new file mode 100644 index 00000000..2d7e9082 --- /dev/null +++ b/tractor/trionics/_ordering.py @@ -0,0 +1,89 @@ +from __future__ import annotations +from heapq import ( + heappush, + heappop +) + +import trio +import msgspec + + +class OrderedPayload(msgspec.Struct, frozen=True): + index: int + payload: bytes + + @classmethod + def from_msg(cls, msg: bytes) -> OrderedPayload: + return msgspec.msgpack.decode(msg, type=OrderedPayload) + + def encode(self) -> bytes: + return msgspec.msgpack.encode(self) + + +def order_send_channel( + channel: trio.abc.SendChannel[bytes], + start_index: int = 0 +): + + next_index = start_index + send_lock = trio.StrictFIFOLock() + + channel._send = channel.send + channel._aclose = channel.aclose + + async def send(msg: bytes): + nonlocal next_index + async with send_lock: + await channel._send( + OrderedPayload( + index=next_index, + payload=msg + ).encode() + ) + next_index += 1 + + async def aclose(): + async with send_lock: + await channel._aclose() + + channel.send = send + channel.aclose = aclose + + +def order_receive_channel( + channel: trio.abc.ReceiveChannel[bytes], + start_index: int = 0 +): + next_index = start_index + pqueue = [] + + channel._receive = channel.receive + + def can_pop_next() -> bool: + return ( + len(pqueue) > 0 + and + pqueue[0][0] == next_index + ) + + async def drain_to_heap(): + while not can_pop_next(): + msg = await channel._receive() + msg = OrderedPayload.from_msg(msg) + heappush(pqueue, (msg.index, msg.payload)) + + def pop_next(): + nonlocal next_index + _, msg = heappop(pqueue) + next_index += 1 + return msg + + async def receive() -> bytes: + if can_pop_next(): + return pop_next() + + await drain_to_heap() + + return pop_next() + + channel.receive = receive -- 2.34.1 From d8d01e8b3c62f2ca58c3fdf608f1bc46a82f928b Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Fri, 4 Apr 2025 02:47:27 -0300 Subject: [PATCH 20/41] Add header to generic chan orderers --- tractor/trionics/_ordering.py | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/tractor/trionics/_ordering.py b/tractor/trionics/_ordering.py index 2d7e9082..0cc89b4b 100644 --- a/tractor/trionics/_ordering.py +++ b/tractor/trionics/_ordering.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 . +''' +Helpers to guarantee ordering of messages through a unordered channel + +''' from __future__ import annotations from heapq import ( heappush, -- 2.34.1 From d942f073e0b36d8a3ef9aac12c291c5df6b225e7 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Fri, 4 Apr 2025 02:57:38 -0300 Subject: [PATCH 21/41] Enable ordering assertion & simplify some parts of test --- tests/test_ringd.py | 42 +++++++++++++++++++++++------------------- 1 file changed, 23 insertions(+), 19 deletions(-) diff --git a/tests/test_ringd.py b/tests/test_ringd.py index 3eda428a..9c6bec69 100644 --- a/tests/test_ringd.py +++ b/tests/test_ringd.py @@ -1,3 +1,5 @@ +from contextlib import asynccontextmanager as acm + import trio import tractor import msgspec @@ -107,8 +109,7 @@ class RemoveChannelMsg(Struct, frozen=True, tag=True): class RangeMsg(Struct, frozen=True, tag=True): - start: int - end: int + size: int ControlMessages = AddChannelMsg | RemoveChannelMsg | RangeMsg @@ -151,17 +152,17 @@ async def subscriber_child(ctx: tractor.Context): while True: await range_event.wait() range_event = trio.Event() - for i in range(range_msg.start, range_msg.end): + for i in range(range_msg.size): recv = int.from_bytes(await subs.receive()) - # if recv != i: - # raise AssertionError( - # f'received: {recv} expected: {i}' - # ) + if recv != i: + raise AssertionError( + f'received: {recv} expected: {i}' + ) - log.info(f'received: {recv} expected: {i}') + log.info(f'received: {recv}') await stream.send(b'valid range') - log.info('FINISHED RANGE') + log.info('finished range') log.info('subscriber exit') @@ -170,10 +171,9 @@ async def subscriber_child(ctx: tractor.Context): async def publisher_child(ctx: tractor.Context): await ctx.started() async with ( - open_ringbuf_publisher(batch_size=1, guarantee_order=True) as pub, + open_ringbuf_publisher(batch_size=100, guarantee_order=True) as pub, ctx.open_stream() as stream ): - abs_index = 0 async for msg in stream: msg = msgspec.msgpack.decode(msg, type=ControlMessages) match msg: @@ -184,10 +184,9 @@ async def publisher_child(ctx: tractor.Context): await pub.remove_channel(msg.name) case RangeMsg(): - for i in range(msg.start, msg.end): + for i in range(msg.size): await pub.send(i.to_bytes(4)) - log.info(f'sent {i}, index: {abs_index}') - abs_index += 1 + log.info(f'sent {i}') await stream.send(b'ack') @@ -243,21 +242,26 @@ def test_pubsub(): async def remove_channel(name: str): await send_wait_ack(RemoveChannelMsg(name=name).encode()) - async def send_range(start: int, end: int): - await send_wait_ack(RangeMsg(start=start, end=end).encode()) + async def send_range(size: int): + await send_wait_ack(RangeMsg(size=size).encode()) range_ack = await recv_stream.receive() assert range_ack == b'valid range' # simple test, open one channel and send 0..100 range ring_name = 'ring-first' await add_channel(ring_name) - await send_range(0, 100) + await send_range(100) await remove_channel(ring_name) # redo ring_name = 'ring-redo' await add_channel(ring_name) - await send_range(0, 100) + await send_range(100) + await remove_channel(ring_name) + + # try using same ring name + await add_channel(ring_name) + await send_range(100) await remove_channel(ring_name) # multi chan test @@ -268,7 +272,7 @@ def test_pubsub(): for name in ring_names: await add_channel(name) - await send_range(0, 300) + await send_range(1000) for name in ring_names: await remove_channel(name) -- 2.34.1 From 3a1eda9d6d39b38925e97657229d134a52e113ac Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Fri, 4 Apr 2025 02:59:50 -0300 Subject: [PATCH 22/41] Fix test docstring --- tests/test_ringd.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/test_ringd.py b/tests/test_ringd.py index 9c6bec69..4b5c792e 100644 --- a/tests/test_ringd.py +++ b/tests/test_ringd.py @@ -196,10 +196,9 @@ async def publisher_child(ctx: tractor.Context): def test_pubsub(): ''' - Spawn ringd actor and two childs that access same ringbuf through ringd. - - Both will use `ringd.open_ringbuf` to allocate the ringbuf, then attach to - them as sender and receiver. + Spawn to childs a publisher and a subscriber, use context streams + to dynamically test different scenarios with different channel + configurations between them. ''' async def main(): -- 2.34.1 From 1451feb15923698cf17cfb2f65f3d1b4ae2a2761 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sun, 6 Apr 2025 17:02:15 -0300 Subject: [PATCH 23/41] Adhere to trio semantics on channels for closed and busy resource cases --- tests/test_ringbuf.py | 2 +- tractor/ipc/_ringbuf/__init__.py | 210 +++++++++++++++++++++---------- 2 files changed, 147 insertions(+), 65 deletions(-) diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index fc177976..6dfeae72 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -56,6 +56,7 @@ async def child_read_shm( print(f'\n\telapsed ms: {elapsed_ms}') print(f'\tmsg/sec: {int(msg_amount / elapsed):,}') print(f'\tbytes/sec: {int(recvd_bytes / elapsed):,}') + print(f'\treceived msgs: {msg_amount:,}') print(f'\treceived bytes: {recvd_bytes:,}') return recvd_hash.hexdigest() @@ -165,7 +166,6 @@ def test_ringbuf( await send_p.cancel_actor() await recv_p.cancel_actor() - trio.run(main) diff --git a/tractor/ipc/_ringbuf/__init__.py b/tractor/ipc/_ringbuf/__init__.py index 58d90dab..2dfd8d82 100644 --- a/tractor/ipc/_ringbuf/__init__.py +++ b/tractor/ipc/_ringbuf/__init__.py @@ -200,23 +200,28 @@ class RingBufferSendChannel(trio.abc.SendChannel[bytes]): self._eof_event = EventFD(self._token.eof_eventfd, 'w') # current write pointer - self._ptr = 0 + self._ptr: int = 0 # when `batch_size` > 1 store messages on `self._batch` and write them # all, once `len(self._batch) == `batch_size` self._batch: list[bytes] = [] - self._cleanup = cleanup + # close shm & fds on exit? + self._cleanup: bool = cleanup + + # have we closed this ringbuf? + # set to `False` on `.open()` + self._is_closed: bool = True + + # ensure no concurrent `.send_all()` calls + self._send_all_lock = trio.StrictFIFOLock() + + # ensure no concurrent `.send()` calls self._send_lock = trio.StrictFIFOLock() - @acm - async def _maybe_lock(self) -> AsyncContextManager[None]: - if self._send_lock.locked(): - yield - return - - async with self._send_lock: - yield + @property + def closed(self) -> bool: + return self._is_closed @property def name(self) -> str: @@ -252,7 +257,13 @@ class RingBufferSendChannel(trio.abc.SendChannel[bytes]): await self._wrap_event.read() async def send_all(self, data: Buffer): - async with self._maybe_lock(): + if self.closed: + raise trio.ClosedResourceError + + if self._send_all_lock.locked(): + raise trio.BusyResourceError + + async with self._send_all_lock: # while data is larger than the remaining buf target_ptr = self.ptr + len(data) while target_ptr > self.size: @@ -274,13 +285,16 @@ class RingBufferSendChannel(trio.abc.SendChannel[bytes]): self._ptr = target_ptr async def wait_send_all_might_not_block(self): - raise NotImplementedError + return async def flush( self, new_batch_size: int | None = None ) -> None: - async with self._maybe_lock(): + if self.closed: + raise trio.ClosedResourceError + + async with self._send_lock: for msg in self._batch: await self.send_all(msg) @@ -289,7 +303,13 @@ class RingBufferSendChannel(trio.abc.SendChannel[bytes]): self.batch_size = new_batch_size async def send(self, value: bytes) -> None: - async with self._maybe_lock(): + if self.closed: + raise trio.ClosedResourceError + + if self._send_lock.locked(): + raise trio.BusyResourceError + + async with self._send_lock: msg: bytes = struct.pack(" None: - async with self._send_lock: - await self.flush(new_batch_size=1) - await self.send(b'') - def open(self): try: self._shm = SharedMemory( @@ -314,12 +329,13 @@ class RingBufferSendChannel(trio.abc.SendChannel[bytes]): self._write_event.open() self._wrap_event.open() self._eof_event.open() + self._is_closed = False except Exception as e: e.add_note(f'while opening sender for {self._token.as_msg()}') raise e - def close(self): + def _close(self): self._eof_event.write( self._ptr if self._ptr > 0 else self.size ) @@ -330,8 +346,14 @@ class RingBufferSendChannel(trio.abc.SendChannel[bytes]): self._eof_event.close() self._shm.close() + self._is_closed = True + async def aclose(self): - self.close() + if not self.closed: + await self.send(b'') + await self.flush() + + self._close() async def __aenter__(self): self.open() @@ -362,6 +384,16 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): self._cleanup: bool = cleanup + self._is_closed: bool = True + + self._receive_some_lock = trio.StrictFIFOLock() + self._receive_exactly_lock = trio.StrictFIFOLock() + self._receive_lock = trio.StrictFIFOLock() + + @property + def closed(self) -> bool: + return self._is_closed + @property def name(self) -> str: if not self._shm: @@ -409,12 +441,25 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): Try to receive any bytes we can without blocking or raise `trio.WouldBlock`. + Returns b'' when no more bytes can be read (EOF signaled & read all). + ''' if max_bytes < 1: raise ValueError("max_bytes must be >= 1") - delta = self._write_ptr - self._ptr + # in case `end_ptr` is set that means eof was signaled. + # it will be >= `write_ptr`, use it for delta calc + highest_ptr = max(self._write_ptr, self._end_ptr) + + delta = highest_ptr - self._ptr + + # no more bytes to read if delta == 0: + # if `end_ptr` is set that means we read all bytes before EOF + if self._end_ptr > 0: + return b'' + + # signal the need to wait on `write_event` raise trio.WouldBlock # dont overflow caller @@ -442,35 +487,47 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): Can return < max_bytes. ''' - try: - return self.receive_nowait(max_bytes=max_bytes) + if self.closed: + raise trio.ClosedResourceError - except trio.WouldBlock: - # we have read all we can, see if new data is available - if self._end_ptr < 0: - # if we havent been signaled about EOF yet - try: - delta = await self._write_event.read() - self._write_ptr += delta + if self._receive_some_lock.locked(): + raise trio.BusyResourceError - except EFDReadCancelled: - # while waiting for new data `self._write_event` was closed - # this means writer signaled EOF - if self._end_ptr > 0: - # final self._write_ptr modification and recalculate delta - self._write_ptr = self._end_ptr - delta = self._end_ptr - self._ptr + async with self._receive_some_lock: + try: + # attempt direct read + return self.receive_nowait(max_bytes=max_bytes) - else: - # shouldnt happen cause self._eof_monitor_task always sets - # self._end_ptr before closing self._write_event - raise InternalError( - 'self._write_event.read cancelled but self._end_ptr is not set' - ) + except trio.WouldBlock as e: + # we have read all we can, see if new data is available + if self._end_ptr == -1: + # if we havent been signaled about EOF yet + try: + # wait next write and advance `write_ptr` + delta = await self._write_event.read() + self._write_ptr += delta + # yield lock and re-enter - else: - # no more bytes to read and self._end_ptr set, EOF reached - return b'' + except EFDReadCancelled: + # while waiting for new data `self._write_event` was closed + # this means writer signaled EOF + if self._end_ptr > 0: + # receive_nowait will handle read until EOF + return self.receive_nowait(max_bytes=max_bytes) + + else: + # shouldnt happen because self._eof_monitor_task always sets + # self._end_ptr before closing self._write_event + raise InternalError( + 'self._write_event.read cancelled but self._end_ptr is not set' + ) + + else: + # shouldnt happen because receive_nowait does not raise + # trio.WouldBlock when `end_ptr` is set + raise InternalError( + 'self._end_ptr is set but receive_nowait raised trio.WouldBlock' + ) from e return await self.receive_some(max_bytes=max_bytes) @@ -479,32 +536,50 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): Fetch bytes until we read exactly `num_bytes` or EOF. ''' - payload = b'' - while len(payload) < num_bytes: - remaining = num_bytes - len(payload) + if self.closed: + raise trio.ClosedResourceError - new_bytes = await self.receive_some( - max_bytes=remaining - ) + if self._receive_exactly_lock.locked(): + raise trio.BusyResourceError - if new_bytes == b'': + async with self._receive_exactly_lock: + payload = b'' + while len(payload) < num_bytes: + remaining = num_bytes - len(payload) + + new_bytes = await self.receive_some( + max_bytes=remaining + ) + + if new_bytes == b'': + break + + payload += new_bytes + + if payload == b'': raise trio.EndOfChannel - payload += new_bytes - - return payload + return payload async def receive(self) -> bytes: ''' Receive a complete payload ''' - header: bytes = await self.receive_exactly(4) - size: int - size, = struct.unpack(" AsyncContextManager[RingBufferSendChannel]: @@ -574,6 +653,7 @@ async def attach_to_ringbuf_sender( ''' async with RingBufferSendChannel( token, + batch_size=batch_size, cleanup=cleanup ) as sender: yield sender @@ -644,8 +724,9 @@ class RingBufferChannel(trio.abc.Channel[bytes]): async def attach_to_ringbuf_channel( token_in: RBToken, token_out: RBToken, + batch_size: int = 1, cleanup_in: bool = True, - cleanup_out: bool = True + cleanup_out: bool = True, ) -> AsyncContextManager[trio.StapledStream]: ''' Attach to two previously opened `RBToken`s and return a `RingBufferChannel` @@ -658,6 +739,7 @@ async def attach_to_ringbuf_channel( ) as receiver, attach_to_ringbuf_sender( token_out, + batch_size=batch_size, cleanup=cleanup_out ) as sender, ): -- 2.34.1 From 8e1f95881c24abeab72886981ab142fde4b80f46 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sun, 6 Apr 2025 21:04:18 -0300 Subject: [PATCH 24/41] Add trio resource semantics to eventfd --- tests/test_eventfd.py | 42 ++++++++++++++++++++++++++++++++++++---- tractor/linux/eventfd.py | 39 +++++++++++++++++++++++++++---------- 2 files changed, 67 insertions(+), 14 deletions(-) diff --git a/tests/test_eventfd.py b/tests/test_eventfd.py index 3d757169..3432048b 100644 --- a/tests/test_eventfd.py +++ b/tests/test_eventfd.py @@ -1,13 +1,13 @@ import trio import pytest -from tractor.ipc import ( +from tractor.linux.eventfd import ( open_eventfd, EFDReadCancelled, EventFD ) -def test_eventfd_read_cancellation(): +def test_read_cancellation(): ''' Ensure EventFD.read raises EFDReadCancelled if EventFD.close() is called. @@ -15,7 +15,7 @@ def test_eventfd_read_cancellation(): ''' fd = open_eventfd() - async def _read(event: EventFD): + async def bg_read(event: EventFD): with pytest.raises(EFDReadCancelled): await event.read() @@ -25,8 +25,42 @@ def test_eventfd_read_cancellation(): EventFD(fd, 'w') as event, trio.fail_after(3) ): - n.start_soon(_read, event) + n.start_soon(bg_read, event) await trio.sleep(0.2) event.close() trio.run(main) + + +def test_read_trio_semantics(): + ''' + Ensure EventFD.read raises trio.ClosedResourceError and + trio.BusyResourceError. + + ''' + + fd = open_eventfd() + + async def bg_read(event: EventFD): + try: + await event.read() + + except EFDReadCancelled: + ... + + async def main(): + async with trio.open_nursery() as n: + + # start background read and attempt + # foreground read, should be busy + with EventFD(fd, 'w') as event: + n.start_soon(bg_read, event) + await trio.sleep(0.2) + with pytest.raises(trio.BusyResourceError): + await event.read() + + # attempt read after close + with pytest.raises(trio.ClosedResourceError): + await event.read() + + trio.run(main) diff --git a/tractor/linux/eventfd.py b/tractor/linux/eventfd.py index f262c051..8ddf3669 100644 --- a/tractor/linux/eventfd.py +++ b/tractor/linux/eventfd.py @@ -129,12 +129,21 @@ class EventFD: self._omode: str = omode self._fobj = None self._cscope: trio.CancelScope | None = None + self._is_closed: bool = True + self._read_lock = trio.StrictFIFOLock() + + @property + def closed(self) -> bool: + return self._is_closed @property def fd(self) -> int | None: return self._fd def write(self, value: int) -> int: + if self.closed: + raise trio.ClosedResourceError + return write_eventfd(self._fd, value) async def read(self) -> int: @@ -145,19 +154,26 @@ class EventFD: in order to make it cancellable when `self.close()` is called. ''' - self._cscope = trio.CancelScope() - with self._cscope: - return await trio.to_thread.run_sync( - read_eventfd, self._fd, - abandon_on_cancel=True - ) + if self.closed: + raise trio.ClosedResourceError - if self._cscope.cancelled_caught: - raise EFDReadCancelled + if self._read_lock.locked(): + raise trio.BusyResourceError - self._cscope = None + async with self._read_lock: + self._cscope = trio.CancelScope() + with self._cscope: + return await trio.to_thread.run_sync( + read_eventfd, self._fd, + abandon_on_cancel=True + ) - def read_direct(self) -> int: + if self._cscope.cancelled_caught: + raise EFDReadCancelled + + self._cscope = None + + def read_nowait(self) -> int: ''' Direct call to `read_eventfd(self.fd)`, unless `eventfd` was opened with `EFD_NONBLOCK` its gonna block the thread. @@ -167,6 +183,7 @@ class EventFD: def open(self): self._fobj = os.fdopen(self._fd, self._omode) + self._is_closed = False def close(self): if self._fobj: @@ -179,6 +196,8 @@ class EventFD: if self._cscope: self._cscope.cancel() + self._is_closed = True + def __enter__(self): self.open() return self -- 2.34.1 From 853aa740aad927ab0bca008fee8fa533ed95cdc9 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sun, 6 Apr 2025 21:16:55 -0300 Subject: [PATCH 25/41] RingBufferReceiveChannel fixes for the non clean eof case, add comments --- tractor/ipc/_ringbuf/__init__.py | 88 +++++++++++++++++++++----------- 1 file changed, 57 insertions(+), 31 deletions(-) diff --git a/tractor/ipc/_ringbuf/__init__.py b/tractor/ipc/_ringbuf/__init__.py index 2dfd8d82..2b2b549e 100644 --- a/tractor/ipc/_ringbuf/__init__.py +++ b/tractor/ipc/_ringbuf/__init__.py @@ -108,19 +108,14 @@ def open_ringbuf( create=True ) try: - with ( - EventFD(open_eventfd(), 'r') as write_event, - EventFD(open_eventfd(), 'r') as wrap_event, - EventFD(open_eventfd(), 'r') as eof_event, - ): - token = RBToken( - shm_name=shm_name, - write_eventfd=write_event.fd, - wrap_eventfd=wrap_event.fd, - eof_eventfd=eof_event.fd, - buf_size=buf_size - ) - yield token + token = RBToken( + shm_name=shm_name, + write_eventfd=open_eventfd(), + wrap_eventfd=open_eventfd(), + eof_eventfd=open_eventfd(), + buf_size=buf_size + ) + yield token finally: shm.unlink() @@ -219,6 +214,9 @@ class RingBufferSendChannel(trio.abc.SendChannel[bytes]): # ensure no concurrent `.send()` calls self._send_lock = trio.StrictFIFOLock() + # ensure no concurrent `.flush()` calls + self._flush_lock = trio.StrictFIFOLock() + @property def closed(self) -> bool: return self._is_closed @@ -294,7 +292,7 @@ class RingBufferSendChannel(trio.abc.SendChannel[bytes]): if self.closed: raise trio.ClosedResourceError - async with self._send_lock: + async with self._flush_lock: for msg in self._batch: await self.send_all(msg) @@ -312,6 +310,9 @@ class RingBufferSendChannel(trio.abc.SendChannel[bytes]): async with self._send_lock: msg: bytes = struct.pack(" 0: + await self.flush() + await self.send_all(msg) return @@ -349,9 +350,8 @@ class RingBufferSendChannel(trio.abc.SendChannel[bytes]): self._is_closed = True async def aclose(self): - if not self.closed: - await self.send(b'') - await self.flush() + if self.closed: + return self._close() @@ -374,20 +374,37 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): cleanup: bool = True, ): self._token = RBToken.from_msg(token) + + # ringbuf os resources self._shm: SharedMemory | None = None self._write_event = EventFD(self._token.write_eventfd, 'w') self._wrap_event = EventFD(self._token.wrap_eventfd, 'r') self._eof_event = EventFD(self._token.eof_eventfd, 'r') + + # current read ptr self._ptr: int = 0 + + # current write_ptr (max bytes we can read from buf) self._write_ptr: int = 0 + + # end ptr is used when EOF is signaled, it will contain maximun + # readable position on buf self._end_ptr: int = -1 + # close shm & fds on exit? self._cleanup: bool = cleanup + # have we closed this ringbuf? + # set to `False` on `.open()` self._is_closed: bool = True + # ensure no concurrent `.receive_some()` calls self._receive_some_lock = trio.StrictFIFOLock() + + # ensure no concurrent `.receive_exactly()` calls self._receive_exactly_lock = trio.StrictFIFOLock() + + # ensure no concurrent `.receive()` calls self._receive_lock = trio.StrictFIFOLock() @property @@ -416,6 +433,10 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): def wrap_fd(self) -> int: return self._wrap_event.fd + @property + def eof_was_signaled(self) -> bool: + return self._end_ptr != -1 + async def _eof_monitor_task(self): ''' Long running EOF event monitor, automatically run in bg by @@ -428,7 +449,6 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): ''' try: self._end_ptr = await self._eof_event.read() - self._write_event.close() except EFDReadCancelled: ... @@ -436,6 +456,11 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): except trio.Cancelled: ... + finally: + # closing write_event should trigger `EFDReadCancelled` + # on any pending read + self._write_event.close() + def receive_nowait(self, max_bytes: int = _DEFAULT_RB_SIZE) -> bytes: ''' Try to receive any bytes we can without blocking or raise @@ -456,7 +481,7 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): # no more bytes to read if delta == 0: # if `end_ptr` is set that means we read all bytes before EOF - if self._end_ptr > 0: + if self.eof_was_signaled: return b'' # signal the need to wait on `write_event` @@ -500,7 +525,7 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): except trio.WouldBlock as e: # we have read all we can, see if new data is available - if self._end_ptr == -1: + if not self.eof_was_signaled: # if we havent been signaled about EOF yet try: # wait next write and advance `write_ptr` @@ -510,17 +535,15 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): except EFDReadCancelled: # while waiting for new data `self._write_event` was closed - # this means writer signaled EOF - if self._end_ptr > 0: - # receive_nowait will handle read until EOF + try: + # if eof was signaled receive no wait will not raise + # trio.WouldBlock and will push remaining until EOF return self.receive_nowait(max_bytes=max_bytes) - else: - # shouldnt happen because self._eof_monitor_task always sets - # self._end_ptr before closing self._write_event - raise InternalError( - 'self._write_event.read cancelled but self._end_ptr is not set' - ) + except trio.WouldBlock: + # eof was not signaled but `self._wrap_event` is closed + # this means send side closed without EOF signal + return b'' else: # shouldnt happen because receive_nowait does not raise @@ -533,7 +556,7 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): async def receive_exactly(self, num_bytes: int) -> bytes: ''' - Fetch bytes until we read exactly `num_bytes` or EOF. + Fetch bytes until we read exactly `num_bytes` or EOC. ''' if self.closed: @@ -563,7 +586,7 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): async def receive(self) -> bytes: ''' - Receive a complete payload + Receive a complete payload or raise EOC ''' if self.closed: @@ -607,6 +630,9 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): self._is_closed = True async def aclose(self): + if self.closed: + return + self.close() async def __aenter__(self): -- 2.34.1 From 171545e4fbff33adbd4e0be3dd9105cd6086c9b8 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sun, 6 Apr 2025 21:19:39 -0300 Subject: [PATCH 26/41] Add trio resource semantics to ring pubsub --- tractor/ipc/_ringbuf/_pubsub.py | 179 ++++++++++++++++++++------------ 1 file changed, 113 insertions(+), 66 deletions(-) diff --git a/tractor/ipc/_ringbuf/_pubsub.py b/tractor/ipc/_ringbuf/_pubsub.py index 4d5e0d20..37d54308 100644 --- a/tractor/ipc/_ringbuf/_pubsub.py +++ b/tractor/ipc/_ringbuf/_pubsub.py @@ -58,7 +58,7 @@ class ChannelInfo: cancel_scope: trio.CancelScope -class ChannelManager(trio.abc.AsyncResource, Generic[ChannelType]): +class ChannelManager(Generic[ChannelType]): ''' Helper for managing channel resources and their handler tasks with cancellation, add or remove channels dynamically! @@ -89,18 +89,15 @@ class ChannelManager(trio.abc.AsyncResource, Generic[ChannelType]): # methods that modify self._channels should be ordered by FIFO self._lock = trio.StrictFIFOLock() - @acm - async def maybe_lock(self): - ''' - If lock is not held, acquire + self._is_closed: bool = True - ''' - if self._lock.locked(): - yield - return + @property + def closed(self) -> bool: + return self._is_closed - async with self._lock: - yield + @property + def lock(self) -> trio.StrictFIFOLock: + return self._lock @property def channels(self) -> list[ChannelInfo]: @@ -139,7 +136,7 @@ class ChannelManager(trio.abc.AsyncResource, Generic[ChannelType]): with cancel_scope: await self._channel_task(info) - await self._maybe_destroy_channel(name) + self._maybe_destroy_channel(name) def _find_channel(self, name: str) -> tuple[int, ChannelInfo] | None: ''' @@ -156,25 +153,27 @@ class ChannelManager(trio.abc.AsyncResource, Generic[ChannelType]): return None - async def _maybe_destroy_channel(self, name: str): + def _maybe_destroy_channel(self, name: str): ''' If channel exists cancel its scope and remove from internal _channels list. ''' - async with self.maybe_lock(): - maybe_entry = self._find_channel(name) - if maybe_entry: - i, info = maybe_entry - info.cancel_scope.cancel() - del self._channels[i] + maybe_entry = self._find_channel(name) + if maybe_entry: + i, info = maybe_entry + info.cancel_scope.cancel() + del self._channels[i] async def add_channel(self, name: str, **kwargs): ''' Add a new channel to be handled ''' - async with self.maybe_lock(): + if self.closed: + raise trio.ClosedResourceError + + async with self._lock: await self._n.start(partial( self._channel_handler_task, name, @@ -186,8 +185,11 @@ class ChannelManager(trio.abc.AsyncResource, Generic[ChannelType]): Remove a channel and stop its handling ''' - async with self.maybe_lock(): - await self._maybe_destroy_channel(name) + if self.closed: + raise trio.ClosedResourceError + + async with self._lock: + self._maybe_destroy_channel(name) # if that was last channel reset connect event if len(self) == 0: @@ -198,6 +200,9 @@ class ChannelManager(trio.abc.AsyncResource, Generic[ChannelType]): Wait until at least one channel added ''' + if self.closed: + raise trio.ClosedResourceError + await self._connect_event.wait() self._connect_event = trio.Event() @@ -212,10 +217,18 @@ class ChannelManager(trio.abc.AsyncResource, Generic[ChannelType]): raise KeyError(f'Channel {name} not found!') - async def aclose(self) -> None: - async with self.maybe_lock(): - for info in self._channels: - await self.remove_channel(info.name) + def open(self): + self._is_closed = False + + async def close(self) -> None: + if self.closed: + log.warning('tried to close ChannelManager but its already closed...') + return + + for info in self._channels: + await self.remove_channel(info.name) + + self._is_closed = True ''' @@ -264,6 +277,12 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): self._next_turn: int = 0 + self._is_closed: bool = True + + @property + def closed(self) -> bool: + return self._is_closed + @property def batch_size(self) -> int: return self._batch_size @@ -289,36 +308,10 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): name: str, must_exist: bool = False ): - ''' - Store additional runtime info for channel and add channel to underlying - ChannelManager - - ''' await self._chanmngr.add_channel(name, must_exist=must_exist) async def remove_channel(self, name: str): - ''' - Send EOF to channel (does `channel.flush` also) then remove from - `ChannelManager` acquire both `self._send_lock` and - `self._chanmngr.maybe_lock()` in order to ensure no channel - modifications or sends happen concurrenty - ''' - async with self._chanmngr.maybe_lock(): - # ensure all pending messages are sent - info = self.get_channel(name) - - try: - while True: - msg = info.channel.rchan.receive_nowait() - await info.channel.ring.send(msg) - - except trio.WouldBlock: - await info.channel.ring.flush() - - await info.channel.schan.aclose() - - # finally remove from ChannelManager - await self._chanmngr.remove_channel(name) + await self._chanmngr.remove_channel(name) @acm async def _open_channel( @@ -345,6 +338,13 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): schan=schan, rchan=rchan ) + try: + while True: + msg = rchan.receive_nowait() + await ring.send(msg) + + except trio.WouldBlock: + ... async def _channel_task(self, info: ChannelInfo) -> None: ''' @@ -352,8 +352,12 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): payloads update event then drain all into send channel. ''' - async for msg in info.channel.rchan: - await info.channel.ring.send(msg) + try: + async for msg in info.channel.rchan: + await info.channel.ring.send(msg) + + except trio.Cancelled: + ... async def send(self, msg: bytes): ''' @@ -365,6 +369,12 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): variables dont happen out of order. ''' + if self.closed: + raise trio.ClosedResourceError + + if self._send_lock.locked(): + raise trio.BusyResourceError + async with self._send_lock: # wait at least one decoder connected if len(self.channels) == 0: @@ -378,8 +388,23 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): self._next_turn += 1 + async def flush(self, new_batch_size: int | None = None): + async with self._chanmngr.lock: + for info in self.channels: + await info.channel.ring.flush(new_batch_size=new_batch_size) + + async def __aenter__(self): + self._chanmngr.open() + self._is_closed = False + return self + async def aclose(self) -> None: - await self._chanmngr.aclose() + if self.closed: + log.warning('tried to close RingBufferPublisher but its already closed...') + return + + await self._chanmngr.close() + self._is_closed = True @acm @@ -445,6 +470,14 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): self._schan, self._rchan = trio.open_memory_channel(0) + self._is_closed: bool = True + + self._receive_lock = trio.StrictFIFOLock() + + @property + def closed(self) -> bool: + return self._is_closed + @property def channels(self) -> list[ChannelInfo]: return self._chanmngr.channels @@ -453,17 +486,9 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): return self._chanmngr[name] async def add_channel(self, name: str, must_exist: bool = False): - ''' - Add new input channel by name - - ''' await self._chanmngr.add_channel(name, must_exist=must_exist) async def remove_channel(self, name: str): - ''' - Remove an input channel by name - - ''' await self._chanmngr.remove_channel(name) @acm @@ -506,14 +531,36 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): except trio.EndOfChannel: break + except trio.ClosedResourceError: + break + async def receive(self) -> bytes: ''' Receive next in order msg ''' - return await self._rchan.receive() + if self.closed: + raise trio.ClosedResourceError + + if self._receive_lock.locked(): + raise trio.BusyResourceError + + async with self._receive_lock: + return await self._rchan.receive() + + async def __aenter__(self): + self._is_closed = False + self._chanmngr.open() + return self async def aclose(self) -> None: - await self._chanmngr.aclose() + if self.closed: + log.warning('tried to close RingBufferSubscriber but its already closed...') + return + + await self._chanmngr.close() + await self._schan.aclose() + await self._rchan.aclose() + self._is_closed = True @acm async def open_ringbuf_subscriber( -- 2.34.1 From b2f6c298f5ca23452cb688fca847ec7f6ca046c1 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sun, 6 Apr 2025 21:59:14 -0300 Subject: [PATCH 27/41] Refactor generate_sample_messages to be a generator and use numpy --- tests/test_ringbuf.py | 49 ++++++++----- tractor/_testing/samples.py | 135 ++++++++++++++++++++---------------- 2 files changed, 108 insertions(+), 76 deletions(-) diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index 6dfeae72..8f549b7c 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -14,7 +14,7 @@ from tractor.ipc._ringbuf import ( ) from tractor._testing.samples import ( generate_single_byte_msgs, - generate_sample_messages + RandomBytesGenerator ) # in case you don't want to melt your cores, uncomment dis! @@ -80,18 +80,22 @@ async def child_write_shm( Attach to ringbuf and send all generated messages. ''' - sent_hash, msgs, _total_bytes = generate_sample_messages( + rng = RandomBytesGenerator( msg_amount, rand_min=rand_min, rand_max=rand_max, ) - await ctx.started(sent_hash) + await ctx.started() print('writer started') async with attach_to_ringbuf_sender(token, cleanup=False) as sender: - for msg in msgs: + for msg in rng: await sender.send(msg) + if rng.msgs_generated % rng.recommended_log_interval == 0: + print(f'wrote {rng.total_msgs} msgs') + print('writer exit') + return rng.hexdigest @pytest.mark.parametrize( @@ -153,12 +157,14 @@ def test_ringbuf( msg_amount=msg_amount, rand_min=rand_min, rand_max=rand_max, - ) as (_sctx, sent_hash), + ) as (sctx, _), + recv_p.open_context( child_read_shm, token=token, - ) as (rctx, _sent), + ) as (rctx, _), ): + sent_hash = await sctx.result() recvd_hash = await rctx.result() assert sent_hash == recvd_hash @@ -300,7 +306,7 @@ async def child_channel_sender( token_out: RBToken ): import random - _hash, msgs, _total_bytes = generate_sample_messages( + rng = RandomBytesGenerator( random.randint(msg_amount_min, msg_amount_max), rand_min=256, rand_max=1024, @@ -309,10 +315,14 @@ async def child_channel_sender( token_in, token_out ) as chan: - await ctx.started(msgs) - for msg in msgs: + await ctx.started() + for msg in rng: await chan.send(msg) + await chan.send(b'bye') + await chan.receive() + return rng.hexdigest + def test_channel(): @@ -327,7 +337,7 @@ def test_channel(): attach_to_ringbuf_channel(send_token, recv_token) as chan, tractor.open_nursery() as an ): - recv_p = await an.start_actor( + sender = await an.start_actor( 'test_ringbuf_transport_sender', enable_modules=[__name__], proc_kwargs={ @@ -335,19 +345,26 @@ def test_channel(): } ) async with ( - recv_p.open_context( + sender.open_context( child_channel_sender, msg_amount_min=msg_amount_min, msg_amount_max=msg_amount_max, token_in=recv_token, token_out=send_token - ) as (ctx, msgs), + ) as (ctx, _), ): - recv_msgs = [] + recvd_hash = hashlib.sha256() async for msg in chan: - recv_msgs.append(msg) + if msg == b'bye': + await chan.send(b'bye') + break - await recv_p.cancel_actor() - assert recv_msgs == msgs + recvd_hash.update(msg) + + sent_hash = await ctx.result() + + assert recvd_hash.hexdigest() == sent_hash + + await an.cancel() trio.run(main) diff --git a/tractor/_testing/samples.py b/tractor/_testing/samples.py index f8671332..fcf41dfa 100644 --- a/tractor/_testing/samples.py +++ b/tractor/_testing/samples.py @@ -1,84 +1,99 @@ -import os -import random import hashlib +import numpy as np def generate_single_byte_msgs(amount: int) -> bytes: ''' - Generate a byte instance of len `amount` with: - - ``` - byte_at_index(i) = (i % 10).encode() - ``` - - this results in constantly repeating sequences of: - - b'0123456789' + Generate a byte instance of length `amount` with repeating ASCII digits 0..9. ''' - return b''.join(str(i % 10).encode() for i in range(amount)) + # array [0, 1, 2, ..., amount-1], take mod 10 => [0..9], and map 0->'0'(48) + # up to 9->'9'(57). + arr = np.arange(amount, dtype=np.uint8) % 10 + # move into ascii space + arr += 48 + return arr.tobytes() -def generate_sample_messages( - amount: int, - rand_min: int = 0, - rand_max: int = 0, - silent: bool = False, -) -> tuple[str, list[bytes], int]: +class RandomBytesGenerator: ''' Generate bytes msgs for tests. - Messages will have the following format: + messages will have the following format: - ``` - b'[{i:08}]' + os.urandom(random.randint(rand_min, rand_max)) - ``` + b'[{i:08}]' + random_bytes so for message index 25: - b'[00000025]' + random_bytes + b'[00000025]' + random_bytes + + also generates sha256 hash of msgs. ''' - msgs = [] - size = 0 - log_interval = None - if not silent: - print(f'\ngenerating {amount} messages...') + def __init__( + self, + amount: int, + rand_min: int = 0, + rand_max: int = 0 + ): + if rand_max < rand_min: + raise ValueError('rand_max must be >= rand_min') - # calculate an apropiate log interval based on - # max message size - max_msg_size = 10 + rand_max + self._amount = amount + self._rand_min = rand_min + self._rand_max = rand_max + self._index = 0 + self._hasher = hashlib.sha256() + self._total_bytes = 0 + + self._lengths = np.random.randint( + rand_min, + rand_max + 1, + size=amount, + dtype=np.int32 + ) + + def __iter__(self): + return self + + def __next__(self) -> bytes: + if self._index == self._amount: + raise StopIteration + + header = f'[{self._index:08}]'.encode('utf-8') + + length = int(self._lengths[self._index]) + msg = header + np.random.bytes(length) + + self._hasher.update(msg) + self._total_bytes += length + self._index += 1 + + return msg + + @property + def hexdigest(self) -> str: + return self._hasher.hexdigest() + + @property + def total_bytes(self) -> int: + return self._total_bytes + + @property + def total_msgs(self) -> int: + return self._amount + + @property + def msgs_generated(self) -> int: + return self._index + + @property + def recommended_log_interval(self) -> int: + max_msg_size = 10 + self._rand_max if max_msg_size <= 32 * 1024: - log_interval = 10_000 + return 10_000 else: - log_interval = 1000 - - payload_hash = hashlib.sha256() - 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) - - payload_hash.update(msg) - msgs.append(msg) - - if ( - not silent - and - i > 0 - and - i % log_interval == 0 - ): - print(f'{i} generated') - - if not silent: - print(f'done, {size:,} bytes in total') - - return payload_hash.hexdigest(), msgs, size + return 1000 -- 2.34.1 From e4868ded54aff412bae6f635148ece97e2406dd4 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 10 Apr 2025 11:27:52 -0300 Subject: [PATCH 28/41] Tweaks to make cancellation happen correctly on ringbuf receiver & fix test log msg --- tests/test_ringbuf.py | 2 +- tractor/ipc/_ringbuf/__init__.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index 8f549b7c..2befeafc 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -92,7 +92,7 @@ async def child_write_shm( await sender.send(msg) if rng.msgs_generated % rng.recommended_log_interval == 0: - print(f'wrote {rng.total_msgs} msgs') + print(f'wrote {rng.msgs_generated} msgs') print('writer exit') return rng.hexdigest diff --git a/tractor/ipc/_ringbuf/__init__.py b/tractor/ipc/_ringbuf/__init__.py index 2b2b549e..2c74b88e 100644 --- a/tractor/ipc/_ringbuf/__init__.py +++ b/tractor/ipc/_ringbuf/__init__.py @@ -533,7 +533,7 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): self._write_ptr += delta # yield lock and re-enter - except EFDReadCancelled: + except (EFDReadCancelled, trio.Cancelled): # while waiting for new data `self._write_event` was closed try: # if eof was signaled receive no wait will not raise @@ -654,7 +654,7 @@ async def attach_to_ringbuf_receiver( Launches `receiver._eof_monitor_task` in a `trio.Nursery`. ''' async with ( - trio.open_nursery() as n, + trio.open_nursery(strict_exception_groups=False) as n, RingBufferReceiveChannel( token, cleanup=cleanup -- 2.34.1 From 5d6fa643ba15a45f994cebf2bda593247567e095 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 10 Apr 2025 13:13:08 -0300 Subject: [PATCH 29/41] Better APIs for ringd and pubsub Pubsub: Remove un-necesary ChannelManager locking mechanism Make ChannelManager.close wait for all channel removals Make publisher turn switch configurable with `msgs_per_turn` variable Fix batch_size setter on publisher Add broadcast to publisher Add endpoints on pubsub for remote actors to dynamically add and remove channels Ringd: Add fifo lock and use it on methods that modify _rings state Add comments Break up ringd.open_ringbuf apis into attach_, open_ & maybe_open_ When attaching its no longer a long running context, only on opens Adapt ringd test to new apis --- tests/test_ringd.py | 21 +- tractor/ipc/_ringbuf/_pubsub.py | 465 ++++++++++++++++++++++++-------- tractor/ipc/_ringbuf/_ringd.py | 364 ++++++++++++++++++------- 3 files changed, 617 insertions(+), 233 deletions(-) diff --git a/tests/test_ringd.py b/tests/test_ringd.py index 4b5c792e..e08b7c1c 100644 --- a/tests/test_ringd.py +++ b/tests/test_ringd.py @@ -1,5 +1,3 @@ -from contextlib import asynccontextmanager as acm - import trio import tractor import msgspec @@ -40,7 +38,7 @@ async def send_child( ring_name: str ): async with ( - ringd.open_ringbuf(ring_name) as token, + ringd.attach_ringbuf(ring_name) as token, attach_to_ringbuf_sender(token) as chan, ): @@ -63,9 +61,7 @@ def test_ringd(): async with ( tractor.open_nursery() as an, - ringd.open_ringd( - loglevel='info' - ) + ringd.open_ringd() ): recv_portal = await an.start_actor( 'recv', @@ -133,10 +129,10 @@ async def subscriber_child(ctx: tractor.Context): msg = msgspec.msgpack.decode(msg, type=ControlMessages) match msg: case AddChannelMsg(): - await subs.add_channel(msg.name, must_exist=False) + await subs.add_channel(msg.name) case RemoveChannelMsg(): - await subs.remove_channel(msg.name) + subs.remove_channel(msg.name) case RangeMsg(): range_msg = msg @@ -171,7 +167,7 @@ async def subscriber_child(ctx: tractor.Context): async def publisher_child(ctx: tractor.Context): await ctx.started() async with ( - open_ringbuf_publisher(batch_size=100, guarantee_order=True) as pub, + open_ringbuf_publisher(guarantee_order=True) as pub, ctx.open_stream() as stream ): async for msg in stream: @@ -181,7 +177,7 @@ async def publisher_child(ctx: tractor.Context): await pub.add_channel(msg.name, must_exist=True) case RemoveChannelMsg(): - await pub.remove_channel(msg.name) + pub.remove_channel(msg.name) case RangeMsg(): for i in range(msg.size): @@ -258,11 +254,6 @@ def test_pubsub(): await send_range(100) await remove_channel(ring_name) - # try using same ring name - await add_channel(ring_name) - await send_range(100) - await remove_channel(ring_name) - # multi chan test ring_names = [] for i in range(3): diff --git a/tractor/ipc/_ringbuf/_pubsub.py b/tractor/ipc/_ringbuf/_pubsub.py index 37d54308..e2575ab6 100644 --- a/tractor/ipc/_ringbuf/_pubsub.py +++ b/tractor/ipc/_ringbuf/_pubsub.py @@ -86,19 +86,14 @@ class ChannelManager(Generic[ChannelType]): # store channel runtime variables self._channels: list[ChannelInfo] = [] - # methods that modify self._channels should be ordered by FIFO - self._lock = trio.StrictFIFOLock() - self._is_closed: bool = True + self._teardown = trio.Event() + @property def closed(self) -> bool: return self._is_closed - @property - def lock(self) -> trio.StrictFIFOLock: - return self._lock - @property def channels(self) -> list[ChannelInfo]: return self._channels @@ -106,8 +101,8 @@ class ChannelManager(Generic[ChannelType]): async def _channel_handler_task( self, name: str, - task_status: trio.TASK_STATUS_IGNORED, - **kwargs + must_exist: bool = False, + task_status=trio.TASK_STATUS_IGNORED, ): ''' Open channel resources, add to internal data structures, signal channel @@ -119,7 +114,7 @@ class ChannelManager(Generic[ChannelType]): kwargs are proxied to `self._open_channel` acm. ''' - async with self._open_channel(name, **kwargs) as chan: + async with self._open_channel(name, must_exist=must_exist) as chan: cancel_scope = trio.CancelScope() info = ChannelInfo( name=name, @@ -138,6 +133,9 @@ class ChannelManager(Generic[ChannelType]): self._maybe_destroy_channel(name) + if len(self) == 0: + self._teardown.set() + def _find_channel(self, name: str) -> tuple[int, ChannelInfo] | None: ''' Given a channel name maybe return its index and value from @@ -165,7 +163,7 @@ class ChannelManager(Generic[ChannelType]): info.cancel_scope.cancel() del self._channels[i] - async def add_channel(self, name: str, **kwargs): + async def add_channel(self, name: str, must_exist: bool = False): ''' Add a new channel to be handled @@ -173,14 +171,13 @@ class ChannelManager(Generic[ChannelType]): if self.closed: raise trio.ClosedResourceError - async with self._lock: - await self._n.start(partial( - self._channel_handler_task, - name, - **kwargs - )) + await self._n.start(partial( + self._channel_handler_task, + name, + must_exist=must_exist + )) - async def remove_channel(self, name: str): + def remove_channel(self, name: str): ''' Remove a channel and stop its handling @@ -188,12 +185,11 @@ class ChannelManager(Generic[ChannelType]): if self.closed: raise trio.ClosedResourceError - async with self._lock: - self._maybe_destroy_channel(name) + self._maybe_destroy_channel(name) - # if that was last channel reset connect event - if len(self) == 0: - self._connect_event = trio.Event() + # if that was last channel reset connect event + if len(self) == 0: + self._connect_event = trio.Event() async def wait_for_channel(self): ''' @@ -226,7 +222,18 @@ class ChannelManager(Generic[ChannelType]): return for info in self._channels: - await self.remove_channel(info.name) + if info.channel.closed: + continue + + self.remove_channel(info.name) + + try: + await self._teardown.wait() + + except trio.Cancelled: + # log.exception('close was cancelled') + raise + self._is_closed = True @@ -236,12 +243,6 @@ Ring buffer publisher & subscribe pattern mediated by `ringd` actor. ''' -@dataclass -class PublisherChannels: - ring: RingBufferSendChannel - schan: trio.MemorySendChannel - rchan: trio.MemoryReceiveChannel - class RingBufferPublisher(trio.abc.SendChannel[bytes]): ''' @@ -259,24 +260,32 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): # new ringbufs created will have this buf_size buf_size: int = 10 * 1024, + # amount of msgs to each ring before switching turns + msgs_per_turn: int = 1, + # global batch size for all channels batch_size: int = 1 ): self._buf_size = buf_size self._batch_size: int = batch_size + self.msgs_per_turn = msgs_per_turn - self._chanmngr = ChannelManager[PublisherChannels]( + # helper to manage acms + long running tasks + self._chanmngr = ChannelManager[RingBufferSendChannel]( n, self._open_channel, self._channel_task ) - # methods that send data over the channels need to be acquire send lock - # in order to guarantee order of operations + # ensure no concurrent `.send()` calls self._send_lock = trio.StrictFIFOLock() + # index of channel to be used for next send self._next_turn: int = 0 - + # amount of messages sent this turn + self._turn_msgs: int = 0 + # have we closed this publisher? + # set to `False` on `.__aenter__()` self._is_closed: bool = True @property @@ -288,14 +297,31 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): return self._batch_size @batch_size.setter - def set_batch_size(self, value: int) -> None: + def batch_size(self, value: int) -> None: for info in self.channels: - info.channel.ring.batch_size = value + info.channel.batch_size = value @property def channels(self) -> list[ChannelInfo]: return self._chanmngr.channels + def _get_next_turn(self) -> int: + ''' + Maybe switch turn and reset self._turn_msgs or just increment it. + Return current turn + ''' + if self._turn_msgs == self.msgs_per_turn: + self._turn_msgs = 0 + self._next_turn += 1 + + if self._next_turn >= len(self.channels): + self._next_turn = 0 + + else: + self._turn_msgs += 1 + + return self._next_turn + def get_channel(self, name: str) -> ChannelInfo: ''' Get underlying ChannelInfo from name @@ -310,8 +336,8 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): ): await self._chanmngr.add_channel(name, must_exist=must_exist) - async def remove_channel(self, name: str): - await self._chanmngr.remove_channel(name) + def remove_channel(self, name: str): + self._chanmngr.remove_channel(name) @acm async def _open_channel( @@ -320,41 +346,45 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): name: str, must_exist: bool = False - ) -> AsyncContextManager[PublisherChannels]: + ) -> AsyncContextManager[RingBufferSendChannel]: ''' Open a ringbuf through `ringd` and attach as send side ''' - async with ( - ringd.open_ringbuf( - name=name, - buf_size=self._buf_size, - must_exist=must_exist, - ) as token, - attach_to_ringbuf_sender(token) as ring, - ): - schan, rchan = trio.open_memory_channel(0) - yield PublisherChannels( - ring=ring, - schan=schan, - rchan=rchan - ) - try: - while True: - msg = rchan.receive_nowait() - await ring.send(msg) + if must_exist: + ringd_fn = ringd.attach_ringbuf + kwargs = {} - except trio.WouldBlock: - ... + else: + ringd_fn = ringd.open_ringbuf + kwargs = {'buf_size': self._buf_size} + + async with ( + ringd_fn( + name=name, + **kwargs + ) as token, + + attach_to_ringbuf_sender( + token, + batch_size=self._batch_size + ) as ring, + ): + yield ring + # try: + # # ensure all messages are sent + # await ring.flush() + + # except Exception as e: + # e.add_note(f'while closing ringbuf send channel {name}') + # log.exception(e) async def _channel_task(self, info: ChannelInfo) -> None: ''' - Forever get current runtime info for channel, wait on its next pending - payloads update event then drain all into send channel. + Wait forever until channel cancellation ''' try: - async for msg in info.channel.rchan: - await info.channel.ring.send(msg) + await trio.sleep_forever() except trio.Cancelled: ... @@ -362,11 +392,9 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): async def send(self, msg: bytes): ''' If no output channels connected, wait until one, then fetch the next - channel based on turn, add the indexed payload and update - `self._next_turn` & `self._next_index`. + channel based on turn. - Needs to acquire `self._send_lock` to make sure updates to turn & index - variables dont happen out of order. + Needs to acquire `self._send_lock` to ensure no concurrent calls. ''' if self.closed: @@ -380,18 +408,28 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): if len(self.channels) == 0: await self._chanmngr.wait_for_channel() - if self._next_turn >= len(self.channels): - self._next_turn = 0 + turn = self._get_next_turn() - info = self.channels[self._next_turn] - await info.channel.schan.send(msg) + info = self.channels[turn] + await info.channel.send(msg) - self._next_turn += 1 + async def broadcast(self, msg: bytes): + ''' + Send a msg to all channels, if no channels connected, does nothing. + ''' + if self.closed: + raise trio.ClosedResourceError + + for info in self.channels: + await info.channel.send(msg) async def flush(self, new_batch_size: int | None = None): - async with self._chanmngr.lock: - for info in self.channels: - await info.channel.ring.flush(new_batch_size=new_batch_size) + for info in self.channels: + try: + await info.channel.flush(new_batch_size=new_batch_size) + + except trio.ClosedResourceError: + ... async def __aenter__(self): self._chanmngr.open() @@ -403,41 +441,12 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): log.warning('tried to close RingBufferPublisher but its already closed...') return - await self._chanmngr.close() + with trio.CancelScope(shield=True): + await self._chanmngr.close() + self._is_closed = True -@acm -async def open_ringbuf_publisher( - - buf_size: int = 10 * 1024, - batch_size: int = 1, - guarantee_order: bool = False, - force_cancel: bool = False - -) -> AsyncContextManager[RingBufferPublisher]: - ''' - Open a new ringbuf publisher - - ''' - async with ( - trio.open_nursery() as n, - RingBufferPublisher( - n, - buf_size=buf_size, - batch_size=batch_size - ) as publisher - ): - if guarantee_order: - order_send_channel(publisher) - - yield publisher - - if force_cancel: - # implicitly cancel any running channel handler task - n.cancel_scope.cancel() - - class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): ''' Use ChannelManager to create a multi ringbuf receiver that can @@ -458,10 +467,15 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): self, n: trio.Nursery, + # new ringbufs created will have this buf_size + buf_size: int = 10 * 1024, + # if connecting to a publisher that has already sent messages set # to the next expected payload index this subscriber will receive start_index: int = 0 ): + self._buf_size = buf_size + self._chanmngr = ChannelManager[RingBufferReceiveChannel]( n, self._open_channel, @@ -488,8 +502,8 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): async def add_channel(self, name: str, must_exist: bool = False): await self._chanmngr.add_channel(name, must_exist=must_exist) - async def remove_channel(self, name: str): - await self._chanmngr.remove_channel(name) + def remove_channel(self, name: str): + self._chanmngr.remove_channel(name) @acm async def _open_channel( @@ -502,11 +516,20 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): ''' Open a ringbuf through `ringd` and attach as receiver side ''' + if must_exist: + ringd_fn = ringd.attach_ringbuf + kwargs = {} + + else: + ringd_fn = ringd.open_ringbuf + kwargs = {'buf_size': self._buf_size} + async with ( - ringd.open_ringbuf( + ringd_fn( name=name, - must_exist=must_exist, + **kwargs ) as token, + attach_to_ringbuf_receiver(token) as chan ): yield chan @@ -554,7 +577,6 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): async def aclose(self) -> None: if self.closed: - log.warning('tried to close RingBufferSubscriber but its already closed...') return await self._chanmngr.close() @@ -562,26 +584,241 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): await self._rchan.aclose() self._is_closed = True + +''' +Actor module for managing publisher & subscriber channels remotely through +`tractor.context` rpc +''' + +_publisher: RingBufferPublisher | None = None +_subscriber: RingBufferSubscriber | None = None + + +def set_publisher(pub: RingBufferPublisher): + global _publisher + + if _publisher: + raise RuntimeError( + f'publisher already set on {tractor.current_actor()}' + ) + + _publisher = pub + + +def set_subscriber(sub: RingBufferSubscriber): + global _subscriber + + if _subscriber: + raise RuntimeError( + f'subscriber already set on {tractor.current_actor()}' + ) + + _subscriber = sub + + +def get_publisher() -> RingBufferPublisher: + global _publisher + + if not _publisher: + raise RuntimeError( + f'{tractor.current_actor()} tried to get publisher' + 'but it\'s not set' + ) + + return _publisher + + +def get_subscriber() -> RingBufferSubscriber: + global _subscriber + + if not _subscriber: + raise RuntimeError( + f'{tractor.current_actor()} tried to get subscriber' + 'but it\'s not set' + ) + + return _subscriber + + +@tractor.context +async def open_pub_channel( + ctx: tractor.Context, + ring_name: str, + must_exist: bool = False +): + publisher = get_publisher() + await publisher.add_channel( + ring_name, + must_exist=must_exist + ) + + await ctx.started() + + try: + await trio.sleep_forever() + + finally: + try: + publisher.remove_channel(ring_name) + + except trio.ClosedResourceError: + ... + + +@acm +async def open_pub_channel_at( + actor_name: str, + ring_name: str, + must_exist: bool = False +): + async with ( + tractor.find_actor(actor_name) as portal, + portal.open_context( + open_pub_channel, + ring_name=ring_name, + must_exist=must_exist + ) as (ctx, _) + ): + yield + await ctx.cancel() + + +@tractor.context +async def open_sub_channel( + ctx: tractor.Context, + ring_name: str, + must_exist: bool = False +): + subscriber = get_subscriber() + await subscriber.add_channel( + ring_name, + must_exist=must_exist + ) + + await ctx.started() + + try: + await trio.sleep_forever() + + finally: + try: + subscriber.remove_channel(ring_name) + + except trio.ClosedResourceError: + ... + + +@acm +async def open_sub_channel_at( + actor_name: str, + ring_name: str, + must_exist: bool = False +): + async with ( + tractor.find_actor(actor_name) as portal, + portal.open_context( + open_sub_channel, + ring_name=ring_name, + must_exist=must_exist + ) as (ctx, _) + ): + yield + await ctx.cancel() + + +''' +High level helpers to open publisher & subscriber +''' + + +@acm +async def open_ringbuf_publisher( + # buf size for created rings + buf_size: int = 10 * 1024, + + # global batch size for channels + batch_size: int = 1, + + # messages before changing output channel + msgs_per_turn: int = 1, + + # ensure subscriber receives in same order publisher sent + # causes it to use wrapped payloads which contain the og + # index + guarantee_order: bool = False, + + # explicit nursery cancel call on cleanup + force_cancel: bool = False, + + # on creation, set the `_publisher` global in order to use the provided + # tractor.context & helper utils for adding and removing new channels from + # remote actors + set_module_var: bool = True + +) -> AsyncContextManager[RingBufferPublisher]: + ''' + Open a new ringbuf publisher + + ''' + async with ( + trio.open_nursery(strict_exception_groups=False) as n, + RingBufferPublisher( + n, + buf_size=buf_size, + batch_size=batch_size + ) as publisher + ): + if guarantee_order: + order_send_channel(publisher) + + if set_module_var: + set_publisher(publisher) + + try: + yield publisher + + finally: + if force_cancel: + # implicitly cancel any running channel handler task + n.cancel_scope.cancel() + + @acm async def open_ringbuf_subscriber( + # buf size for created rings + buf_size: int = 10 * 1024, + # expect indexed payloads and unwrap them in order guarantee_order: bool = False, - force_cancel: bool = False + # explicit nursery cancel call on cleanup + force_cancel: bool = False, + + # on creation, set the `_subscriber` global in order to use the provided + # tractor.context & helper utils for adding and removing new channels from + # remote actors + set_module_var: bool = True ) -> AsyncContextManager[RingBufferPublisher]: ''' Open a new ringbuf subscriber ''' async with ( - trio.open_nursery() as n, + trio.open_nursery(strict_exception_groups=False) as n, RingBufferSubscriber( n, + buf_size=buf_size ) as subscriber ): + # maybe monkey patch `.receive` to use indexed payloads if guarantee_order: order_receive_channel(subscriber) + # maybe set global module var for remote actor channel updates + if set_module_var: + global _subscriber + set_subscriber(subscriber) + yield subscriber if force_cancel: diff --git a/tractor/ipc/_ringbuf/_ringd.py b/tractor/ipc/_ringbuf/_ringd.py index 24c3e530..51818e34 100644 --- a/tractor/ipc/_ringbuf/_ringd.py +++ b/tractor/ipc/_ringbuf/_ringd.py @@ -25,6 +25,7 @@ open_ringbuf acm, will automatically contact ringd. ''' import os import tempfile +from typing import AsyncContextManager from pathlib import Path from contextlib import ( asynccontextmanager as acm @@ -33,111 +34,118 @@ from dataclasses import dataclass import trio import tractor -from tractor.linux import send_fds, recv_fds +from tractor.linux import ( + send_fds, + recv_fds, +) import tractor.ipc._ringbuf as ringbuf from tractor.ipc._ringbuf import RBToken log = tractor.log.get_logger(__name__) -# log = tractor.log.get_console_log(level='info') -class RingNotFound(Exception): - ... +''' +Daemon implementation + +''' -_ringd_actor_name = 'ringd' -_root_key = _ringd_actor_name + f'-{os.getpid()}' +_ringd_actor_name: str = 'ringd' + + +_root_name: str = f'{_ringd_actor_name}-{os.getpid()}' + + +def _make_ring_name(name: str) -> str: + ''' + User provided ring names will be prefixed by the ringd actor name and pid. + ''' + return f'{_root_name}.{name}' @dataclass class RingInfo: token: RBToken creator: str - unlink: trio.Event() _rings: dict[str, RingInfo] = {} +_ring_lock = trio.StrictFIFOLock() def _maybe_get_ring(name: str) -> RingInfo | None: - if name in _rings: - return _rings[name] + ''' + Maybe return RingInfo for a given name str - return None + ''' + # if full name was passed, strip root name + if _root_name in name: + name = name.replace(f'{_root_name}.', '') + + return _rings.get(name, None) + + +def _get_ring(name: str) -> RingInfo: + ''' + Return a RingInfo for a given name or raise + ''' + info = _maybe_get_ring(name) + + if not info: + raise RuntimeError(f'Ring \"{name}\" not found!') + + return info def _insert_ring(name: str, info: RingInfo): + ''' + Add a new ring + ''' + if name in _rings: + raise RuntimeError(f'A ring with name {name} already exists!') + _rings[name] = info def _destroy_ring(name: str): + ''' + Delete information about a ring + ''' + if name not in _rings: + raise RuntimeError(f'Tried to delete non existant {name} ring!') + del _rings[name] -async def _attach_to_ring( - ringd_pid: int, - ring_name: str -) -> RBToken: - actor = tractor.current_actor() - - fd_amount = 3 - sock_path = str( - Path(tempfile.gettempdir()) - / - f'ringd-{ringd_pid}-{ring_name}-to-{actor.name}.sock' - ) - - log.info(f'trying to attach to ring {ring_name}...') - - async with ( - tractor.find_actor(_ringd_actor_name) as ringd, - ringd.open_context( - _pass_fds, - name=ring_name, - sock_path=sock_path - ) as (ctx, token), - ): - fds = await recv_fds(sock_path, fd_amount) - log.info( - f'received fds: {fds}' - ) - - token = RBToken.from_msg(token) - - write, wrap, eof = fds - - return RBToken( - shm_name=token.shm_name, - write_eventfd=write, - wrap_eventfd=wrap, - eof_eventfd=eof, - buf_size=token.buf_size - ) - - @tractor.context async def _pass_fds( ctx: tractor.Context, name: str, sock_path: str ): - global _rings - info = _maybe_get_ring(name) + ''' + Ringd endpoint to request passing fds of a ring. - if not info: - raise RingNotFound(f'Ring \"{name}\" not found!') + Supports passing fullname or not (ringd actor name and pid before ring + name). - token = info.token + See `_attach_to_ring` function for usage. + ''' + async with _ring_lock: + # get ring fds or raise error + token = _get_ring(name).token - async with send_fds(token.fds, sock_path): - log.info(f'connected to {sock_path} for fd passing') - await ctx.started(token) + # start fd passing context using socket on `sock_path` + async with send_fds(token.fds, sock_path): + log.info(f'connected to {sock_path} for fd passing') + # use started to signal socket is ready and send token in order for + # client to get extra info like buf_size + await ctx.started(token) + # send_fds will block until receive side acks - log.info(f'fds {token.fds} sent') - - return token + log.info(f'ring {name} fds: {token.fds}, sent') @tractor.context @@ -145,60 +153,105 @@ async def _open_ringbuf( ctx: tractor.Context, caller: str, name: str, - buf_size: int = 10 * 1024, - must_exist: bool = False, + buf_size: int = 10 * 1024 ): - global _root_key, _rings - log.info(f'maybe open ring {name} from {caller}, must_exist = {must_exist}') + ''' + Ringd endpoint to create and allocate resources for a new ring. - info = _maybe_get_ring(name) + ''' + await _ring_lock.acquire() + maybe_info = _maybe_get_ring(name) - if info: - log.info(f'ring {name} exists, {caller} attached') - - await ctx.started(os.getpid()) - - async with ctx.open_stream() as stream: - await stream.receive() - - info.unlink.set() - - log.info(f'{caller} detached from ring {name}') - - return - - if must_exist: - raise RingNotFound( - f'Tried to open_ringbuf but it doesn\'t exist: {name}' + if maybe_info: + raise RuntimeError( + f'Tried to create ringbuf but it already exists: {name}' ) + fullname = _make_ring_name(name) + with ringbuf.open_ringbuf( - _root_key + name, + fullname, buf_size=buf_size ) as token: - unlink_event = trio.Event() + _insert_ring( name, RingInfo( token=token, creator=caller, - unlink=unlink_event, ) ) - log.info(f'ring {name} created by {caller}') - await ctx.started(os.getpid()) - async with ctx.open_stream() as stream: - await stream.receive() + _ring_lock.release() - await unlink_event.wait() - _destroy_ring(name) + # yield full ring name to rebuild token after fd passing + await ctx.started(fullname) - log.info(f'ring {name} destroyed by {caller}') + # await ctx cancel to remove ring from tracking and cleanup + try: + log.info(f'ring {name} created by {caller}') + await trio.sleep_forever() + + finally: + _destroy_ring(name) + + log.info(f'ring {name} destroyed by {caller}') + + +@tractor.context +async def _attach_ringbuf( + ctx: tractor.Context, + caller: str, + name: str +) -> str: + ''' + Ringd endpoint to "attach" to an existing ring, this just ensures ring + actually exists and returns its full name. + ''' + async with _ring_lock: + info = _maybe_get_ring(name) + + if not info: + raise RuntimeError( + f'{caller} tried to open_ringbuf but it doesn\'t exist: {name}' + ) + + await ctx.started() + + # return full ring name to rebuild token after fd passing + return info.token.shm_name + + +@tractor.context +async def _maybe_open_ringbuf( + ctx: tractor.Context, + caller: str, + name: str, + buf_size: int = 10 * 1024, +): + ''' + If ring already exists attach, if not create it. + ''' + maybe_info = _maybe_get_ring(name) + + if maybe_info: + return await _attach_ringbuf(ctx, caller, name) + + return await _open_ringbuf(ctx, caller, name, buf_size=buf_size) + + +''' +Ringd client side helpers + +''' @acm async def open_ringd(**kwargs) -> tractor.Portal: + ''' + Spawn new ringd actor. + + ''' async with tractor.open_nursery(**kwargs) as an: portal = await an.start_actor( _ringd_actor_name, @@ -210,21 +263,69 @@ async def open_ringd(**kwargs) -> tractor.Portal: @acm async def wait_for_ringd() -> tractor.Portal: + ''' + Wait for ringd actor to be up. + + ''' async with tractor.wait_for_actor( _ringd_actor_name ) as portal: yield portal +async def _request_ring_fds( + fullname: str +) -> RBToken: + ''' + Private helper to fetch ring fds from ringd actor. + ''' + actor = tractor.current_actor() + + fd_amount = 3 + sock_path = str( + Path(tempfile.gettempdir()) + / + f'{fullname}-to-{actor.name}.sock' + ) + + log.info(f'trying to attach to {fullname}...') + + async with ( + tractor.find_actor(_ringd_actor_name) as ringd, + + ringd.open_context( + _pass_fds, + name=fullname, + sock_path=sock_path + ) as (ctx, token), + ): + fds = await recv_fds(sock_path, fd_amount) + write, wrap, eof = fds + log.info( + f'received fds, write: {write}, wrap: {wrap}, eof: {eof}' + ) + + token = RBToken.from_msg(token) + + return RBToken( + shm_name=fullname, + write_eventfd=write, + wrap_eventfd=wrap, + eof_eventfd=eof, + buf_size=token.buf_size + ) + + + @acm async def open_ringbuf( - name: str, buf_size: int = 10 * 1024, +) -> AsyncContextManager[RBToken]: + ''' + Create a new ring and retrieve its fds. - must_exist: bool = False, - -) -> RBToken: + ''' actor = tractor.current_actor() async with ( wait_for_ringd() as ringd, @@ -234,12 +335,67 @@ async def open_ringbuf( caller=actor.name, name=name, buf_size=buf_size, - must_exist=must_exist - ) as (rd_ctx, ringd_pid), - - rd_ctx.open_stream() as _stream, + ) as (ctx, fullname), ): - token = await _attach_to_ring(ringd_pid, name) - log.info(f'attached to {token}') + token = await _request_ring_fds(fullname) + log.info(f'{actor.name} opened {token}') + try: + yield token + + finally: + with trio.CancelScope(shield=True): + await ctx.cancel() + + +@acm +async def attach_ringbuf( + name: str, +) -> AsyncContextManager[RBToken]: + ''' + Attach to an existing ring and retreive its fds. + + ''' + actor = tractor.current_actor() + async with ( + wait_for_ringd() as ringd, + + ringd.open_context( + _attach_ringbuf, + caller=actor.name, + name=name, + ) as (ctx, _), + ): + fullname = await ctx.wait_for_result() + token = await _request_ring_fds(fullname) + log.info(f'{actor.name} attached {token}') yield token + +@acm +async def maybe_open_ringbuf( + name: str, + buf_size: int = 10 * 1024, +) -> AsyncContextManager[RBToken]: + ''' + Attach or create a ring and retreive its fds. + + ''' + actor = tractor.current_actor() + async with ( + wait_for_ringd() as ringd, + + ringd.open_context( + _maybe_open_ringbuf, + caller=actor.name, + name=name, + buf_size=buf_size, + ) as (ctx, fullname), + ): + token = await _request_ring_fds(fullname) + log.info(f'{actor.name} opened {token}') + try: + yield token + + finally: + with trio.CancelScope(shield=True): + await ctx.cancel() -- 2.34.1 From 39dccbdde757eb94a84366459259f8b0b6d0ddd2 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sun, 13 Apr 2025 13:31:44 -0300 Subject: [PATCH 30/41] Add owner semantics to RBToken Stop exporting `_ringbuf` on `tractor.ipc` Use absolute imports on `_ringbuf` module Add more comments and acm helpers for ringbuf allocation functions Create generic FD sharing actor module in `tractor.linux._fdshare` Include original allocator actor name as `owner` in RBToken Auto share FDs of allocated ringbufs On `attach_ringbuf_*` functions request fds from owner Adapt all ringbuf tests to new system --- tests/test_ringbuf.py | 280 ++++++++++++++++--------------- tractor/ipc/__init__.py | 19 --- tractor/ipc/_ringbuf/__init__.py | 232 ++++++++++++++++++++----- tractor/linux/__init__.py | 4 - tractor/linux/_fdshare.py | 129 ++++++++++++++ 5 files changed, 463 insertions(+), 201 deletions(-) diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index 2befeafc..3e6a5734 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -7,6 +7,7 @@ import pytest import tractor from tractor.ipc._ringbuf import ( open_ringbuf, + open_ringbuf_pair, attach_to_ringbuf_receiver, attach_to_ringbuf_sender, attach_to_ringbuf_channel, @@ -68,7 +69,7 @@ async def child_write_shm( msg_amount: int, rand_min: int, rand_max: int, - token: RBToken, + buf_size: int ) -> None: ''' Sub-actor used in `test_ringbuf` @@ -85,9 +86,12 @@ async def child_write_shm( rand_min=rand_min, rand_max=rand_max, ) - await ctx.started() - print('writer started') - async with attach_to_ringbuf_sender(token, cleanup=False) as sender: + async with ( + open_ringbuf('test_ringbuf', buf_size=buf_size) as token, + attach_to_ringbuf_sender(token) as sender + ): + await ctx.started(token) + print('writer started') for msg in rng: await sender.send(msg) @@ -133,55 +137,53 @@ def test_ringbuf( ''' async def main(): - with open_ringbuf( - 'test_ringbuf', - buf_size=buf_size - ) as token: - proc_kwargs = {'pass_fds': token.fds} + async with tractor.open_nursery() as an: + send_p = await an.start_actor( + 'ring_sender', + enable_modules=[ + __name__, + 'tractor.linux._fdshare' + ], + ) + recv_p = await an.start_actor( + 'ring_receiver', + enable_modules=[ + __name__, + 'tractor.linux._fdshare' + ], + ) + async with ( + send_p.open_context( + child_write_shm, + msg_amount=msg_amount, + rand_min=rand_min, + rand_max=rand_max, + buf_size=buf_size + ) as (sctx, token), - 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, - token=token, - msg_amount=msg_amount, - rand_min=rand_min, - rand_max=rand_max, - ) as (sctx, _), + recv_p.open_context( + child_read_shm, + token=token, + ) as (rctx, _), + ): + sent_hash = await sctx.result() + recvd_hash = await rctx.result() - recv_p.open_context( - child_read_shm, - token=token, - ) as (rctx, _), - ): - sent_hash = await sctx.result() - recvd_hash = await rctx.result() + assert sent_hash == recvd_hash - assert sent_hash == recvd_hash - - await send_p.cancel_actor() - await recv_p.cancel_actor() + await an.cancel() trio.run(main) @tractor.context -async def child_blocked_receiver( - ctx: tractor.Context, - token: RBToken -): - async with attach_to_ringbuf_receiver(token) as receiver: - await ctx.started() +async def child_blocked_receiver(ctx: tractor.Context): + async with ( + open_ringbuf('test_ring_cancel_reader') as token, + + attach_to_ringbuf_receiver(token) as receiver + ): + await ctx.started(token) await receiver.receive_some() @@ -192,26 +194,23 @@ def test_reader_cancel(): ''' async def main(): - with open_ringbuf('test_ring_cancel_reader') as token: + async with tractor.open_nursery() as an: + recv_p = await an.start_actor( + 'ring_blocked_receiver', + enable_modules=[ + __name__, + 'tractor.linux._fdshare' + ], + ) async with ( - tractor.open_nursery() as an, - attach_to_ringbuf_sender(token) as _sender, + recv_p.open_context( + child_blocked_receiver, + ) as (sctx, token), + + attach_to_ringbuf_sender(token), ): - recv_p = await an.start_actor( - 'ring_blocked_receiver', - enable_modules=[__name__], - proc_kwargs={ - 'pass_fds': token.fds - } - ) - async with ( - recv_p.open_context( - child_blocked_receiver, - token=token - ) as (sctx, _sent), - ): - await trio.sleep(1) - await an.cancel() + await trio.sleep(.1) + await an.cancel() with pytest.raises(tractor._exceptions.ContextCancelled): @@ -219,12 +218,16 @@ def test_reader_cancel(): @tractor.context -async def child_blocked_sender( - ctx: tractor.Context, - token: RBToken -): - async with attach_to_ringbuf_sender(token) as sender: - await ctx.started() +async def child_blocked_sender(ctx: tractor.Context): + async with ( + open_ringbuf( + 'test_ring_cancel_sender', + buf_size=1 + ) as token, + + attach_to_ringbuf_sender(token) as sender + ): + await ctx.started(token) await sender.send_all(b'this will wrap') @@ -235,26 +238,23 @@ def test_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.fds - } - ) - async with ( - recv_p.open_context( - child_blocked_sender, - token=token - ) as (sctx, _sent), - ): - await trio.sleep(1) - await an.cancel() + async with tractor.open_nursery() as an: + recv_p = await an.start_actor( + 'ring_blocked_sender', + enable_modules=[ + __name__, + 'tractor.linux._fdshare' + ], + ) + async with ( + recv_p.open_context( + child_blocked_sender, + ) as (sctx, token), + + attach_to_ringbuf_receiver(token) + ): + await trio.sleep(.1) + await an.cancel() with pytest.raises(tractor._exceptions.ContextCancelled): @@ -274,24 +274,28 @@ def test_receiver_max_bytes(): msgs = [] async def main(): - with open_ringbuf( - 'test_ringbuf_max_bytes', - buf_size=10 - ) as token: - async with ( - trio.open_nursery() as n, - attach_to_ringbuf_sender(token, cleanup=False) as sender, - attach_to_ringbuf_receiver(token, cleanup=False) as receiver - ): - async def _send_and_close(): - await sender.send_all(msg) - await sender.aclose() + async with ( + tractor.open_nursery(), + open_ringbuf( + 'test_ringbuf_max_bytes', + buf_size=10 + ) as token, - n.start_soon(_send_and_close) - while len(msgs) < len(msg): - msg_part = await receiver.receive_some(max_bytes=1) - assert len(msg_part) == 1 - msgs.append(msg_part) + trio.open_nursery() as n, + + attach_to_ringbuf_sender(token, cleanup=False) as sender, + + attach_to_ringbuf_receiver(token, cleanup=False) as receiver + ): + async def _send_and_close(): + await sender.send_all(msg) + await sender.aclose() + + n.start_soon(_send_and_close) + while len(msgs) < len(msg): + msg_part = await receiver.receive_some(max_bytes=1) + assert len(msg_part) == 1 + msgs.append(msg_part) trio.run(main) assert msg == b''.join(msgs) @@ -329,42 +333,46 @@ def test_channel(): msg_amount_min = 100 msg_amount_max = 1000 + mods = [ + __name__, + 'tractor.linux._fdshare' + ] + async def main(): - with tractor.ipc.open_ringbuf_pair( - 'test_ringbuf_transport' - ) as (send_token, recv_token): + async with ( + tractor.open_nursery(enable_modules=mods) as an, + + open_ringbuf_pair( + 'test_ringbuf_transport' + ) as (send_token, recv_token), + + attach_to_ringbuf_channel(send_token, recv_token) as chan, + ): + sender = await an.start_actor( + 'test_ringbuf_transport_sender', + enable_modules=mods, + ) async with ( - attach_to_ringbuf_channel(send_token, recv_token) as chan, - tractor.open_nursery() as an + sender.open_context( + child_channel_sender, + msg_amount_min=msg_amount_min, + msg_amount_max=msg_amount_max, + token_in=recv_token, + token_out=send_token + ) as (ctx, _), ): - sender = await an.start_actor( - 'test_ringbuf_transport_sender', - enable_modules=[__name__], - proc_kwargs={ - 'pass_fds': send_token.fds + recv_token.fds - } - ) - async with ( - sender.open_context( - child_channel_sender, - msg_amount_min=msg_amount_min, - msg_amount_max=msg_amount_max, - token_in=recv_token, - token_out=send_token - ) as (ctx, _), - ): - recvd_hash = hashlib.sha256() - async for msg in chan: - if msg == b'bye': - await chan.send(b'bye') - break + recvd_hash = hashlib.sha256() + async for msg in chan: + if msg == b'bye': + await chan.send(b'bye') + break - recvd_hash.update(msg) + recvd_hash.update(msg) - sent_hash = await ctx.result() + sent_hash = await ctx.result() - assert recvd_hash.hexdigest() == sent_hash + assert recvd_hash.hexdigest() == sent_hash - await an.cancel() + await an.cancel() trio.run(main) diff --git a/tractor/ipc/__init__.py b/tractor/ipc/__init__.py index f2f42839..37c3c8ed 100644 --- a/tractor/ipc/__init__.py +++ b/tractor/ipc/__init__.py @@ -17,26 +17,7 @@ A modular IPC layer supporting the power of cross-process SC! ''' -import platform - from ._chan import ( _connect_chan as _connect_chan, Channel as Channel ) - -if platform.system() == 'Linux': - from ._ringbuf import ( - RBToken as RBToken, - - open_ringbuf as open_ringbuf, - open_ringbuf_pair as open_ringbuf_pair, - - RingBufferSendChannel as RingBufferSendChannel, - attach_to_ringbuf_sender as attach_to_ringbuf_sender, - - RingBufferReceiveChannel as RingBufferReceiveChannel, - attach_to_ringbuf_receiver as attach_to_ringbuf_receiver, - - RingBufferChannel as RingBufferChannel, - attach_to_ringbuf_channel as attach_to_ringbuf_channel, - ) diff --git a/tractor/ipc/_ringbuf/__init__.py b/tractor/ipc/_ringbuf/__init__.py index 2c74b88e..0d96d1a3 100644 --- a/tractor/ipc/_ringbuf/__init__.py +++ b/tractor/ipc/_ringbuf/__init__.py @@ -35,16 +35,22 @@ from msgspec import ( to_builtins ) -from ...log import get_logger -from ..._exceptions import ( +from tractor.log import get_logger +from tractor._exceptions import ( InternalError ) -from .._mp_bs import disable_mantracker -from ...linux.eventfd import ( +from tractor.ipc._mp_bs import disable_mantracker +from tractor.linux._fdshare import ( + share_fds, + unshare_fds, + request_fds_from +) +from tractor.linux.eventfd import ( open_eventfd, EFDReadCancelled, EventFD ) +from tractor._state import current_actor log = get_logger(__name__) @@ -57,17 +63,19 @@ _DEFAULT_RB_SIZE = 10 * 1024 class RBToken(Struct, frozen=True): ''' - RingBuffer token contains necesary info to open the three - eventfds and the shared memory + RingBuffer token contains necesary info to open resources of a ringbuf, + even in the case that ringbuf was not allocated by current actor. ''' + owner: str # if owner != `current_actor().name` we must use fdshare + shm_name: str write_eventfd: int # used to signal writer ptr advance wrap_eventfd: int # used to signal reader ready after wrap around eof_eventfd: int # used to signal writer closed - buf_size: int + buf_size: int # size in bytes of underlying shared memory buffer def as_msg(self): return to_builtins(self) @@ -81,10 +89,6 @@ class RBToken(Struct, frozen=True): @property def fds(self) -> tuple[int, int, int]: - ''' - Useful for `pass_fds` params - - ''' return ( self.write_eventfd, self.wrap_eventfd, @@ -92,38 +96,137 @@ class RBToken(Struct, frozen=True): ) -@cm -def open_ringbuf( +def alloc_ringbuf( shm_name: str, buf_size: int = _DEFAULT_RB_SIZE, -) -> ContextManager[RBToken]: +) -> tuple[SharedMemory, RBToken]: ''' - Handle resources for a ringbuf (shm, eventfd), yield `RBToken` to - be used with `attach_to_ringbuf_sender` and `attach_to_ringbuf_receiver` - + Allocate OS resources for a ringbuf. ''' shm = SharedMemory( name=shm_name, size=buf_size, create=True ) - try: - token = RBToken( - shm_name=shm_name, - write_eventfd=open_eventfd(), - wrap_eventfd=open_eventfd(), - eof_eventfd=open_eventfd(), - buf_size=buf_size - ) - yield token - - finally: - shm.unlink() + token = RBToken( + owner=current_actor().name, + shm_name=shm_name, + write_eventfd=open_eventfd(), + wrap_eventfd=open_eventfd(), + eof_eventfd=open_eventfd(), + buf_size=buf_size + ) + # register fds for sharing + share_fds( + shm_name, + token.fds, + ) + return shm, token @cm -def open_ringbuf_pair( - name: str, +def open_ringbuf_sync( + shm_name: str, + buf_size: int = _DEFAULT_RB_SIZE, +) -> ContextManager[RBToken]: + ''' + Handle resources for a ringbuf (shm, eventfd), yield `RBToken` to + be used with `attach_to_ringbuf_sender` and `attach_to_ringbuf_receiver`, + post yield maybe unshare fds and unlink shared memory + + ''' + shm: SharedMemory | None = None + token: RBToken | None = None + try: + shm, token = alloc_ringbuf(shm_name, buf_size=buf_size) + yield token + + finally: + if token: + unshare_fds(shm_name) + + if shm: + shm.unlink() + +@acm +async def open_ringbuf( + shm_name: str, + buf_size: int = _DEFAULT_RB_SIZE, +) -> AsyncContextManager[RBToken]: + ''' + Helper to use `open_ringbuf_sync` inside an async with block. + + ''' + with open_ringbuf_sync( + shm_name, + buf_size=buf_size + ) as token: + yield token + + +@cm +def open_ringbufs_sync( + shm_names: list[str], + buf_sizes: int | list[str] = _DEFAULT_RB_SIZE, +) -> ContextManager[tuple[RBToken]]: + ''' + Handle resources for multiple ringbufs at once. + + ''' + # maybe convert single int into list + if isinstance(buf_sizes, int): + buf_size = [buf_sizes] * len(shm_names) + + # ensure len(shm_names) == len(buf_sizes) + if ( + isinstance(buf_sizes, list) + and + len(buf_sizes) != len(shm_names) + ): + raise ValueError( + 'Expected buf_size list to be same length as shm_names' + ) + + # allocate resources + rings: list[tuple[SharedMemory, RBToken]] = [ + alloc_ringbuf(shm_name, buf_size=buf_size) + for shm_name, buf_size in zip(shm_names, buf_size) + ] + + try: + yield tuple([token for _, token in rings]) + + finally: + # attempt fd unshare and shm unlink for each + for shm, token in rings: + try: + unshare_fds(token.shm_name) + + except RuntimeError: + log.exception(f'while unsharing fds of {token}') + + shm.unlink() + + +@acm +async def open_ringbufs( + shm_names: list[str], + buf_sizes: int | list[str] = _DEFAULT_RB_SIZE, +) -> AsyncContextManager[tuple[RBToken]]: + ''' + Helper to use `open_ringbufs_sync` inside an async with block. + + ''' + with open_ringbufs_sync( + shm_names, + buf_sizes=buf_sizes + ) as tokens: + yield tokens + + +@cm +def open_ringbuf_pair_sync( + shm_name: str, buf_size: int = _DEFAULT_RB_SIZE ) -> ContextManager[tuple(RBToken, RBToken)]: ''' @@ -131,18 +234,30 @@ def open_ringbuf_pair( bidirectional messaging. ''' - with ( - open_ringbuf( - name + '.send', - buf_size=buf_size - ) as send_token, + with open_ringbufs_sync( + [ + f'{shm_name}.send', + f'{shm_name}.recv' + ], + buf_sizes=buf_size + ) as tokens: + yield tokens - open_ringbuf( - name + '.recv', - buf_size=buf_size - ) as recv_token - ): - yield send_token, recv_token + +@acm +async def open_ringbuf_pair( + shm_name: str, + buf_size: int = _DEFAULT_RB_SIZE +) -> AsyncContextManager[tuple[RBToken, RBToken]]: + ''' + Helper to use `open_ringbuf_pair_sync` inside an async with block. + + ''' + with open_ringbuf_pair_sync( + shm_name, + buf_size=buf_size + ) as tokens: + yield tokens Buffer = bytes | bytearray | memoryview @@ -640,6 +755,29 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): return self +async def _maybe_obtain_shared_resources(token: RBToken): + token = RBToken.from_msg(token) + + # maybe token wasn't allocated by current actor + if token.owner != current_actor().name: + # use fdshare module to retrieve a copy of the FDs + fds = await request_fds_from( + token.owner, + token.shm_name + ) + write, wrap, eof = fds + # rebuild token using FDs copies + token = RBToken( + owner=token.owner, + shm_name=token.shm_name, + write_eventfd=write, + wrap_eventfd=wrap, + eof_eventfd=eof, + buf_size=token.buf_size + ) + + return token + @acm async def attach_to_ringbuf_receiver( @@ -651,8 +789,13 @@ async def attach_to_ringbuf_receiver( Attach a RingBufferReceiveChannel from a previously opened RBToken. + Requires tractor runtime to be up in order to support opening a ringbuf + originally allocated by a different actor. + Launches `receiver._eof_monitor_task` in a `trio.Nursery`. ''' + token = await _maybe_obtain_shared_resources(token) + async with ( trio.open_nursery(strict_exception_groups=False) as n, RingBufferReceiveChannel( @@ -676,7 +819,12 @@ async def attach_to_ringbuf_sender( Attach a RingBufferSendChannel from a previously opened RBToken. + Requires tractor runtime to be up in order to support opening a ringbuf + originally allocated by a different actor. + ''' + token = await _maybe_obtain_shared_resources(token) + async with RingBufferSendChannel( token, batch_size=batch_size, diff --git a/tractor/linux/__init__.py b/tractor/linux/__init__.py index 211a0040..33526d14 100644 --- a/tractor/linux/__init__.py +++ b/tractor/linux/__init__.py @@ -13,7 +13,3 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . -from ._fdshare import ( - send_fds as send_fds, - recv_fds as recv_fds -) diff --git a/tractor/linux/_fdshare.py b/tractor/linux/_fdshare.py index 16a748b8..bc2385ef 100644 --- a/tractor/linux/_fdshare.py +++ b/tractor/linux/_fdshare.py @@ -21,13 +21,19 @@ https://github.com/python/cpython/blob/275056a7fdcbe36aaac494b4183ae59943a338eb/ ''' import os import array +import tempfile +from pathlib import Path from typing import AsyncContextManager from contextlib import asynccontextmanager as acm import trio +import tractor from trio import socket +log = tractor.log.get_logger(__name__) + + class FDSharingError(Exception): ... @@ -157,3 +163,126 @@ async def recv_fds(sock_path: str, amount: int) -> tuple: ) return tuple(a) + + +''' +Share FD actor module + +Add "tractor.linux._fdshare" to enabled modules on actors to allow sharing of +FDs with other actors. + +Use `share_fds` function to register a set of fds with a name, then other +actors can use `request_fds_from` function to retrieve the fds. + +Use `unshare_fds` to disable sharing of a set of FDs. + +''' + +FDType = tuple[int] + +_fds: dict[str, FDType] = {} + + +def maybe_get_fds(name: str) -> FDType | None: + ''' + Get registered FDs with a given name or return None + + ''' + return _fds.get(name, None) + + +def get_fds(name: str) -> FDType: + ''' + Get registered FDs with a given name or raise + ''' + fds = maybe_get_fds(name) + + if not fds: + raise RuntimeError(f'No FDs with name {name} found!') + + return fds + + +def share_fds( + name: str, + fds: tuple[int], +) -> None: + ''' + Register a set of fds to be shared under a given name. + + ''' + maybe_fds = maybe_get_fds(name) + if maybe_fds: + raise RuntimeError(f'share FDs: {maybe_fds} already tied to name {name}') + + _fds[name] = fds + + +def unshare_fds(name: str) -> None: + ''' + Unregister a set of fds to disable sharing them. + + ''' + get_fds(name) # raise if not exists + + del _fds[name] + + +@tractor.context +async def _pass_fds( + ctx: tractor.Context, + name: str, + sock_path: str +) -> None: + ''' + Endpoint to request a set of FDs from current actor, will use `ctx.started` + to send original FDs, then `send_fds` will block until remote side finishes + the `recv_fds` call. + + ''' + # get fds or raise error + fds = get_fds(name) + + # start fd passing context using socket on `sock_path` + async with send_fds(fds, sock_path): + # send original fds through ctx.started + await ctx.started(fds) + + +async def request_fds_from( + actor_name: str, + fds_name: str +) -> FDType: + ''' + Use this function to retreive shared FDs from `actor_name`. + + ''' + this_actor = tractor.current_actor() + + # create a temporary path for the UDS sock + sock_path = str( + Path(tempfile.gettempdir()) + / + f'{fds_name}-from-{actor_name}-to-{this_actor.name}.sock' + ) + + async with ( + tractor.find_actor(actor_name) as portal, + + portal.open_context( + _pass_fds, + name=fds_name, + sock_path=sock_path + ) as (ctx, fds_info), + ): + # get original FDs + og_fds = fds_info + + # retrieve copies of FDs + fds = await recv_fds(sock_path, len(og_fds)) + + log.info( + f'{this_actor.name} received fds: {og_fds} -> {fds}' + ) + + return fds -- 2.34.1 From f5513ba0059ef4a7504ff18ec0c36b6411366d76 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sun, 13 Apr 2025 13:36:02 -0300 Subject: [PATCH 31/41] Adapt ringbuf pubsub to new RBToken owner system --- tests/test_ring_pubsub.py | 182 +++++++++++++++ tests/test_ringd.py | 272 ---------------------- tractor/ipc/_ringbuf/_pubsub.py | 330 ++++++++++++-------------- tractor/ipc/_ringbuf/_ringd.py | 401 -------------------------------- 4 files changed, 336 insertions(+), 849 deletions(-) create mode 100644 tests/test_ring_pubsub.py delete mode 100644 tests/test_ringd.py delete mode 100644 tractor/ipc/_ringbuf/_ringd.py diff --git a/tests/test_ring_pubsub.py b/tests/test_ring_pubsub.py new file mode 100644 index 00000000..3bdbeb0a --- /dev/null +++ b/tests/test_ring_pubsub.py @@ -0,0 +1,182 @@ +from typing import AsyncContextManager +from contextlib import asynccontextmanager as acm + +import trio +import pytest +import tractor + +from tractor.trionics import gather_contexts + +from tractor.ipc._ringbuf import open_ringbufs +from tractor.ipc._ringbuf._pubsub import ( + open_ringbuf_publisher, + open_ringbuf_subscriber, + get_publisher, + get_subscriber, + open_pub_channel_at, + open_sub_channel_at +) + + +log = tractor.log.get_console_log(level='info') + + +@tractor.context +async def publish_range( + ctx: tractor.Context, + size: int +): + pub = get_publisher() + await ctx.started() + for i in range(size): + await pub.send(i.to_bytes(4)) + log.info(f'sent {i}') + + await pub.flush() + + log.info('range done') + + +@tractor.context +async def subscribe_range( + ctx: tractor.Context, + size: int +): + sub = get_subscriber() + await ctx.started() + + for i in range(size): + recv = int.from_bytes(await sub.receive()) + if recv != i: + raise AssertionError( + f'received: {recv} expected: {i}' + ) + + log.info(f'received: {recv}') + + log.info('range done') + + +@tractor.context +async def subscriber_child(ctx: tractor.Context): + try: + async with open_ringbuf_subscriber(guarantee_order=True): + await ctx.started() + await trio.sleep_forever() + + finally: + log.info('subscriber exit') + + +@tractor.context +async def publisher_child( + ctx: tractor.Context, + batch_size: int +): + try: + async with open_ringbuf_publisher( + guarantee_order=True, + batch_size=batch_size + ): + await ctx.started() + await trio.sleep_forever() + + finally: + log.info('publisher exit') + + +@acm +async def open_pubsub_test_actors( + + ring_names: list[str], + size: int, + batch_size: int + +) -> AsyncContextManager[tuple[tractor.Portal, tractor.Portal]]: + + with trio.fail_after(5): + async with tractor.open_nursery( + enable_modules=[ + 'tractor.linux._fdshare' + ] + ) as an: + modules = [ + __name__, + 'tractor.linux._fdshare', + 'tractor.ipc._ringbuf._pubsub' + ] + sub_portal = await an.start_actor( + 'sub', + enable_modules=modules + ) + pub_portal = await an.start_actor( + 'pub', + enable_modules=modules + ) + + async with ( + sub_portal.open_context(subscriber_child) as (long_rctx, _), + pub_portal.open_context( + publisher_child, + batch_size=batch_size + ) as (long_sctx, _), + ): + with open_ringbufs(ring_names) as tokens: + async with ( + gather_contexts([ + open_sub_channel_at('sub', ring) + for ring in tokens + ]), + gather_contexts([ + open_pub_channel_at('pub', ring) + for ring in tokens + ]), + sub_portal.open_context(subscribe_range, size=size) as (rctx, _), + pub_portal.open_context(publish_range, size=size) as (sctx, _) + ): + yield + + await long_sctx.cancel() + await long_rctx.cancel() + + await an.cancel() + + +@pytest.mark.parametrize( + ('ring_names', 'size', 'batch_size'), + [ + ( + ['ring-first'], + 100, + 1 + ), + ( + ['ring-first'], + 69, + 1 + ), + ( + [f'multi-ring-{i}' for i in range(3)], + 1000, + 100 + ), + ], + ids=[ + 'simple', + 'redo-simple', + 'multi-ring', + ] +) +def test_pubsub( + request, + ring_names: list[str], + size: int, + batch_size: int +): + async def main(): + async with open_pubsub_test_actors( + ring_names, size, batch_size + ): + ... + + trio.run(main) diff --git a/tests/test_ringd.py b/tests/test_ringd.py deleted file mode 100644 index e08b7c1c..00000000 --- a/tests/test_ringd.py +++ /dev/null @@ -1,272 +0,0 @@ -import trio -import tractor -import msgspec - -from tractor.ipc import ( - attach_to_ringbuf_receiver, - attach_to_ringbuf_sender -) -from tractor.ipc._ringbuf._pubsub import ( - open_ringbuf_publisher, - open_ringbuf_subscriber -) - -import tractor.ipc._ringbuf._ringd as ringd - - -log = tractor.log.get_console_log(level='info') - - -@tractor.context -async def recv_child( - ctx: tractor.Context, - ring_name: str -): - async with ( - ringd.open_ringbuf(ring_name) as token, - - attach_to_ringbuf_receiver(token) as chan, - ): - await ctx.started() - async for msg in chan: - log.info(f'received {int.from_bytes(msg)}') - - -@tractor.context -async def send_child( - ctx: tractor.Context, - ring_name: str -): - async with ( - ringd.attach_ringbuf(ring_name) as token, - - attach_to_ringbuf_sender(token) as chan, - ): - await ctx.started() - for i in range(100): - await chan.send(i.to_bytes(4)) - log.info(f'sent {i}') - - - -def test_ringd(): - ''' - Spawn ringd actor and two childs that access same ringbuf through ringd. - - Both will use `ringd.open_ringbuf` to allocate the ringbuf, then attach to - them as sender and receiver. - - ''' - async def main(): - async with ( - tractor.open_nursery() as an, - - ringd.open_ringd() - ): - recv_portal = await an.start_actor( - 'recv', - enable_modules=[__name__] - ) - send_portal = await an.start_actor( - 'send', - enable_modules=[__name__] - ) - - async with ( - recv_portal.open_context( - recv_child, - ring_name='ring' - ) as (rctx, _), - - send_portal.open_context( - send_child, - ring_name='ring' - ) as (sctx, _), - ): - ... - - await an.cancel() - - trio.run(main) - - -class Struct(msgspec.Struct): - - def encode(self) -> bytes: - return msgspec.msgpack.encode(self) - - -class AddChannelMsg(Struct, frozen=True, tag=True): - name: str - - -class RemoveChannelMsg(Struct, frozen=True, tag=True): - name: str - - -class RangeMsg(Struct, frozen=True, tag=True): - size: int - - -ControlMessages = AddChannelMsg | RemoveChannelMsg | RangeMsg - - -@tractor.context -async def subscriber_child(ctx: tractor.Context): - await ctx.started() - async with ( - open_ringbuf_subscriber(guarantee_order=True) as subs, - trio.open_nursery() as n, - ctx.open_stream() as stream - ): - range_msg = None - range_event = trio.Event() - range_scope = trio.CancelScope() - - async def _control_listen_task(): - nonlocal range_msg, range_event - async for msg in stream: - msg = msgspec.msgpack.decode(msg, type=ControlMessages) - match msg: - case AddChannelMsg(): - await subs.add_channel(msg.name) - - case RemoveChannelMsg(): - subs.remove_channel(msg.name) - - case RangeMsg(): - range_msg = msg - range_event.set() - - await stream.send(b'ack') - - range_scope.cancel() - - n.start_soon(_control_listen_task) - - with range_scope: - while True: - await range_event.wait() - range_event = trio.Event() - for i in range(range_msg.size): - recv = int.from_bytes(await subs.receive()) - if recv != i: - raise AssertionError( - f'received: {recv} expected: {i}' - ) - - log.info(f'received: {recv}') - - await stream.send(b'valid range') - log.info('finished range') - - log.info('subscriber exit') - - -@tractor.context -async def publisher_child(ctx: tractor.Context): - await ctx.started() - async with ( - open_ringbuf_publisher(guarantee_order=True) as pub, - ctx.open_stream() as stream - ): - async for msg in stream: - msg = msgspec.msgpack.decode(msg, type=ControlMessages) - match msg: - case AddChannelMsg(): - await pub.add_channel(msg.name, must_exist=True) - - case RemoveChannelMsg(): - pub.remove_channel(msg.name) - - case RangeMsg(): - for i in range(msg.size): - await pub.send(i.to_bytes(4)) - log.info(f'sent {i}') - - await stream.send(b'ack') - - log.info('publisher exit') - - - -def test_pubsub(): - ''' - Spawn to childs a publisher and a subscriber, use context streams - to dynamically test different scenarios with different channel - configurations between them. - - ''' - async def main(): - async with ( - tractor.open_nursery( - loglevel='info', - # debug_mode=True, - # enable_stack_on_sig=True - ) as an, - - ringd.open_ringd() - ): - recv_portal = await an.start_actor( - 'recv', - enable_modules=[__name__] - ) - send_portal = await an.start_actor( - 'send', - enable_modules=[__name__] - ) - - async with ( - recv_portal.open_context(subscriber_child) as (rctx, _), - rctx.open_stream() as recv_stream, - send_portal.open_context(publisher_child) as (sctx, _), - sctx.open_stream() as send_stream, - ): - async def send_wait_ack(msg: bytes): - await recv_stream.send(msg) - ack = await recv_stream.receive() - assert ack == b'ack' - - await send_stream.send(msg) - ack = await send_stream.receive() - assert ack == b'ack' - - async def add_channel(name: str): - await send_wait_ack(AddChannelMsg(name=name).encode()) - - async def remove_channel(name: str): - await send_wait_ack(RemoveChannelMsg(name=name).encode()) - - async def send_range(size: int): - await send_wait_ack(RangeMsg(size=size).encode()) - range_ack = await recv_stream.receive() - assert range_ack == b'valid range' - - # simple test, open one channel and send 0..100 range - ring_name = 'ring-first' - await add_channel(ring_name) - await send_range(100) - await remove_channel(ring_name) - - # redo - ring_name = 'ring-redo' - await add_channel(ring_name) - await send_range(100) - await remove_channel(ring_name) - - # multi chan test - ring_names = [] - for i in range(3): - ring_names.append(f'multi-ring-{i}') - - for name in ring_names: - await add_channel(name) - - await send_range(1000) - - for name in ring_names: - await remove_channel(name) - - await an.cancel() - - trio.run(main) diff --git a/tractor/ipc/_ringbuf/_pubsub.py b/tractor/ipc/_ringbuf/_pubsub.py index e2575ab6..40adf611 100644 --- a/tractor/ipc/_ringbuf/_pubsub.py +++ b/tractor/ipc/_ringbuf/_pubsub.py @@ -31,7 +31,8 @@ from dataclasses import dataclass import trio import tractor -from tractor.ipc import ( +from tractor.ipc._ringbuf import ( + RBToken, RingBufferSendChannel, RingBufferReceiveChannel, attach_to_ringbuf_sender, @@ -42,7 +43,8 @@ from tractor.trionics import ( order_send_channel, order_receive_channel ) -import tractor.ipc._ringbuf._ringd as ringd + +import tractor.linux._fdshare as fdshare log = tractor.log.get_logger(__name__) @@ -53,9 +55,10 @@ ChannelType = TypeVar('ChannelType') @dataclass class ChannelInfo: - name: str + token: RBToken channel: ChannelType cancel_scope: trio.CancelScope + teardown: trio.Event class ChannelManager(Generic[ChannelType]): @@ -88,8 +91,6 @@ class ChannelManager(Generic[ChannelType]): self._is_closed: bool = True - self._teardown = trio.Event() - @property def closed(self) -> bool: return self._is_closed @@ -100,9 +101,9 @@ class ChannelManager(Generic[ChannelType]): async def _channel_handler_task( self, - name: str, - must_exist: bool = False, + token: RBToken, task_status=trio.TASK_STATUS_IGNORED, + **kwargs ): ''' Open channel resources, add to internal data structures, signal channel @@ -114,12 +115,16 @@ class ChannelManager(Generic[ChannelType]): kwargs are proxied to `self._open_channel` acm. ''' - async with self._open_channel(name, must_exist=must_exist) as chan: + async with self._open_channel( + token, + **kwargs + ) as chan: cancel_scope = trio.CancelScope() info = ChannelInfo( - name=name, + token=token, channel=chan, - cancel_scope=cancel_scope + cancel_scope=cancel_scope, + teardown=trio.Event() ) self._channels.append(info) @@ -131,10 +136,7 @@ class ChannelManager(Generic[ChannelType]): with cancel_scope: await self._channel_task(info) - self._maybe_destroy_channel(name) - - if len(self) == 0: - self._teardown.set() + self._maybe_destroy_channel(token.shm_name) def _find_channel(self, name: str) -> tuple[int, ChannelInfo] | None: ''' @@ -145,7 +147,7 @@ class ChannelManager(Generic[ChannelType]): ''' for entry in enumerate(self._channels): i, info = entry - if info.name == name: + if info.token.shm_name == name: return entry return None @@ -161,9 +163,14 @@ class ChannelManager(Generic[ChannelType]): if maybe_entry: i, info = maybe_entry info.cancel_scope.cancel() + info.teardown.set() del self._channels[i] - async def add_channel(self, name: str, must_exist: bool = False): + async def add_channel( + self, + token: RBToken, + **kwargs + ): ''' Add a new channel to be handled @@ -173,11 +180,11 @@ class ChannelManager(Generic[ChannelType]): await self._n.start(partial( self._channel_handler_task, - name, - must_exist=must_exist + RBToken.from_msg(token), + **kwargs )) - def remove_channel(self, name: str): + async def remove_channel(self, name: str): ''' Remove a channel and stop its handling @@ -185,8 +192,18 @@ class ChannelManager(Generic[ChannelType]): if self.closed: raise trio.ClosedResourceError + maybe_entry = self._find_channel(name) + if not maybe_entry: + # return + raise RuntimeError( + f'tried to remove channel {name} but if does not exist' + ) + + i, info = maybe_entry self._maybe_destroy_channel(name) + await info.teardown.wait() + # if that was last channel reset connect event if len(self) == 0: self._connect_event = trio.Event() @@ -225,15 +242,7 @@ class ChannelManager(Generic[ChannelType]): if info.channel.closed: continue - self.remove_channel(info.name) - - try: - await self._teardown.wait() - - except trio.Cancelled: - # log.exception('close was cancelled') - raise - + await self.remove_channel(info.name) self._is_closed = True @@ -257,16 +266,12 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): self, n: trio.Nursery, - # new ringbufs created will have this buf_size - buf_size: int = 10 * 1024, - # amount of msgs to each ring before switching turns msgs_per_turn: int = 1, # global batch size for all channels batch_size: int = 1 ): - self._buf_size = buf_size self._batch_size: int = batch_size self.msgs_per_turn = msgs_per_turn @@ -331,63 +336,32 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): async def add_channel( self, - name: str, - must_exist: bool = False + token: RBToken, ): - await self._chanmngr.add_channel(name, must_exist=must_exist) + await self._chanmngr.add_channel(token) - def remove_channel(self, name: str): - self._chanmngr.remove_channel(name) + async def remove_channel(self, name: str): + await self._chanmngr.remove_channel(name) @acm async def _open_channel( self, - name: str, - must_exist: bool = False + token: RBToken ) -> AsyncContextManager[RingBufferSendChannel]: - ''' - Open a ringbuf through `ringd` and attach as send side - ''' - if must_exist: - ringd_fn = ringd.attach_ringbuf - kwargs = {} - - else: - ringd_fn = ringd.open_ringbuf - kwargs = {'buf_size': self._buf_size} - - async with ( - ringd_fn( - name=name, - **kwargs - ) as token, - - attach_to_ringbuf_sender( - token, - batch_size=self._batch_size - ) as ring, - ): + async with attach_to_ringbuf_sender( + token, + batch_size=self._batch_size + ) as ring: yield ring - # try: - # # ensure all messages are sent - # await ring.flush() - - # except Exception as e: - # e.add_note(f'while closing ringbuf send channel {name}') - # log.exception(e) async def _channel_task(self, info: ChannelInfo) -> None: ''' Wait forever until channel cancellation ''' - try: - await trio.sleep_forever() - - except trio.Cancelled: - ... + await trio.sleep_forever() async def send(self, msg: bytes): ''' @@ -441,8 +415,7 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): log.warning('tried to close RingBufferPublisher but its already closed...') return - with trio.CancelScope(shield=True): - await self._chanmngr.close() + await self._chanmngr.close() self._is_closed = True @@ -467,15 +440,10 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): self, n: trio.Nursery, - # new ringbufs created will have this buf_size - buf_size: int = 10 * 1024, - # if connecting to a publisher that has already sent messages set # to the next expected payload index this subscriber will receive start_index: int = 0 ): - self._buf_size = buf_size - self._chanmngr = ChannelManager[RingBufferReceiveChannel]( n, self._open_channel, @@ -499,40 +467,24 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): def get_channel(self, name: str): return self._chanmngr[name] - async def add_channel(self, name: str, must_exist: bool = False): - await self._chanmngr.add_channel(name, must_exist=must_exist) + async def add_channel( + self, + token: RBToken + ): + await self._chanmngr.add_channel(token) - def remove_channel(self, name: str): - self._chanmngr.remove_channel(name) + async def remove_channel(self, name: str): + await self._chanmngr.remove_channel(name) @acm async def _open_channel( self, - name: str, - must_exist: bool = False + token: RBToken - ) -> AsyncContextManager[RingBufferReceiveChannel]: - ''' - Open a ringbuf through `ringd` and attach as receiver side - ''' - if must_exist: - ringd_fn = ringd.attach_ringbuf - kwargs = {} - - else: - ringd_fn = ringd.open_ringbuf - kwargs = {'buf_size': self._buf_size} - - async with ( - ringd_fn( - name=name, - **kwargs - ) as token, - - attach_to_ringbuf_receiver(token) as chan - ): - yield chan + ) -> AsyncContextManager[RingBufferSendChannel]: + async with attach_to_ringbuf_receiver(token) as ring: + yield ring async def _channel_task(self, info: ChannelInfo) -> None: ''' @@ -582,6 +534,7 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): await self._chanmngr.close() await self._schan.aclose() await self._rchan.aclose() + self._is_closed = True @@ -641,89 +594,128 @@ def get_subscriber() -> RingBufferSubscriber: @tractor.context -async def open_pub_channel( +async def _add_pub_channel( ctx: tractor.Context, - ring_name: str, - must_exist: bool = False + token: RBToken ): publisher = get_publisher() - await publisher.add_channel( - ring_name, - must_exist=must_exist - ) - await ctx.started() + await publisher.add_channel(token) - try: - await trio.sleep_forever() - finally: - try: - publisher.remove_channel(ring_name) - - except trio.ClosedResourceError: - ... +@tractor.context +async def _remove_pub_channel( + ctx: tractor.Context, + ring_name: str +): + publisher = get_publisher() + await ctx.started() + maybe_token = fdshare.maybe_get_fds(ring_name) + if maybe_token: + await publisher.remove_channel(ring_name) @acm async def open_pub_channel_at( actor_name: str, - ring_name: str, - must_exist: bool = False + token: RBToken, + cleanup: bool = True, ): async with ( tractor.find_actor(actor_name) as portal, + portal.open_context( - open_pub_channel, - ring_name=ring_name, - must_exist=must_exist + _add_pub_channel, + token=token ) as (ctx, _) ): + ... + + try: yield - await ctx.cancel() + + except trio.Cancelled: + log.exception( + 'open_pub_channel_at got cancelled!\n' + f'\tactor_name = {actor_name}\n' + f'\ttoken = {token}\n' + ) + raise + + finally: + if not cleanup: + return + + async with tractor.find_actor(actor_name) as portal: + if portal: + async with portal.open_context( + _remove_pub_channel, + ring_name=token.shm_name + ) as (ctx, _): + ... @tractor.context -async def open_sub_channel( +async def _add_sub_channel( ctx: tractor.Context, - ring_name: str, - must_exist: bool = False + token: RBToken ): subscriber = get_subscriber() - await subscriber.add_channel( - ring_name, - must_exist=must_exist - ) - await ctx.started() + await subscriber.add_channel(token) - try: - await trio.sleep_forever() - finally: - try: - subscriber.remove_channel(ring_name) - - except trio.ClosedResourceError: - ... +@tractor.context +async def _remove_sub_channel( + ctx: tractor.Context, + ring_name: str +): + subscriber = get_subscriber() + await ctx.started() + maybe_token = fdshare.maybe_get_fds(ring_name) + if maybe_token: + await subscriber.remove_channel(ring_name) @acm async def open_sub_channel_at( actor_name: str, - ring_name: str, - must_exist: bool = False + token: RBToken, + cleanup: bool = True, ): async with ( tractor.find_actor(actor_name) as portal, + portal.open_context( - open_sub_channel, - ring_name=ring_name, - must_exist=must_exist + _add_sub_channel, + token=token ) as (ctx, _) ): + ... + + try: yield - await ctx.cancel() + + except trio.Cancelled: + log.exception( + 'open_sub_channel_at got cancelled!\n' + f'\tactor_name = {actor_name}\n' + f'\ttoken = {token}\n' + ) + raise + + finally: + if not cleanup: + return + + async with tractor.find_actor(actor_name) as portal: + if portal: + async with portal.open_context( + _remove_sub_channel, + ring_name=token.shm_name + ) as (ctx, _): + ... + ''' @@ -733,9 +725,6 @@ High level helpers to open publisher & subscriber @acm async def open_ringbuf_publisher( - # buf size for created rings - buf_size: int = 10 * 1024, - # global batch size for channels batch_size: int = 1, @@ -747,9 +736,6 @@ async def open_ringbuf_publisher( # index guarantee_order: bool = False, - # explicit nursery cancel call on cleanup - force_cancel: bool = False, - # on creation, set the `_publisher` global in order to use the provided # tractor.context & helper utils for adding and removing new channels from # remote actors @@ -764,7 +750,6 @@ async def open_ringbuf_publisher( trio.open_nursery(strict_exception_groups=False) as n, RingBufferPublisher( n, - buf_size=buf_size, batch_size=batch_size ) as publisher ): @@ -777,23 +762,17 @@ async def open_ringbuf_publisher( try: yield publisher - finally: - if force_cancel: - # implicitly cancel any running channel handler task - n.cancel_scope.cancel() + except trio.Cancelled: + with trio.CancelScope(shield=True): + await publisher.aclose() + raise @acm async def open_ringbuf_subscriber( - # buf size for created rings - buf_size: int = 10 * 1024, - # expect indexed payloads and unwrap them in order guarantee_order: bool = False, - # explicit nursery cancel call on cleanup - force_cancel: bool = False, - # on creation, set the `_subscriber` global in order to use the provided # tractor.context & helper utils for adding and removing new channels from # remote actors @@ -805,10 +784,7 @@ async def open_ringbuf_subscriber( ''' async with ( trio.open_nursery(strict_exception_groups=False) as n, - RingBufferSubscriber( - n, - buf_size=buf_size - ) as subscriber + RingBufferSubscriber(n) as subscriber ): # maybe monkey patch `.receive` to use indexed payloads if guarantee_order: @@ -819,8 +795,10 @@ async def open_ringbuf_subscriber( global _subscriber set_subscriber(subscriber) - yield subscriber + try: + yield subscriber - if force_cancel: - # implicitly cancel any running channel handler task - n.cancel_scope.cancel() + except trio.Cancelled: + with trio.CancelScope(shield=True): + await subscriber.aclose() + raise diff --git a/tractor/ipc/_ringbuf/_ringd.py b/tractor/ipc/_ringbuf/_ringd.py deleted file mode 100644 index 51818e34..00000000 --- a/tractor/ipc/_ringbuf/_ringd.py +++ /dev/null @@ -1,401 +0,0 @@ -# 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 . -''' -Actor to broker ringbuf resources, creates and allocates -the resources, then automatically does fd passing. - -call open_ringd in your root actor - -then on actors that need a ringbuf resource use - -open_ringbuf acm, will automatically contact ringd. -''' -import os -import tempfile -from typing import AsyncContextManager -from pathlib import Path -from contextlib import ( - asynccontextmanager as acm -) -from dataclasses import dataclass - -import trio -import tractor -from tractor.linux import ( - send_fds, - recv_fds, -) - -import tractor.ipc._ringbuf as ringbuf -from tractor.ipc._ringbuf import RBToken - - -log = tractor.log.get_logger(__name__) - - -''' -Daemon implementation - -''' - - -_ringd_actor_name: str = 'ringd' - - -_root_name: str = f'{_ringd_actor_name}-{os.getpid()}' - - -def _make_ring_name(name: str) -> str: - ''' - User provided ring names will be prefixed by the ringd actor name and pid. - ''' - return f'{_root_name}.{name}' - - -@dataclass -class RingInfo: - token: RBToken - creator: str - - -_rings: dict[str, RingInfo] = {} -_ring_lock = trio.StrictFIFOLock() - - -def _maybe_get_ring(name: str) -> RingInfo | None: - ''' - Maybe return RingInfo for a given name str - - ''' - # if full name was passed, strip root name - if _root_name in name: - name = name.replace(f'{_root_name}.', '') - - return _rings.get(name, None) - - -def _get_ring(name: str) -> RingInfo: - ''' - Return a RingInfo for a given name or raise - ''' - info = _maybe_get_ring(name) - - if not info: - raise RuntimeError(f'Ring \"{name}\" not found!') - - return info - - -def _insert_ring(name: str, info: RingInfo): - ''' - Add a new ring - ''' - if name in _rings: - raise RuntimeError(f'A ring with name {name} already exists!') - - _rings[name] = info - - -def _destroy_ring(name: str): - ''' - Delete information about a ring - ''' - if name not in _rings: - raise RuntimeError(f'Tried to delete non existant {name} ring!') - - del _rings[name] - - -@tractor.context -async def _pass_fds( - ctx: tractor.Context, - name: str, - sock_path: str -): - ''' - Ringd endpoint to request passing fds of a ring. - - Supports passing fullname or not (ringd actor name and pid before ring - name). - - See `_attach_to_ring` function for usage. - ''' - async with _ring_lock: - # get ring fds or raise error - token = _get_ring(name).token - - # start fd passing context using socket on `sock_path` - async with send_fds(token.fds, sock_path): - log.info(f'connected to {sock_path} for fd passing') - # use started to signal socket is ready and send token in order for - # client to get extra info like buf_size - await ctx.started(token) - # send_fds will block until receive side acks - - log.info(f'ring {name} fds: {token.fds}, sent') - - -@tractor.context -async def _open_ringbuf( - ctx: tractor.Context, - caller: str, - name: str, - buf_size: int = 10 * 1024 -): - ''' - Ringd endpoint to create and allocate resources for a new ring. - - ''' - await _ring_lock.acquire() - maybe_info = _maybe_get_ring(name) - - if maybe_info: - raise RuntimeError( - f'Tried to create ringbuf but it already exists: {name}' - ) - - fullname = _make_ring_name(name) - - with ringbuf.open_ringbuf( - fullname, - buf_size=buf_size - ) as token: - - _insert_ring( - name, - RingInfo( - token=token, - creator=caller, - ) - ) - - _ring_lock.release() - - # yield full ring name to rebuild token after fd passing - await ctx.started(fullname) - - # await ctx cancel to remove ring from tracking and cleanup - try: - log.info(f'ring {name} created by {caller}') - await trio.sleep_forever() - - finally: - _destroy_ring(name) - - log.info(f'ring {name} destroyed by {caller}') - - -@tractor.context -async def _attach_ringbuf( - ctx: tractor.Context, - caller: str, - name: str -) -> str: - ''' - Ringd endpoint to "attach" to an existing ring, this just ensures ring - actually exists and returns its full name. - ''' - async with _ring_lock: - info = _maybe_get_ring(name) - - if not info: - raise RuntimeError( - f'{caller} tried to open_ringbuf but it doesn\'t exist: {name}' - ) - - await ctx.started() - - # return full ring name to rebuild token after fd passing - return info.token.shm_name - - -@tractor.context -async def _maybe_open_ringbuf( - ctx: tractor.Context, - caller: str, - name: str, - buf_size: int = 10 * 1024, -): - ''' - If ring already exists attach, if not create it. - ''' - maybe_info = _maybe_get_ring(name) - - if maybe_info: - return await _attach_ringbuf(ctx, caller, name) - - return await _open_ringbuf(ctx, caller, name, buf_size=buf_size) - - -''' -Ringd client side helpers - -''' - - -@acm -async def open_ringd(**kwargs) -> tractor.Portal: - ''' - Spawn new ringd actor. - - ''' - async with tractor.open_nursery(**kwargs) as an: - portal = await an.start_actor( - _ringd_actor_name, - enable_modules=[__name__] - ) - yield portal - await an.cancel() - - -@acm -async def wait_for_ringd() -> tractor.Portal: - ''' - Wait for ringd actor to be up. - - ''' - async with tractor.wait_for_actor( - _ringd_actor_name - ) as portal: - yield portal - - -async def _request_ring_fds( - fullname: str -) -> RBToken: - ''' - Private helper to fetch ring fds from ringd actor. - ''' - actor = tractor.current_actor() - - fd_amount = 3 - sock_path = str( - Path(tempfile.gettempdir()) - / - f'{fullname}-to-{actor.name}.sock' - ) - - log.info(f'trying to attach to {fullname}...') - - async with ( - tractor.find_actor(_ringd_actor_name) as ringd, - - ringd.open_context( - _pass_fds, - name=fullname, - sock_path=sock_path - ) as (ctx, token), - ): - fds = await recv_fds(sock_path, fd_amount) - write, wrap, eof = fds - log.info( - f'received fds, write: {write}, wrap: {wrap}, eof: {eof}' - ) - - token = RBToken.from_msg(token) - - return RBToken( - shm_name=fullname, - write_eventfd=write, - wrap_eventfd=wrap, - eof_eventfd=eof, - buf_size=token.buf_size - ) - - - -@acm -async def open_ringbuf( - name: str, - buf_size: int = 10 * 1024, -) -> AsyncContextManager[RBToken]: - ''' - Create a new ring and retrieve its fds. - - ''' - actor = tractor.current_actor() - async with ( - wait_for_ringd() as ringd, - - ringd.open_context( - _open_ringbuf, - caller=actor.name, - name=name, - buf_size=buf_size, - ) as (ctx, fullname), - ): - token = await _request_ring_fds(fullname) - log.info(f'{actor.name} opened {token}') - try: - yield token - - finally: - with trio.CancelScope(shield=True): - await ctx.cancel() - - -@acm -async def attach_ringbuf( - name: str, -) -> AsyncContextManager[RBToken]: - ''' - Attach to an existing ring and retreive its fds. - - ''' - actor = tractor.current_actor() - async with ( - wait_for_ringd() as ringd, - - ringd.open_context( - _attach_ringbuf, - caller=actor.name, - name=name, - ) as (ctx, _), - ): - fullname = await ctx.wait_for_result() - token = await _request_ring_fds(fullname) - log.info(f'{actor.name} attached {token}') - yield token - - -@acm -async def maybe_open_ringbuf( - name: str, - buf_size: int = 10 * 1024, -) -> AsyncContextManager[RBToken]: - ''' - Attach or create a ring and retreive its fds. - - ''' - actor = tractor.current_actor() - async with ( - wait_for_ringd() as ringd, - - ringd.open_context( - _maybe_open_ringbuf, - caller=actor.name, - name=name, - buf_size=buf_size, - ) as (ctx, fullname), - ): - token = await _request_ring_fds(fullname) - log.info(f'{actor.name} opened {token}') - try: - yield token - - finally: - with trio.CancelScope(shield=True): - await ctx.cancel() -- 2.34.1 From d60a49a85349bc03463316610c58ec4276cd9cfb Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sun, 13 Apr 2025 14:02:09 -0300 Subject: [PATCH 32/41] Check if fdshare module is enable on share_fds function --- tractor/linux/_fdshare.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tractor/linux/_fdshare.py b/tractor/linux/_fdshare.py index bc2385ef..50fc16f9 100644 --- a/tractor/linux/_fdshare.py +++ b/tractor/linux/_fdshare.py @@ -211,6 +211,13 @@ def share_fds( Register a set of fds to be shared under a given name. ''' + this_actor = tractor.current_actor() + if __name__ not in this_actor.enable_modules: + raise RuntimeError( + f'Tried to share FDs {fds} with name {name}, but ' + f'module {__name__} is not enabled in actor {this_actor.name}!' + ) + maybe_fds = maybe_get_fds(name) if maybe_fds: raise RuntimeError(f'share FDs: {maybe_fds} already tied to name {name}') -- 2.34.1 From 9b16eeed2f99b0995823660279e72ed7992363db Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sun, 13 Apr 2025 14:02:41 -0300 Subject: [PATCH 33/41] Fix chan manager close remove_channel call --- tractor/ipc/_ringbuf/_pubsub.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tractor/ipc/_ringbuf/_pubsub.py b/tractor/ipc/_ringbuf/_pubsub.py index 40adf611..4b386463 100644 --- a/tractor/ipc/_ringbuf/_pubsub.py +++ b/tractor/ipc/_ringbuf/_pubsub.py @@ -242,7 +242,7 @@ class ChannelManager(Generic[ChannelType]): if info.channel.closed: continue - await self.remove_channel(info.name) + await self.remove_channel(info.token.shm_name) self._is_closed = True -- 2.34.1 From 4ca1aaeaeb93a85312fd0552c1baf21348e8fa4f Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sun, 13 Apr 2025 14:16:39 -0300 Subject: [PATCH 34/41] Only set shield flag when trio nursery mode is used --- tractor/trionics/_mngrs.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tractor/trionics/_mngrs.py b/tractor/trionics/_mngrs.py index 9a5ed156..24b4fde8 100644 --- a/tractor/trionics/_mngrs.py +++ b/tractor/trionics/_mngrs.py @@ -70,7 +70,8 @@ async def maybe_open_nursery( yield nursery else: async with lib.open_nursery(**kwargs) as nursery: - nursery.cancel_scope.shield = shield + if lib == trio: + nursery.cancel_scope.shield = shield yield nursery -- 2.34.1 From 06103d1f44e80e68b3a0dda92614962a9bf6a12d Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Sun, 13 Apr 2025 14:06:03 -0300 Subject: [PATCH 35/41] Disable parent channel append on get_peer_by_name to_scan --- tractor/_discovery.py | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/tractor/_discovery.py b/tractor/_discovery.py index fd3e4b1c..478538c6 100644 --- a/tractor/_discovery.py +++ b/tractor/_discovery.py @@ -121,9 +121,14 @@ def get_peer_by_name( actor: Actor = current_actor() server: IPCServer = actor.ipc_server to_scan: dict[tuple, list[Channel]] = server._peers.copy() - pchan: Channel|None = actor._parent_chan - if pchan: - to_scan[pchan.uid].append(pchan) + + # 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 + # which causes actor exit to hang waiting forever on + # `actor._no_more_peers.wait()` in `_runtime.async_main` + # pchan: Channel|None = actor._parent_chan + # if pchan: + # to_scan[pchan.uid].append(pchan) for aid, chans in to_scan.items(): _, peer_name = aid -- 2.34.1 From 59521cd4db05661ed5802204b15823af77299126 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 17 Apr 2025 12:07:40 -0300 Subject: [PATCH 36/41] Add fix for cases where sockname len > 100 --- tractor/linux/_fdshare.py | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/tractor/linux/_fdshare.py b/tractor/linux/_fdshare.py index 50fc16f9..84681455 100644 --- a/tractor/linux/_fdshare.py +++ b/tractor/linux/_fdshare.py @@ -22,6 +22,7 @@ https://github.com/python/cpython/blob/275056a7fdcbe36aaac494b4183ae59943a338eb/ import os import array import tempfile +from uuid import uuid4 from pathlib import Path from typing import AsyncContextManager from contextlib import asynccontextmanager as acm @@ -273,6 +274,26 @@ async def request_fds_from( f'{fds_name}-from-{actor_name}-to-{this_actor.name}.sock' ) + # having a socket path length > 100 aprox can cause: + # OSError: AF_UNIX path too long + # https://pubs.opengroup.org/onlinepubs/9699919799/basedefs/sys_un.h.html#tag_13_67_04 + + # attempt sock path creation with smaller names + if len(sock_path) > 100: + sock_path = str( + Path(tempfile.gettempdir()) + / + f'{fds_name}-to-{this_actor.name}.sock' + ) + + if len(sock_path) > 100: + # just use uuid4 + sock_path = str( + Path(tempfile.gettempdir()) + / + f'pass-fds-{uuid4()}.sock' + ) + async with ( tractor.find_actor(actor_name) as portal, -- 2.34.1 From 86e09a80f454e8999fab009454ee6dd8fe6bc311 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Thu, 17 Apr 2025 12:08:16 -0300 Subject: [PATCH 37/41] Log warning instead of exception on pubsub cancelled --- tractor/ipc/_ringbuf/_pubsub.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tractor/ipc/_ringbuf/_pubsub.py b/tractor/ipc/_ringbuf/_pubsub.py index 4b386463..6a33e42a 100644 --- a/tractor/ipc/_ringbuf/_pubsub.py +++ b/tractor/ipc/_ringbuf/_pubsub.py @@ -635,7 +635,7 @@ async def open_pub_channel_at( yield except trio.Cancelled: - log.exception( + log.warning( 'open_pub_channel_at got cancelled!\n' f'\tactor_name = {actor_name}\n' f'\ttoken = {token}\n' @@ -697,7 +697,7 @@ async def open_sub_channel_at( yield except trio.Cancelled: - log.exception( + log.warning( 'open_sub_channel_at got cancelled!\n' f'\tactor_name = {actor_name}\n' f'\ttoken = {token}\n' -- 2.34.1 From 8799cf3b78d3d26ff1d481fdbf9ba735e2c5441d Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Tue, 22 Apr 2025 01:45:44 -0300 Subject: [PATCH 38/41] Add optional msgpack encoder & decoder to ringbuf apis --- tractor/ipc/_ringbuf/__init__.py | 81 ++++++++++++++++++++++++++------ 1 file changed, 67 insertions(+), 14 deletions(-) diff --git a/tractor/ipc/_ringbuf/__init__.py b/tractor/ipc/_ringbuf/__init__.py index 0d96d1a3..f9a88813 100644 --- a/tractor/ipc/_ringbuf/__init__.py +++ b/tractor/ipc/_ringbuf/__init__.py @@ -20,6 +20,7 @@ IPC Reliable RingBuffer implementation from __future__ import annotations import struct from typing import ( + TypeVar, ContextManager, AsyncContextManager ) @@ -34,6 +35,10 @@ from msgspec import ( Struct, to_builtins ) +from msgspec.msgpack import ( + Encoder, + Decoder, +) from tractor.log import get_logger from tractor._exceptions import ( @@ -277,7 +282,10 @@ next full payload. ''' -class RingBufferSendChannel(trio.abc.SendChannel[bytes]): +PayloadT = TypeVar('PayloadT') + + +class RingBufferSendChannel(trio.abc.SendChannel[PayloadT]): ''' Ring Buffer sender side implementation @@ -298,7 +306,8 @@ class RingBufferSendChannel(trio.abc.SendChannel[bytes]): self, token: RBToken, batch_size: int = 1, - cleanup: bool = False + cleanup: bool = False, + encoder: Encoder | None = None ): self._token = RBToken.from_msg(token) self.batch_size = batch_size @@ -319,6 +328,8 @@ class RingBufferSendChannel(trio.abc.SendChannel[bytes]): # close shm & fds on exit? self._cleanup: bool = cleanup + self._enc: Encoder | None = encoder + # have we closed this ringbuf? # set to `False` on `.open()` self._is_closed: bool = True @@ -415,15 +426,22 @@ class RingBufferSendChannel(trio.abc.SendChannel[bytes]): if new_batch_size: self.batch_size = new_batch_size - async def send(self, value: bytes) -> None: + async def send(self, value: PayloadT) -> None: if self.closed: raise trio.ClosedResourceError if self._send_lock.locked(): raise trio.BusyResourceError + raw_value: bytes = ( + value + if isinstance(value, bytes) + else + self._enc.encode(value) + ) + async with self._send_lock: - msg: bytes = struct.pack(" 0: await self.flush() @@ -475,7 +493,7 @@ class RingBufferSendChannel(trio.abc.SendChannel[bytes]): return self -class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): +class RingBufferReceiveChannel(trio.abc.ReceiveChannel[PayloadT]): ''' Ring Buffer receiver side implementation @@ -487,6 +505,7 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): self, token: RBToken, cleanup: bool = True, + decoder: Decoder | None = None ): self._token = RBToken.from_msg(token) @@ -513,6 +532,8 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): # set to `False` on `.open()` self._is_closed: bool = True + self._dec: Decoder | None = decoder + # ensure no concurrent `.receive_some()` calls self._receive_some_lock = trio.StrictFIFOLock() @@ -648,7 +669,11 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): self._write_ptr += delta # yield lock and re-enter - except (EFDReadCancelled, trio.Cancelled): + except ( + EFDReadCancelled, # read was cancelled with cscope + trio.Cancelled, # read got cancelled from outside + trio.BrokenResourceError # OSError EBADF happened while reading + ): # while waiting for new data `self._write_event` was closed try: # if eof was signaled receive no wait will not raise @@ -699,7 +724,7 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): return payload - async def receive(self) -> bytes: + async def receive(self, raw: bool = False) -> PayloadT: ''' Receive a complete payload or raise EOC @@ -717,7 +742,27 @@ class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]): if size == 0: raise trio.EndOfChannel - return await self.receive_exactly(size) + raw_msg = await self.receive_exactly(size) + if raw: + return raw_msg + + return ( + raw_msg + if not self._dec + else self._dec.decode(raw_msg) + ) + + async def iter_raw_pairs(self) -> tuple[bytes, PayloadT]: + if not self._dec: + raise RuntimeError('iter_raw_pair requires decoder') + + while True: + try: + raw = await self.receive(raw=True) + yield raw, self._dec.decode(raw) + + except trio.EndOfChannel: + break def open(self): try: @@ -782,7 +827,8 @@ async def _maybe_obtain_shared_resources(token: RBToken): async def attach_to_ringbuf_receiver( token: RBToken, - cleanup: bool = True + cleanup: bool = True, + decoder: Decoder | None = None ) -> AsyncContextManager[RingBufferReceiveChannel]: ''' @@ -800,7 +846,8 @@ async def attach_to_ringbuf_receiver( trio.open_nursery(strict_exception_groups=False) as n, RingBufferReceiveChannel( token, - cleanup=cleanup + cleanup=cleanup, + decoder=decoder ) as receiver ): n.start_soon(receiver._eof_monitor_task) @@ -812,7 +859,8 @@ async def attach_to_ringbuf_sender( token: RBToken, batch_size: int = 1, - cleanup: bool = True + cleanup: bool = True, + encoder: Encoder | None = None ) -> AsyncContextManager[RingBufferSendChannel]: ''' @@ -828,7 +876,8 @@ async def attach_to_ringbuf_sender( async with RingBufferSendChannel( token, batch_size=batch_size, - cleanup=cleanup + cleanup=cleanup, + encoder=encoder ) as sender: yield sender @@ -901,6 +950,8 @@ async def attach_to_ringbuf_channel( batch_size: int = 1, cleanup_in: bool = True, cleanup_out: bool = True, + encoder: Encoder | None = None, + decoder: Decoder | None = None ) -> AsyncContextManager[trio.StapledStream]: ''' Attach to two previously opened `RBToken`s and return a `RingBufferChannel` @@ -909,12 +960,14 @@ async def attach_to_ringbuf_channel( async with ( attach_to_ringbuf_receiver( token_in, - cleanup=cleanup_in + cleanup=cleanup_in, + decoder=decoder ) as receiver, attach_to_ringbuf_sender( token_out, batch_size=batch_size, - cleanup=cleanup_out + cleanup=cleanup_out, + encoder=encoder ) as sender, ): yield RingBufferChannel(sender, receiver) -- 2.34.1 From a553446619eec994f4cbb1bf814c0b15132597d7 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Tue, 22 Apr 2025 01:46:41 -0300 Subject: [PATCH 39/41] Pubsub topics, enc & decoders Implicit aclose on all channels on ChannelManager aclose Implicit nursery cancel on pubsub acms Use long running actor portal for open_{pub,sub}_channel_at fns Add optional encoder/decoder on pubsub Add topic system for multiple pub or sub on same actor Add wait fn for sub and pub channel register --- tractor/ipc/_ringbuf/_pubsub.py | 292 ++++++++++++++++++-------------- 1 file changed, 161 insertions(+), 131 deletions(-) diff --git a/tractor/ipc/_ringbuf/_pubsub.py b/tractor/ipc/_ringbuf/_pubsub.py index 6a33e42a..7de1d9b2 100644 --- a/tractor/ipc/_ringbuf/_pubsub.py +++ b/tractor/ipc/_ringbuf/_pubsub.py @@ -31,8 +31,14 @@ from dataclasses import dataclass import trio import tractor +from msgspec.msgpack import ( + Encoder, + Decoder +) + from tractor.ipc._ringbuf import ( RBToken, + PayloadT, RingBufferSendChannel, RingBufferReceiveChannel, attach_to_ringbuf_sender, @@ -242,6 +248,7 @@ class ChannelManager(Generic[ChannelType]): if info.channel.closed: continue + await info.channel.aclose() await self.remove_channel(info.token.shm_name) self._is_closed = True @@ -253,7 +260,7 @@ Ring buffer publisher & subscribe pattern mediated by `ringd` actor. ''' -class RingBufferPublisher(trio.abc.SendChannel[bytes]): +class RingBufferPublisher(trio.abc.SendChannel[PayloadT]): ''' Use ChannelManager to create a multi ringbuf round robin sender that can dynamically add or remove more outputs. @@ -270,13 +277,16 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): msgs_per_turn: int = 1, # global batch size for all channels - batch_size: int = 1 + batch_size: int = 1, + + encoder: Encoder | None = None ): self._batch_size: int = batch_size self.msgs_per_turn = msgs_per_turn + self._enc = encoder # helper to manage acms + long running tasks - self._chanmngr = ChannelManager[RingBufferSendChannel]( + self._chanmngr = ChannelManager[RingBufferSendChannel[PayloadT]]( n, self._open_channel, self._channel_task @@ -349,10 +359,11 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): self, token: RBToken - ) -> AsyncContextManager[RingBufferSendChannel]: + ) -> AsyncContextManager[RingBufferSendChannel[PayloadT]]: async with attach_to_ringbuf_sender( token, - batch_size=self._batch_size + batch_size=self._batch_size, + encoder=self._enc ) as ring: yield ring @@ -387,7 +398,7 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): info = self.channels[turn] await info.channel.send(msg) - async def broadcast(self, msg: bytes): + async def broadcast(self, msg: PayloadT): ''' Send a msg to all channels, if no channels connected, does nothing. ''' @@ -406,8 +417,8 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): ... async def __aenter__(self): - self._chanmngr.open() self._is_closed = False + self._chanmngr.open() return self async def aclose(self) -> None: @@ -420,7 +431,7 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]): self._is_closed = True -class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): +class RingBufferSubscriber(trio.abc.ReceiveChannel[PayloadT]): ''' Use ChannelManager to create a multi ringbuf receiver that can dynamically add or remove more inputs and combine all into a single output. @@ -440,11 +451,10 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): self, n: trio.Nursery, - # if connecting to a publisher that has already sent messages set - # to the next expected payload index this subscriber will receive - start_index: int = 0 + decoder: Decoder | None = None ): - self._chanmngr = ChannelManager[RingBufferReceiveChannel]( + self._dec = decoder + self._chanmngr = ChannelManager[RingBufferReceiveChannel[PayloadT]]( n, self._open_channel, self._channel_task @@ -483,7 +493,10 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): token: RBToken ) -> AsyncContextManager[RingBufferSendChannel]: - async with attach_to_ringbuf_receiver(token) as ring: + async with attach_to_ringbuf_receiver( + token, + decoder=self._dec + ) as ring: yield ring async def _channel_task(self, info: ChannelInfo) -> None: @@ -509,7 +522,7 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]): except trio.ClosedResourceError: break - async def receive(self) -> bytes: + async def receive(self) -> PayloadT: ''' Receive next in order msg ''' @@ -543,73 +556,74 @@ Actor module for managing publisher & subscriber channels remotely through `tractor.context` rpc ''' -_publisher: RingBufferPublisher | None = None -_subscriber: RingBufferSubscriber | None = None +@dataclass +class PublisherEntry: + publisher: RingBufferPublisher | None = None + is_set: trio.Event = trio.Event() -def set_publisher(pub: RingBufferPublisher): - global _publisher +_publishers: dict[str, PublisherEntry] = {} - if _publisher: + +def maybe_init_publisher(topic: str) -> PublisherEntry: + entry = _publishers.get(topic, None) + if not entry: + entry = PublisherEntry() + _publishers[topic] = entry + + return entry + + +def set_publisher(topic: str, pub: RingBufferPublisher): + global _publishers + + entry = _publishers.get(topic, None) + if not entry: + entry = maybe_init_publisher(topic) + + if entry.publisher: raise RuntimeError( - f'publisher already set on {tractor.current_actor()}' + f'publisher for topic {topic} already set on {tractor.current_actor()}' ) - _publisher = pub + entry.publisher = pub + entry.is_set.set() -def set_subscriber(sub: RingBufferSubscriber): - global _subscriber - - if _subscriber: - raise RuntimeError( - f'subscriber already set on {tractor.current_actor()}' - ) - - _subscriber = sub - - -def get_publisher() -> RingBufferPublisher: - global _publisher - - if not _publisher: +def get_publisher(topic: str) -> RingBufferPublisher: + entry = _publishers.get(topic, None) + if not entry or not entry.publisher: raise RuntimeError( f'{tractor.current_actor()} tried to get publisher' 'but it\'s not set' ) - return _publisher + return entry.publisher -def get_subscriber() -> RingBufferSubscriber: - global _subscriber - - if not _subscriber: - raise RuntimeError( - f'{tractor.current_actor()} tried to get subscriber' - 'but it\'s not set' - ) - - return _subscriber +async def wait_publisher(topic: str) -> RingBufferPublisher: + entry = maybe_init_publisher(topic) + await entry.is_set.wait() + return entry.publisher @tractor.context async def _add_pub_channel( ctx: tractor.Context, + topic: str, token: RBToken ): - publisher = get_publisher() - await ctx.started() + publisher = await wait_publisher(topic) await publisher.add_channel(token) @tractor.context async def _remove_pub_channel( ctx: tractor.Context, + topic: str, ring_name: str ): - publisher = get_publisher() - await ctx.started() + publisher = await wait_publisher(topic) maybe_token = fdshare.maybe_get_fds(ring_name) if maybe_token: await publisher.remove_channel(ring_name) @@ -619,59 +633,92 @@ async def _remove_pub_channel( async def open_pub_channel_at( actor_name: str, token: RBToken, - cleanup: bool = True, + topic: str = 'default', ): - async with ( - tractor.find_actor(actor_name) as portal, + async with tractor.find_actor(actor_name) as portal: + await portal.run(_add_pub_channel, topic=topic, token=token) + try: + yield - portal.open_context( - _add_pub_channel, - token=token - ) as (ctx, _) - ): - ... + except trio.Cancelled: + log.warning( + 'open_pub_channel_at got cancelled!\n' + f'\tactor_name = {actor_name}\n' + f'\ttoken = {token}\n' + ) + raise - try: - yield + await portal.run(_remove_pub_channel, topic=topic, ring_name=token.shm_name) - except trio.Cancelled: - log.warning( - 'open_pub_channel_at got cancelled!\n' - f'\tactor_name = {actor_name}\n' - f'\ttoken = {token}\n' + +@dataclass +class SubscriberEntry: + subscriber: RingBufferSubscriber | None = None + is_set: trio.Event = trio.Event() + + +_subscribers: dict[str, SubscriberEntry] = {} + + +def maybe_init_subscriber(topic: str) -> SubscriberEntry: + entry = _subscribers.get(topic, None) + if not entry: + entry = SubscriberEntry() + _subscribers[topic] = entry + + return entry + + +def set_subscriber(topic: str, sub: RingBufferSubscriber): + global _subscribers + + entry = _subscribers.get(topic, None) + if not entry: + entry = maybe_init_subscriber(topic) + + if entry.subscriber: + raise RuntimeError( + f'subscriber for topic {topic} already set on {tractor.current_actor()}' ) - raise - finally: - if not cleanup: - return + entry.subscriber = sub + entry.is_set.set() - async with tractor.find_actor(actor_name) as portal: - if portal: - async with portal.open_context( - _remove_pub_channel, - ring_name=token.shm_name - ) as (ctx, _): - ... + +def get_subscriber(topic: str) -> RingBufferSubscriber: + entry = _subscribers.get(topic, None) + if not entry or not entry.subscriber: + raise RuntimeError( + f'{tractor.current_actor()} tried to get subscriber' + 'but it\'s not set' + ) + + return entry.subscriber + + +async def wait_subscriber(topic: str) -> RingBufferSubscriber: + entry = maybe_init_subscriber(topic) + await entry.is_set.wait() + return entry.subscriber @tractor.context async def _add_sub_channel( ctx: tractor.Context, + topic: str, token: RBToken ): - subscriber = get_subscriber() - await ctx.started() + subscriber = await wait_subscriber(topic) await subscriber.add_channel(token) @tractor.context async def _remove_sub_channel( ctx: tractor.Context, + topic: str, ring_name: str ): - subscriber = get_subscriber() - await ctx.started() + subscriber = await wait_subscriber(topic) maybe_token = fdshare.maybe_get_fds(ring_name) if maybe_token: await subscriber.remove_channel(ring_name) @@ -681,41 +728,22 @@ async def _remove_sub_channel( async def open_sub_channel_at( actor_name: str, token: RBToken, - cleanup: bool = True, + topic: str = 'default', ): - async with ( - tractor.find_actor(actor_name) as portal, + async with tractor.find_actor(actor_name) as portal: + await portal.run(_add_sub_channel, topic=topic, token=token) + try: + yield - portal.open_context( - _add_sub_channel, - token=token - ) as (ctx, _) - ): - ... - - try: - yield - - except trio.Cancelled: - log.warning( - 'open_sub_channel_at got cancelled!\n' - f'\tactor_name = {actor_name}\n' - f'\ttoken = {token}\n' - ) - raise - - finally: - if not cleanup: - return - - async with tractor.find_actor(actor_name) as portal: - if portal: - async with portal.open_context( - _remove_sub_channel, - ring_name=token.shm_name - ) as (ctx, _): - ... + except trio.Cancelled: + log.warning( + 'open_sub_channel_at got cancelled!\n' + f'\tactor_name = {actor_name}\n' + f'\ttoken = {token}\n' + ) + raise + await portal.run(_remove_sub_channel, topic=topic, ring_name=token.shm_name) ''' @@ -725,12 +753,17 @@ High level helpers to open publisher & subscriber @acm async def open_ringbuf_publisher( + # name to distinguish this publisher + topic: str = 'default', + # global batch size for channels batch_size: int = 1, # messages before changing output channel msgs_per_turn: int = 1, + encoder: Encoder | None = None, + # ensure subscriber receives in same order publisher sent # causes it to use wrapped payloads which contain the og # index @@ -750,26 +783,28 @@ async def open_ringbuf_publisher( trio.open_nursery(strict_exception_groups=False) as n, RingBufferPublisher( n, - batch_size=batch_size + batch_size=batch_size, + encoder=encoder, ) as publisher ): if guarantee_order: order_send_channel(publisher) if set_module_var: - set_publisher(publisher) + set_publisher(topic, publisher) - try: - yield publisher + yield publisher - except trio.Cancelled: - with trio.CancelScope(shield=True): - await publisher.aclose() - raise + n.cancel_scope.cancel() @acm async def open_ringbuf_subscriber( + # name to distinguish this subscriber + topic: str = 'default', + + decoder: Decoder | None = None, + # expect indexed payloads and unwrap them in order guarantee_order: bool = False, @@ -784,7 +819,7 @@ async def open_ringbuf_subscriber( ''' async with ( trio.open_nursery(strict_exception_groups=False) as n, - RingBufferSubscriber(n) as subscriber + RingBufferSubscriber(n, decoder=decoder) as subscriber ): # maybe monkey patch `.receive` to use indexed payloads if guarantee_order: @@ -792,13 +827,8 @@ async def open_ringbuf_subscriber( # maybe set global module var for remote actor channel updates if set_module_var: - global _subscriber - set_subscriber(subscriber) + set_subscriber(topic, subscriber) - try: - yield subscriber + yield subscriber - except trio.Cancelled: - with trio.CancelScope(shield=True): - await subscriber.aclose() - raise + n.cancel_scope.cancel() -- 2.34.1 From 7766caf623f7519c2afa3c554a3063c222c80c52 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Tue, 22 Apr 2025 01:51:21 -0300 Subject: [PATCH 40/41] Detect OSError errno EBADF and re-raise as trio.BrokenResourceError on EventFD reads --- tractor/linux/eventfd.py | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/tractor/linux/eventfd.py b/tractor/linux/eventfd.py index 8ddf3669..1b00a190 100644 --- a/tractor/linux/eventfd.py +++ b/tractor/linux/eventfd.py @@ -163,10 +163,17 @@ class EventFD: async with self._read_lock: self._cscope = trio.CancelScope() with self._cscope: - return await trio.to_thread.run_sync( - read_eventfd, self._fd, - abandon_on_cancel=True - ) + try: + return await trio.to_thread.run_sync( + read_eventfd, self._fd, + abandon_on_cancel=True + ) + + except OSError as e: + if e.errno != errno.EBADF: + raise + + raise trio.BrokenResourceError if self._cscope.cancelled_caught: raise EFDReadCancelled -- 2.34.1 From 053078ce8fd52284d767b200e77bb2bd23b5c812 Mon Sep 17 00:00:00 2001 From: Guillermo Rodriguez Date: Tue, 22 Apr 2025 05:38:32 -0300 Subject: [PATCH 41/41] Fix rb non ipc case and tests in general --- tests/test_ring_pubsub.py | 31 ++++++------ tests/test_ringbuf.py | 16 +++--- tractor/ipc/_ringbuf/__init__.py | 85 ++++++++++++++++++++++---------- tractor/ipc/_ringbuf/_pubsub.py | 4 +- 4 files changed, 86 insertions(+), 50 deletions(-) diff --git a/tests/test_ring_pubsub.py b/tests/test_ring_pubsub.py index 3bdbeb0a..b3b0dade 100644 --- a/tests/test_ring_pubsub.py +++ b/tests/test_ring_pubsub.py @@ -120,21 +120,24 @@ async def open_pubsub_test_actors( publisher_child, batch_size=batch_size ) as (long_sctx, _), + + open_ringbufs(ring_names) as tokens, + + gather_contexts([ + open_sub_channel_at('sub', ring) + for ring in tokens + ]), + gather_contexts([ + open_pub_channel_at('pub', ring) + for ring in tokens + ]), + sub_portal.open_context(subscribe_range, size=size) as (rctx, _), + pub_portal.open_context(publish_range, size=size) as (sctx, _) ): - with open_ringbufs(ring_names) as tokens: - async with ( - gather_contexts([ - open_sub_channel_at('sub', ring) - for ring in tokens - ]), - gather_contexts([ - open_pub_channel_at('pub', ring) - for ring in tokens - ]), - sub_portal.open_context(subscribe_range, size=size) as (rctx, _), - pub_portal.open_context(publish_range, size=size) as (sctx, _) - ): - yield + yield + + await rctx.wait_for_result() + await sctx.wait_for_result() await long_sctx.cancel() await long_rctx.cancel() diff --git a/tests/test_ringbuf.py b/tests/test_ringbuf.py index 3e6a5734..1f6a1927 100644 --- a/tests/test_ringbuf.py +++ b/tests/test_ringbuf.py @@ -18,9 +18,6 @@ from tractor._testing.samples import ( RandomBytesGenerator ) -# in case you don't want to melt your cores, uncomment dis! -pytestmark = pytest.mark.skip - @tractor.context async def child_read_shm( @@ -273,19 +270,24 @@ def test_receiver_max_bytes(): msg = generate_single_byte_msgs(100) msgs = [] + rb_common = { + 'cleanup': False, + 'is_ipc': False + } + async def main(): async with ( - tractor.open_nursery(), open_ringbuf( 'test_ringbuf_max_bytes', - buf_size=10 + buf_size=10, + is_ipc=False ) as token, trio.open_nursery() as n, - attach_to_ringbuf_sender(token, cleanup=False) as sender, + attach_to_ringbuf_sender(token, **rb_common) as sender, - attach_to_ringbuf_receiver(token, cleanup=False) as receiver + attach_to_ringbuf_receiver(token, **rb_common) as receiver ): async def _send_and_close(): await sender.send_all(msg) diff --git a/tractor/ipc/_ringbuf/__init__.py b/tractor/ipc/_ringbuf/__init__.py index f9a88813..12943707 100644 --- a/tractor/ipc/_ringbuf/__init__.py +++ b/tractor/ipc/_ringbuf/__init__.py @@ -72,7 +72,7 @@ class RBToken(Struct, frozen=True): even in the case that ringbuf was not allocated by current actor. ''' - owner: str # if owner != `current_actor().name` we must use fdshare + owner: str | None # if owner != `current_actor().name` we must use fdshare shm_name: str @@ -104,6 +104,7 @@ class RBToken(Struct, frozen=True): def alloc_ringbuf( shm_name: str, buf_size: int = _DEFAULT_RB_SIZE, + is_ipc: bool = True ) -> tuple[SharedMemory, RBToken]: ''' Allocate OS resources for a ringbuf. @@ -114,18 +115,21 @@ def alloc_ringbuf( create=True ) token = RBToken( - owner=current_actor().name, + owner=current_actor().name if is_ipc else None, shm_name=shm_name, write_eventfd=open_eventfd(), wrap_eventfd=open_eventfd(), eof_eventfd=open_eventfd(), buf_size=buf_size ) - # register fds for sharing - share_fds( - shm_name, - token.fds, - ) + + if is_ipc: + # register fds for sharing + share_fds( + shm_name, + token.fds, + ) + return shm, token @@ -133,6 +137,7 @@ def alloc_ringbuf( def open_ringbuf_sync( shm_name: str, buf_size: int = _DEFAULT_RB_SIZE, + is_ipc: bool = True ) -> ContextManager[RBToken]: ''' Handle resources for a ringbuf (shm, eventfd), yield `RBToken` to @@ -143,11 +148,15 @@ def open_ringbuf_sync( shm: SharedMemory | None = None token: RBToken | None = None try: - shm, token = alloc_ringbuf(shm_name, buf_size=buf_size) + shm, token = alloc_ringbuf( + shm_name, + buf_size=buf_size, + is_ipc=is_ipc + ) yield token finally: - if token: + if token and is_ipc: unshare_fds(shm_name) if shm: @@ -157,6 +166,7 @@ def open_ringbuf_sync( async def open_ringbuf( shm_name: str, buf_size: int = _DEFAULT_RB_SIZE, + is_ipc: bool = True ) -> AsyncContextManager[RBToken]: ''' Helper to use `open_ringbuf_sync` inside an async with block. @@ -164,7 +174,8 @@ async def open_ringbuf( ''' with open_ringbuf_sync( shm_name, - buf_size=buf_size + buf_size=buf_size, + is_ipc=is_ipc ) as token: yield token @@ -173,6 +184,7 @@ async def open_ringbuf( def open_ringbufs_sync( shm_names: list[str], buf_sizes: int | list[str] = _DEFAULT_RB_SIZE, + is_ipc: bool = True ) -> ContextManager[tuple[RBToken]]: ''' Handle resources for multiple ringbufs at once. @@ -194,7 +206,11 @@ def open_ringbufs_sync( # allocate resources rings: list[tuple[SharedMemory, RBToken]] = [ - alloc_ringbuf(shm_name, buf_size=buf_size) + alloc_ringbuf( + shm_name, + buf_size=buf_size, + is_ipc=is_ipc + ) for shm_name, buf_size in zip(shm_names, buf_size) ] @@ -204,11 +220,12 @@ def open_ringbufs_sync( finally: # attempt fd unshare and shm unlink for each for shm, token in rings: - try: - unshare_fds(token.shm_name) + if is_ipc: + try: + unshare_fds(token.shm_name) - except RuntimeError: - log.exception(f'while unsharing fds of {token}') + except RuntimeError: + log.exception(f'while unsharing fds of {token}') shm.unlink() @@ -217,6 +234,7 @@ def open_ringbufs_sync( async def open_ringbufs( shm_names: list[str], buf_sizes: int | list[str] = _DEFAULT_RB_SIZE, + is_ipc: bool = True ) -> AsyncContextManager[tuple[RBToken]]: ''' Helper to use `open_ringbufs_sync` inside an async with block. @@ -224,7 +242,8 @@ async def open_ringbufs( ''' with open_ringbufs_sync( shm_names, - buf_sizes=buf_sizes + buf_sizes=buf_sizes, + is_ipc=is_ipc ) as tokens: yield tokens @@ -232,7 +251,8 @@ async def open_ringbufs( @cm def open_ringbuf_pair_sync( shm_name: str, - buf_size: int = _DEFAULT_RB_SIZE + buf_size: int = _DEFAULT_RB_SIZE, + is_ipc: bool = True ) -> ContextManager[tuple(RBToken, RBToken)]: ''' Handle resources for a ringbuf pair to be used for @@ -244,7 +264,8 @@ def open_ringbuf_pair_sync( f'{shm_name}.send', f'{shm_name}.recv' ], - buf_sizes=buf_size + buf_sizes=buf_size, + is_ipc=is_ipc ) as tokens: yield tokens @@ -252,7 +273,8 @@ def open_ringbuf_pair_sync( @acm async def open_ringbuf_pair( shm_name: str, - buf_size: int = _DEFAULT_RB_SIZE + buf_size: int = _DEFAULT_RB_SIZE, + is_ipc: bool = True ) -> AsyncContextManager[tuple[RBToken, RBToken]]: ''' Helper to use `open_ringbuf_pair_sync` inside an async with block. @@ -260,7 +282,8 @@ async def open_ringbuf_pair( ''' with open_ringbuf_pair_sync( shm_name, - buf_size=buf_size + buf_size=buf_size, + is_ipc=is_ipc ) as tokens: yield tokens @@ -828,7 +851,8 @@ async def attach_to_ringbuf_receiver( token: RBToken, cleanup: bool = True, - decoder: Decoder | None = None + decoder: Decoder | None = None, + is_ipc: bool = True ) -> AsyncContextManager[RingBufferReceiveChannel]: ''' @@ -840,7 +864,8 @@ async def attach_to_ringbuf_receiver( Launches `receiver._eof_monitor_task` in a `trio.Nursery`. ''' - token = await _maybe_obtain_shared_resources(token) + if is_ipc: + token = await _maybe_obtain_shared_resources(token) async with ( trio.open_nursery(strict_exception_groups=False) as n, @@ -860,7 +885,8 @@ async def attach_to_ringbuf_sender( token: RBToken, batch_size: int = 1, cleanup: bool = True, - encoder: Encoder | None = None + encoder: Encoder | None = None, + is_ipc: bool = True ) -> AsyncContextManager[RingBufferSendChannel]: ''' @@ -871,7 +897,8 @@ async def attach_to_ringbuf_sender( originally allocated by a different actor. ''' - token = await _maybe_obtain_shared_resources(token) + if is_ipc: + token = await _maybe_obtain_shared_resources(token) async with RingBufferSendChannel( token, @@ -951,7 +978,9 @@ async def attach_to_ringbuf_channel( cleanup_in: bool = True, cleanup_out: bool = True, encoder: Encoder | None = None, - decoder: Decoder | None = None + decoder: Decoder | None = None, + sender_ipc: bool = True, + receiver_ipc: bool = True ) -> AsyncContextManager[trio.StapledStream]: ''' Attach to two previously opened `RBToken`s and return a `RingBufferChannel` @@ -961,13 +990,15 @@ async def attach_to_ringbuf_channel( attach_to_ringbuf_receiver( token_in, cleanup=cleanup_in, - decoder=decoder + decoder=decoder, + is_ipc=receiver_ipc ) as receiver, attach_to_ringbuf_sender( token_out, batch_size=batch_size, cleanup=cleanup_out, - encoder=encoder + encoder=encoder, + is_ipc=sender_ipc ) as sender, ): yield RingBufferChannel(sender, receiver) diff --git a/tractor/ipc/_ringbuf/_pubsub.py b/tractor/ipc/_ringbuf/_pubsub.py index 7de1d9b2..c85de9ca 100644 --- a/tractor/ipc/_ringbuf/_pubsub.py +++ b/tractor/ipc/_ringbuf/_pubsub.py @@ -590,7 +590,7 @@ def set_publisher(topic: str, pub: RingBufferPublisher): entry.is_set.set() -def get_publisher(topic: str) -> RingBufferPublisher: +def get_publisher(topic: str = 'default') -> RingBufferPublisher: entry = _publishers.get(topic, None) if not entry or not entry.publisher: raise RuntimeError( @@ -685,7 +685,7 @@ def set_subscriber(topic: str, sub: RingBufferSubscriber): entry.is_set.set() -def get_subscriber(topic: str) -> RingBufferSubscriber: +def get_subscriber(topic: str = 'default') -> RingBufferSubscriber: entry = _subscribers.get(topic, None) if not entry or not entry.subscriber: raise RuntimeError( -- 2.34.1