Compare commits

...

37 Commits

Author SHA1 Message Date
Guillermo Rodriguez 9b5341b670
Log warning instead of exception on pubsub cancelled 2025-04-17 12:08:16 -03:00
Guillermo Rodriguez a49dbc7b97
Add fix for cases where sockname len > 100 2025-04-17 12:07:40 -03:00
Guillermo Rodriguez 6c24787f77
Disable parent channel append on get_peer_by_name to_scan 2025-04-13 14:19:22 -03:00
Guillermo Rodriguez 71ff519d71
Only set shield flag when trio nursery mode is used 2025-04-13 14:19:02 -03:00
Guillermo Rodriguez ddfb1d0a7b
Fix chan manager close remove_channel call 2025-04-13 14:02:41 -03:00
Guillermo Rodriguez 19ef8df9fd
Check if fdshare module is enable on share_fds function 2025-04-13 14:02:09 -03:00
Guillermo Rodriguez 73099fa1ba
Adapt ringbuf pubsub to new RBToken owner system 2025-04-13 13:36:02 -03:00
Guillermo Rodriguez b66ef12c2e
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
2025-04-13 13:31:44 -03:00
Guillermo Rodriguez 8f400f6524
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
2025-04-10 13:13:08 -03:00
Guillermo Rodriguez 8f5d968078
Tweaks to make cancellation happen correctly on ringbuf receiver & fix test log msg 2025-04-10 11:27:52 -03:00
Guillermo Rodriguez 3327da8847
Refactor generate_sample_messages to be a generator and use numpy 2025-04-06 21:59:14 -03:00
Guillermo Rodriguez a15b852b18
Add trio resource semantics to ring pubsub 2025-04-06 21:19:39 -03:00
Guillermo Rodriguez 7b668c2f33
RingBufferReceiveChannel fixes for the non clean eof case, add comments 2025-04-06 21:16:55 -03:00
Guillermo Rodriguez 2bf8ce84cf
Add trio resource semantics to eventfd 2025-04-06 21:04:18 -03:00
Guillermo Rodriguez 0de70172fc
Adhere to trio semantics on channels for closed and busy resource cases 2025-04-06 17:02:15 -03:00
Guillermo Rodriguez 5fbd6aced7
Fix test docstring 2025-04-04 02:59:50 -03:00
Guillermo Rodriguez 1c5bebfca6
Enable ordering assertion & simplify some parts of test 2025-04-04 02:57:38 -03:00
Guillermo Rodriguez 53c98958b1
Add header to generic chan orderers 2025-04-04 02:47:27 -03:00
Guillermo Rodriguez 3c1873c68a
Fully test and fix bugs on _ringbuf._pubsub
Add generic channel orderer
2025-04-04 02:44:45 -03:00
Guillermo Rodriguez 0b9c2de3ad
Improve ringd ringbuf lifecycle
Unlink sock after use in fdshare
2025-04-04 02:41:50 -03:00
Guillermo Rodriguez eb20e5ea8d
Rename RingBuff -> RingBuffer
Combine RingBuffer stream and channel apis
Implement RingBufferReceiveChannel.receive_nowait
Make msg generator calculate hash
2025-04-04 02:36:59 -03:00
Guillermo Rodriguez 70d72fd173
Woops fix old typing Self stuff 2025-04-03 13:11:00 -03:00
Guillermo Rodriguez 26efa3b25f
Add header and fix white lines 2025-04-03 12:35:09 -03:00
Guillermo Rodriguez 20f7ba4c2c
Switch to using typing.Protocl instead of abc.ABC on ChannelManager, improve abstraction and add comments 2025-04-03 12:34:40 -03:00
Guillermo Rodriguez 57d48a3af7
Improve error handling in fdshare functions, add comments 2025-04-03 11:48:07 -03:00
Guillermo Rodriguez eef98bc826
Dont use relative import on ringd 2025-04-02 15:17:56 -03:00
Guillermo Rodriguez dfc0254995
recv_fds doesnt need to be an acm 2025-04-02 15:00:40 -03:00
Guillermo Rodriguez b8d1fd6978
Add LICENSE headers and module docstring on new files 2025-04-02 14:56:48 -03:00
Guillermo Rodriguez 739e728065
Add ringd test, which also tests fd share 2025-04-02 14:45:21 -03:00
Guillermo Rodriguez d01ad12b3b
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
2025-04-02 14:39:02 -03:00
Guillermo Rodriguez bab265b2d8
Important RingBuffBytesSender fix on non batched mode! & downgrade nix-shell python to lowest supported 2025-03-27 20:36:46 -03:00
Guillermo Rodriguez 010874bed5
Catch trio cancellation on RingBuffReceiver bg eof listener task, add batched mode to RingBuffBytesSender 2025-03-27 20:36:46 -03:00
Guillermo Rodriguez ea010ab46a
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
2025-03-27 20:36:46 -03:00
Guillermo Rodriguez be7fc89ae9
Add direct ctx managers for RB channels 2025-03-27 20:36:46 -03:00
Guillermo Rodriguez 2a9a78651b
Improve test_ringbuf test, drop MsgTransport ring buf impl for now in favour of a trio.abc.Channel[bytes] impl, add docstrings 2025-03-27 20:36:46 -03:00
Guillermo Rodriguez be818a720a
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
2025-03-27 20:36:46 -03:00
Guillermo Rodriguez ba353bf46f
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
2025-03-27 20:36:46 -03:00
20 changed files with 2910 additions and 427 deletions

View File

@ -13,6 +13,6 @@ pkgs.mkShell {
shellHook = ''
set -e
uv venv .venv --python=3.12
uv venv .venv --python=3.11
'';
}

View File

@ -63,6 +63,7 @@ dev = [
"pyperclip>=1.9.0",
"prompt-toolkit>=3.0.50",
"xonsh>=0.19.2",
"numpy>=2.2.4", # used for fast test sample gen
]
# TODO, add these with sane versions; were originally in
# `requirements-docs.txt`..

View File

@ -0,0 +1,66 @@
import trio
import pytest
from tractor.linux.eventfd import (
open_eventfd,
EFDReadCancelled,
EventFD
)
def test_read_cancellation():
'''
Ensure EventFD.read raises EFDReadCancelled if EventFD.close()
is called.
'''
fd = open_eventfd()
async def bg_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(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)

View File

@ -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)

View File

@ -1,35 +1,50 @@
import time
import hashlib
import trio
import pytest
import tractor
from tractor.ipc import (
from tractor.ipc._ringbuf import (
open_ringbuf,
open_ringbuf_pair,
attach_to_ringbuf_receiver,
attach_to_ringbuf_sender,
attach_to_ringbuf_channel,
RBToken,
RingBuffSender,
RingBuffReceiver
)
from tractor._testing.samples import generate_sample_messages
from tractor._testing.samples import (
generate_single_byte_msgs,
RandomBytesGenerator
)
@tractor.context
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()
recvd_bytes += len(msg)
) -> str:
'''
Sub-actor used in `test_ringbuf`.
# make sure we dont hold any memoryviews
# before the ctx manager aclose()
msg = None
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')
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)
end_ts = time.time()
elapsed = end_ts - start_ts
@ -38,6 +53,10 @@ 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()
@tractor.context
@ -46,17 +65,37 @@ async def child_write_shm(
msg_amount: int,
rand_min: int,
rand_max: int,
token: RBToken,
buf_size: int
) -> 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.
'''
rng = RandomBytesGenerator(
msg_amount,
rand_min=rand_min,
rand_max=rand_max,
)
await ctx.started(total_bytes)
async with RingBuffSender(token) as sender:
for msg in msgs:
await sender.send_all(msg)
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)
if rng.msgs_generated % rng.recommended_log_interval == 0:
print(f'wrote {rng.msgs_generated} msgs')
print('writer exit')
return rng.hexdigest
@pytest.mark.parametrize(
@ -83,83 +122,90 @@ def test_ringbuf(
rand_max: int,
buf_size: int
):
async def main():
with open_ringbuf(
'test_ringbuf',
buf_size=buf_size
) as token:
proc_kwargs = {
'pass_fds': (token.write_eventfd, token.wrap_eventfd)
}
'''
- Open a new ring buf on root actor
- 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
common_kwargs = {
'msg_amount': msg_amount,
'token': token,
}
'''
async def main():
async with tractor.open_nursery() as an:
send_p = await an.start_actor(
'ring_sender',
enable_modules=[__name__],
proc_kwargs=proc_kwargs
enable_modules=[
__name__,
'tractor.linux._fdshare'
],
)
recv_p = await an.start_actor(
'ring_receiver',
enable_modules=[__name__],
proc_kwargs=proc_kwargs
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,
**common_kwargs
) as (sctx, total_bytes),
buf_size=buf_size
) as (sctx, token),
recv_p.open_context(
child_read_shm,
**common_kwargs,
total_bytes=total_bytes,
) as (sctx, _sent),
token=token,
) as (rctx, _),
):
await recv_p.result()
sent_hash = await sctx.result()
recvd_hash = await rctx.result()
await send_p.cancel_actor()
await recv_p.cancel_actor()
assert sent_hash == recvd_hash
await an.cancel()
trio.run(main)
@tractor.context
async def child_blocked_receiver(
ctx: tractor.Context,
token: RBToken
):
async with RingBuffReceiver(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()
def test_ring_reader_cancel():
def test_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 (
tractor.open_nursery() as an,
RingBuffSender(token) as _sender,
):
async with tractor.open_nursery() as an:
recv_p = await an.start_actor(
'ring_blocked_receiver',
enable_modules=[__name__],
proc_kwargs={
'pass_fds': (token.write_eventfd, token.wrap_eventfd)
}
enable_modules=[
__name__,
'tractor.linux._fdshare'
],
)
async with (
recv_p.open_context(
child_blocked_receiver,
token=token
) as (sctx, _sent),
) as (sctx, token),
attach_to_ringbuf_sender(token),
):
await trio.sleep(1)
await trio.sleep(.1)
await an.cancel()
@ -168,38 +214,161 @@ def test_ring_reader_cancel():
@tractor.context
async def child_blocked_sender(
ctx: tractor.Context,
token: RBToken
):
async with RingBuffSender(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')
def test_ring_sender_cancel():
def test_sender_cancel():
'''
Test that a sender blocked on eventfd(2) read responds to
cancellation.
'''
async def main():
with open_ringbuf(
'test_ring_cancel_sender',
buf_size=1
) as token:
async with tractor.open_nursery() as an:
recv_p = await an.start_actor(
'ring_blocked_sender',
enable_modules=[__name__],
proc_kwargs={
'pass_fds': (token.write_eventfd, token.wrap_eventfd)
}
enable_modules=[
__name__,
'tractor.linux._fdshare'
],
)
async with (
recv_p.open_context(
child_blocked_sender,
token=token
) as (sctx, _sent),
) as (sctx, token),
attach_to_ringbuf_receiver(token)
):
await trio.sleep(1)
await trio.sleep(.1)
await an.cancel()
with pytest.raises(tractor._exceptions.ContextCancelled):
trio.run(main)
def test_receiver_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 = generate_single_byte_msgs(100)
msgs = []
async def main():
async with (
tractor.open_nursery(),
open_ringbuf(
'test_ringbuf_max_bytes',
buf_size=10
) as token,
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)
@tractor.context
async def child_channel_sender(
ctx: tractor.Context,
msg_amount_min: int,
msg_amount_max: int,
token_in: RBToken,
token_out: RBToken
):
import random
rng = RandomBytesGenerator(
random.randint(msg_amount_min, msg_amount_max),
rand_min=256,
rand_max=1024,
)
async with attach_to_ringbuf_channel(
token_in,
token_out
) as chan:
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():
msg_amount_min = 100
msg_amount_max = 1000
mods = [
__name__,
'tractor.linux._fdshare'
]
async def main():
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 (
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.update(msg)
sent_hash = await ctx.result()
assert recvd_hash.hexdigest() == sent_hash
await an.cancel()
trio.run(main)

View File

@ -114,9 +114,15 @@ def get_peer_by_name(
'''
actor: Actor = current_actor()
to_scan: dict[tuple, list[Channel]] = actor._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 and pchan.uid not in to_scan:
# to_scan[pchan.uid].append(pchan)
for aid, chans in to_scan.items():
_, peer_name = aid

View File

@ -1,35 +1,99 @@
import os
import random
import hashlib
import numpy as np
def generate_sample_messages(
def generate_single_byte_msgs(amount: int) -> bytes:
'''
Generate a byte instance of length `amount` with repeating ASCII digits 0..9.
'''
# 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()
class RandomBytesGenerator:
'''
Generate bytes msgs for tests.
messages will have the following format:
b'[{i:08}]' + random_bytes
so for message index 25:
b'[00000025]' + random_bytes
also generates sha256 hash of msgs.
'''
def __init__(
self,
amount: int,
rand_min: int = 0,
rand_max: int = 0,
silent: bool = False
) -> tuple[list[bytes], int]:
rand_max: int = 0
):
if rand_max < rand_min:
raise ValueError('rand_max must be >= rand_min')
msgs = []
size = 0
self._amount = amount
self._rand_min = rand_min
self._rand_max = rand_max
self._index = 0
self._hasher = hashlib.sha256()
self._total_bytes = 0
if not silent:
print(f'\ngenerating {amount} messages...')
self._lengths = np.random.randint(
rand_min,
rand_max + 1,
size=amount,
dtype=np.int32
)
for i in range(amount):
msg = f'[{i:08}]'.encode('utf-8')
def __iter__(self):
return self
if rand_max > 0:
msg += os.urandom(
random.randint(rand_min, rand_max))
def __next__(self) -> bytes:
if self._index == self._amount:
raise StopIteration
size += len(msg)
header = f'[{self._index:08}]'.encode('utf-8')
msgs.append(msg)
length = int(self._lengths[self._index])
msg = header + np.random.bytes(length)
if not silent and i and i % 10_000 == 0:
print(f'{i} generated')
self._hasher.update(msg)
self._total_bytes += length
self._index += 1
if not silent:
print(f'done, {size:,} bytes in total')
return msg
return msgs, size
@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:
return 10_000
else:
return 1000

View File

@ -13,10 +13,6 @@
# You should have received a copy of the GNU Affero General Public License
# along with this program. If not, see <https://www.gnu.org/licenses/>.
import platform
from ._transport import MsgTransport as MsgTransport
from ._tcp import (
@ -29,22 +25,3 @@ from ._chan import (
get_msg_transport as get_msg_transport,
Channel as Channel
)
if platform.system() == 'Linux':
from ._linux import (
EFD_SEMAPHORE as EFD_SEMAPHORE,
EFD_CLOEXEC as EFD_CLOEXEC,
EFD_NONBLOCK as EFD_NONBLOCK,
open_eventfd as open_eventfd,
write_eventfd as write_eventfd,
read_eventfd as read_eventfd,
close_eventfd as close_eventfd,
EventFD as EventFD,
)
from ._ringbuf import (
RBToken as RBToken,
RingBuffSender as RingBuffSender,
RingBuffReceiver as RingBuffReceiver,
open_ringbuf as open_ringbuf
)

View File

@ -1,253 +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 <https://www.gnu.org/licenses/>.
'''
IPC Reliable RingBuffer implementation
'''
from __future__ import annotations
from contextlib import contextmanager as cm
from multiprocessing.shared_memory import SharedMemory
import trio
from msgspec import (
Struct,
to_builtins
)
from ._linux import (
EFD_NONBLOCK,
open_eventfd,
EventFD
)
from ._mp_bs import disable_mantracker
disable_mantracker()
class RBToken(Struct, frozen=True):
'''
RingBuffer token contains necesary info to open the two
eventfds and the shared memory
'''
shm_name: str
write_eventfd: int
wrap_eventfd: int
buf_size: int
def as_msg(self):
return to_builtins(self)
@classmethod
def from_msg(cls, msg: dict) -> RBToken:
if isinstance(msg, RBToken):
return msg
return RBToken(**msg)
@cm
def open_ringbuf(
shm_name: str,
buf_size: int = 10 * 1024,
write_efd_flags: int = 0,
wrap_efd_flags: int = 0
) -> RBToken:
shm = SharedMemory(
name=shm_name,
size=buf_size,
create=True
)
try:
token = RBToken(
shm_name=shm_name,
write_eventfd=open_eventfd(flags=write_efd_flags),
wrap_eventfd=open_eventfd(flags=wrap_efd_flags),
buf_size=buf_size
)
yield token
finally:
shm.unlink()
class RingBuffSender(trio.abc.SendStream):
'''
IPC Reliable Ring Buffer sender side implementation
`eventfd(2)` is used for wrap around sync, and also to signal
writes to the reader.
'''
def __init__(
self,
token: RBToken,
start_ptr: int = 0,
):
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._ptr = start_ptr
@property
def key(self) -> str:
return self._shm.name
@property
def size(self) -> int:
return self._shm.size
@property
def ptr(self) -> int:
return self._ptr
@property
def write_fd(self) -> int:
return self._write_event.fd
@property
def wrap_fd(self) -> int:
return self._wrap_event.fd
async def send_all(self, data: bytes | bytearray | memoryview):
# while data is larger than the remaining buf
target_ptr = self.ptr + len(data)
while target_ptr > self.size:
# write all bytes that fit
remaining = self.size - self.ptr
self._shm.buf[self.ptr:] = data[:remaining]
# signal write and wait for reader wrap around
self._write_event.write(remaining)
await self._wrap_event.read()
# wrap around and trim already written bytes
self._ptr = 0
data = data[remaining:]
target_ptr = self._ptr + len(data)
# remaining data fits on buffer
self._shm.buf[self.ptr:target_ptr] = data
self._write_event.write(len(data))
self._ptr = target_ptr
async def wait_send_all_might_not_block(self):
raise NotImplementedError
async def aclose(self):
self._write_event.close()
self._wrap_event.close()
self._shm.close()
async def __aenter__(self):
self._write_event.open()
self._wrap_event.open()
return self
class RingBuffReceiver(trio.abc.ReceiveStream):
'''
IPC Reliable Ring Buffer receiver side implementation
`eventfd(2)` is used for wrap around sync, and also to signal
writes to the reader.
'''
def __init__(
self,
token: RBToken,
start_ptr: int = 0,
flags: int = 0
):
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._ptr = start_ptr
self._flags = flags
@property
def key(self) -> str:
return self._shm.name
@property
def size(self) -> int:
return self._shm.size
@property
def ptr(self) -> int:
return self._ptr
@property
def write_fd(self) -> int:
return self._write_event.fd
@property
def wrap_fd(self) -> int:
return self._wrap_event.fd
async def receive_some(
self,
max_bytes: int | None = None,
nb_timeout: float = 0.1
) -> memoryview:
# if non blocking eventfd enabled, do polling
# until next write, this allows signal handling
if self._flags | EFD_NONBLOCK:
delta = None
while delta is None:
try:
delta = await self._write_event.read()
except OSError as e:
if e.errno == 'EAGAIN':
continue
raise e
else:
delta = await self._write_event.read()
# fetch next segment and advance ptr
next_ptr = self._ptr + delta
segment = self._shm.buf[self._ptr:next_ptr]
self._ptr = next_ptr
if self.ptr == self.size:
# reached the end, signal wrap around
self._ptr = 0
self._wrap_event.write(1)
return segment
async def aclose(self):
self._write_event.close()
self._wrap_event.close()
self._shm.close()
async def __aenter__(self):
self._write_event.open()
self._wrap_event.open()
return self

View File

@ -0,0 +1,920 @@
# 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 <https://www.gnu.org/licenses/>.
'''
IPC Reliable RingBuffer implementation
'''
from __future__ import annotations
import struct
from typing import (
ContextManager,
AsyncContextManager
)
from contextlib import (
contextmanager as cm,
asynccontextmanager as acm
)
from multiprocessing.shared_memory import SharedMemory
import trio
from msgspec import (
Struct,
to_builtins
)
from tractor.log import get_logger
from tractor._exceptions import (
InternalError
)
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__)
disable_mantracker()
_DEFAULT_RB_SIZE = 10 * 1024
class RBToken(Struct, frozen=True):
'''
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 # size in bytes of underlying shared memory buffer
def as_msg(self):
return to_builtins(self)
@classmethod
def from_msg(cls, msg: dict) -> RBToken:
if isinstance(msg, RBToken):
return msg
return RBToken(**msg)
@property
def fds(self) -> tuple[int, int, int]:
return (
self.write_eventfd,
self.wrap_eventfd,
self.eof_eventfd
)
def alloc_ringbuf(
shm_name: str,
buf_size: int = _DEFAULT_RB_SIZE,
) -> tuple[SharedMemory, RBToken]:
'''
Allocate OS resources for a ringbuf.
'''
shm = SharedMemory(
name=shm_name,
size=buf_size,
create=True
)
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_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)]:
'''
Handle resources for a ringbuf pair to be used for
bidirectional messaging.
'''
with open_ringbufs_sync(
[
f'{shm_name}.send',
f'{shm_name}.recv'
],
buf_sizes=buf_size
) as tokens:
yield tokens
@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
'''
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 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: 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] = []
# 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()
# ensure no concurrent `.flush()` calls
self._flush_lock = trio.StrictFIFOLock()
@property
def closed(self) -> bool:
return self._is_closed
@property
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._token.buf_size
@property
def ptr(self) -> int:
return self._ptr
@property
def write_fd(self) -> int:
return self._write_event.fd
@property
def wrap_fd(self) -> int:
return self._wrap_event.fd
@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):
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:
# 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._wait_wrap()
# wrap around and trim already written bytes
self._ptr = 0
data = data[remaining:]
target_ptr = self._ptr + len(data)
# remaining data fits on buffer
self._shm.buf[self.ptr:target_ptr] = data
self._write_event.write(len(data))
self._ptr = target_ptr
async def wait_send_all_might_not_block(self):
return
async def flush(
self,
new_batch_size: int | None = None
) -> None:
if self.closed:
raise trio.ClosedResourceError
async with self._flush_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:
if self.closed:
raise trio.ClosedResourceError
if self._send_lock.locked():
raise trio.BusyResourceError
async with self._send_lock:
msg: bytes = struct.pack("<I", len(value)) + value
if self.batch_size == 1:
if len(self._batch) > 0:
await self.flush()
await self.send_all(msg)
return
self._batch.append(msg)
if self.must_flush:
await self.flush()
def open(self):
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()
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):
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()
self._is_closed = True
async def aclose(self):
if self.closed:
return
self._close()
async def __aenter__(self):
self.open()
return self
class RingBufferReceiveChannel(trio.abc.ReceiveChannel[bytes]):
'''
Ring Buffer receiver side implementation
Do not use directly! manage with `attach_to_ringbuf_receiver`
after having opened a ringbuf context with `open_ringbuf`.
'''
def __init__(
self,
token: RBToken,
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
def closed(self) -> bool:
return self._is_closed
@property
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._token.buf_size
@property
def ptr(self) -> int:
return self._ptr
@property
def write_fd(self) -> int:
return self._write_event.fd
@property
def wrap_fd(self) -> int:
return self._wrap_event.fd
@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
`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()
except EFDReadCancelled:
...
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
`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")
# 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.eof_was_signaled:
return b''
# signal the need to wait on `write_event`
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 self.closed:
raise trio.ClosedResourceError
if self._receive_some_lock.locked():
raise trio.BusyResourceError
async with self._receive_some_lock:
try:
# attempt direct read
return self.receive_nowait(max_bytes=max_bytes)
except trio.WouldBlock as e:
# we have read all we can, see if new data is available
if not self.eof_was_signaled:
# 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
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
# trio.WouldBlock and will push remaining until EOF
return self.receive_nowait(max_bytes=max_bytes)
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
# 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)
async def receive_exactly(self, num_bytes: int) -> bytes:
'''
Fetch bytes until we read exactly `num_bytes` or EOC.
'''
if self.closed:
raise trio.ClosedResourceError
if self._receive_exactly_lock.locked():
raise trio.BusyResourceError
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
return payload
async def receive(self) -> bytes:
'''
Receive a complete payload or raise EOC
'''
if self.closed:
raise trio.ClosedResourceError
if self._receive_lock.locked():
raise trio.BusyResourceError
async with self._receive_lock:
header: bytes = await self.receive_exactly(4)
size: int
size, = struct.unpack("<I", header)
if size == 0:
raise trio.EndOfChannel
return await self.receive_exactly(size)
def open(self):
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()
self._is_closed = False
except Exception as e:
e.add_note(f'while opening receiver for {self._token.as_msg()}')
raise e
def close(self):
if self._cleanup:
self._write_event.close()
self._wrap_event.close()
self._eof_event.close()
self._shm.close()
self._is_closed = True
async def aclose(self):
if self.closed:
return
self.close()
async def __aenter__(self):
self.open()
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(
token: RBToken,
cleanup: bool = True
) -> AsyncContextManager[RingBufferReceiveChannel]:
'''
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(
token,
cleanup=cleanup
) as receiver
):
n.start_soon(receiver._eof_monitor_task)
yield receiver
@acm
async def attach_to_ringbuf_sender(
token: RBToken,
batch_size: int = 1,
cleanup: bool = True
) -> AsyncContextManager[RingBufferSendChannel]:
'''
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,
cleanup=cleanup
) as sender:
yield sender
class RingBufferChannel(trio.abc.Channel[bytes]):
'''
Combine `RingBufferSendChannel` and `RingBufferReceiveChannel`
in order to expose the bidirectional `trio.abc.Channel` API.
'''
def __init__(
self,
sender: RingBufferSendChannel,
receiver: RingBufferReceiveChannel
):
self._sender = sender
self._receiver = receiver
@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 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
) -> 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()
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()
async def aclose(self):
await self._receiver.aclose()
await self._sender.aclose()
@acm
async def attach_to_ringbuf_channel(
token_in: RBToken,
token_out: RBToken,
batch_size: int = 1,
cleanup_in: bool = True,
cleanup_out: bool = True,
) -> AsyncContextManager[trio.StapledStream]:
'''
Attach to two previously opened `RBToken`s and return a `RingBufferChannel`
'''
async with (
attach_to_ringbuf_receiver(
token_in,
cleanup=cleanup_in
) as receiver,
attach_to_ringbuf_sender(
token_out,
batch_size=batch_size,
cleanup=cleanup_out
) as sender,
):
yield RingBufferChannel(sender, receiver)

View File

@ -0,0 +1,804 @@
# 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 <https://www.gnu.org/licenses/>.
'''
Ring buffer ipc publish-subscribe mechanism brokered by ringd
can dynamically add new outputs (publisher) or inputs (subscriber)
'''
from typing import (
TypeVar,
Generic,
Callable,
Awaitable,
AsyncContextManager
)
from functools import partial
from contextlib import asynccontextmanager as acm
from dataclasses import dataclass
import trio
import tractor
from tractor.ipc._ringbuf import (
RBToken,
RingBufferSendChannel,
RingBufferReceiveChannel,
attach_to_ringbuf_sender,
attach_to_ringbuf_receiver
)
from tractor.trionics import (
order_send_channel,
order_receive_channel
)
import tractor.linux._fdshare as fdshare
log = tractor.log.get_logger(__name__)
ChannelType = TypeVar('ChannelType')
@dataclass
class ChannelInfo:
token: RBToken
channel: ChannelType
cancel_scope: trio.CancelScope
teardown: trio.Event
class ChannelManager(Generic[ChannelType]):
'''
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] = []
self._is_closed: bool = True
@property
def closed(self) -> bool:
return self._is_closed
@property
def channels(self) -> list[ChannelInfo]:
return self._channels
async def _channel_handler_task(
self,
token: RBToken,
task_status=trio.TASK_STATUS_IGNORED,
**kwargs
):
'''
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(
token,
**kwargs
) as chan:
cancel_scope = trio.CancelScope()
info = ChannelInfo(
token=token,
channel=chan,
cancel_scope=cancel_scope,
teardown=trio.Event()
)
self._channels.append(info)
if len(self) == 1:
self._connect_event.set()
task_status.started()
with cancel_scope:
await self._channel_task(info)
self._maybe_destroy_channel(token.shm_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.
Only use after acquiring lock.
'''
for entry in enumerate(self._channels):
i, info = entry
if info.token.shm_name == name:
return entry
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
info.cancel_scope.cancel()
info.teardown.set()
del self._channels[i]
async def add_channel(
self,
token: RBToken,
**kwargs
):
'''
Add a new channel to be handled
'''
if self.closed:
raise trio.ClosedResourceError
await self._n.start(partial(
self._channel_handler_task,
RBToken.from_msg(token),
**kwargs
))
async def remove_channel(self, name: str):
'''
Remove a channel and stop its handling
'''
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()
async def wait_for_channel(self):
'''
Wait until at least one channel added
'''
if self.closed:
raise trio.ClosedResourceError
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!')
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:
if info.channel.closed:
continue
await self.remove_channel(info.token.shm_name)
self._is_closed = True
'''
Ring buffer publisher & subscribe pattern mediated by `ringd` actor.
'''
class RingBufferPublisher(trio.abc.SendChannel[bytes]):
'''
Use ChannelManager to create a multi ringbuf round robin sender that can
dynamically add or remove more outputs.
Don't instantiate directly, use `open_ringbuf_publisher` acm to manage its
lifecycle.
'''
def __init__(
self,
n: trio.Nursery,
# 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._batch_size: int = batch_size
self.msgs_per_turn = msgs_per_turn
# helper to manage acms + long running tasks
self._chanmngr = ChannelManager[RingBufferSendChannel](
n,
self._open_channel,
self._channel_task
)
# 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
def closed(self) -> bool:
return self._is_closed
@property
def batch_size(self) -> int:
return self._batch_size
@batch_size.setter
def batch_size(self, value: int) -> None:
for info in self.channels:
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
'''
return self._chanmngr[name]
async def add_channel(
self,
token: RBToken,
):
await self._chanmngr.add_channel(token)
async def remove_channel(self, name: str):
await self._chanmngr.remove_channel(name)
@acm
async def _open_channel(
self,
token: RBToken
) -> AsyncContextManager[RingBufferSendChannel]:
async with attach_to_ringbuf_sender(
token,
batch_size=self._batch_size
) as ring:
yield ring
async def _channel_task(self, info: ChannelInfo) -> None:
'''
Wait forever until channel cancellation
'''
await trio.sleep_forever()
async def send(self, msg: bytes):
'''
If no output channels connected, wait until one, then fetch the next
channel based on turn.
Needs to acquire `self._send_lock` to ensure no concurrent calls.
'''
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:
await self._chanmngr.wait_for_channel()
turn = self._get_next_turn()
info = self.channels[turn]
await info.channel.send(msg)
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):
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()
self._is_closed = False
return self
async def aclose(self) -> None:
if self.closed:
log.warning('tried to close RingBufferPublisher but its already closed...')
return
await self._chanmngr.close()
self._is_closed = True
class RingBufferSubscriber(trio.abc.ReceiveChannel[bytes]):
'''
Use ChannelManager to create a multi ringbuf receiver that can
dynamically add or remove more inputs and combine all into a single output.
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
):
self._chanmngr = ChannelManager[RingBufferReceiveChannel](
n,
self._open_channel,
self._channel_task
)
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
def get_channel(self, name: str):
return self._chanmngr[name]
async def add_channel(
self,
token: RBToken
):
await self._chanmngr.add_channel(token)
async def remove_channel(self, name: str):
await self._chanmngr.remove_channel(name)
@acm
async def _open_channel(
self,
token: RBToken
) -> AsyncContextManager[RingBufferSendChannel]:
async with attach_to_ringbuf_receiver(token) as ring:
yield ring
async def _channel_task(self, info: ChannelInfo) -> None:
'''
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`.
'''
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
except trio.ClosedResourceError:
break
async def receive(self) -> bytes:
'''
Receive next in order msg
'''
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:
if self.closed:
return
await self._chanmngr.close()
await self._schan.aclose()
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 _add_pub_channel(
ctx: tractor.Context,
token: RBToken
):
publisher = get_publisher()
await ctx.started()
await publisher.add_channel(token)
@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,
token: RBToken,
cleanup: bool = True,
):
async with (
tractor.find_actor(actor_name) as portal,
portal.open_context(
_add_pub_channel,
token=token
) as (ctx, _)
):
...
try:
yield
except trio.Cancelled:
log.warning(
'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 _add_sub_channel(
ctx: tractor.Context,
token: RBToken
):
subscriber = get_subscriber()
await ctx.started()
await subscriber.add_channel(token)
@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,
token: RBToken,
cleanup: bool = True,
):
async with (
tractor.find_actor(actor_name) as portal,
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, _):
...
'''
High level helpers to open publisher & subscriber
'''
@acm
async def open_ringbuf_publisher(
# 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,
# 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,
batch_size=batch_size
) as publisher
):
if guarantee_order:
order_send_channel(publisher)
if set_module_var:
set_publisher(publisher)
try:
yield publisher
except trio.Cancelled:
with trio.CancelScope(shield=True):
await publisher.aclose()
raise
@acm
async def open_ringbuf_subscriber(
# expect indexed payloads and unwrap them in order
guarantee_order: 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(strict_exception_groups=False) as n,
RingBufferSubscriber(n) 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)
try:
yield subscriber
except trio.Cancelled:
with trio.CancelScope(shield=True):
await subscriber.aclose()
raise

View File

@ -26,7 +26,6 @@ import struct
from typing import (
Any,
Callable,
Type,
)
import msgspec

View File

@ -41,10 +41,10 @@ class MsgTransport(Protocol[MsgType]):
# eventual msg definition/types?
# - https://docs.python.org/3/library/typing.html#typing.Protocol
stream: trio.SocketStream
stream: trio.abc.Stream
drained: list[MsgType]
def __init__(self, stream: trio.SocketStream) -> None:
def __init__(self, stream: trio.abc.Stream) -> None:
...
# XXX: should this instead be called `.sendall()`?

View File

@ -0,0 +1,15 @@
# 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 <https://www.gnu.org/licenses/>.

View File

@ -0,0 +1,316 @@
# 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 <https://www.gnu.org/licenses/>.
'''
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 os
import array
import tempfile
from uuid import uuid4
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):
...
@acm
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 # 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)]
)
# finally wait receiver ack
if await conn.recv(1) != b'A':
raise FDSharingError('did not receive acknowledgement of fd')
conn.close()
sock.close()
os.unlink(sock_path)
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 FDSharingError(f'Expected to receive {amount} fds from {sock_path}, but got EOF')
# 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
# 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}'
)
# unpack SCM_RIGHTS msg
cmsg_level, cmsg_type, cmsg_data = ancdata[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}'
)
if cmsg_type != socket.SCM_RIGHTS:
raise FDSharingError(
f'Expected CMSG type to be SCM_RIGHTS({socket.SCM_RIGHTS}) but got {cmsg_type}'
)
# 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}'
)
# 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)
'''
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.
'''
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}')
_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'
)
# 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,
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

View File

@ -14,7 +14,7 @@
# You should have received a copy of the GNU Affero General Public License
# along with this program. If not, see <https://www.gnu.org/licenses/>.
'''
Linux specifics, for now we are only exposing EventFD
Expose libc eventfd APIs
'''
import os
@ -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,27 +128,76 @@ class EventFD:
self._fd: int = fd
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:
'''
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.
'''
if self.closed:
raise trio.ClosedResourceError
if self._read_lock.locked():
raise trio.BusyResourceError
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
)
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.
'''
return read_eventfd(self._fd)
def open(self):
self._fobj = os.fdopen(self._fd, self._omode)
self._is_closed = False
def close(self):
if self._fobj:
try:
self._fobj.close()
except OSError:
...
if self._cscope:
self._cscope.cancel()
self._is_closed = True
def __enter__(self):
self.open()
return self

View File

@ -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
)

View File

@ -70,6 +70,7 @@ async def maybe_open_nursery(
yield nursery
else:
async with lib.open_nursery(**kwargs) as nursery:
if lib == trio:
nursery.cancel_scope.shield = shield
yield nursery

View File

@ -0,0 +1,108 @@
# 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 <https://www.gnu.org/licenses/>.
'''
Helpers to guarantee ordering of messages through a unordered channel
'''
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

50
uv.lock
View File

@ -180,6 +180,54 @@ wheels = [
{ url = "https://files.pythonhosted.org/packages/23/d8/f15b40611c2d5753d1abb0ca0da0c75348daf1252220e5dda2867bd81062/msgspec-0.19.0-cp313-cp313-win_amd64.whl", hash = "sha256:317050bc0f7739cb30d257ff09152ca309bf5a369854bbf1e57dffc310c1f20f", size = 187432 },
]
[[package]]
name = "numpy"
version = "2.2.4"
source = { registry = "https://pypi.org/simple" }
sdist = { url = "https://files.pythonhosted.org/packages/e1/78/31103410a57bc2c2b93a3597340a8119588571f6a4539067546cb9a0bfac/numpy-2.2.4.tar.gz", hash = "sha256:9ba03692a45d3eef66559efe1d1096c4b9b75c0986b5dff5530c378fb8331d4f", size = 20270701 }
wheels = [
{ url = "https://files.pythonhosted.org/packages/16/fb/09e778ee3a8ea0d4dc8329cca0a9c9e65fed847d08e37eba74cb7ed4b252/numpy-2.2.4-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:e9e0a277bb2eb5d8a7407e14688b85fd8ad628ee4e0c7930415687b6564207a4", size = 21254989 },
{ url = "https://files.pythonhosted.org/packages/a2/0a/1212befdbecab5d80eca3cde47d304cad986ad4eec7d85a42e0b6d2cc2ef/numpy-2.2.4-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:9eeea959168ea555e556b8188da5fa7831e21d91ce031e95ce23747b7609f8a4", size = 14425910 },
{ url = "https://files.pythonhosted.org/packages/2b/3e/e7247c1d4f15086bb106c8d43c925b0b2ea20270224f5186fa48d4fb5cbd/numpy-2.2.4-cp311-cp311-macosx_14_0_arm64.whl", hash = "sha256:bd3ad3b0a40e713fc68f99ecfd07124195333f1e689387c180813f0e94309d6f", size = 5426490 },
{ url = "https://files.pythonhosted.org/packages/5d/fa/aa7cd6be51419b894c5787a8a93c3302a1ed4f82d35beb0613ec15bdd0e2/numpy-2.2.4-cp311-cp311-macosx_14_0_x86_64.whl", hash = "sha256:cf28633d64294969c019c6df4ff37f5698e8326db68cc2b66576a51fad634880", size = 6967754 },
{ url = "https://files.pythonhosted.org/packages/d5/ee/96457c943265de9fadeb3d2ffdbab003f7fba13d971084a9876affcda095/numpy-2.2.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2fa8fa7697ad1646b5c93de1719965844e004fcad23c91228aca1cf0800044a1", size = 14373079 },
{ url = "https://files.pythonhosted.org/packages/c5/5c/ceefca458559f0ccc7a982319f37ed07b0d7b526964ae6cc61f8ad1b6119/numpy-2.2.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f4162988a360a29af158aeb4a2f4f09ffed6a969c9776f8f3bdee9b06a8ab7e5", size = 16428819 },
{ url = "https://files.pythonhosted.org/packages/22/31/9b2ac8eee99e001eb6add9fa27514ef5e9faf176169057a12860af52704c/numpy-2.2.4-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:892c10d6a73e0f14935c31229e03325a7b3093fafd6ce0af704be7f894d95687", size = 15881470 },
{ url = "https://files.pythonhosted.org/packages/f0/dc/8569b5f25ff30484b555ad8a3f537e0225d091abec386c9420cf5f7a2976/numpy-2.2.4-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:db1f1c22173ac1c58db249ae48aa7ead29f534b9a948bc56828337aa84a32ed6", size = 18218144 },
{ url = "https://files.pythonhosted.org/packages/5e/05/463c023a39bdeb9bb43a99e7dee2c664cb68d5bb87d14f92482b9f6011cc/numpy-2.2.4-cp311-cp311-win32.whl", hash = "sha256:ea2bb7e2ae9e37d96835b3576a4fa4b3a97592fbea8ef7c3587078b0068b8f09", size = 6606368 },
{ url = "https://files.pythonhosted.org/packages/8b/72/10c1d2d82101c468a28adc35de6c77b308f288cfd0b88e1070f15b98e00c/numpy-2.2.4-cp311-cp311-win_amd64.whl", hash = "sha256:f7de08cbe5551911886d1ab60de58448c6df0f67d9feb7d1fb21e9875ef95e91", size = 12947526 },
{ url = "https://files.pythonhosted.org/packages/a2/30/182db21d4f2a95904cec1a6f779479ea1ac07c0647f064dea454ec650c42/numpy-2.2.4-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:a7b9084668aa0f64e64bd00d27ba5146ef1c3a8835f3bd912e7a9e01326804c4", size = 20947156 },
{ url = "https://files.pythonhosted.org/packages/24/6d/9483566acfbda6c62c6bc74b6e981c777229d2af93c8eb2469b26ac1b7bc/numpy-2.2.4-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:dbe512c511956b893d2dacd007d955a3f03d555ae05cfa3ff1c1ff6df8851854", size = 14133092 },
{ url = "https://files.pythonhosted.org/packages/27/f6/dba8a258acbf9d2bed2525cdcbb9493ef9bae5199d7a9cb92ee7e9b2aea6/numpy-2.2.4-cp312-cp312-macosx_14_0_arm64.whl", hash = "sha256:bb649f8b207ab07caebba230d851b579a3c8711a851d29efe15008e31bb4de24", size = 5163515 },
{ url = "https://files.pythonhosted.org/packages/62/30/82116199d1c249446723c68f2c9da40d7f062551036f50b8c4caa42ae252/numpy-2.2.4-cp312-cp312-macosx_14_0_x86_64.whl", hash = "sha256:f34dc300df798742b3d06515aa2a0aee20941c13579d7a2f2e10af01ae4901ee", size = 6696558 },
{ url = "https://files.pythonhosted.org/packages/0e/b2/54122b3c6df5df3e87582b2e9430f1bdb63af4023c739ba300164c9ae503/numpy-2.2.4-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c3f7ac96b16955634e223b579a3e5798df59007ca43e8d451a0e6a50f6bfdfba", size = 14084742 },
{ url = "https://files.pythonhosted.org/packages/02/e2/e2cbb8d634151aab9528ef7b8bab52ee4ab10e076509285602c2a3a686e0/numpy-2.2.4-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4f92084defa704deadd4e0a5ab1dc52d8ac9e8a8ef617f3fbb853e79b0ea3592", size = 16134051 },
{ url = "https://files.pythonhosted.org/packages/8e/21/efd47800e4affc993e8be50c1b768de038363dd88865920439ef7b422c60/numpy-2.2.4-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:7a4e84a6283b36632e2a5b56e121961f6542ab886bc9e12f8f9818b3c266bfbb", size = 15578972 },
{ url = "https://files.pythonhosted.org/packages/04/1e/f8bb88f6157045dd5d9b27ccf433d016981032690969aa5c19e332b138c0/numpy-2.2.4-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:11c43995255eb4127115956495f43e9343736edb7fcdb0d973defd9de14cd84f", size = 17898106 },
{ url = "https://files.pythonhosted.org/packages/2b/93/df59a5a3897c1f036ae8ff845e45f4081bb06943039ae28a3c1c7c780f22/numpy-2.2.4-cp312-cp312-win32.whl", hash = "sha256:65ef3468b53269eb5fdb3a5c09508c032b793da03251d5f8722b1194f1790c00", size = 6311190 },
{ url = "https://files.pythonhosted.org/packages/46/69/8c4f928741c2a8efa255fdc7e9097527c6dc4e4df147e3cadc5d9357ce85/numpy-2.2.4-cp312-cp312-win_amd64.whl", hash = "sha256:2aad3c17ed2ff455b8eaafe06bcdae0062a1db77cb99f4b9cbb5f4ecb13c5146", size = 12644305 },
{ url = "https://files.pythonhosted.org/packages/2a/d0/bd5ad792e78017f5decfb2ecc947422a3669a34f775679a76317af671ffc/numpy-2.2.4-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:1cf4e5c6a278d620dee9ddeb487dc6a860f9b199eadeecc567f777daace1e9e7", size = 20933623 },
{ url = "https://files.pythonhosted.org/packages/c3/bc/2b3545766337b95409868f8e62053135bdc7fa2ce630aba983a2aa60b559/numpy-2.2.4-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:1974afec0b479e50438fc3648974268f972e2d908ddb6d7fb634598cdb8260a0", size = 14148681 },
{ url = "https://files.pythonhosted.org/packages/6a/70/67b24d68a56551d43a6ec9fe8c5f91b526d4c1a46a6387b956bf2d64744e/numpy-2.2.4-cp313-cp313-macosx_14_0_arm64.whl", hash = "sha256:79bd5f0a02aa16808fcbc79a9a376a147cc1045f7dfe44c6e7d53fa8b8a79392", size = 5148759 },
{ url = "https://files.pythonhosted.org/packages/1c/8b/e2fc8a75fcb7be12d90b31477c9356c0cbb44abce7ffb36be39a0017afad/numpy-2.2.4-cp313-cp313-macosx_14_0_x86_64.whl", hash = "sha256:3387dd7232804b341165cedcb90694565a6015433ee076c6754775e85d86f1fc", size = 6683092 },
{ url = "https://files.pythonhosted.org/packages/13/73/41b7b27f169ecf368b52533edb72e56a133f9e86256e809e169362553b49/numpy-2.2.4-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6f527d8fdb0286fd2fd97a2a96c6be17ba4232da346931d967a0630050dfd298", size = 14081422 },
{ url = "https://files.pythonhosted.org/packages/4b/04/e208ff3ae3ddfbafc05910f89546382f15a3f10186b1f56bd99f159689c2/numpy-2.2.4-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bce43e386c16898b91e162e5baaad90c4b06f9dcbe36282490032cec98dc8ae7", size = 16132202 },
{ url = "https://files.pythonhosted.org/packages/fe/bc/2218160574d862d5e55f803d88ddcad88beff94791f9c5f86d67bd8fbf1c/numpy-2.2.4-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:31504f970f563d99f71a3512d0c01a645b692b12a63630d6aafa0939e52361e6", size = 15573131 },
{ url = "https://files.pythonhosted.org/packages/a5/78/97c775bc4f05abc8a8426436b7cb1be806a02a2994b195945600855e3a25/numpy-2.2.4-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:81413336ef121a6ba746892fad881a83351ee3e1e4011f52e97fba79233611fd", size = 17894270 },
{ url = "https://files.pythonhosted.org/packages/b9/eb/38c06217a5f6de27dcb41524ca95a44e395e6a1decdc0c99fec0832ce6ae/numpy-2.2.4-cp313-cp313-win32.whl", hash = "sha256:f486038e44caa08dbd97275a9a35a283a8f1d2f0ee60ac260a1790e76660833c", size = 6308141 },
{ url = "https://files.pythonhosted.org/packages/52/17/d0dd10ab6d125c6d11ffb6dfa3423c3571befab8358d4f85cd4471964fcd/numpy-2.2.4-cp313-cp313-win_amd64.whl", hash = "sha256:207a2b8441cc8b6a2a78c9ddc64d00d20c303d79fba08c577752f080c4007ee3", size = 12636885 },
{ url = "https://files.pythonhosted.org/packages/fa/e2/793288ede17a0fdc921172916efb40f3cbc2aa97e76c5c84aba6dc7e8747/numpy-2.2.4-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:8120575cb4882318c791f839a4fd66161a6fa46f3f0a5e613071aae35b5dd8f8", size = 20961829 },
{ url = "https://files.pythonhosted.org/packages/3a/75/bb4573f6c462afd1ea5cbedcc362fe3e9bdbcc57aefd37c681be1155fbaa/numpy-2.2.4-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:a761ba0fa886a7bb33c6c8f6f20213735cb19642c580a931c625ee377ee8bd39", size = 14161419 },
{ url = "https://files.pythonhosted.org/packages/03/68/07b4cd01090ca46c7a336958b413cdbe75002286295f2addea767b7f16c9/numpy-2.2.4-cp313-cp313t-macosx_14_0_arm64.whl", hash = "sha256:ac0280f1ba4a4bfff363a99a6aceed4f8e123f8a9b234c89140f5e894e452ecd", size = 5196414 },
{ url = "https://files.pythonhosted.org/packages/a5/fd/d4a29478d622fedff5c4b4b4cedfc37a00691079623c0575978d2446db9e/numpy-2.2.4-cp313-cp313t-macosx_14_0_x86_64.whl", hash = "sha256:879cf3a9a2b53a4672a168c21375166171bc3932b7e21f622201811c43cdd3b0", size = 6709379 },
{ url = "https://files.pythonhosted.org/packages/41/78/96dddb75bb9be730b87c72f30ffdd62611aba234e4e460576a068c98eff6/numpy-2.2.4-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f05d4198c1bacc9124018109c5fba2f3201dbe7ab6e92ff100494f236209c960", size = 14051725 },
{ url = "https://files.pythonhosted.org/packages/00/06/5306b8199bffac2a29d9119c11f457f6c7d41115a335b78d3f86fad4dbe8/numpy-2.2.4-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e2f085ce2e813a50dfd0e01fbfc0c12bbe5d2063d99f8b29da30e544fb6483b8", size = 16101638 },
{ url = "https://files.pythonhosted.org/packages/fa/03/74c5b631ee1ded596945c12027649e6344614144369fd3ec1aaced782882/numpy-2.2.4-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:92bda934a791c01d6d9d8e038363c50918ef7c40601552a58ac84c9613a665bc", size = 15571717 },
{ url = "https://files.pythonhosted.org/packages/cb/dc/4fc7c0283abe0981e3b89f9b332a134e237dd476b0c018e1e21083310c31/numpy-2.2.4-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:ee4d528022f4c5ff67332469e10efe06a267e32f4067dc76bb7e2cddf3cd25ff", size = 17879998 },
{ url = "https://files.pythonhosted.org/packages/e5/2b/878576190c5cfa29ed896b518cc516aecc7c98a919e20706c12480465f43/numpy-2.2.4-cp313-cp313t-win32.whl", hash = "sha256:05c076d531e9998e7e694c36e8b349969c56eadd2cdcd07242958489d79a7286", size = 6366896 },
{ url = "https://files.pythonhosted.org/packages/3e/05/eb7eec66b95cf697f08c754ef26c3549d03ebd682819f794cb039574a0a6/numpy-2.2.4-cp313-cp313t-win_amd64.whl", hash = "sha256:188dcbca89834cc2e14eb2f106c96d6d46f200fe0200310fc29089657379c58d", size = 12739119 },
]
[[package]]
name = "outcome"
version = "1.3.0.post0"
@ -361,6 +409,7 @@ dependencies = [
[package.dev-dependencies]
dev = [
{ name = "greenback" },
{ name = "numpy" },
{ name = "pexpect" },
{ name = "prompt-toolkit" },
{ name = "pyperclip" },
@ -383,6 +432,7 @@ requires-dist = [
[package.metadata.requires-dev]
dev = [
{ name = "greenback", specifier = ">=1.2.1,<2" },
{ name = "numpy", specifier = ">=2.2.4" },
{ name = "pexpect", specifier = ">=4.9.0,<5" },
{ name = "prompt-toolkit", specifier = ">=3.0.50" },
{ name = "pyperclip", specifier = ">=1.9.0" },