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
one_ring_to_rule_them_all
Guillermo Rodriguez 2025-04-10 13:13:08 -03:00
parent e4868ded54
commit 5d6fa643ba
No known key found for this signature in database
GPG Key ID: 002CC5F1E6BDA53E
3 changed files with 617 additions and 233 deletions

View File

@ -1,5 +1,3 @@
from contextlib import asynccontextmanager as acm
import trio import trio
import tractor import tractor
import msgspec import msgspec
@ -40,7 +38,7 @@ async def send_child(
ring_name: str ring_name: str
): ):
async with ( async with (
ringd.open_ringbuf(ring_name) as token, ringd.attach_ringbuf(ring_name) as token,
attach_to_ringbuf_sender(token) as chan, attach_to_ringbuf_sender(token) as chan,
): ):
@ -63,9 +61,7 @@ def test_ringd():
async with ( async with (
tractor.open_nursery() as an, tractor.open_nursery() as an,
ringd.open_ringd( ringd.open_ringd()
loglevel='info'
)
): ):
recv_portal = await an.start_actor( recv_portal = await an.start_actor(
'recv', 'recv',
@ -133,10 +129,10 @@ async def subscriber_child(ctx: tractor.Context):
msg = msgspec.msgpack.decode(msg, type=ControlMessages) msg = msgspec.msgpack.decode(msg, type=ControlMessages)
match msg: match msg:
case AddChannelMsg(): case AddChannelMsg():
await subs.add_channel(msg.name, must_exist=False) await subs.add_channel(msg.name)
case RemoveChannelMsg(): case RemoveChannelMsg():
await subs.remove_channel(msg.name) subs.remove_channel(msg.name)
case RangeMsg(): case RangeMsg():
range_msg = msg range_msg = msg
@ -171,7 +167,7 @@ async def subscriber_child(ctx: tractor.Context):
async def publisher_child(ctx: tractor.Context): async def publisher_child(ctx: tractor.Context):
await ctx.started() await ctx.started()
async with ( 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 ctx.open_stream() as stream
): ):
async for msg in 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) await pub.add_channel(msg.name, must_exist=True)
case RemoveChannelMsg(): case RemoveChannelMsg():
await pub.remove_channel(msg.name) pub.remove_channel(msg.name)
case RangeMsg(): case RangeMsg():
for i in range(msg.size): for i in range(msg.size):
@ -258,11 +254,6 @@ def test_pubsub():
await send_range(100) await send_range(100)
await remove_channel(ring_name) 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 # multi chan test
ring_names = [] ring_names = []
for i in range(3): for i in range(3):

View File

@ -86,19 +86,14 @@ class ChannelManager(Generic[ChannelType]):
# store channel runtime variables # store channel runtime variables
self._channels: list[ChannelInfo] = [] self._channels: list[ChannelInfo] = []
# methods that modify self._channels should be ordered by FIFO
self._lock = trio.StrictFIFOLock()
self._is_closed: bool = True self._is_closed: bool = True
self._teardown = trio.Event()
@property @property
def closed(self) -> bool: def closed(self) -> bool:
return self._is_closed return self._is_closed
@property
def lock(self) -> trio.StrictFIFOLock:
return self._lock
@property @property
def channels(self) -> list[ChannelInfo]: def channels(self) -> list[ChannelInfo]:
return self._channels return self._channels
@ -106,8 +101,8 @@ class ChannelManager(Generic[ChannelType]):
async def _channel_handler_task( async def _channel_handler_task(
self, self,
name: str, name: str,
task_status: trio.TASK_STATUS_IGNORED, must_exist: bool = False,
**kwargs task_status=trio.TASK_STATUS_IGNORED,
): ):
''' '''
Open channel resources, add to internal data structures, signal channel 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. 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() cancel_scope = trio.CancelScope()
info = ChannelInfo( info = ChannelInfo(
name=name, name=name,
@ -138,6 +133,9 @@ class ChannelManager(Generic[ChannelType]):
self._maybe_destroy_channel(name) self._maybe_destroy_channel(name)
if len(self) == 0:
self._teardown.set()
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 Given a channel name maybe return its index and value from
@ -165,7 +163,7 @@ class ChannelManager(Generic[ChannelType]):
info.cancel_scope.cancel() info.cancel_scope.cancel()
del self._channels[i] 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 Add a new channel to be handled
@ -173,14 +171,13 @@ class ChannelManager(Generic[ChannelType]):
if self.closed: if self.closed:
raise trio.ClosedResourceError raise trio.ClosedResourceError
async with self._lock:
await self._n.start(partial( await self._n.start(partial(
self._channel_handler_task, self._channel_handler_task,
name, name,
**kwargs must_exist=must_exist
)) ))
async def remove_channel(self, name: str): def remove_channel(self, name: str):
''' '''
Remove a channel and stop its handling Remove a channel and stop its handling
@ -188,7 +185,6 @@ class ChannelManager(Generic[ChannelType]):
if self.closed: if self.closed:
raise trio.ClosedResourceError 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 that was last channel reset connect event
@ -226,7 +222,18 @@ class ChannelManager(Generic[ChannelType]):
return return
for info in self._channels: 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 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]): class RingBufferPublisher(trio.abc.SendChannel[bytes]):
''' '''
@ -259,24 +260,32 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]):
# new ringbufs created will have this buf_size # new ringbufs created will have this buf_size
buf_size: int = 10 * 1024, 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 # global batch size for all channels
batch_size: int = 1 batch_size: int = 1
): ):
self._buf_size = buf_size self._buf_size = buf_size
self._batch_size: int = batch_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, n,
self._open_channel, self._open_channel,
self._channel_task self._channel_task
) )
# methods that send data over the channels need to be acquire send lock # ensure no concurrent `.send()` calls
# in order to guarantee order of operations
self._send_lock = trio.StrictFIFOLock() self._send_lock = trio.StrictFIFOLock()
# index of channel to be used for next send
self._next_turn: int = 0 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 self._is_closed: bool = True
@property @property
@ -288,14 +297,31 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]):
return self._batch_size return self._batch_size
@batch_size.setter @batch_size.setter
def set_batch_size(self, value: int) -> None: def batch_size(self, value: int) -> None:
for info in self.channels: for info in self.channels:
info.channel.ring.batch_size = value info.channel.batch_size = value
@property @property
def channels(self) -> list[ChannelInfo]: def channels(self) -> list[ChannelInfo]:
return self._chanmngr.channels 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: def get_channel(self, name: str) -> ChannelInfo:
''' '''
Get underlying ChannelInfo from name 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) await self._chanmngr.add_channel(name, must_exist=must_exist)
async def remove_channel(self, name: str): def remove_channel(self, name: str):
await self._chanmngr.remove_channel(name) self._chanmngr.remove_channel(name)
@acm @acm
async def _open_channel( async def _open_channel(
@ -320,41 +346,45 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]):
name: str, name: str,
must_exist: bool = False must_exist: bool = False
) -> AsyncContextManager[PublisherChannels]: ) -> AsyncContextManager[RingBufferSendChannel]:
''' '''
Open a ringbuf through `ringd` and attach as send side Open a ringbuf through `ringd` and attach as send side
''' '''
async with ( if must_exist:
ringd.open_ringbuf( ringd_fn = ringd.attach_ringbuf
name=name, kwargs = {}
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)
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: async def _channel_task(self, info: ChannelInfo) -> None:
''' '''
Forever get current runtime info for channel, wait on its next pending Wait forever until channel cancellation
payloads update event then drain all into send channel.
''' '''
try: try:
async for msg in info.channel.rchan: await trio.sleep_forever()
await info.channel.ring.send(msg)
except trio.Cancelled: except trio.Cancelled:
... ...
@ -362,11 +392,9 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]):
async def send(self, msg: bytes): async def send(self, msg: bytes):
''' '''
If no output channels connected, wait until one, then fetch the next If no output channels connected, wait until one, then fetch the next
channel based on turn, add the indexed payload and update channel based on turn.
`self._next_turn` & `self._next_index`.
Needs to acquire `self._send_lock` to make sure updates to turn & index Needs to acquire `self._send_lock` to ensure no concurrent calls.
variables dont happen out of order.
''' '''
if self.closed: if self.closed:
@ -380,18 +408,28 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]):
if len(self.channels) == 0: if len(self.channels) == 0:
await self._chanmngr.wait_for_channel() await self._chanmngr.wait_for_channel()
if self._next_turn >= len(self.channels): turn = self._get_next_turn()
self._next_turn = 0
info = self.channels[self._next_turn] info = self.channels[turn]
await info.channel.schan.send(msg) 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 def flush(self, new_batch_size: int | None = None):
async with self._chanmngr.lock:
for info in self.channels: for info in self.channels:
await info.channel.ring.flush(new_batch_size=new_batch_size) try:
await info.channel.flush(new_batch_size=new_batch_size)
except trio.ClosedResourceError:
...
async def __aenter__(self): async def __aenter__(self):
self._chanmngr.open() self._chanmngr.open()
@ -403,41 +441,12 @@ class RingBufferPublisher(trio.abc.SendChannel[bytes]):
log.warning('tried to close RingBufferPublisher but its already closed...') log.warning('tried to close RingBufferPublisher but its already closed...')
return return
with trio.CancelScope(shield=True):
await self._chanmngr.close() await self._chanmngr.close()
self._is_closed = True 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]): class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]):
''' '''
Use ChannelManager to create a multi ringbuf receiver that can Use ChannelManager to create a multi ringbuf receiver that can
@ -458,10 +467,15 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]):
self, self,
n: trio.Nursery, 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 # if connecting to a publisher that has already sent messages set
# to the next expected payload index this subscriber will receive # to the next expected payload index this subscriber will receive
start_index: int = 0 start_index: int = 0
): ):
self._buf_size = buf_size
self._chanmngr = ChannelManager[RingBufferReceiveChannel]( self._chanmngr = ChannelManager[RingBufferReceiveChannel](
n, n,
self._open_channel, self._open_channel,
@ -488,8 +502,8 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]):
async def add_channel(self, name: str, must_exist: bool = False): async def add_channel(self, name: str, must_exist: bool = False):
await self._chanmngr.add_channel(name, must_exist=must_exist) await self._chanmngr.add_channel(name, must_exist=must_exist)
async def remove_channel(self, name: str): def remove_channel(self, name: str):
await self._chanmngr.remove_channel(name) self._chanmngr.remove_channel(name)
@acm @acm
async def _open_channel( async def _open_channel(
@ -502,11 +516,20 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]):
''' '''
Open a ringbuf through `ringd` and attach as receiver side 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 ( async with (
ringd.open_ringbuf( ringd_fn(
name=name, name=name,
must_exist=must_exist, **kwargs
) as token, ) as token,
attach_to_ringbuf_receiver(token) as chan attach_to_ringbuf_receiver(token) as chan
): ):
yield chan yield chan
@ -554,7 +577,6 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]):
async def aclose(self) -> None: async def aclose(self) -> None:
if self.closed: if self.closed:
log.warning('tried to close RingBufferSubscriber but its already closed...')
return return
await self._chanmngr.close() await self._chanmngr.close()
@ -562,26 +584,241 @@ class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]):
await self._rchan.aclose() await self._rchan.aclose()
self._is_closed = True 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 @acm
async def open_ringbuf_subscriber( 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, 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]: ) -> AsyncContextManager[RingBufferPublisher]:
''' '''
Open a new ringbuf subscriber Open a new ringbuf subscriber
''' '''
async with ( async with (
trio.open_nursery() as n, trio.open_nursery(strict_exception_groups=False) as n,
RingBufferSubscriber( RingBufferSubscriber(
n, n,
buf_size=buf_size
) as subscriber ) as subscriber
): ):
# maybe monkey patch `.receive` to use indexed payloads
if guarantee_order: if guarantee_order:
order_receive_channel(subscriber) 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 yield subscriber
if force_cancel: if force_cancel:

View File

@ -25,6 +25,7 @@ open_ringbuf acm, will automatically contact ringd.
''' '''
import os import os
import tempfile import tempfile
from typing import AsyncContextManager
from pathlib import Path from pathlib import Path
from contextlib import ( from contextlib import (
asynccontextmanager as acm asynccontextmanager as acm
@ -33,111 +34,118 @@ from dataclasses import dataclass
import trio import trio
import tractor import tractor
from tractor.linux import send_fds, recv_fds from tractor.linux import (
send_fds,
recv_fds,
)
import tractor.ipc._ringbuf as ringbuf import tractor.ipc._ringbuf as ringbuf
from tractor.ipc._ringbuf import RBToken from tractor.ipc._ringbuf import RBToken
log = tractor.log.get_logger(__name__) log = tractor.log.get_logger(__name__)
# log = tractor.log.get_console_log(level='info')
class RingNotFound(Exception): '''
... Daemon implementation
'''
_ringd_actor_name = 'ringd' _ringd_actor_name: str = 'ringd'
_root_key = _ringd_actor_name + f'-{os.getpid()}'
_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 @dataclass
class RingInfo: class RingInfo:
token: RBToken token: RBToken
creator: str creator: str
unlink: trio.Event()
_rings: dict[str, RingInfo] = {} _rings: dict[str, RingInfo] = {}
_ring_lock = trio.StrictFIFOLock()
def _maybe_get_ring(name: str) -> RingInfo | None: 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): 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 _rings[name] = info
def _destroy_ring(name: str): 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] 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 @tractor.context
async def _pass_fds( async def _pass_fds(
ctx: tractor.Context, ctx: tractor.Context,
name: str, name: str,
sock_path: str sock_path: str
): ):
global _rings '''
info = _maybe_get_ring(name) Ringd endpoint to request passing fds of a ring.
if not info: Supports passing fullname or not (ringd actor name and pid before ring
raise RingNotFound(f'Ring \"{name}\" not found!') 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
# start fd passing context using socket on `sock_path`
async with send_fds(token.fds, sock_path): async with send_fds(token.fds, sock_path):
log.info(f'connected to {sock_path} for fd passing') 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) await ctx.started(token)
# send_fds will block until receive side acks
log.info(f'fds {token.fds} sent') log.info(f'ring {name} fds: {token.fds}, sent')
return token
@tractor.context @tractor.context
@ -145,60 +153,105 @@ async def _open_ringbuf(
ctx: tractor.Context, ctx: tractor.Context,
caller: str, caller: str,
name: str, name: str,
buf_size: int = 10 * 1024, buf_size: int = 10 * 1024
must_exist: bool = False,
): ):
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: if maybe_info:
log.info(f'ring {name} exists, {caller} attached') raise RuntimeError(
f'Tried to create ringbuf but it already exists: {name}'
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}'
) )
fullname = _make_ring_name(name)
with ringbuf.open_ringbuf( with ringbuf.open_ringbuf(
_root_key + name, fullname,
buf_size=buf_size buf_size=buf_size
) as token: ) as token:
unlink_event = trio.Event()
_insert_ring( _insert_ring(
name, name,
RingInfo( RingInfo(
token=token, token=token,
creator=caller, creator=caller,
unlink=unlink_event,
) )
) )
_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}') log.info(f'ring {name} created by {caller}')
await ctx.started(os.getpid()) await trio.sleep_forever()
async with ctx.open_stream() as stream: finally:
await stream.receive()
await unlink_event.wait()
_destroy_ring(name) _destroy_ring(name)
log.info(f'ring {name} destroyed by {caller}') 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 @acm
async def open_ringd(**kwargs) -> tractor.Portal: async def open_ringd(**kwargs) -> tractor.Portal:
'''
Spawn new ringd actor.
'''
async with tractor.open_nursery(**kwargs) as an: async with tractor.open_nursery(**kwargs) as an:
portal = await an.start_actor( portal = await an.start_actor(
_ringd_actor_name, _ringd_actor_name,
@ -210,21 +263,69 @@ async def open_ringd(**kwargs) -> tractor.Portal:
@acm @acm
async def wait_for_ringd() -> tractor.Portal: async def wait_for_ringd() -> tractor.Portal:
'''
Wait for ringd actor to be up.
'''
async with tractor.wait_for_actor( async with tractor.wait_for_actor(
_ringd_actor_name _ringd_actor_name
) as portal: ) as portal:
yield 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 @acm
async def open_ringbuf( async def open_ringbuf(
name: str, name: str,
buf_size: int = 10 * 1024, buf_size: int = 10 * 1024,
) -> AsyncContextManager[RBToken]:
'''
Create a new ring and retrieve its fds.
must_exist: bool = False, '''
) -> RBToken:
actor = tractor.current_actor() actor = tractor.current_actor()
async with ( async with (
wait_for_ringd() as ringd, wait_for_ringd() as ringd,
@ -234,12 +335,67 @@ async def open_ringbuf(
caller=actor.name, caller=actor.name,
name=name, name=name,
buf_size=buf_size, buf_size=buf_size,
must_exist=must_exist ) as (ctx, fullname),
) as (rd_ctx, ringd_pid),
rd_ctx.open_stream() as _stream,
): ):
token = await _attach_to_ring(ringd_pid, name) token = await _request_ring_fds(fullname)
log.info(f'attached to {token}') log.info(f'{actor.name} opened {token}')
try:
yield token 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()