Compare commits

...

52 Commits

Author SHA1 Message Date
Tyler Goodlet 10f9b505ee Add `psutil` to `--dev` / testing deps 2025-04-01 22:26:56 -04:00
Tyler Goodlet 0c60914cc4 Factor `breakpoint()` blocking into `@acm`
Call it `maybe_block_bp()` can wrap the `open_root_actor()` body with
it. Main reason is to guarantee we can bp inside actor runtime bootup as
needed when debugging internals! Prolly should factor this to another
module tho?

ALSO, ensure we RTE on recurrent entries to `open_root_actor()` from
within an existing tree! There was actually `test_spawning` test somehow
getting away with this!? Should never be possible or allowed!
2025-04-01 22:25:29 -04:00
Tyler Goodlet 1cb2337c7c Add an `Actor.pformat()`
And map `.__repr__/__str__` to it and add various new fields to fill it
out,
- drop `self.uid` as var and instead add `Actor._aid: Aid` and proxy to
  it for the various `.name/.uid/.pid` properties as well as a new
  `.aid` field.
 |_ the `Aid.pid` addition is also included.

Other improvements,
- flip to a sync call to `Address.close_listener()`.
- track the `async_main()` parent task as `Actor._task`.
- add exception logging around failure to bind due to already-in-use
  when calling `add.open_listener()` in `._stream_forever()`; sometimes
  the error might be overridden by something else during the
  runtime-failure unwind..
2025-04-01 22:21:28 -04:00
Tyler Goodlet 46e775ce6d Add a `MsgpackTransport.pformat()`
And map `.__repr__/__str__` to it. Also adjust to new
`Address.proto_key` and add a #TODO for a `.get_peers()`.
2025-04-01 22:01:51 -04:00
Tyler Goodlet 89993a4e3a Even more `tractor._addr.Address` simplifying
Namely reducing the duplication of class-fields and `TypeVar`s used
for parametrizing the `Address` protocol type,
- drop all of the `TypeVar` types and just stick with all concrete addrs
  types inheriting from `Address` only.
- rename `Address.name_key` -> `.proto_key`.
- rename `Address.address_type` -> `.unwrapped_type`
- rename `.namespace` -> `.bindspace` to better reflect that this "part"
  of the address represents the possible "space for binding endpoints".
 |_ also linux already uses "namespace" to mean the `netns` and i'd
   prefer to stick with their semantics for that.
- add `TCPAddress/UDSAddress.def_bindspace` values.
- drop commented `.open_stream()` method; never used.
- simplify `UnwrappedAdress` to just a `tuple` of union types.
- add logging to `USDAddress.open_listener()` for now.
- adjust `tractor.ipc/_uds/tcp` transport to use new addr field names.
2025-04-01 21:53:03 -04:00
Tyler Goodlet a28659c3cd Handle broken-pipes from `MsgpackTransport.send()`
Much like we already do in the `._iter_packets()` async-generator which
delivers to `.recv()` and `async for`, handle the `''[Errno 32] Broken
pipe'` case that can show up with unix-domain-socket usage.

Seems like the cause is due to how fast the socket can be torn down
during a registry addr channel ping where,
- the sending side can break the connection faster then the pong side
  can prep its handshake msg,
- the pong side tries to send it's handshake pkt via
  `.SocketStream.send_all()` after the breakage and then raises
  `trio.BrokenResourceError`.
2025-04-01 12:56:28 -04:00
Tyler Goodlet 012b5fd8c2 Emphasize internal error block header-comment a bit 2025-03-30 23:57:52 -04:00
Tyler Goodlet 84a888f381 Bit of multi-line styling for `LocalPortal` 2025-03-30 22:54:19 -04:00
Tyler Goodlet c1141c1f66 Adjust `._child` instantiation of `Actor` to use newly named `uuid` arg 2025-03-30 22:53:36 -04:00
Tyler Goodlet bfa31f94c4 Add `bidict` pkg as dep since used in `._addr` for now 2025-03-30 22:52:46 -04:00
Tyler Goodlet 8040ae6994 Adjust lowlevel-tb hiding logic for `MsgStream`
Such that whenev the `self._ctx.chan._exc is trans_err` we suppress.
I.e. when the `Channel._exc: Exception|None` error **is the same as**
set by the `._rpc.process_messages()` loop (that is, set to the
underlying transport layer error), we suppress the lowlevel tb,
otherwise we deliver the full tb since likely something at the lowlevel
that we aren't detecting changed/signalled/is-relevant!
2025-03-30 22:45:44 -04:00
Tyler Goodlet 08ac89b807 Slight typing and multi-line styling tweaks in `.ipc` sugpkg 2025-03-30 22:42:51 -04:00
Tyler Goodlet e904af679b Add a big boi `Channel.pformat()/__repr__()`
Much like how `Context` has been implemented, try to give tons of high
level details on all the lower level encapsulated primitives, namely the
`.msgstream/.transport` and any useful runtime state.

B)

Impl deats,
- adjust `.from_addr()` to only call `._addr.wrap_address()` when we
  detect `addr` is unwrapped.
- add another `log.runtime()` using the new `.__repr__()` in
  `Channel.from_addr()`.
- change to `UnwrappedAddress` as in prior commits.
2025-03-30 22:35:35 -04:00
Tyler Goodlet 6a5ccc2425 Allocate bind-addrs in subactors
Previously whenever an `ActorNursery.start_actor()` call did not receive
a `bind_addrs` arg we would allocate the default `(localhost, 0)` pairs
in the parent, for UDS this obviously won't work nor is it ideal bc it's
nicer to have the actor to be a socket server (who calls
`Address.open_listener()`) define the socket-file-name containing their
unique ID info such as pid, actor-uuid etc.

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

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

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

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

Related refinements,
- set `.layer_key: int = 4` for the "transport layer" ;)
- tweak some typing and multi-line unpacking in `.ipc/_tcp`.
2025-03-30 21:14:12 -04:00
Tyler Goodlet 9de192390a Rework/simplify transport addressing
A few things that can fundamentally change,

- UDS addresses now always encapsulate the local and remote pid such
  that it denotes each side's process much like a TCP *port*.
  |_ `.__init__()` takes a new `maybe_pid: int`.
  |_ this required changes to the `.ipc._uds` backend which will come in
     an subsequent commit!
  |_ `UDSAddress.address_type` becomes a `tuple[str, int]` just like the
      TCP case.
  |_ adjust `wrap_address()` to match.
- use a new `_state.get_rt_dir() -> Path` as the default location for
  UDS socket file: now under `XDG_RUNTIME_DIR'/tractor/` subdir by
  default.
- re-implement `USDAddress.get_random()` to use both the local
  `Actor.uid` (if available) and at least the pid for its socket file
  name.

Removals,
- drop the loop generated `_default_addrs`, simplify to just
  `_default_lo_addrs` for per-transport default registry addresses.
  |_ change to `_address_types: dict[str, Type[Address]]` instead of
     separate types `list`.
  |_ adjust `is_wrapped_addr()` to just check `in _addr_types.values()`.
- comment out `Address.open_stream()` it's unused and i think the wrong
  place for this API.

Renames,
- from `AddressTypes` -> `UnwrappedAddress`, since it's a simple type
  union and all this type set is, is the simple python data-structures
  we encode to for the wire.
  |_ see note about possibly implementing the `.[un]wrap()` stuff as
     `msgspec` codec `enc/dec_hook()`s instead!

Additions,
- add a `mk_uuid()` to be used throughout the runtime including for
  generating the `Aid.uuid` part.
- tons of notes around follow up refinements!
2025-03-30 19:35:02 -04:00
Guillermo Rodriguez efd11f7d74
Trying to make full suite pass with uds 2025-03-27 20:37:52 -03:00
Guillermo Rodriguez 76cee99fc2
Finally switch to using address protocol in all runtime 2025-03-27 20:37:52 -03:00
Guillermo Rodriguez 5f50206d84
Add root and random addr getters on MsgTransport type 2025-03-27 20:37:52 -03:00
Guillermo Rodriguez a47a7a39b1
Starting to make tractor.ipc.Channel work with multiple MsgTransports 2025-03-27 20:37:52 -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
Guillermo Rodriguez 9b2161506f
Break out transport protocol and tcp specifics into their own submodules under tractor.ipc 2025-03-27 20:36:46 -03:00
Guillermo Rodriguez 6b155849b7
Add buf_size to RBToken and add sender cancel test, move disable_mantracker to its own _mp_bs module 2025-03-27 20:36:46 -03:00
Guillermo Rodriguez 59c8c7bfe3
Make ring buf api use pickle-able RBToken 2025-03-27 20:36:46 -03:00
Guillermo Rodriguez 6ac6fd56c0
Address some of fomo\'s comments 2025-03-27 20:36:46 -03:00
Guillermo Rodriguez f799e9ac51
Handle cancelation on EventFD.read 2025-03-27 20:36:46 -03:00
Guillermo Rodriguez 9980bb2bd0
Add module headers and fix spacing on tractor._ipc._linux 2025-03-27 20:36:45 -03:00
Guillermo Rodriguez 8de9ab291e
Move RingBuffSender|Receiver to its own tractor.ipc._ringbuf module 2025-03-27 20:36:45 -03:00
Guillermo Rodriguez 1a83626f26
Move linux specifics from tractor.ipc._shm into tractor.ipc._linux 2025-03-27 20:36:45 -03:00
Guillermo Rodriguez 6b4d08d030
Move tractor._shm to tractor.ipc._shm 2025-03-27 20:36:45 -03:00
Guillermo Rodriguez 7b8b9d6805
move tractor._ipc.py into tractor.ipc._chan.py 2025-03-27 20:36:45 -03:00
Guillermo Rodriguez 5afe0a0264
General improvements
EventFD class now expects the fd to already be init with open_eventfd
RingBuff Sender and Receiver fully manage SharedMemory and EventFD lifecycles, no aditional ctx mngrs needed
Separate ring buf tests into its own test bed
Add parametrization to test and cancellation
Add docstrings
Add simple testing data gen module .samples
2025-03-27 20:36:45 -03:00
Guillermo Rodriguez eeb9a7d61b
IPC ring bug impl with async read 2025-03-27 20:36:45 -03:00
Tyler Goodlet 5cee222353 Updates from latest `piker.data._sharedmem` changes 2025-03-27 17:54:04 -04:00
Tyler Goodlet 8ebb1f09de Pass `str` dtype for `use_str` case 2025-03-27 17:54:04 -04:00
Tyler Goodlet 2683a7f33a Allocate size-specced "empty" sequence from default values by type 2025-03-27 17:54:04 -04:00
Tyler Goodlet 255209f881 Mod define `_USE_POSIX`, add a of of todos 2025-03-27 17:54:04 -04:00
Tyler Goodlet 9a0d529b18 Parametrize rw test with variable frame sizes
Demonstrates fixed size frame-oriented reads by the child where the
parent only transmits a "read" stream msg on "frame fill events" such
that the child incrementally reads the shm list data (much like in
a real-time-buffered streaming system).
2025-03-27 17:54:04 -04:00
Tyler Goodlet 1c441b0986 Add `ShmList` slice support in `.__getitem__()` 2025-03-27 17:54:04 -04:00
Tyler Goodlet afbdb50a30 Rename token type to `NDToken` in the style of `nptyping` 2025-03-27 17:54:04 -04:00
Tyler Goodlet e46033cbe7 Don't require runtime (for now), type annot fixing 2025-03-27 17:54:04 -04:00
Tyler Goodlet c932bb5911 Add repetitive attach to existing segment test 2025-03-27 17:54:04 -04:00
Tyler Goodlet 33482d8f41 Add initial readers-writer shm list tests 2025-03-27 17:54:04 -04:00
Tyler Goodlet 7ae194baed Add `ShmList` wrapping the stdlib's `ShareableList`
First attempt at getting `multiprocessing.shared_memory.ShareableList`
working; we wrap the stdlib type with a readonly attr and a `.key` for
cross-actor lookup. Also, rename all `numpy` specific routines to have
a `ndarray` suffix in the func names.
2025-03-27 17:54:04 -04:00
Tyler Goodlet ef7ca49e9b Initial module import from `piker.data._sharemem`
More or less a verbatim copy-paste minus some edgy variable naming and
internal `piker` module imports. There is a bunch of OHLC related
defaults that need to be dropped and we need to adjust to an optional
dependence on `numpy` by supporting shared lists as per the mp docs.
2025-03-27 17:54:04 -04:00
42 changed files with 4787 additions and 1002 deletions

19
default.nix 100644
View File

@ -0,0 +1,19 @@
{ pkgs ? import <nixpkgs> {} }:
let
nativeBuildInputs = with pkgs; [
stdenv.cc.cc.lib
uv
];
in
pkgs.mkShell {
inherit nativeBuildInputs;
LD_LIBRARY_PATH = pkgs.lib.makeLibraryPath nativeBuildInputs;
TMPDIR = "/tmp";
shellHook = ''
set -e
uv venv .venv --python=3.11
'';
}

View File

@ -9,7 +9,7 @@ async def main(service_name):
async with tractor.open_nursery() as an:
await an.start_actor(service_name)
async with tractor.get_registry('127.0.0.1', 1616) as portal:
async with tractor.get_registry() as portal:
print(f"Arbiter is listening on {portal.channel}")
async with tractor.wait_for_actor(service_name) as sockaddr:

View File

@ -45,6 +45,8 @@ dependencies = [
"pdbp>=1.6,<2", # windows only (from `pdbp`)
# typed IPC msging
"msgspec>=0.19.0",
"cffi>=1.17.1",
"bidict>=0.23.1",
]
# ------ project ------
@ -62,6 +64,7 @@ dev = [
"pyperclip>=1.9.0",
"prompt-toolkit>=3.0.50",
"xonsh>=0.19.2",
"psutil>=7.0.0",
]
# TODO, add these with sane versions; were originally in
# `requirements-docs.txt`..

View File

@ -26,7 +26,7 @@ async def test_reg_then_unreg(reg_addr):
portal = await n.start_actor('actor', enable_modules=[__name__])
uid = portal.channel.uid
async with tractor.get_registry(*reg_addr) as aportal:
async with tractor.get_registry(reg_addr) as aportal:
# this local actor should be the arbiter
assert actor is aportal.actor
@ -160,7 +160,7 @@ async def spawn_and_check_registry(
async with tractor.open_root_actor(
registry_addrs=[reg_addr],
):
async with tractor.get_registry(*reg_addr) as portal:
async with tractor.get_registry(reg_addr) as portal:
# runtime needs to be up to call this
actor = tractor.current_actor()
@ -300,7 +300,7 @@ async def close_chans_before_nursery(
async with tractor.open_root_actor(
registry_addrs=[reg_addr],
):
async with tractor.get_registry(*reg_addr) as aportal:
async with tractor.get_registry(reg_addr) as aportal:
try:
get_reg = partial(unpack_reg, aportal)

View File

@ -66,6 +66,9 @@ def run_example_in_subproc(
# due to backpressure!!!
proc = testdir.popen(
cmdargs,
stdin=subprocess.PIPE,
stdout=subprocess.PIPE,
stderr=subprocess.PIPE,
**kwargs,
)
assert not proc.returncode
@ -119,10 +122,14 @@ def test_example(
code = ex.read()
with run_example_in_subproc(code) as proc:
proc.wait()
err, _ = proc.stderr.read(), proc.stdout.read()
# print(f'STDERR: {err}')
# print(f'STDOUT: {out}')
err = None
try:
if not proc.poll():
_, err = proc.communicate(timeout=15)
except subprocess.TimeoutExpired as e:
proc.kill()
err = e.stderr
# if we get some gnarly output let's aggregate and raise
if err:

View File

@ -0,0 +1,32 @@
import trio
import pytest
from tractor.ipc import (
open_eventfd,
EFDReadCancelled,
EventFD
)
def test_eventfd_read_cancellation():
'''
Ensure EventFD.read raises EFDReadCancelled if EventFD.close()
is called.
'''
fd = open_eventfd()
async def _read(event: EventFD):
with pytest.raises(EFDReadCancelled):
await event.read()
async def main():
async with trio.open_nursery() as n:
with (
EventFD(fd, 'w') as event,
trio.fail_after(3)
):
n.start_soon(_read, event)
await trio.sleep(0.2)
event.close()
trio.run(main)

View File

@ -871,7 +871,7 @@ async def serve_subactors(
)
await ipc.send((
peer.chan.uid,
peer.chan.raddr,
peer.chan.raddr.unwrap(),
))
print('Spawner exiting spawn serve loop!')

View File

@ -38,7 +38,7 @@ async def test_self_is_registered_localportal(reg_addr):
"Verify waiting on the arbiter to register itself using a local portal."
actor = tractor.current_actor()
assert actor.is_arbiter
async with tractor.get_registry(*reg_addr) as portal:
async with tractor.get_registry(reg_addr) as portal:
assert isinstance(portal, tractor._portal.LocalPortal)
with trio.fail_after(0.2):

View File

@ -32,7 +32,7 @@ def test_abort_on_sigint(daemon):
@tractor_test
async def test_cancel_remote_arbiter(daemon, reg_addr):
assert not tractor.current_actor().is_arbiter
async with tractor.get_registry(*reg_addr) as portal:
async with tractor.get_registry(reg_addr) as portal:
await portal.cancel_actor()
time.sleep(0.1)
@ -41,7 +41,7 @@ async def test_cancel_remote_arbiter(daemon, reg_addr):
# no arbiter socket should exist
with pytest.raises(OSError):
async with tractor.get_registry(*reg_addr) as portal:
async with tractor.get_registry(reg_addr) as portal:
pass

View File

@ -0,0 +1,423 @@
import time
import hashlib
import trio
import pytest
import tractor
from tractor.ipc import (
open_ringbuf,
attach_to_ringbuf_receiver,
attach_to_ringbuf_sender,
attach_to_ringbuf_stream,
attach_to_ringbuf_channel,
RBToken,
)
from tractor._testing.samples import (
generate_single_byte_msgs,
generate_sample_messages
)
@tractor.context
async def child_read_shm(
ctx: tractor.Context,
msg_amount: int,
token: RBToken,
) -> str:
'''
Sub-actor used in `test_ringbuf`.
Attach to a ringbuf and receive all messages until end of stream.
Keep track of how many bytes received and also calculate
sha256 of the whole byte stream.
Calculate and print performance stats, finally return calculated
hash.
'''
await ctx.started()
print('reader started')
recvd_bytes = 0
recvd_hash = hashlib.sha256()
start_ts = time.time()
async with attach_to_ringbuf_receiver(token) as receiver:
async for msg in receiver:
recvd_hash.update(msg)
recvd_bytes += len(msg)
end_ts = time.time()
elapsed = end_ts - start_ts
elapsed_ms = int(elapsed * 1000)
print(f'\n\telapsed ms: {elapsed_ms}')
print(f'\tmsg/sec: {int(msg_amount / elapsed):,}')
print(f'\tbytes/sec: {int(recvd_bytes / elapsed):,}')
print(f'\treceived bytes: {recvd_bytes:,}')
return recvd_hash.hexdigest()
@tractor.context
async def child_write_shm(
ctx: tractor.Context,
msg_amount: int,
rand_min: int,
rand_max: int,
token: RBToken,
) -> None:
'''
Sub-actor used in `test_ringbuf`
Generate `msg_amount` payloads with
`random.randint(rand_min, rand_max)` random bytes at the end,
Calculate sha256 hash and send it to parent on `ctx.started`.
Attach to ringbuf and send all generated messages.
'''
msgs, _total_bytes = generate_sample_messages(
msg_amount,
rand_min=rand_min,
rand_max=rand_max,
)
print('writer hashing payload...')
sent_hash = hashlib.sha256(b''.join(msgs)).hexdigest()
print('writer done hashing.')
await ctx.started(sent_hash)
print('writer started')
async with attach_to_ringbuf_sender(token, cleanup=False) as sender:
for msg in msgs:
await sender.send_all(msg)
print('writer exit')
@pytest.mark.parametrize(
'msg_amount,rand_min,rand_max,buf_size',
[
# simple case, fixed payloads, large buffer
(100_000, 0, 0, 10 * 1024),
# guaranteed wrap around on every write
(100, 10 * 1024, 20 * 1024, 10 * 1024),
# large payload size, but large buffer
(10_000, 256 * 1024, 512 * 1024, 10 * 1024 * 1024)
],
ids=[
'fixed_payloads_large_buffer',
'wrap_around_every_write',
'large_payloads_large_buffer',
]
)
def test_ringbuf(
msg_amount: int,
rand_min: int,
rand_max: int,
buf_size: int
):
'''
- 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
'''
async def main():
with open_ringbuf(
'test_ringbuf',
buf_size=buf_size
) as token:
proc_kwargs = {'pass_fds': token.fds}
async with tractor.open_nursery() as an:
send_p = await an.start_actor(
'ring_sender',
enable_modules=[__name__],
proc_kwargs=proc_kwargs
)
recv_p = await an.start_actor(
'ring_receiver',
enable_modules=[__name__],
proc_kwargs=proc_kwargs
)
async with (
send_p.open_context(
child_write_shm,
token=token,
msg_amount=msg_amount,
rand_min=rand_min,
rand_max=rand_max,
) as (_sctx, sent_hash),
recv_p.open_context(
child_read_shm,
token=token,
msg_amount=msg_amount
) as (rctx, _sent),
):
recvd_hash = await rctx.result()
assert sent_hash == recvd_hash
await send_p.cancel_actor()
await recv_p.cancel_actor()
trio.run(main)
@tractor.context
async def child_blocked_receiver(
ctx: tractor.Context,
token: RBToken
):
async with attach_to_ringbuf_receiver(token) as receiver:
await ctx.started()
await receiver.receive_some()
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,
attach_to_ringbuf_sender(token) as _sender,
):
recv_p = await an.start_actor(
'ring_blocked_receiver',
enable_modules=[__name__],
proc_kwargs={
'pass_fds': token.fds
}
)
async with (
recv_p.open_context(
child_blocked_receiver,
token=token
) as (sctx, _sent),
):
await trio.sleep(1)
await an.cancel()
with pytest.raises(tractor._exceptions.ContextCancelled):
trio.run(main)
@tractor.context
async def child_blocked_sender(
ctx: tractor.Context,
token: RBToken
):
async with attach_to_ringbuf_sender(token) as sender:
await ctx.started()
await sender.send_all(b'this will wrap')
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.fds
}
)
async with (
recv_p.open_context(
child_blocked_sender,
token=token
) as (sctx, _sent),
):
await trio.sleep(1)
await an.cancel()
with pytest.raises(tractor._exceptions.ContextCancelled):
trio.run(main)
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():
with open_ringbuf(
'test_ringbuf_max_bytes',
buf_size=10
) as token:
async with (
trio.open_nursery() as n,
attach_to_ringbuf_sender(token, cleanup=False) as sender,
attach_to_ringbuf_receiver(token, cleanup=False) as receiver
):
async def _send_and_close():
await sender.send_all(msg)
await sender.aclose()
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)
def test_stapled_ringbuf():
'''
Open two ringbufs and give tokens to tasks (swap them such that in/out tokens
are inversed on each task) which will open the streams and use trio.StapledStream
to have a single bidirectional stream.
Then take turns to send and receive messages.
'''
msg = generate_single_byte_msgs(100)
pair_0_msgs = []
pair_1_msgs = []
pair_0_done = trio.Event()
pair_1_done = trio.Event()
async def pair_0(token_in: RBToken, token_out: RBToken):
async with attach_to_ringbuf_stream(
token_in,
token_out,
cleanup_in=False,
cleanup_out=False
) as stream:
# first turn to send
await stream.send_all(msg)
# second turn to receive
while len(pair_0_msgs) != len(msg):
_msg = await stream.receive_some(max_bytes=1)
pair_0_msgs.append(_msg)
pair_0_done.set()
await pair_1_done.wait()
async def pair_1(token_in: RBToken, token_out: RBToken):
async with attach_to_ringbuf_stream(
token_in,
token_out,
cleanup_in=False,
cleanup_out=False
) as stream:
# first turn to receive
while len(pair_1_msgs) != len(msg):
_msg = await stream.receive_some(max_bytes=1)
pair_1_msgs.append(_msg)
# second turn to send
await stream.send_all(msg)
pair_1_done.set()
await pair_0_done.wait()
async def main():
with tractor.ipc.open_ringbuf_pair(
'test_stapled_ringbuf'
) as (token_0, token_1):
async with trio.open_nursery() as n:
n.start_soon(pair_0, token_0, token_1)
n.start_soon(pair_1, token_1, token_0)
trio.run(main)
assert msg == b''.join(pair_0_msgs)
assert msg == b''.join(pair_1_msgs)
@tractor.context
async def child_channel_sender(
ctx: tractor.Context,
msg_amount_min: int,
msg_amount_max: int,
token_in: RBToken,
token_out: RBToken
):
import random
msgs, _total_bytes = generate_sample_messages(
random.randint(msg_amount_min, msg_amount_max),
rand_min=256,
rand_max=1024,
)
async with attach_to_ringbuf_channel(
token_in,
token_out
) as chan:
await ctx.started(msgs)
for msg in msgs:
await chan.send(msg)
def test_channel():
msg_amount_min = 100
msg_amount_max = 1000
async def main():
with tractor.ipc.open_ringbuf_pair(
'test_ringbuf_transport'
) as (token_0, token_1):
async with (
attach_to_ringbuf_channel(token_0, token_1) as chan,
tractor.open_nursery() as an
):
recv_p = await an.start_actor(
'test_ringbuf_transport_sender',
enable_modules=[__name__],
proc_kwargs={
'pass_fds': token_0.fds + token_1.fds
}
)
async with (
recv_p.open_context(
child_channel_sender,
msg_amount_min=msg_amount_min,
msg_amount_max=msg_amount_max,
token_in=token_1,
token_out=token_0
) as (ctx, msgs),
):
recv_msgs = []
async for msg in chan:
recv_msgs.append(msg)
await recv_p.cancel_actor()
assert recv_msgs == msgs
trio.run(main)

167
tests/test_shm.py 100644
View File

@ -0,0 +1,167 @@
"""
Shared mem primitives and APIs.
"""
import uuid
# import numpy
import pytest
import trio
import tractor
from tractor.ipc._shm import (
open_shm_list,
attach_shm_list,
)
@tractor.context
async def child_attach_shml_alot(
ctx: tractor.Context,
shm_key: str,
) -> None:
await ctx.started(shm_key)
# now try to attach a boatload of times in a loop..
for _ in range(1000):
shml = attach_shm_list(
key=shm_key,
readonly=False,
)
assert shml.shm.name == shm_key
await trio.sleep(0.001)
def test_child_attaches_alot():
async def main():
async with tractor.open_nursery() as an:
# allocate writeable list in parent
key = f'shml_{uuid.uuid4()}'
shml = open_shm_list(
key=key,
)
portal = await an.start_actor(
'shm_attacher',
enable_modules=[__name__],
)
async with (
portal.open_context(
child_attach_shml_alot,
shm_key=shml.key,
) as (ctx, start_val),
):
assert start_val == key
await ctx.result()
await portal.cancel_actor()
trio.run(main)
@tractor.context
async def child_read_shm_list(
ctx: tractor.Context,
shm_key: str,
use_str: bool,
frame_size: int,
) -> None:
# attach in child
shml = attach_shm_list(
key=shm_key,
# dtype=str if use_str else float,
)
await ctx.started(shml.key)
async with ctx.open_stream() as stream:
async for i in stream:
print(f'(child): reading shm list index: {i}')
if use_str:
expect = str(float(i))
else:
expect = float(i)
if frame_size == 1:
val = shml[i]
assert expect == val
print(f'(child): reading value: {val}')
else:
frame = shml[i - frame_size:i]
print(f'(child): reading frame: {frame}')
@pytest.mark.parametrize(
'use_str',
[False, True],
ids=lambda i: f'use_str_values={i}',
)
@pytest.mark.parametrize(
'frame_size',
[1, 2**6, 2**10],
ids=lambda i: f'frame_size={i}',
)
def test_parent_writer_child_reader(
use_str: bool,
frame_size: int,
):
async def main():
async with tractor.open_nursery(
# debug_mode=True,
) as an:
portal = await an.start_actor(
'shm_reader',
enable_modules=[__name__],
debug_mode=True,
)
# allocate writeable list in parent
key = 'shm_list'
seq_size = int(2 * 2 ** 10)
shml = open_shm_list(
key=key,
size=seq_size,
dtype=str if use_str else float,
readonly=False,
)
async with (
portal.open_context(
child_read_shm_list,
shm_key=key,
use_str=use_str,
frame_size=frame_size,
) as (ctx, sent),
ctx.open_stream() as stream,
):
assert sent == key
for i in range(seq_size):
val = float(i)
if use_str:
val = str(val)
# print(f'(parent): writing {val}')
shml[i] = val
# only on frame fills do we
# signal to the child that a frame's
# worth is ready.
if (i % frame_size) == 0:
print(f'(parent): signalling frame full on {val}')
await stream.send(i)
else:
print(f'(parent): signalling final frame on {val}')
await stream.send(i)
await portal.cancel_actor()
trio.run(main)

View File

@ -77,7 +77,7 @@ async def movie_theatre_question():
async def test_movie_theatre_convo(start_method):
"""The main ``tractor`` routine.
"""
async with tractor.open_nursery() as n:
async with tractor.open_nursery(debug_mode=True) as n:
portal = await n.start_actor(
'frank',

View File

@ -64,7 +64,7 @@ from ._root import (
run_daemon as run_daemon,
open_root_actor as open_root_actor,
)
from ._ipc import Channel as Channel
from .ipc import Channel as Channel
from ._portal import Portal as Portal
from ._runtime import Actor as Actor
# from . import hilevel as hilevel

497
tractor/_addr.py 100644
View File

@ -0,0 +1,497 @@
# 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/>.
from __future__ import annotations
from pathlib import Path
import os
# import tempfile
from uuid import uuid4
from typing import (
Protocol,
ClassVar,
# TypeVar,
# Union,
Type,
TYPE_CHECKING,
)
from bidict import bidict
# import trio
from trio import (
socket,
SocketListener,
open_tcp_listeners,
)
from .log import get_logger
from ._state import (
get_rt_dir,
current_actor,
is_root_process,
)
if TYPE_CHECKING:
from ._runtime import Actor
log = get_logger(__name__)
# TODO, maybe breakout the netns key to a struct?
# class NetNs(Struct)[str, int]:
# ...
# TODO, can't we just use a type alias
# for this? namely just some `tuple[str, int, str, str]`?
#
# -[ ] would also just be simpler to keep this as SockAddr[tuple]
# or something, implying it's just a simple pair of values which can
# presumably be mapped to all transports?
# -[ ] `pydoc socket.socket.getsockname()` delivers a 4-tuple for
# ipv6 `(hostaddr, port, flowinfo, scope_id)`.. so how should we
# handle that?
# -[ ] as a further alternative to this wrap()/unwrap() approach we
# could just implement `enc/dec_hook()`s for the `Address`-types
# and just deal with our internal objs directly and always and
# leave it to the codec layer to figure out marshalling?
# |_ would mean only one spot to do the `.unwrap()` (which we may
# end up needing to call from the hook()s anyway?)
# -[x] rename to `UnwrappedAddress[Descriptor]` ??
# seems like the right name as per,
# https://www.geeksforgeeks.org/introduction-to-address-descriptor/
#
UnwrappedAddress = (
# tcp/udp/uds
tuple[
str, # host/domain(tcp), filesys-dir(uds)
int|str, # port/path(uds)
]
# ?TODO? should we also include another 2 fields from
# our `Aid` msg such that we include the runtime `Actor.uid`
# of `.name` and `.uuid`?
# - would ensure uniqueness across entire net?
# - allows for easier runtime-level filtering of "actors by
# service name"
)
# TODO, maybe rename to `SocketAddress`?
class Address(Protocol):
proto_key: ClassVar[str]
unwrapped_type: ClassVar[UnwrappedAddress]
# TODO, i feel like an `.is_bound()` is a better thing to
# support?
# Lke, what use does this have besides a noop and if it's not
# valid why aren't we erroring on creation/use?
@property
def is_valid(self) -> bool:
...
# TODO, maybe `.netns` is a better name?
@property
def namespace(self) -> tuple[str, int]|None:
'''
The if-available, OS-specific "network namespace" key.
'''
...
@property
def bindspace(self) -> str:
'''
Deliver the socket address' "bindable space" from
a `socket.socket.bind()` and thus from the perspective of
specific transport protocol domain.
I.e. for most (layer-4) network-socket protocols this is
normally the ipv4/6 address, for UDS this is normally
a filesystem (sub-directory).
For (distributed) network protocols this is normally the routing
layer's domain/(ip-)address, though it might also include a "network namespace"
key different then the default.
For local-host-only transports this is either an explicit
namespace (with types defined by the OS: netns, Cgroup, IPC,
pid, etc. on linux) or failing that the sub-directory in the
filesys in which socket/shm files are located *under*.
'''
...
@classmethod
def from_addr(cls, addr: UnwrappedAddress) -> Address:
...
def unwrap(self) -> UnwrappedAddress:
'''
Deliver the underying minimum field set in
a primitive python data type-structure.
'''
...
@classmethod
def get_random(
cls,
current_actor: Actor,
bindspace: str|None = None,
) -> Address:
...
# TODO, this should be something like a `.get_def_registar_addr()`
# or similar since,
# - it should be a **host singleton** (not root/tree singleton)
# - we **only need this value** when one isn't provided to the
# runtime at boot and we want to implicitly provide a host-wide
# registrar.
# - each rooted-actor-tree should likely have its own
# micro-registry (likely the root being it), also see
@classmethod
def get_root(cls) -> Address:
...
def __repr__(self) -> str:
...
def __eq__(self, other) -> bool:
...
async def open_listener(
self,
**kwargs,
) -> SocketListener:
...
async def close_listener(self):
...
class TCPAddress(Address):
proto_key: str = 'tcp'
unwrapped_type: type = tuple[str, int]
def_bindspace: str = '127.0.0.1'
def __init__(
self,
host: str,
port: int
):
if (
not isinstance(host, str)
or
not isinstance(port, int)
):
raise TypeError(
f'Expected host {host!r} to be str and port {port!r} to be int'
)
self._host: str = host
self._port: int = port
@property
def is_valid(self) -> bool:
return self._port != 0
@property
def bindspace(self) -> str:
return self._host
@property
def domain(self) -> str:
return self._host
@classmethod
def from_addr(
cls,
addr: tuple[str, int]
) -> TCPAddress:
return TCPAddress(addr[0], addr[1])
def unwrap(self) -> tuple[str, int]:
return (
self._host,
self._port,
)
@classmethod
def get_random(
cls,
current_actor: Actor,
bindspace: str = def_bindspace,
) -> TCPAddress:
return TCPAddress(bindspace, 0)
@classmethod
def get_root(cls) -> Address:
return TCPAddress(
'127.0.0.1',
1616,
)
def __repr__(self) -> str:
return (
f'{type(self).__name__}[{self.unwrap()}]'
)
def __eq__(self, other) -> bool:
if not isinstance(other, TCPAddress):
raise TypeError(
f'Can not compare {type(other)} with {type(self)}'
)
return (
self._host == other._host
and
self._port == other._port
)
async def open_listener(
self,
**kwargs,
) -> SocketListener:
listeners: list[SocketListener] = await open_tcp_listeners(
host=self._host,
port=self._port,
**kwargs
)
assert len(listeners) == 1
listener = listeners[0]
self._host, self._port = listener.socket.getsockname()[:2]
return listener
async def close_listener(self):
...
class UDSAddress(Address):
# TODO, maybe we should use better field and value
# -[x] really this is a `.protocol_key` not a "name" of anything.
# -[ ] consider a 'unix' proto-key instead?
# -[ ] need to check what other mult-transport frameworks do
# like zmq, nng, uri-spec et al!
proto_key: str = 'uds'
unwrapped_type: type = tuple[str, int]
def_bindspace: Path = get_rt_dir()
def __init__(
self,
filepath: str|Path,
maybe_pid: int,
# ^XXX, in the sense you can also pass
# a "non-real-world-process-id" such as is handy to represent
# our host-local default "port-like" key for the very first
# root actor to create a registry address.
):
self._filepath: Path = Path(filepath).absolute()
self._pid: int = maybe_pid
@property
def is_valid(self) -> bool:
'''
We block socket files not allocated under the runtime subdir.
'''
return self.bindspace in self._filepath.parents
@property
def bindspace(self) -> Path:
'''
We replicate the "ip-set-of-hosts" part of a UDS socket as
just the sub-directory in which we allocate socket files.
'''
return self.def_bindspace
@classmethod
def from_addr(
cls,
addr: tuple[Path, int]
) -> UDSAddress:
return UDSAddress(
filepath=addr[0],
maybe_pid=addr[1],
)
def unwrap(self) -> tuple[Path, int]:
return (
str(self._filepath),
# XXX NOTE, since this gets passed DIRECTLY to
# `open_unix_socket_w_passcred()` above!
self._pid,
)
@classmethod
def get_random(
cls,
bindspace: Path|None = None, # default netns
) -> UDSAddress:
bs: Path = bindspace or get_rt_dir()
pid: int = os.getpid()
actor: Actor|None = current_actor(
err_on_no_runtime=False,
)
if actor:
sockname: str = '::'.join(actor.uid) + f'@{pid}'
else:
prefix: str = '<unknown-actor>'
if is_root_process():
prefix: str = 'root'
sockname: str = f'{prefix}@{pid}'
sockpath: Path = Path(f'{bs}/{sockname}.sock')
return UDSAddress(
# filename=f'{tempfile.gettempdir()}/{uuid4()}.sock'
filepath=sockpath,
maybe_pid=pid,
)
@classmethod
def get_root(cls) -> Address:
def_uds_filepath: Path = (
get_rt_dir()
/
'registry@1616.sock'
)
return UDSAddress(
filepath=def_uds_filepath,
maybe_pid=1616
)
def __repr__(self) -> str:
return (
f'{type(self).__name__}'
f'['
f'({self._filepath}, {self._pid})'
f']'
)
def __eq__(self, other) -> bool:
if not isinstance(other, UDSAddress):
raise TypeError(
f'Can not compare {type(other)} with {type(self)}'
)
return self._filepath == other._filepath
# async def open_listener(self, **kwargs) -> SocketListener:
async def open_listener(
self,
**kwargs,
) -> SocketListener:
self._sock = socket.socket(
socket.AF_UNIX,
socket.SOCK_STREAM
)
log.info(
f'Attempting to bind UDS socket\n'
f'>[\n'
f'|_{self}\n'
)
await self._sock.bind(self._filepath)
self._sock.listen(1)
log.info(
f'Listening on UDS socket\n'
f'[>\n'
f' |_{self}\n'
)
return SocketListener(self._sock)
def close_listener(self):
self._sock.close()
os.unlink(self._filepath)
preferred_transport: str = 'uds'
_address_types: bidict[str, Type[Address]] = {
'tcp': TCPAddress,
'uds': UDSAddress
}
# TODO! really these are discovery sys default addrs ONLY useful for
# when none is provided to a root actor on first boot.
_default_lo_addrs: dict[
str,
UnwrappedAddress
] = {
'tcp': TCPAddress.get_root().unwrap(),
'uds': UDSAddress.get_root().unwrap(),
}
def get_address_cls(name: str) -> Type[Address]:
return _address_types[name]
def is_wrapped_addr(addr: any) -> bool:
return type(addr) in _address_types.values()
def mk_uuid() -> str:
'''
Encapsulate creation of a uuid4 as `str` as used
for creating `Actor.uid: tuple[str, str]` and/or
`.msg.types.Aid`.
'''
return str(uuid4())
def wrap_address(
addr: UnwrappedAddress
) -> Address:
if is_wrapped_addr(addr):
return addr
cls: Type|None = None
match addr:
case (
str()|Path(),
int(),
):
cls = UDSAddress
case tuple() | list():
cls = TCPAddress
case None:
cls: Type[Address] = get_address_cls(preferred_transport)
addr: UnwrappedAddress = cls.get_root().unwrap()
case _:
raise TypeError(
f'Can not wrap address {type(addr)}\n'
f'{addr!r}\n'
)
return cls.from_addr(addr)
def default_lo_addrs(
transports: list[str],
) -> list[Type[Address]]:
'''
Return the default, host-singleton, registry address
for an input transport key set.
'''
return [
_default_lo_addrs[transport]
for transport in transports
]

View File

@ -31,8 +31,12 @@ def parse_uid(arg):
return str(name), str(uuid) # ensures str encoding
def parse_ipaddr(arg):
host, port = literal_eval(arg)
return (str(host), int(port))
try:
return literal_eval(arg)
except (ValueError, SyntaxError):
# UDS: try to interpret as a straight up str
return arg
if __name__ == "__main__":
@ -46,8 +50,8 @@ if __name__ == "__main__":
args = parser.parse_args()
subactor = Actor(
args.uid[0],
uid=args.uid[1],
name=args.uid[0],
uuid=args.uid[1],
loglevel=args.loglevel,
spawn_method="trio"
)

View File

@ -89,7 +89,7 @@ from .msg import (
pretty_struct,
_ops as msgops,
)
from ._ipc import (
from .ipc import (
Channel,
)
from ._streaming import (
@ -105,7 +105,7 @@ from ._state import (
if TYPE_CHECKING:
from ._portal import Portal
from ._runtime import Actor
from ._ipc import MsgTransport
from .ipc import MsgTransport
from .devx._frame_stack import (
CallerInfo,
)
@ -859,19 +859,10 @@ class Context:
@property
def dst_maddr(self) -> str:
chan: Channel = self.chan
dst_addr, dst_port = chan.raddr
trans: MsgTransport = chan.transport
# cid: str = self.cid
# cid_head, cid_tail = cid[:6], cid[-6:]
return (
f'/ipv4/{dst_addr}'
f'/{trans.name_key}/{dst_port}'
# f'/{self.chan.uid[0]}'
# f'/{self.cid}'
# f'/cid={cid_head}..{cid_tail}'
# TODO: ? not use this ^ right ?
)
return trans.maddr
dmaddr = dst_maddr

View File

@ -29,7 +29,13 @@ from contextlib import asynccontextmanager as acm
from tractor.log import get_logger
from .trionics import gather_contexts
from ._ipc import _connect_chan, Channel
from .ipc import _connect_chan, Channel
from ._addr import (
UnwrappedAddress,
Address,
preferred_transport,
wrap_address
)
from ._portal import (
Portal,
open_portal,
@ -49,9 +55,7 @@ log = get_logger(__name__)
@acm
async def get_registry(
host: str,
port: int,
addr: UnwrappedAddress|None = None,
) -> AsyncGenerator[
Portal | LocalPortal | None,
None,
@ -69,13 +73,15 @@ async def get_registry(
# (likely a re-entrant call from the arbiter actor)
yield LocalPortal(
actor,
Channel((host, port))
Channel(transport=None)
# ^XXX, we DO NOT actually provide nor connect an
# underlying transport since this is merely an API shim.
)
else:
# TODO: try to look pre-existing connection from
# `Actor._peers` and use it instead?
async with (
_connect_chan(host, port) as chan,
_connect_chan(addr) as chan,
open_portal(chan) as regstr_ptl,
):
yield regstr_ptl
@ -89,11 +95,10 @@ async def get_root(
# TODO: rename mailbox to `_root_maddr` when we finally
# add and impl libp2p multi-addrs?
host, port = _runtime_vars['_root_mailbox']
assert host is not None
addr = _runtime_vars['_root_mailbox']
async with (
_connect_chan(host, port) as chan,
_connect_chan(addr) as chan,
open_portal(chan, **kwargs) as portal,
):
yield portal
@ -134,10 +139,10 @@ def get_peer_by_name(
@acm
async def query_actor(
name: str,
regaddr: tuple[str, int]|None = None,
regaddr: UnwrappedAddress|None = None,
) -> AsyncGenerator[
tuple[str, int]|None,
UnwrappedAddress|None,
None,
]:
'''
@ -163,31 +168,31 @@ async def query_actor(
return
reg_portal: Portal
regaddr: tuple[str, int] = regaddr or actor.reg_addrs[0]
async with get_registry(*regaddr) as reg_portal:
regaddr: Address = wrap_address(regaddr) or actor.reg_addrs[0]
async with get_registry(regaddr) as reg_portal:
# TODO: return portals to all available actors - for now
# just the last one that registered
sockaddr: tuple[str, int] = await reg_portal.run_from_ns(
addr: UnwrappedAddress = await reg_portal.run_from_ns(
'self',
'find_actor',
name=name,
)
yield sockaddr
yield addr
@acm
async def maybe_open_portal(
addr: tuple[str, int],
addr: UnwrappedAddress,
name: str,
):
async with query_actor(
name=name,
regaddr=addr,
) as sockaddr:
) as addr:
pass
if sockaddr:
async with _connect_chan(*sockaddr) as chan:
if addr:
async with _connect_chan(addr) as chan:
async with open_portal(chan) as portal:
yield portal
else:
@ -197,7 +202,8 @@ async def maybe_open_portal(
@acm
async def find_actor(
name: str,
registry_addrs: list[tuple[str, int]]|None = None,
registry_addrs: list[UnwrappedAddress]|None = None,
enable_transports: list[str] = [preferred_transport],
only_first: bool = True,
raise_on_none: bool = False,
@ -224,15 +230,15 @@ async def find_actor(
# XXX NOTE: make sure to dynamically read the value on
# every call since something may change it globally (eg.
# like in our discovery test suite)!
from . import _root
from ._addr import default_lo_addrs
registry_addrs = (
_runtime_vars['_registry_addrs']
or
_root._default_lo_addrs
default_lo_addrs(enable_transports)
)
maybe_portals: list[
AsyncContextManager[tuple[str, int]]
AsyncContextManager[UnwrappedAddress]
] = list(
maybe_open_portal(
addr=addr,
@ -274,7 +280,7 @@ async def find_actor(
@acm
async def wait_for_actor(
name: str,
registry_addr: tuple[str, int] | None = None,
registry_addr: UnwrappedAddress | None = None,
) -> AsyncGenerator[Portal, None]:
'''
@ -291,7 +297,7 @@ async def wait_for_actor(
yield peer_portal
return
regaddr: tuple[str, int] = (
regaddr: UnwrappedAddress = (
registry_addr
or
actor.reg_addrs[0]
@ -299,8 +305,8 @@ async def wait_for_actor(
# TODO: use `.trionics.gather_contexts()` like
# above in `find_actor()` as well?
reg_portal: Portal
async with get_registry(*regaddr) as reg_portal:
sockaddrs = await reg_portal.run_from_ns(
async with get_registry(regaddr) as reg_portal:
addrs = await reg_portal.run_from_ns(
'self',
'wait_for_actor',
name=name,
@ -308,8 +314,8 @@ async def wait_for_actor(
# get latest registered addr by default?
# TODO: offer multi-portal yields in multi-homed case?
sockaddr: tuple[str, int] = sockaddrs[-1]
addr: UnwrappedAddress = addrs[-1]
async with _connect_chan(*sockaddr) as chan:
async with _connect_chan(addr) as chan:
async with open_portal(chan) as portal:
yield portal

View File

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

View File

@ -65,7 +65,7 @@ if TYPE_CHECKING:
from ._context import Context
from .log import StackLevelAdapter
from ._stream import MsgStream
from ._ipc import Channel
from .ipc import Channel
log = get_logger('tractor')

View File

@ -43,7 +43,7 @@ from .trionics import maybe_open_nursery
from ._state import (
current_actor,
)
from ._ipc import Channel
from .ipc import Channel
from .log import get_logger
from .msg import (
# Error,
@ -107,6 +107,10 @@ class Portal:
# point.
self._expect_result_ctx: Context|None = None
self._streams: set[MsgStream] = set()
# TODO, this should be PRIVATE (and never used publicly)! since it's just
# a cached ref to the local runtime instead of calling
# `current_actor()` everywhere.. XD
self.actor: Actor = current_actor()
@property
@ -504,8 +508,12 @@ class LocalPortal:
return it's result.
'''
obj = self.actor if ns == 'self' else importlib.import_module(ns)
func = getattr(obj, func_name)
obj = (
self.actor
if ns == 'self'
else importlib.import_module(ns)
)
func: Callable = getattr(obj, func_name)
return await func(**kwargs)

View File

@ -18,7 +18,9 @@
Root actor runtime ignition(s).
'''
from contextlib import asynccontextmanager as acm
from contextlib import (
asynccontextmanager as acm,
)
from functools import partial
import importlib
import inspect
@ -26,7 +28,10 @@ import logging
import os
import signal
import sys
from typing import Callable
from typing import (
Any,
Callable,
)
import warnings
@ -43,33 +48,107 @@ from .devx import _debug
from . import _spawn
from . import _state
from . import log
from ._ipc import _connect_chan
from ._exceptions import is_multi_cancelled
# set at startup and after forks
_default_host: str = '127.0.0.1'
_default_port: int = 1616
# default registry always on localhost
_default_lo_addrs: list[tuple[str, int]] = [(
_default_host,
_default_port,
)]
from .ipc import (
_connect_chan,
)
from ._addr import (
Address,
UnwrappedAddress,
default_lo_addrs,
mk_uuid,
preferred_transport,
wrap_address,
)
from ._exceptions import (
ActorFailure,
is_multi_cancelled,
)
logger = log.get_logger('tractor')
# TODO: stick this in a `@acm` defined in `devx._debug`?
# -[ ] also maybe consider making this a `wrapt`-deco to
# save an indent level?
#
@acm
async def maybe_block_bp(
debug_mode: bool,
maybe_enable_greenback: bool,
) -> bool:
# Override the global debugger hook to make it play nice with
# ``trio``, see much discussion in:
# https://github.com/python-trio/trio/issues/1155#issuecomment-742964018
builtin_bp_handler: Callable = sys.breakpointhook
orig_bp_path: str|None = os.environ.get(
'PYTHONBREAKPOINT',
None,
)
bp_blocked: bool
if (
debug_mode
and maybe_enable_greenback
and (
maybe_mod := await _debug.maybe_init_greenback(
raise_not_found=False,
)
)
):
logger.info(
f'Found `greenback` installed @ {maybe_mod}\n'
'Enabling `tractor.pause_from_sync()` support!\n'
)
os.environ['PYTHONBREAKPOINT'] = (
'tractor.devx._debug._sync_pause_from_builtin'
)
_state._runtime_vars['use_greenback'] = True
bp_blocked = False
else:
# TODO: disable `breakpoint()` by default (without
# `greenback`) since it will break any multi-actor
# usage by a clobbered TTY's stdstreams!
def block_bps(*args, **kwargs):
raise RuntimeError(
'Trying to use `breakpoint()` eh?\n\n'
'Welp, `tractor` blocks `breakpoint()` built-in calls by default!\n'
'If you need to use it please install `greenback` and set '
'`debug_mode=True` when opening the runtime '
'(either via `.open_nursery()` or `open_root_actor()`)\n'
)
sys.breakpointhook = block_bps
# lol ok,
# https://docs.python.org/3/library/sys.html#sys.breakpointhook
os.environ['PYTHONBREAKPOINT'] = "0"
bp_blocked = True
try:
yield bp_blocked
finally:
# restore any prior built-in `breakpoint()` hook state
if builtin_bp_handler is not None:
sys.breakpointhook = builtin_bp_handler
if orig_bp_path is not None:
os.environ['PYTHONBREAKPOINT'] = orig_bp_path
else:
# clear env back to having no entry
os.environ.pop('PYTHONBREAKPOINT', None)
@acm
async def open_root_actor(
*,
# defaults are above
registry_addrs: list[tuple[str, int]]|None = None,
registry_addrs: list[UnwrappedAddress]|None = None,
# defaults are above
arbiter_addr: tuple[str, int]|None = None,
arbiter_addr: tuple[UnwrappedAddress]|None = None,
enable_transports: list[str] = [preferred_transport],
name: str|None = 'root',
@ -111,350 +190,323 @@ async def open_root_actor(
Runtime init entry point for ``tractor``.
'''
_debug.hide_runtime_frames()
__tracebackhide__: bool = hide_tb
# TODO: stick this in a `@cm` defined in `devx._debug`?
#
# Override the global debugger hook to make it play nice with
# ``trio``, see much discussion in:
# https://github.com/python-trio/trio/issues/1155#issuecomment-742964018
builtin_bp_handler: Callable = sys.breakpointhook
orig_bp_path: str|None = os.environ.get(
'PYTHONBREAKPOINT',
None,
)
if (
debug_mode
and maybe_enable_greenback
and (
maybe_mod := await _debug.maybe_init_greenback(
raise_not_found=False,
)
# XXX NEVER allow nested actor-trees!
if already_actor := _state.current_actor(err_on_no_runtime=False):
rtvs: dict[str, Any] = _state._runtime_vars
root_mailbox: list[str, int] = rtvs['_root_mailbox']
registry_addrs: list[list[str, int]] = rtvs['_registry_addrs']
raise ActorFailure(
f'A current actor already exists !?\n'
f'({already_actor}\n'
f'\n'
f'You can NOT open a second root actor from within '
f'an existing tree and the current root of this '
f'already exists !!\n'
f'\n'
f'_root_mailbox: {root_mailbox!r}\n'
f'_registry_addrs: {registry_addrs!r}\n'
)
async with maybe_block_bp(
debug_mode=debug_mode,
maybe_enable_greenback=maybe_enable_greenback,
):
logger.info(
f'Found `greenback` installed @ {maybe_mod}\n'
'Enabling `tractor.pause_from_sync()` support!\n'
)
os.environ['PYTHONBREAKPOINT'] = (
'tractor.devx._debug._sync_pause_from_builtin'
)
_state._runtime_vars['use_greenback'] = True
_debug.hide_runtime_frames()
__tracebackhide__: bool = hide_tb
else:
# TODO: disable `breakpoint()` by default (without
# `greenback`) since it will break any multi-actor
# usage by a clobbered TTY's stdstreams!
def block_bps(*args, **kwargs):
raise RuntimeError(
'Trying to use `breakpoint()` eh?\n\n'
'Welp, `tractor` blocks `breakpoint()` built-in calls by default!\n'
'If you need to use it please install `greenback` and set '
'`debug_mode=True` when opening the runtime '
'(either via `.open_nursery()` or `open_root_actor()`)\n'
# attempt to retreive ``trio``'s sigint handler and stash it
# on our debugger lock state.
_debug.DebugStatus._trio_handler = signal.getsignal(signal.SIGINT)
# mark top most level process as root actor
_state._runtime_vars['_is_root'] = True
# caps based rpc list
enable_modules = (
enable_modules
or
[]
)
if rpc_module_paths:
warnings.warn(
"`rpc_module_paths` is now deprecated, use "
" `enable_modules` instead.",
DeprecationWarning,
stacklevel=2,
)
enable_modules.extend(rpc_module_paths)
if start_method is not None:
_spawn.try_set_start_method(start_method)
# TODO! remove this ASAP!
if arbiter_addr is not None:
warnings.warn(
'`arbiter_addr` is now deprecated\n'
'Use `registry_addrs: list[tuple]` instead..',
DeprecationWarning,
stacklevel=2,
)
registry_addrs = [arbiter_addr]
if not registry_addrs:
registry_addrs: list[UnwrappedAddress] = default_lo_addrs(
enable_transports
)
sys.breakpointhook = block_bps
# lol ok,
# https://docs.python.org/3/library/sys.html#sys.breakpointhook
os.environ['PYTHONBREAKPOINT'] = "0"
assert registry_addrs
# attempt to retreive ``trio``'s sigint handler and stash it
# on our debugger lock state.
_debug.DebugStatus._trio_handler = signal.getsignal(signal.SIGINT)
loglevel = (
loglevel
or log._default_loglevel
).upper()
# mark top most level process as root actor
_state._runtime_vars['_is_root'] = True
# caps based rpc list
enable_modules = (
enable_modules
or
[]
)
if rpc_module_paths:
warnings.warn(
"`rpc_module_paths` is now deprecated, use "
" `enable_modules` instead.",
DeprecationWarning,
stacklevel=2,
)
enable_modules.extend(rpc_module_paths)
if start_method is not None:
_spawn.try_set_start_method(start_method)
if arbiter_addr is not None:
warnings.warn(
'`arbiter_addr` is now deprecated\n'
'Use `registry_addrs: list[tuple]` instead..',
DeprecationWarning,
stacklevel=2,
)
registry_addrs = [arbiter_addr]
registry_addrs: list[tuple[str, int]] = (
registry_addrs
or
_default_lo_addrs
)
assert registry_addrs
loglevel = (
loglevel
or log._default_loglevel
).upper()
if (
debug_mode
and _spawn._spawn_method == 'trio'
):
_state._runtime_vars['_debug_mode'] = True
# expose internal debug module to every actor allowing for
# use of ``await tractor.pause()``
enable_modules.append('tractor.devx._debug')
# if debug mode get's enabled *at least* use that level of
# logging for some informative console prompts.
if (
logging.getLevelName(
# lul, need the upper case for the -> int map?
# sweet "dynamic function behaviour" stdlib...
loglevel,
) > logging.getLevelName('PDB')
debug_mode
and _spawn._spawn_method == 'trio'
):
loglevel = 'PDB'
_state._runtime_vars['_debug_mode'] = True
# expose internal debug module to every actor allowing for
# use of ``await tractor.pause()``
enable_modules.append('tractor.devx._debug')
# if debug mode get's enabled *at least* use that level of
# logging for some informative console prompts.
if (
logging.getLevelName(
# lul, need the upper case for the -> int map?
# sweet "dynamic function behaviour" stdlib...
loglevel,
) > logging.getLevelName('PDB')
):
loglevel = 'PDB'
elif debug_mode:
raise RuntimeError(
"Debug mode is only supported for the `trio` backend!"
)
assert loglevel
_log = log.get_console_log(loglevel)
assert _log
# TODO: factor this into `.devx._stackscope`!!
if (
debug_mode
and
enable_stack_on_sig
):
from .devx._stackscope import enable_stack_on_sig
enable_stack_on_sig()
# closed into below ping task-func
ponged_addrs: list[tuple[str, int]] = []
async def ping_tpt_socket(
addr: tuple[str, int],
timeout: float = 1,
) -> None:
'''
Attempt temporary connection to see if a registry is
listening at the requested address by a tranport layer
ping.
If a connection can't be made quickly we assume none no
server is listening at that addr.
'''
try:
# TODO: this connect-and-bail forces us to have to
# carefully rewrap TCP 104-connection-reset errors as
# EOF so as to avoid propagating cancel-causing errors
# to the channel-msg loop machinery. Likely it would
# be better to eventually have a "discovery" protocol
# with basic handshake instead?
with trio.move_on_after(timeout):
async with _connect_chan(*addr):
ponged_addrs.append(addr)
except OSError:
# TODO: make this a "discovery" log level?
logger.info(
f'No actor registry found @ {addr}\n'
)
async with trio.open_nursery() as tn:
for addr in registry_addrs:
tn.start_soon(
ping_tpt_socket,
tuple(addr), # TODO: just drop this requirement?
)
trans_bind_addrs: list[tuple[str, int]] = []
# Create a new local root-actor instance which IS NOT THE
# REGISTRAR
if ponged_addrs:
if ensure_registry:
elif debug_mode:
raise RuntimeError(
f'Failed to open `{name}`@{ponged_addrs}: '
'registry socket(s) already bound'
"Debug mode is only supported for the `trio` backend!"
)
# we were able to connect to an arbiter
logger.info(
f'Registry(s) seem(s) to exist @ {ponged_addrs}'
)
assert loglevel
_log = log.get_console_log(loglevel)
assert _log
actor = Actor(
name=name or 'anonymous',
registry_addrs=ponged_addrs,
loglevel=loglevel,
enable_modules=enable_modules,
)
# DO NOT use the registry_addrs as the transport server
# addrs for this new non-registar, root-actor.
for host, port in ponged_addrs:
# NOTE: zero triggers dynamic OS port allocation
trans_bind_addrs.append((host, 0))
# Start this local actor as the "registrar", aka a regular
# actor who manages the local registry of "mailboxes" of
# other process-tree-local sub-actors.
else:
# NOTE that if the current actor IS THE REGISTAR, the
# following init steps are taken:
# - the tranport layer server is bound to each (host, port)
# pair defined in provided registry_addrs, or the default.
trans_bind_addrs = registry_addrs
# - it is normally desirable for any registrar to stay up
# indefinitely until either all registered (child/sub)
# actors are terminated (via SC supervision) or,
# a re-election process has taken place.
# NOTE: all of ^ which is not implemented yet - see:
# https://github.com/goodboy/tractor/issues/216
# https://github.com/goodboy/tractor/pull/348
# https://github.com/goodboy/tractor/issues/296
actor = Arbiter(
name or 'registrar',
registry_addrs=registry_addrs,
loglevel=loglevel,
enable_modules=enable_modules,
)
# XXX, in case the root actor runtime was actually run from
# `tractor.to_asyncio.run_as_asyncio_guest()` and NOt
# `.trio.run()`.
actor._infected_aio = _state._runtime_vars['_is_infected_aio']
# Start up main task set via core actor-runtime nurseries.
try:
# assign process-local actor
_state._current_actor = actor
# start local channel-server and fake the portal API
# NOTE: this won't block since we provide the nursery
ml_addrs_str: str = '\n'.join(
f'@{addr}' for addr in trans_bind_addrs
)
logger.info(
f'Starting local {actor.uid} on the following transport addrs:\n'
f'{ml_addrs_str}'
)
# start the actor runtime in a new task
async with trio.open_nursery(
strict_exception_groups=False,
# ^XXX^ TODO? instead unpack any RAE as per "loose" style?
) as nursery:
# ``_runtime.async_main()`` creates an internal nursery
# and blocks here until any underlying actor(-process)
# tree has terminated thereby conducting so called
# "end-to-end" structured concurrency throughout an
# entire hierarchical python sub-process set; all
# "actor runtime" primitives are SC-compat and thus all
# transitively spawned actors/processes must be as
# well.
await nursery.start(
partial(
async_main,
actor,
accept_addrs=trans_bind_addrs,
parent_addr=None
)
)
try:
yield actor
except (
Exception,
BaseExceptionGroup,
) as err:
# TODO, in beginning to handle the subsubactor with
# crashed grandparent cases..
#
# was_locked: bool = await _debug.maybe_wait_for_debugger(
# child_in_debug=True,
# )
# XXX NOTE XXX see equiv note inside
# `._runtime.Actor._stream_handler()` where in the
# non-root or root-that-opened-this-mahually case we
# wait for the local actor-nursery to exit before
# exiting the transport channel handler.
entered: bool = await _debug._maybe_enter_pm(
err,
api_frame=inspect.currentframe(),
debug_filter=debug_filter,
)
if (
not entered
and
not is_multi_cancelled(
err,
)
):
logger.exception('Root actor crashed\n')
# ALWAYS re-raise any error bubbled up from the
# runtime!
raise
finally:
# NOTE: not sure if we'll ever need this but it's
# possibly better for even more determinism?
# logger.cancel(
# f'Waiting on {len(nurseries)} nurseries in root..')
# nurseries = actor._actoruid2nursery.values()
# async with trio.open_nursery() as tempn:
# for an in nurseries:
# tempn.start_soon(an.exited.wait)
logger.info(
'Closing down root actor'
)
await actor.cancel(None) # self cancel
finally:
_state._current_actor = None
_state._last_actor_terminated = actor
# restore built-in `breakpoint()` hook state
# TODO: factor this into `.devx._stackscope`!!
if (
debug_mode
and
maybe_enable_greenback
enable_stack_on_sig
):
if builtin_bp_handler is not None:
sys.breakpointhook = builtin_bp_handler
from .devx._stackscope import enable_stack_on_sig
enable_stack_on_sig()
if orig_bp_path is not None:
os.environ['PYTHONBREAKPOINT'] = orig_bp_path
# closed into below ping task-func
ponged_addrs: list[UnwrappedAddress] = []
else:
# clear env back to having no entry
os.environ.pop('PYTHONBREAKPOINT', None)
async def ping_tpt_socket(
addr: UnwrappedAddress,
timeout: float = 1,
) -> None:
'''
Attempt temporary connection to see if a registry is
listening at the requested address by a tranport layer
ping.
logger.runtime("Root actor terminated")
If a connection can't be made quickly we assume none no
server is listening at that addr.
'''
try:
# TODO: this connect-and-bail forces us to have to
# carefully rewrap TCP 104-connection-reset errors as
# EOF so as to avoid propagating cancel-causing errors
# to the channel-msg loop machinery. Likely it would
# be better to eventually have a "discovery" protocol
# with basic handshake instead?
with trio.move_on_after(timeout):
async with _connect_chan(addr):
ponged_addrs.append(addr)
except OSError:
# TODO: make this a "discovery" log level?
logger.info(
f'No actor registry found @ {addr}\n'
)
async with trio.open_nursery() as tn:
for addr in registry_addrs:
tn.start_soon(
ping_tpt_socket,
addr,
)
trans_bind_addrs: list[UnwrappedAddress] = []
# Create a new local root-actor instance which IS NOT THE
# REGISTRAR
if ponged_addrs:
if ensure_registry:
raise RuntimeError(
f'Failed to open `{name}`@{ponged_addrs}: '
'registry socket(s) already bound'
)
# we were able to connect to an arbiter
logger.info(
f'Registry(s) seem(s) to exist @ {ponged_addrs}'
)
actor = Actor(
name=name or 'anonymous',
uuid=mk_uuid(),
registry_addrs=ponged_addrs,
loglevel=loglevel,
enable_modules=enable_modules,
)
# DO NOT use the registry_addrs as the transport server
# addrs for this new non-registar, root-actor.
for addr in ponged_addrs:
waddr: Address = wrap_address(addr)
trans_bind_addrs.append(
waddr.get_random(bindspace=waddr.bindspace)
)
# Start this local actor as the "registrar", aka a regular
# actor who manages the local registry of "mailboxes" of
# other process-tree-local sub-actors.
else:
# NOTE that if the current actor IS THE REGISTAR, the
# following init steps are taken:
# - the tranport layer server is bound to each addr
# pair defined in provided registry_addrs, or the default.
trans_bind_addrs = registry_addrs
# - it is normally desirable for any registrar to stay up
# indefinitely until either all registered (child/sub)
# actors are terminated (via SC supervision) or,
# a re-election process has taken place.
# NOTE: all of ^ which is not implemented yet - see:
# https://github.com/goodboy/tractor/issues/216
# https://github.com/goodboy/tractor/pull/348
# https://github.com/goodboy/tractor/issues/296
actor = Arbiter(
name=name or 'registrar',
uuid=mk_uuid(),
registry_addrs=registry_addrs,
loglevel=loglevel,
enable_modules=enable_modules,
)
# XXX, in case the root actor runtime was actually run from
# `tractor.to_asyncio.run_as_asyncio_guest()` and NOt
# `.trio.run()`.
actor._infected_aio = _state._runtime_vars['_is_infected_aio']
# Start up main task set via core actor-runtime nurseries.
try:
# assign process-local actor
_state._current_actor = actor
# start local channel-server and fake the portal API
# NOTE: this won't block since we provide the nursery
ml_addrs_str: str = '\n'.join(
f'@{addr}' for addr in trans_bind_addrs
)
logger.info(
f'Starting local {actor.uid} on the following transport addrs:\n'
f'{ml_addrs_str}'
)
# start the actor runtime in a new task
async with trio.open_nursery(
strict_exception_groups=False,
# ^XXX^ TODO? instead unpack any RAE as per "loose" style?
) as nursery:
# ``_runtime.async_main()`` creates an internal nursery
# and blocks here until any underlying actor(-process)
# tree has terminated thereby conducting so called
# "end-to-end" structured concurrency throughout an
# entire hierarchical python sub-process set; all
# "actor runtime" primitives are SC-compat and thus all
# transitively spawned actors/processes must be as
# well.
await nursery.start(
partial(
async_main,
actor,
accept_addrs=trans_bind_addrs,
parent_addr=None
)
)
try:
yield actor
except (
Exception,
BaseExceptionGroup,
) as err:
# TODO, in beginning to handle the subsubactor with
# crashed grandparent cases..
#
# was_locked: bool = await _debug.maybe_wait_for_debugger(
# child_in_debug=True,
# )
# XXX NOTE XXX see equiv note inside
# `._runtime.Actor._stream_handler()` where in the
# non-root or root-that-opened-this-mahually case we
# wait for the local actor-nursery to exit before
# exiting the transport channel handler.
entered: bool = await _debug._maybe_enter_pm(
err,
api_frame=inspect.currentframe(),
debug_filter=debug_filter,
)
if (
not entered
and
not is_multi_cancelled(
err,
)
):
logger.exception(
'Root actor crashed\n'
f'>x)\n'
f' |_{actor}\n'
)
# ALWAYS re-raise any error bubbled up from the
# runtime!
raise
finally:
# NOTE: not sure if we'll ever need this but it's
# possibly better for even more determinism?
# logger.cancel(
# f'Waiting on {len(nurseries)} nurseries in root..')
# nurseries = actor._actoruid2nursery.values()
# async with trio.open_nursery() as tempn:
# for an in nurseries:
# tempn.start_soon(an.exited.wait)
logger.info(
f'Closing down root actor\n'
f'>)\n'
f'|_{actor}\n'
)
await actor.cancel(None) # self cancel
finally:
_state._current_actor = None
_state._last_actor_terminated = actor
logger.runtime(
f'Root actor terminated\n'
f')>\n'
f' |_{actor}\n'
)
def run_daemon(
@ -462,7 +514,7 @@ def run_daemon(
# runtime kwargs
name: str | None = 'root',
registry_addrs: list[tuple[str, int]] = _default_lo_addrs,
registry_addrs: list[UnwrappedAddress]|None = None,
start_method: str | None = None,
debug_mode: bool = False,

View File

@ -42,7 +42,7 @@ from trio import (
TaskStatus,
)
from ._ipc import Channel
from .ipc import Channel
from ._context import (
Context,
)
@ -1156,7 +1156,7 @@ async def process_messages(
trio.Event(),
)
# runtime-scoped remote (internal) error
# XXX RUNTIME-SCOPED! remote (likely internal) error
# (^- bc no `Error.cid` -^)
#
# NOTE: this is the non-rpc error case, that

View File

@ -52,6 +52,7 @@ import sys
from typing import (
Any,
Callable,
Type,
TYPE_CHECKING,
)
import uuid
@ -73,7 +74,15 @@ from tractor.msg import (
pretty_struct,
types as msgtypes,
)
from ._ipc import Channel
from .ipc import Channel
from ._addr import (
UnwrappedAddress,
Address,
default_lo_addrs,
get_address_cls,
preferred_transport,
wrap_address,
)
from ._context import (
mk_context,
Context,
@ -175,15 +184,15 @@ class Actor:
def __init__(
self,
name: str,
uuid: str,
*,
enable_modules: list[str] = [],
uid: str|None = None,
loglevel: str|None = None,
registry_addrs: list[tuple[str, int]]|None = None,
registry_addrs: list[UnwrappedAddress]|None = None,
spawn_method: str|None = None,
# TODO: remove!
arbiter_addr: tuple[str, int]|None = None,
arbiter_addr: UnwrappedAddress|None = None,
) -> None:
'''
@ -191,12 +200,14 @@ class Actor:
phase (aka before a new process is executed).
'''
self.name = name
self.uid = (
name,
uid or str(uuid.uuid4())
self._aid = msgtypes.Aid(
name=name,
uuid=uuid,
pid=os.getpid(),
)
self._task: trio.Task|None = None
# state
self._cancel_complete = trio.Event()
self._cancel_called_by_remote: tuple[str, tuple]|None = None
self._cancel_called: bool = False
@ -223,7 +234,7 @@ class Actor:
DeprecationWarning,
stacklevel=2,
)
registry_addrs: list[tuple[str, int]] = [arbiter_addr]
registry_addrs: list[UnwrappedAddress] = [arbiter_addr]
# marked by the process spawning backend at startup
# will be None for the parent most process started manually
@ -257,6 +268,7 @@ class Actor:
] = {}
self._listeners: list[trio.abc.Listener] = []
self._listen_addrs: list[Address] = []
self._parent_chan: Channel|None = None
self._forkserver_info: tuple|None = None
@ -269,13 +281,84 @@ class Actor:
# when provided, init the registry addresses property from
# input via the validator.
self._reg_addrs: list[tuple[str, int]] = []
self._reg_addrs: list[UnwrappedAddress] = []
if registry_addrs:
self.reg_addrs: list[tuple[str, int]] = registry_addrs
self.reg_addrs: list[UnwrappedAddress] = registry_addrs
_state._runtime_vars['_registry_addrs'] = registry_addrs
@property
def reg_addrs(self) -> list[tuple[str, int]]:
def aid(self) -> msgtypes.Aid:
'''
This process-singleton-actor's "unique ID" in struct form.
'''
return self._aid
@property
def name(self) -> str:
return self._aid.name
@property
def uid(self) -> tuple[str, str]:
'''
This process-singleton's "unique (cross-host) ID".
Delivered from the `.Aid.name/.uuid` fields as a `tuple` pair
and should be multi-host unique despite a large distributed
process plane.
'''
return (
self._aid.name,
self._aid.uuid,
)
@property
def pid(self) -> int:
return self._aid.pid
def pformat(self) -> str:
ds: str = '='
parent_uid: tuple|None = None
if rent_chan := self._parent_chan:
parent_uid = rent_chan.uid
peers: list[tuple] = list(self._peer_connected)
listen_addrs: str = pformat(self._listen_addrs)
fmtstr: str = (
f' |_id: {self.aid!r}\n'
# f" aid{ds}{self.aid!r}\n"
f" parent{ds}{parent_uid}\n"
f'\n'
f' |_ipc: {len(peers)!r} connected peers\n'
f" peers{ds}{peers!r}\n"
f" _listen_addrs{ds}'{listen_addrs}'\n"
f" _listeners{ds}'{self._listeners}'\n"
f'\n'
f' |_rpc: {len(self._rpc_tasks)} tasks\n'
f" ctxs{ds}{len(self._contexts)}\n"
f'\n'
f' |_runtime: ._task{ds}{self._task!r}\n'
f' _spawn_method{ds}{self._spawn_method}\n'
f' _actoruid2nursery{ds}{self._actoruid2nursery}\n'
f' _forkserver_info{ds}{self._forkserver_info}\n'
f'\n'
f' |_state: "TODO: .repr_state()"\n'
f' _cancel_complete{ds}{self._cancel_complete}\n'
f' _cancel_called_by_remote{ds}{self._cancel_called_by_remote}\n'
f' _cancel_called{ds}{self._cancel_called}\n'
)
return (
'<Actor(\n'
+
fmtstr
+
')>\n'
)
__repr__ = pformat
@property
def reg_addrs(self) -> list[UnwrappedAddress]:
'''
List of (socket) addresses for all known (and contactable)
registry actors.
@ -286,7 +369,7 @@ class Actor:
@reg_addrs.setter
def reg_addrs(
self,
addrs: list[tuple[str, int]],
addrs: list[UnwrappedAddress],
) -> None:
if not addrs:
log.warning(
@ -295,16 +378,7 @@ class Actor:
)
return
# always sanity check the input list since it's critical
# that addrs are correct for discovery sys operation.
for addr in addrs:
if not isinstance(addr, tuple):
raise ValueError(
'Expected `Actor.reg_addrs: list[tuple[str, int]]`\n'
f'Got {addrs}'
)
self._reg_addrs = addrs
self._reg_addrs = addrs
async def wait_for_peer(
self,
@ -423,12 +497,19 @@ class Actor:
try:
uid: tuple|None = await self._do_handshake(chan)
except (
# we need this for ``msgspec`` for some reason?
# for now, it's been put in the stream backend.
TransportClosed,
# ^XXX NOTE, the above wraps `trio` exc types raised
# during various `SocketStream.send/receive_xx()` calls
# under different fault conditions such as,
#
# trio.BrokenResourceError,
# trio.ClosedResourceError,
TransportClosed,
#
# Inside our `.ipc._transport` layer we absorb and
# re-raise our own `TransportClosed` exc such that this
# higher level runtime code can only worry one
# "kinda-error" that we expect to tolerate during
# discovery-sys related pings, queires, DoS etc.
):
# XXX: This may propagate up from `Channel._aiter_recv()`
# and `MsgpackStream._inter_packets()` on a read from the
@ -1024,11 +1105,12 @@ class Actor:
async def _from_parent(
self,
parent_addr: tuple[str, int]|None,
parent_addr: UnwrappedAddress|None,
) -> tuple[
Channel,
list[tuple[str, int]]|None,
list[UnwrappedAddress]|None,
list[str]|None, # preferred tpts
]:
'''
Bootstrap this local actor's runtime config from its parent by
@ -1040,33 +1122,58 @@ class Actor:
# Connect back to the parent actor and conduct initial
# handshake. From this point on if we error, we
# attempt to ship the exception back to the parent.
chan = Channel(
destaddr=parent_addr,
chan = await Channel.from_addr(
addr=wrap_address(parent_addr)
)
await chan.connect()
assert isinstance(chan, Channel)
# TODO: move this into a `Channel.handshake()`?
# Initial handshake: swap names.
await self._do_handshake(chan)
accept_addrs: list[tuple[str, int]]|None = None
accept_addrs: list[UnwrappedAddress]|None = None
if self._spawn_method == "trio":
# Receive post-spawn runtime state from our parent.
spawnspec: msgtypes.SpawnSpec = await chan.recv()
self._spawn_spec = spawnspec
match spawnspec:
case MsgTypeError():
raise spawnspec
case msgtypes.SpawnSpec():
self._spawn_spec = spawnspec
log.runtime(
'Received runtime spec from parent:\n\n'
log.runtime(
'Received runtime spec from parent:\n\n'
# TODO: eventually all these msgs as
# `msgspec.Struct` with a special mode that
# pformats them in multi-line mode, BUT only
# if "trace"/"util" mode is enabled?
f'{pretty_struct.pformat(spawnspec)}\n'
)
# TODO: eventually all these msgs as
# `msgspec.Struct` with a special mode that
# pformats them in multi-line mode, BUT only
# if "trace"/"util" mode is enabled?
f'{pretty_struct.pformat(spawnspec)}\n'
)
accept_addrs: list[tuple[str, int]] = spawnspec.bind_addrs
case _:
raise InternalError(
f'Received invalid non-`SpawnSpec` payload !?\n'
f'{spawnspec}\n'
)
# ^^TODO XXX!! when the `SpawnSpec` fails to decode
# the above will raise a `MsgTypeError` which if we
# do NOT ALSO RAISE it will tried to be pprinted in
# the log.runtime() below..
#
# SO we gotta look at how other `chan.recv()` calls
# are wrapped and do the same for this spec receive!
# -[ ] see `._rpc` likely has the answer?
#
# XXX NOTE, can't be called here in subactor
# bc we haven't yet received the
# `SpawnSpec._runtime_vars: dict` which would
# declare whether `debug_mode` is set!
# breakpoint()
# import pdbp; pdbp.set_trace()
accept_addrs: list[UnwrappedAddress] = spawnspec.bind_addrs
# TODO: another `Struct` for rtvs..
rvs: dict[str, Any] = spawnspec._runtime_vars
@ -1158,6 +1265,9 @@ class Actor:
return (
chan,
accept_addrs,
None,
# ^TODO, preferred tpts list from rent!
# -[ ] need to extend the `SpawnSpec` tho!
)
except OSError: # failed to connect
@ -1173,54 +1283,74 @@ class Actor:
self,
handler_nursery: Nursery,
*,
# (host, port) to bind for channel server
listen_sockaddrs: list[tuple[str, int]]|None = None,
listen_addrs: list[UnwrappedAddress]|None = None,
task_status: TaskStatus[Nursery] = trio.TASK_STATUS_IGNORED,
) -> None:
'''
Start the IPC transport server, begin listening for new connections.
Start the IPC transport server, begin listening/accepting new
`trio.SocketStream` connections.
This will cause an actor to continue living (and thus
blocking at the process/OS-thread level) until
`.cancel_server()` is called.
'''
if listen_sockaddrs is None:
listen_sockaddrs = [(None, 0)]
if listen_addrs is None:
listen_addrs = default_lo_addrs([preferred_transport])
else:
listen_addrs: list[Address] = [
wrap_address(a) for a in listen_addrs
]
self._server_down = trio.Event()
try:
async with trio.open_nursery() as server_n:
for host, port in listen_sockaddrs:
listeners: list[trio.abc.Listener] = await server_n.start(
partial(
trio.serve_tcp,
listeners: list[trio.abc.Listener] = []
for addr in listen_addrs:
try:
listener: trio.abc.Listener = await addr.open_listener()
except OSError as oserr:
if (
'[Errno 98] Address already in use'
in
oserr.args[0]
):
log.exception(
f'Address already in use?\n'
f'{addr}\n'
)
raise
listeners.append(listener)
handler=self._stream_handler,
port=port,
host=host,
await server_n.start(
partial(
trio.serve_listeners,
handler=self._stream_handler,
listeners=listeners,
# NOTE: configured such that new
# connections will stay alive even if
# this server is cancelled!
handler_nursery=handler_nursery,
)
# NOTE: configured such that new
# connections will stay alive even if
# this server is cancelled!
handler_nursery=handler_nursery
)
sockets: list[trio.socket] = [
getattr(listener, 'socket', 'unknown socket')
for listener in listeners
]
log.runtime(
'Started TCP server(s)\n'
f'|_{sockets}\n'
)
self._listeners.extend(listeners)
)
log.runtime(
'Started server(s)\n'
'\n'.join([f'|_{addr}' for addr in listen_addrs])
)
self._listen_addrs.extend(listen_addrs)
self._listeners.extend(listeners)
task_status.started(server_n)
finally:
addr: Address
for addr in listen_addrs:
addr.close_listener()
# signal the server is down since nursery above terminated
self._server_down.set()
@ -1579,26 +1709,21 @@ class Actor:
return False
@property
def accept_addrs(self) -> list[tuple[str, int]]:
def accept_addrs(self) -> list[UnwrappedAddress]:
'''
All addresses to which the transport-channel server binds
and listens for new connections.
'''
# throws OSError on failure
return [
listener.socket.getsockname()
for listener in self._listeners
] # type: ignore
return [a.unwrap() for a in self._listen_addrs]
@property
def accept_addr(self) -> tuple[str, int]:
def accept_addr(self) -> UnwrappedAddress:
'''
Primary address to which the IPC transport server is
bound and listening for new connections.
'''
# throws OSError on failure
return self.accept_addrs[0]
def get_parent(self) -> Portal:
@ -1645,8 +1770,6 @@ class Actor:
chan.aid = aid
uid: tuple[str, str] = (
# str(value[0]),
# str(value[1])
aid.name,
aid.uuid,
)
@ -1670,7 +1793,7 @@ class Actor:
async def async_main(
actor: Actor,
accept_addrs: tuple[str, int]|None = None,
accept_addrs: UnwrappedAddress|None = None,
# XXX: currently ``parent_addr`` is only needed for the
# ``multiprocessing`` backend (which pickles state sent to
@ -1679,7 +1802,7 @@ async def async_main(
# change this to a simple ``is_subactor: bool`` which will
# be False when running as root actor and True when as
# a subactor.
parent_addr: tuple[str, int]|None = None,
parent_addr: UnwrappedAddress|None = None,
task_status: TaskStatus[None] = trio.TASK_STATUS_IGNORED,
) -> None:
@ -1694,6 +1817,8 @@ async def async_main(
the actor's "runtime" and all thus all ongoing RPC tasks.
'''
actor._task: trio.Task = trio.lowlevel.current_task()
# attempt to retreive ``trio``'s sigint handler and stash it
# on our debugger state.
_debug.DebugStatus._trio_handler = signal.getsignal(signal.SIGINT)
@ -1703,13 +1828,15 @@ async def async_main(
# establish primary connection with immediate parent
actor._parent_chan: Channel|None = None
if parent_addr is not None:
if parent_addr is not None:
(
actor._parent_chan,
set_accept_addr_says_rent,
maybe_preferred_transports_says_rent,
) = await actor._from_parent(parent_addr)
accept_addrs: list[UnwrappedAddress] = []
# either it's passed in because we're not a child or
# because we're running in mp mode
if (
@ -1718,6 +1845,18 @@ async def async_main(
set_accept_addr_says_rent is not None
):
accept_addrs = set_accept_addr_says_rent
else:
enable_transports: list[str] = (
maybe_preferred_transports_says_rent
or
[preferred_transport]
)
for transport_key in enable_transports:
transport_cls: Type[Address] = get_address_cls(
transport_key
)
addr: Address = transport_cls.get_random()
accept_addrs.append(addr.unwrap())
# The "root" nursery ensures the channel with the immediate
# parent is kept alive as a resilient service until
@ -1769,7 +1908,7 @@ async def async_main(
partial(
actor._serve_forever,
service_nursery,
listen_sockaddrs=accept_addrs,
listen_addrs=accept_addrs,
)
)
except OSError as oserr:
@ -1785,7 +1924,7 @@ async def async_main(
raise
accept_addrs: list[tuple[str, int]] = actor.accept_addrs
accept_addrs: list[UnwrappedAddress] = actor.accept_addrs
# NOTE: only set the loopback addr for the
# process-tree-global "root" mailbox since
@ -1793,9 +1932,8 @@ async def async_main(
# their root actor over that channel.
if _state._runtime_vars['_is_root']:
for addr in accept_addrs:
host, _ = addr
# TODO: generic 'lo' detector predicate
if '127.0.0.1' in host:
waddr = wrap_address(addr)
if waddr == waddr.get_root():
_state._runtime_vars['_root_mailbox'] = addr
# Register with the arbiter if we're told its addr
@ -1810,24 +1948,21 @@ async def async_main(
# only on unique actor uids?
for addr in actor.reg_addrs:
try:
assert isinstance(addr, tuple)
assert addr[1] # non-zero after bind
waddr = wrap_address(addr)
assert waddr.is_valid
except AssertionError:
await _debug.pause()
async with get_registry(*addr) as reg_portal:
async with get_registry(addr) as reg_portal:
for accept_addr in accept_addrs:
if not accept_addr[1]:
await _debug.pause()
assert accept_addr[1]
accept_addr = wrap_address(accept_addr)
assert accept_addr.is_valid
await reg_portal.run_from_ns(
'self',
'register_actor',
uid=actor.uid,
sockaddr=accept_addr,
addr=accept_addr.unwrap(),
)
is_registered: bool = True
@ -1954,12 +2089,13 @@ async def async_main(
):
failed: bool = False
for addr in actor.reg_addrs:
assert isinstance(addr, tuple)
waddr = wrap_address(addr)
assert waddr.is_valid
with trio.move_on_after(0.5) as cs:
cs.shield = True
try:
async with get_registry(
*addr,
addr,
) as reg_portal:
await reg_portal.run_from_ns(
'self',
@ -2037,7 +2173,7 @@ class Arbiter(Actor):
self._registry: dict[
tuple[str, str],
tuple[str, int],
UnwrappedAddress,
] = {}
self._waiters: dict[
str,
@ -2053,18 +2189,18 @@ class Arbiter(Actor):
self,
name: str,
) -> tuple[str, int]|None:
) -> UnwrappedAddress|None:
for uid, sockaddr in self._registry.items():
for uid, addr in self._registry.items():
if name in uid:
return sockaddr
return addr
return None
async def get_registry(
self
) -> dict[str, tuple[str, int]]:
) -> dict[str, UnwrappedAddress]:
'''
Return current name registry.
@ -2084,7 +2220,7 @@ class Arbiter(Actor):
self,
name: str,
) -> list[tuple[str, int]]:
) -> list[UnwrappedAddress]:
'''
Wait for a particular actor to register.
@ -2092,44 +2228,41 @@ class Arbiter(Actor):
registered.
'''
sockaddrs: list[tuple[str, int]] = []
sockaddr: tuple[str, int]
addrs: list[UnwrappedAddress] = []
addr: UnwrappedAddress
mailbox_info: str = 'Actor registry contact infos:\n'
for uid, sockaddr in self._registry.items():
for uid, addr in self._registry.items():
mailbox_info += (
f'|_uid: {uid}\n'
f'|_sockaddr: {sockaddr}\n\n'
f'|_addr: {addr}\n\n'
)
if name == uid[0]:
sockaddrs.append(sockaddr)
addrs.append(addr)
if not sockaddrs:
if not addrs:
waiter = trio.Event()
self._waiters.setdefault(name, []).append(waiter)
await waiter.wait()
for uid in self._waiters[name]:
if not isinstance(uid, trio.Event):
sockaddrs.append(self._registry[uid])
addrs.append(self._registry[uid])
log.runtime(mailbox_info)
return sockaddrs
return addrs
async def register_actor(
self,
uid: tuple[str, str],
sockaddr: tuple[str, int]
addr: UnwrappedAddress
) -> None:
uid = name, hash = (str(uid[0]), str(uid[1]))
addr = (host, port) = (
str(sockaddr[0]),
int(sockaddr[1]),
)
if port == 0:
waddr: Address = wrap_address(addr)
if not waddr.is_valid:
# should never be 0-dynamic-os-alloc
await _debug.pause()
assert port # should never be 0-dynamic-os-alloc
self._registry[uid] = addr
# pop and signal all waiter events

View File

@ -46,6 +46,7 @@ from tractor._state import (
_runtime_vars,
)
from tractor.log import get_logger
from tractor._addr import UnwrappedAddress
from tractor._portal import Portal
from tractor._runtime import Actor
from tractor._entry import _mp_main
@ -392,14 +393,15 @@ async def new_proc(
errors: dict[tuple[str, str], Exception],
# passed through to actor main
bind_addrs: list[tuple[str, int]],
parent_addr: tuple[str, int],
bind_addrs: list[UnwrappedAddress],
parent_addr: UnwrappedAddress,
_runtime_vars: dict[str, Any], # serialized and sent to _child
*,
infect_asyncio: bool = False,
task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED
task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED,
proc_kwargs: dict[str, any] = {}
) -> None:
@ -419,6 +421,7 @@ async def new_proc(
_runtime_vars, # run time vars
infect_asyncio=infect_asyncio,
task_status=task_status,
proc_kwargs=proc_kwargs
)
@ -429,12 +432,13 @@ async def trio_proc(
errors: dict[tuple[str, str], Exception],
# passed through to actor main
bind_addrs: list[tuple[str, int]],
parent_addr: tuple[str, int],
bind_addrs: list[UnwrappedAddress],
parent_addr: UnwrappedAddress,
_runtime_vars: dict[str, Any], # serialized and sent to _child
*,
infect_asyncio: bool = False,
task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED
task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED,
proc_kwargs: dict[str, any] = {}
) -> None:
'''
@ -475,7 +479,7 @@ async def trio_proc(
proc: trio.Process|None = None
try:
try:
proc: trio.Process = await trio.lowlevel.open_process(spawn_cmd)
proc: trio.Process = await trio.lowlevel.open_process(spawn_cmd, **proc_kwargs)
log.runtime(
'Started new child\n'
f'|_{proc}\n'
@ -517,15 +521,15 @@ async def trio_proc(
# send a "spawning specification" which configures the
# initial runtime state of the child.
await chan.send(
SpawnSpec(
_parent_main_data=subactor._parent_main_data,
enable_modules=subactor.enable_modules,
reg_addrs=subactor.reg_addrs,
bind_addrs=bind_addrs,
_runtime_vars=_runtime_vars,
)
sspec = SpawnSpec(
_parent_main_data=subactor._parent_main_data,
enable_modules=subactor.enable_modules,
reg_addrs=subactor.reg_addrs,
bind_addrs=bind_addrs,
_runtime_vars=_runtime_vars,
)
log.runtime(f'Sending spawn spec: {str(sspec)}')
await chan.send(sspec)
# track subactor in current nursery
curr_actor: Actor = current_actor()
@ -635,12 +639,13 @@ async def mp_proc(
subactor: Actor,
errors: dict[tuple[str, str], Exception],
# passed through to actor main
bind_addrs: list[tuple[str, int]],
parent_addr: tuple[str, int],
bind_addrs: list[UnwrappedAddress],
parent_addr: UnwrappedAddress,
_runtime_vars: dict[str, Any], # serialized and sent to _child
*,
infect_asyncio: bool = False,
task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED
task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED,
proc_kwargs: dict[str, any] = {}
) -> None:

View File

@ -14,14 +14,16 @@
# 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/>.
"""
Per process state
'''
Per actor-process runtime state mgmt APIs.
"""
'''
from __future__ import annotations
from contextvars import (
ContextVar,
)
import os
from pathlib import Path
from typing import (
Any,
TYPE_CHECKING,
@ -143,3 +145,22 @@ def current_ipc_ctx(
f'|_{current_task()}\n'
)
return ctx
# std ODE (mutable) app state location
_rtdir: Path = Path(os.environ['XDG_RUNTIME_DIR'])
def get_rt_dir(
subdir: str = 'tractor'
) -> Path:
'''
Return the user "runtime dir" where most userspace apps stick
their IPC and cache related system util-files; we take hold
of a `'XDG_RUNTIME_DIR'/tractor/` subdir by default.
'''
rtdir: Path = _rtdir / subdir
if not rtdir.is_dir():
rtdir.mkdir()
return rtdir

View File

@ -56,7 +56,7 @@ from tractor.msg import (
if TYPE_CHECKING:
from ._runtime import Actor
from ._context import Context
from ._ipc import Channel
from .ipc import Channel
log = get_logger(__name__)
@ -595,8 +595,17 @@ class MsgStream(trio.abc.Channel):
trio.ClosedResourceError,
trio.BrokenResourceError,
BrokenPipeError,
) as trans_err:
if hide_tb:
) as _trans_err:
trans_err = _trans_err
if (
hide_tb
and
self._ctx.chan._exc is trans_err
# ^XXX, IOW, only if the channel is marked errored
# for the same reason as whatever its underlying
# transport raised, do we keep the full low-level tb
# suppressed from the user.
):
raise type(trans_err)(
*trans_err.args
) from trans_err

View File

@ -22,13 +22,21 @@ from contextlib import asynccontextmanager as acm
from functools import partial
import inspect
from pprint import pformat
from typing import TYPE_CHECKING
from typing import (
TYPE_CHECKING,
)
import typing
import warnings
import trio
from .devx._debug import maybe_wait_for_debugger
from ._addr import (
UnwrappedAddress,
preferred_transport,
mk_uuid,
)
from ._state import current_actor, is_main_process
from .log import get_logger, get_loglevel
from ._runtime import Actor
@ -47,8 +55,6 @@ if TYPE_CHECKING:
log = get_logger(__name__)
_default_bind_addr: tuple[str, int] = ('127.0.0.1', 0)
class ActorNursery:
'''
@ -130,8 +136,9 @@ class ActorNursery:
*,
bind_addrs: list[tuple[str, int]] = [_default_bind_addr],
bind_addrs: list[UnwrappedAddress]|None = None,
rpc_module_paths: list[str]|None = None,
enable_transports: list[str] = [preferred_transport],
enable_modules: list[str]|None = None,
loglevel: str|None = None, # set log level per subactor
debug_mode: bool|None = None,
@ -141,6 +148,7 @@ class ActorNursery:
# a `._ria_nursery` since the dependent APIs have been
# removed!
nursery: trio.Nursery|None = None,
proc_kwargs: dict[str, any] = {}
) -> Portal:
'''
@ -177,7 +185,9 @@ class ActorNursery:
enable_modules.extend(rpc_module_paths)
subactor = Actor(
name,
name=name,
uuid=mk_uuid(),
# modules allowed to invoked funcs from
enable_modules=enable_modules,
loglevel=loglevel,
@ -185,7 +195,7 @@ class ActorNursery:
# verbatim relay this actor's registrar addresses
registry_addrs=current_actor().reg_addrs,
)
parent_addr = self._actor.accept_addr
parent_addr: UnwrappedAddress = self._actor.accept_addr
assert parent_addr
# start a task to spawn a process
@ -204,6 +214,7 @@ class ActorNursery:
parent_addr,
_rtv, # run time vars
infect_asyncio=infect_asyncio,
proc_kwargs=proc_kwargs
)
)
@ -222,11 +233,12 @@ class ActorNursery:
*,
name: str | None = None,
bind_addrs: tuple[str, int] = [_default_bind_addr],
bind_addrs: UnwrappedAddress|None = None,
rpc_module_paths: list[str] | None = None,
enable_modules: list[str] | None = None,
loglevel: str | None = None, # set log level per subactor
infect_asyncio: bool = False,
proc_kwargs: dict[str, any] = {},
**kwargs, # explicit args to ``fn``
@ -257,6 +269,7 @@ class ActorNursery:
# use the run_in_actor nursery
nursery=self._ria_nursery,
infect_asyncio=infect_asyncio,
proc_kwargs=proc_kwargs
)
# XXX: don't allow stream funcs

View File

@ -0,0 +1,81 @@
import os
import random
def generate_single_byte_msgs(amount: int) -> bytes:
'''
Generate a byte instance of len `amount` with:
```
byte_at_index(i) = (i % 10).encode()
```
this results in constantly repeating sequences of:
b'0123456789'
'''
return b''.join(str(i % 10).encode() for i in range(amount))
def generate_sample_messages(
amount: int,
rand_min: int = 0,
rand_max: int = 0,
silent: bool = False,
) -> tuple[list[bytes], int]:
'''
Generate bytes msgs for tests.
Messages will have the following format:
```
b'[{i:08}]' + os.urandom(random.randint(rand_min, rand_max))
```
so for message index 25:
b'[00000025]' + random_bytes
'''
msgs = []
size = 0
log_interval = None
if not silent:
print(f'\ngenerating {amount} messages...')
# calculate an apropiate log interval based on
# max message size
max_msg_size = 10 + rand_max
if max_msg_size <= 32 * 1024:
log_interval = 10_000
else:
log_interval = 1000
for i in range(amount):
msg = f'[{i:08}]'.encode('utf-8')
if rand_max > 0:
msg += os.urandom(
random.randint(rand_min, rand_max))
size += len(msg)
msgs.append(msg)
if (
not silent
and
i > 0
and
i % log_interval == 0
):
print(f'{i} generated')
if not silent:
print(f'done, {size:,} bytes in total')
return msgs, size

View File

@ -91,7 +91,7 @@ from tractor._state import (
if TYPE_CHECKING:
from trio.lowlevel import Task
from threading import Thread
from tractor._ipc import Channel
from tractor.ipc import Channel
from tractor._runtime import (
Actor,
)

View File

@ -0,0 +1,66 @@
# 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/>.
import platform
from ._transport import (
MsgTransportKey as MsgTransportKey,
MsgType as MsgType,
MsgTransport as MsgTransport,
MsgpackTransport as MsgpackTransport
)
from ._tcp import MsgpackTCPStream as MsgpackTCPStream
from ._uds import MsgpackUDSStream as MsgpackUDSStream
from ._types import (
transport_from_addr as transport_from_addr,
transport_from_stream as transport_from_stream,
)
from ._chan import (
_connect_chan as _connect_chan,
Channel as Channel
)
if platform.system() == 'Linux':
from ._linux import (
EFD_SEMAPHORE as EFD_SEMAPHORE,
EFD_CLOEXEC as EFD_CLOEXEC,
EFD_NONBLOCK as EFD_NONBLOCK,
open_eventfd as open_eventfd,
write_eventfd as write_eventfd,
read_eventfd as read_eventfd,
close_eventfd as close_eventfd,
EFDReadCancelled as EFDReadCancelled,
EventFD as EventFD,
)
from ._ringbuf import (
RBToken as RBToken,
open_ringbuf as open_ringbuf,
RingBuffSender as RingBuffSender,
RingBuffReceiver as RingBuffReceiver,
open_ringbuf_pair as open_ringbuf_pair,
attach_to_ringbuf_receiver as attach_to_ringbuf_receiver,
attach_to_ringbuf_sender as attach_to_ringbuf_sender,
attach_to_ringbuf_stream as attach_to_ringbuf_stream,
RingBuffBytesSender as RingBuffBytesSender,
RingBuffBytesReceiver as RingBuffBytesReceiver,
RingBuffChannel as RingBuffChannel,
attach_to_ringbuf_schannel as attach_to_ringbuf_schannel,
attach_to_ringbuf_rchannel as attach_to_ringbuf_rchannel,
attach_to_ringbuf_channel as attach_to_ringbuf_channel,
)

View File

@ -0,0 +1,397 @@
# 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/>.
"""
Inter-process comms abstractions
"""
from __future__ import annotations
from collections.abc import AsyncGenerator
from contextlib import (
asynccontextmanager as acm,
contextmanager as cm,
)
import os
import platform
from pprint import pformat
import typing
from typing import (
Any,
)
import trio
from tractor.ipc._transport import MsgTransport
from tractor.ipc._types import (
transport_from_addr,
transport_from_stream,
)
from tractor._addr import (
is_wrapped_addr,
wrap_address,
Address,
UnwrappedAddress,
)
from tractor.log import get_logger
from tractor._exceptions import (
MsgTypeError,
pack_from_raise,
)
from tractor.msg import MsgCodec
log = get_logger(__name__)
_is_windows = platform.system() == 'Windows'
class Channel:
'''
An inter-process channel for communication between (remote) actors.
Wraps a ``MsgStream``: transport + encoding IPC connection.
Currently we only support ``trio.SocketStream`` for transport
(aka TCP) and the ``msgpack`` interchange format via the ``msgspec``
codec libary.
'''
def __init__(
self,
transport: MsgTransport|None = None,
# TODO: optional reconnection support?
# auto_reconnect: bool = False,
# on_reconnect: typing.Callable[..., typing.Awaitable] = None,
) -> None:
# self._recon_seq = on_reconnect
# self._autorecon = auto_reconnect
# Either created in ``.connect()`` or passed in by
# user in ``.from_stream()``.
self._transport: MsgTransport|None = transport
# set after handshake - always uid of far end
self.uid: tuple[str, str]|None = None
self._aiter_msgs = self._iter_msgs()
self._exc: Exception|None = None
# ^XXX! ONLY set if a remote actor sends an `Error`-msg
self._closed: bool = False
# flag set by ``Portal.cancel_actor()`` indicating remote
# (possibly peer) cancellation of the far end actor
# runtime.
self._cancel_called: bool = False
@property
def stream(self) -> trio.abc.Stream | None:
return self._transport.stream if self._transport else None
@property
def msgstream(self) -> MsgTransport:
log.info(
'`Channel.msgstream` is an old name, use `._transport`'
)
return self._transport
@property
def transport(self) -> MsgTransport:
return self._transport
@classmethod
def from_stream(
cls,
stream: trio.abc.Stream,
) -> Channel:
transport_cls = transport_from_stream(stream)
return Channel(
transport=transport_cls(stream)
)
@classmethod
async def from_addr(
cls,
addr: UnwrappedAddress,
**kwargs
) -> Channel:
if not is_wrapped_addr(addr):
addr: Address = wrap_address(addr)
transport_cls = transport_from_addr(addr)
transport = await transport_cls.connect_to(
addr,
**kwargs,
)
assert transport.raddr == addr
chan = Channel(transport=transport)
log.runtime(
f'Connected channel IPC transport\n'
f'[>\n'
f' |_{chan}\n'
)
return chan
@cm
def apply_codec(
self,
codec: MsgCodec,
) -> None:
'''
Temporarily override the underlying IPC msg codec for
dynamic enforcement of messaging schema.
'''
orig: MsgCodec = self._transport.codec
try:
self._transport.codec = codec
yield
finally:
self._transport.codec = orig
# TODO: do a .src/.dst: str for maddrs?
def pformat(self) -> str:
if not self._transport:
return '<Channel with inactive transport?>'
tpt: MsgTransport = self._transport
tpt_name: str = type(tpt).__name__
tpt_status: str = (
'connected' if self.connected()
else 'closed'
)
return (
f'<Channel(\n'
f' |_status: {tpt_status!r}\n'
f' _closed={self._closed}\n'
f' _cancel_called={self._cancel_called}\n'
f'\n'
f' |_runtime: Actor\n'
f' pid={os.getpid()}\n'
f' uid={self.uid}\n'
f'\n'
f' |_msgstream: {tpt_name}\n'
f' proto={tpt.laddr.proto_key!r}\n'
f' layer={tpt.layer_key!r}\n'
f' laddr={tpt.laddr}\n'
f' raddr={tpt.raddr}\n'
f' codec={tpt.codec_key!r}\n'
f' stream={tpt.stream}\n'
f' maddr={tpt.maddr!r}\n'
f' drained={tpt.drained}\n'
f' _send_lock={tpt._send_lock.statistics()}\n'
f')>\n'
)
# NOTE: making this return a value that can be passed to
# `eval()` is entirely **optional** FYI!
# https://docs.python.org/3/library/functions.html#repr
# https://docs.python.org/3/reference/datamodel.html#object.__repr__
#
# Currently we target **readability** from a (console)
# logging perspective over `eval()`-ability since we do NOT
# target serializing non-struct instances!
# def __repr__(self) -> str:
__str__ = pformat
__repr__ = pformat
@property
def laddr(self) -> Address|None:
return self._transport.laddr if self._transport else None
@property
def raddr(self) -> Address|None:
return self._transport.raddr if self._transport else None
# TODO: something like,
# `pdbp.hideframe_on(errors=[MsgTypeError])`
# instead of the `try/except` hack we have rn..
# seems like a pretty useful thing to have in general
# along with being able to filter certain stack frame(s / sets)
# possibly based on the current log-level?
async def send(
self,
payload: Any,
hide_tb: bool = False,
) -> None:
'''
Send a coded msg-blob over the transport.
'''
__tracebackhide__: bool = hide_tb
try:
log.transport(
'=> send IPC msg:\n\n'
f'{pformat(payload)}\n'
)
# assert self._transport # but why typing?
await self._transport.send(
payload,
hide_tb=hide_tb,
)
except BaseException as _err:
err = _err # bind for introspection
if not isinstance(_err, MsgTypeError):
# assert err
__tracebackhide__: bool = False
else:
try:
assert err.cid
except KeyError:
raise err
raise
async def recv(self) -> Any:
assert self._transport
return await self._transport.recv()
# TODO: auto-reconnect features like 0mq/nanomsg?
# -[ ] implement it manually with nods to SC prot
# possibly on multiple transport backends?
# -> seems like that might be re-inventing scalability
# prots tho no?
# try:
# return await self._transport.recv()
# except trio.BrokenResourceError:
# if self._autorecon:
# await self._reconnect()
# return await self.recv()
# raise
async def aclose(self) -> None:
log.transport(
f'Closing channel to {self.uid} '
f'{self.laddr} -> {self.raddr}'
)
assert self._transport
await self._transport.stream.aclose()
self._closed = True
async def __aenter__(self):
await self.connect()
return self
async def __aexit__(self, *args):
await self.aclose(*args)
def __aiter__(self):
return self._aiter_msgs
# ?TODO? run any reconnection sequence?
# -[ ] prolly should be impl-ed as deco-API?
#
# async def _reconnect(self) -> None:
# """Handle connection failures by polling until a reconnect can be
# established.
# """
# down = False
# while True:
# try:
# with trio.move_on_after(3) as cancel_scope:
# await self.connect()
# cancelled = cancel_scope.cancelled_caught
# if cancelled:
# log.transport(
# "Reconnect timed out after 3 seconds, retrying...")
# continue
# else:
# log.transport("Stream connection re-established!")
# # on_recon = self._recon_seq
# # if on_recon:
# # await on_recon(self)
# break
# except (OSError, ConnectionRefusedError):
# if not down:
# down = True
# log.transport(
# f"Connection to {self.raddr} went down, waiting"
# " for re-establishment")
# await trio.sleep(1)
async def _iter_msgs(
self
) -> AsyncGenerator[Any, None]:
'''
Yield `MsgType` IPC msgs decoded and deliverd from
an underlying `MsgTransport` protocol.
This is a streaming routine alo implemented as an async-gen
func (same a `MsgTransport._iter_pkts()`) gets allocated by
a `.__call__()` inside `.__init__()` where it is assigned to
the `._aiter_msgs` attr.
'''
assert self._transport
while True:
try:
async for msg in self._transport:
match msg:
# NOTE: if transport/interchange delivers
# a type error, we pack it with the far
# end peer `Actor.uid` and relay the
# `Error`-msg upward to the `._rpc` stack
# for normal RAE handling.
case MsgTypeError():
yield pack_from_raise(
local_err=msg,
cid=msg.cid,
# XXX we pack it here bc lower
# layers have no notion of an
# actor-id ;)
src_uid=self.uid,
)
case _:
yield msg
except trio.BrokenResourceError:
# if not self._autorecon:
raise
await self.aclose()
# if self._autorecon: # attempt reconnect
# await self._reconnect()
# continue
def connected(self) -> bool:
return self._transport.connected() if self._transport else False
@acm
async def _connect_chan(
addr: UnwrappedAddress
) -> typing.AsyncGenerator[Channel, None]:
'''
Create and connect a channel with disconnect on context manager
teardown.
'''
chan = await Channel.from_addr(addr)
yield chan
with trio.CancelScope(shield=True):
await chan.aclose()

View File

@ -0,0 +1,187 @@
# 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/>.
'''
Linux specifics, for now we are only exposing EventFD
'''
import os
import errno
import cffi
import trio
ffi = cffi.FFI()
# Declare the C functions and types we plan to use.
# - eventfd: for creating the event file descriptor
# - write: for writing to the file descriptor
# - read: for reading from the file descriptor
# - close: for closing the file descriptor
ffi.cdef(
'''
int eventfd(unsigned int initval, int flags);
ssize_t write(int fd, const void *buf, size_t count);
ssize_t read(int fd, void *buf, size_t count);
int close(int fd);
'''
)
# Open the default dynamic library (essentially 'libc' in most cases)
C = ffi.dlopen(None)
# Constants from <sys/eventfd.h>, if needed.
EFD_SEMAPHORE = 1
EFD_CLOEXEC = 0o2000000
EFD_NONBLOCK = 0o4000
def open_eventfd(initval: int = 0, flags: int = 0) -> int:
'''
Open an eventfd with the given initial value and flags.
Returns the file descriptor on success, otherwise raises OSError.
'''
fd = C.eventfd(initval, flags)
if fd < 0:
raise OSError(errno.errorcode[ffi.errno], 'eventfd failed')
return fd
def write_eventfd(fd: int, value: int) -> int:
'''
Write a 64-bit integer (uint64_t) to the eventfd's counter.
'''
# Create a uint64_t* in C, store `value`
data_ptr = ffi.new('uint64_t *', value)
# Call write(fd, data_ptr, 8)
# We expect to write exactly 8 bytes (sizeof(uint64_t))
ret = C.write(fd, data_ptr, 8)
if ret < 0:
raise OSError(errno.errorcode[ffi.errno], 'write to eventfd failed')
return ret
def read_eventfd(fd: int) -> int:
'''
Read a 64-bit integer (uint64_t) from the eventfd, returning the value.
Reading resets the counter to 0 (unless using EFD_SEMAPHORE).
'''
# Allocate an 8-byte buffer in C for reading
buf = ffi.new('char[]', 8)
ret = C.read(fd, buf, 8)
if ret < 0:
raise OSError(errno.errorcode[ffi.errno], 'read from eventfd failed')
# Convert the 8 bytes we read into a Python integer
data_bytes = ffi.unpack(buf, 8) # returns a Python bytes object of length 8
value = int.from_bytes(data_bytes, byteorder='little', signed=False)
return value
def close_eventfd(fd: int) -> int:
'''
Close the eventfd.
'''
ret = C.close(fd)
if ret < 0:
raise OSError(errno.errorcode[ffi.errno], 'close failed')
class EFDReadCancelled(Exception):
...
class EventFD:
'''
Use a previously opened eventfd(2), meant to be used in
sub-actors after root actor opens the eventfds then passes
them through pass_fds
'''
def __init__(
self,
fd: int,
omode: str
):
self._fd: int = fd
self._omode: str = omode
self._fobj = None
self._cscope: trio.CancelScope | None = None
@property
def fd(self) -> int | None:
return self._fd
def write(self, value: int) -> int:
return write_eventfd(self._fd, value)
async def read(self) -> int:
'''
Async wrapper for `read_eventfd(self.fd)`
`trio.to_thread.run_sync` is used, need to use a `trio.CancelScope`
in order to make it cancellable when `self.close()` is called.
'''
self._cscope = trio.CancelScope()
with self._cscope:
return await trio.to_thread.run_sync(
read_eventfd, self._fd,
abandon_on_cancel=True
)
if self._cscope.cancelled_caught:
raise EFDReadCancelled
self._cscope = None
def read_direct(self) -> int:
'''
Direct call to `read_eventfd(self.fd)`, unless `eventfd` was
opened with `EFD_NONBLOCK` its gonna block the thread.
'''
return read_eventfd(self._fd)
def open(self):
self._fobj = os.fdopen(self._fd, self._omode)
def close(self):
if self._fobj:
try:
self._fobj.close()
except OSError:
...
if self._cscope:
self._cscope.cancel()
def __enter__(self):
self.open()
return self
def __exit__(self, exc_type, exc_value, traceback):
self.close()

View File

@ -0,0 +1,45 @@
# tractor: structured concurrent "actors".
# Copyright 2018-eternity Tyler Goodlet.
# This program is free software: you can redistribute it and/or modify
# it under the terms of the GNU Affero General Public License as published by
# the Free Software Foundation, either version 3 of the License, or
# (at your option) any later version.
# This program is distributed in the hope that it will be useful,
# but WITHOUT ANY WARRANTY; without even the implied warranty of
# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
# GNU Affero General Public License for more details.
# You should have received a copy of the GNU Affero General Public License
# along with this program. If not, see <https://www.gnu.org/licenses/>.
'''
Utils to tame mp non-SC madeness
'''
def disable_mantracker():
'''
Disable all ``multiprocessing``` "resource tracking" machinery since
it's an absolute multi-threaded mess of non-SC madness.
'''
from multiprocessing import resource_tracker as mantracker
# Tell the "resource tracker" thing to fuck off.
class ManTracker(mantracker.ResourceTracker):
def register(self, name, rtype):
pass
def unregister(self, name, rtype):
pass
def ensure_running(self):
pass
# "know your land and know your prey"
# https://www.dailymotion.com/video/x6ozzco
mantracker._resource_tracker = ManTracker()
mantracker.register = mantracker._resource_tracker.register
mantracker.ensure_running = mantracker._resource_tracker.ensure_running
mantracker.unregister = mantracker._resource_tracker.unregister
mantracker.getfd = mantracker._resource_tracker.getfd

View File

@ -0,0 +1,652 @@
# 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 ._linux import (
open_eventfd,
EFDReadCancelled,
EventFD
)
from ._mp_bs import disable_mantracker
from tractor.log import get_logger
from tractor._exceptions import (
InternalError
)
log = get_logger(__name__)
disable_mantracker()
_DEFAULT_RB_SIZE = 10 * 1024
class RBToken(Struct, frozen=True):
'''
RingBuffer token contains necesary info to open the three
eventfds and the shared memory
'''
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
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]:
'''
Useful for `pass_fds` params
'''
return (
self.write_eventfd,
self.wrap_eventfd,
self.eof_eventfd
)
@cm
def open_ringbuf(
shm_name: str,
buf_size: int = _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`
'''
shm = SharedMemory(
name=shm_name,
size=buf_size,
create=True
)
try:
with (
EventFD(open_eventfd(), 'r') as write_event,
EventFD(open_eventfd(), 'r') as wrap_event,
EventFD(open_eventfd(), 'r') as eof_event,
):
token = RBToken(
shm_name=shm_name,
write_eventfd=write_event.fd,
wrap_eventfd=wrap_event.fd,
eof_eventfd=eof_event.fd,
buf_size=buf_size
)
yield token
finally:
shm.unlink()
Buffer = bytes | bytearray | memoryview
'''
IPC Reliable Ring Buffer
`eventfd(2)` is used for wrap around sync, to signal writes to
the reader and end of stream.
'''
class RingBuffSender(trio.abc.SendStream):
'''
Ring Buffer sender side implementation
Do not use directly! manage with `attach_to_ringbuf_sender`
after having opened a ringbuf context with `open_ringbuf`.
'''
def __init__(
self,
token: RBToken,
cleanup: bool = False
):
self._token = RBToken.from_msg(token)
self._shm: SharedMemory | None = None
self._write_event = EventFD(self._token.write_eventfd, 'w')
self._wrap_event = EventFD(self._token.wrap_eventfd, 'r')
self._eof_event = EventFD(self._token.eof_eventfd, 'w')
self._ptr = 0
self._cleanup = cleanup
self._send_lock = trio.StrictFIFOLock()
@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
async def _wait_wrap(self):
await self._wrap_event.read()
async def send_all(self, data: Buffer):
async with self._send_lock:
# while data is larger than the remaining buf
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):
raise NotImplementedError
def open(self):
self._shm = SharedMemory(
name=self._token.shm_name,
size=self._token.buf_size,
create=False
)
self._write_event.open()
self._wrap_event.open()
self._eof_event.open()
def close(self):
self._eof_event.write(
self._ptr if self._ptr > 0 else self.size
)
if self._cleanup:
self._write_event.close()
self._wrap_event.close()
self._eof_event.close()
self._shm.close()
async def aclose(self):
async with self._send_lock:
self.close()
async def __aenter__(self):
self.open()
return self
class RingBuffReceiver(trio.abc.ReceiveStream):
'''
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)
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')
self._ptr: int = 0
self._write_ptr: int = 0
self._end_ptr: int = -1
self._cleanup: bool = cleanup
@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
async def _eof_monitor_task(self):
'''
Long running EOF event monitor, automatically run in bg by
`attach_to_ringbuf_receiver` context manager, if EOF event
is set its value will be the end pointer (highest valid
index to be read from buf, after setting the `self._end_ptr`
we close the write event which should cancel any blocked
`self._write_event.read()`s on it.
'''
try:
self._end_ptr = await self._eof_event.read()
self._write_event.close()
except EFDReadCancelled:
...
except trio.Cancelled:
...
async def receive_some(self, max_bytes: int | None = None) -> bytes:
'''
Receive up to `max_bytes`, if no `max_bytes` is provided
a reasonable default is used.
'''
if max_bytes is None:
max_bytes: int = _DEFAULT_RB_SIZE
if max_bytes < 1:
raise ValueError("max_bytes must be >= 1")
# delta is remaining bytes we havent read
delta = self._write_ptr - self._ptr
if delta == 0:
# we have read all we can, see if new data is available
if self._end_ptr < 0:
# if we havent been signaled about EOF yet
try:
delta = await self._write_event.read()
self._write_ptr += delta
except EFDReadCancelled:
# while waiting for new data `self._write_event` was closed
# this means writer signaled EOF
if self._end_ptr > 0:
# final self._write_ptr modification and recalculate delta
self._write_ptr = self._end_ptr
delta = self._end_ptr - self._ptr
else:
# shouldnt happen cause self._eof_monitor_task always sets
# self._end_ptr before closing self._write_event
raise InternalError(
'self._write_event.read cancelled but self._end_ptr is not set'
)
else:
# no more bytes to read and self._end_ptr set, EOF reached
return b''
# dont overflow caller
delta = min(delta, max_bytes)
target_ptr = self._ptr + delta
# fetch next segment and advance ptr
segment = 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
def open(self):
self._shm = SharedMemory(
name=self._token.shm_name,
size=self._token.buf_size,
create=False
)
self._write_event.open()
self._wrap_event.open()
self._eof_event.open()
def close(self):
if self._cleanup:
self._write_event.close()
self._wrap_event.close()
self._eof_event.close()
self._shm.close()
async def aclose(self):
self.close()
async def __aenter__(self):
self.open()
return self
@acm
async def attach_to_ringbuf_receiver(
token: RBToken,
cleanup: bool = True
) -> AsyncContextManager[RingBuffReceiver]:
'''
Attach a RingBuffReceiver from a previously opened
RBToken.
Launches `receiver._eof_monitor_task` in a `trio.Nursery`.
'''
async with (
trio.open_nursery() as n,
RingBuffReceiver(
token,
cleanup=cleanup
) as receiver
):
n.start_soon(receiver._eof_monitor_task)
yield receiver
@acm
async def attach_to_ringbuf_sender(
token: RBToken,
cleanup: bool = True
) -> AsyncContextManager[RingBuffSender]:
'''
Attach a RingBuffSender from a previously opened
RBToken.
'''
async with RingBuffSender(
token,
cleanup=cleanup
) as sender:
yield sender
@cm
def open_ringbuf_pair(
name: str,
buf_size: int = _DEFAULT_RB_SIZE
) -> ContextManager[tuple(RBToken, RBToken)]:
'''
Handle resources for a ringbuf pair to be used for
bidirectional messaging.
'''
with (
open_ringbuf(
name + '.pair0',
buf_size=buf_size
) as token_0,
open_ringbuf(
name + '.pair1',
buf_size=buf_size
) as token_1
):
yield token_0, token_1
@acm
async def attach_to_ringbuf_stream(
token_in: RBToken,
token_out: RBToken,
cleanup_in: bool = True,
cleanup_out: bool = True
) -> AsyncContextManager[trio.StapledStream]:
'''
Attach a trio.StapledStream from a previously opened
ringbuf pair.
'''
async with (
attach_to_ringbuf_receiver(
token_in,
cleanup=cleanup_in
) as receiver,
attach_to_ringbuf_sender(
token_out,
cleanup=cleanup_out
) as sender,
):
yield trio.StapledStream(sender, receiver)
class RingBuffBytesSender(trio.abc.SendChannel[bytes]):
'''
In order to guarantee full messages are received, all bytes
sent by `RingBuffBytesSender` are preceded with a 4 byte header
which decodes into a uint32 indicating the actual size of the
next payload.
Optional batch mode:
If `batch_size` > 1 messages wont get sent immediately but will be
stored until `batch_size` messages are pending, then it will send
them all at once.
`batch_size` can be changed dynamically but always call, `flush()`
right before.
'''
def __init__(
self,
sender: RingBuffSender,
batch_size: int = 1
):
self._sender = sender
self.batch_size = batch_size
self._batch_msg_len = 0
self._batch: bytes = b''
async def flush(self) -> None:
await self._sender.send_all(self._batch)
self._batch = b''
self._batch_msg_len = 0
async def send(self, value: bytes) -> None:
msg: bytes = struct.pack("<I", len(value)) + value
if self.batch_size == 1:
await self._sender.send_all(msg)
return
self._batch += msg
self._batch_msg_len += 1
if self._batch_msg_len == self.batch_size:
await self.flush()
async def aclose(self) -> None:
await self._sender.aclose()
class RingBuffBytesReceiver(trio.abc.ReceiveChannel[bytes]):
'''
See `RingBuffBytesSender` docstring.
A `tricycle.BufferedReceiveStream` is used for the
`receive_exactly` API.
'''
def __init__(
self,
receiver: RingBuffReceiver
):
self._receiver = receiver
async def _receive_exactly(self, num_bytes: int) -> bytes:
'''
Fetch bytes from receiver until we read exactly `num_bytes`
or end of stream is signaled.
'''
payload = b''
while len(payload) < num_bytes:
remaining = num_bytes - len(payload)
new_bytes = await self._receiver.receive_some(
max_bytes=remaining
)
if new_bytes == b'':
raise trio.EndOfChannel
payload += new_bytes
return payload
async def receive(self) -> bytes:
header: bytes = await self._receive_exactly(4)
size: int
size, = struct.unpack("<I", header)
if size == 0:
raise trio.EndOfChannel
return await self._receive_exactly(size)
async def aclose(self) -> None:
await self._receiver.aclose()
@acm
async def attach_to_ringbuf_rchannel(
token: RBToken,
cleanup: bool = True
) -> AsyncContextManager[RingBuffBytesReceiver]:
'''
Attach a RingBuffBytesReceiver from a previously opened
RBToken.
'''
async with attach_to_ringbuf_receiver(
token, cleanup=cleanup
) as receiver:
yield RingBuffBytesReceiver(receiver)
@acm
async def attach_to_ringbuf_schannel(
token: RBToken,
cleanup: bool = True,
batch_size: int = 1,
) -> AsyncContextManager[RingBuffBytesSender]:
'''
Attach a RingBuffBytesSender from a previously opened
RBToken.
'''
async with attach_to_ringbuf_sender(
token, cleanup=cleanup
) as sender:
yield RingBuffBytesSender(sender, batch_size=batch_size)
class RingBuffChannel(trio.abc.Channel[bytes]):
'''
Combine `RingBuffBytesSender` and `RingBuffBytesReceiver`
in order to expose the bidirectional `trio.abc.Channel` API.
'''
def __init__(
self,
sender: RingBuffBytesSender,
receiver: RingBuffBytesReceiver
):
self._sender = sender
self._receiver = receiver
async def send(self, value: bytes):
await self._sender.send(value)
async def receive(self) -> bytes:
return await self._receiver.receive()
async def aclose(self):
await self._receiver.aclose()
await self._sender.aclose()
@acm
async def attach_to_ringbuf_channel(
token_in: RBToken,
token_out: RBToken,
cleanup_in: bool = True,
cleanup_out: bool = True
) -> AsyncContextManager[RingBuffChannel]:
'''
Attach to an already opened ringbuf pair and return
a `RingBuffChannel`.
'''
async with (
attach_to_ringbuf_rchannel(
token_in,
cleanup=cleanup_in
) as receiver,
attach_to_ringbuf_schannel(
token_out,
cleanup=cleanup_out
) as sender,
):
yield RingBuffChannel(sender, receiver)

812
tractor/ipc/_shm.py 100644
View File

@ -0,0 +1,812 @@
# 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/>.
"""
SC friendly shared memory management geared at real-time
processing.
Support for ``numpy`` compatible array-buffers is provided but is
considered optional within the context of this runtime-library.
"""
from __future__ import annotations
from sys import byteorder
import time
from typing import Optional
from multiprocessing import shared_memory as shm
from multiprocessing.shared_memory import (
SharedMemory,
ShareableList,
)
from msgspec import (
Struct,
to_builtins
)
import tractor
from tractor.ipc._mp_bs import disable_mantracker
from tractor.log import get_logger
_USE_POSIX = getattr(shm, '_USE_POSIX', False)
if _USE_POSIX:
from _posixshmem import shm_unlink
try:
import numpy as np
from numpy.lib import recfunctions as rfn
# TODO ruff complains with,
# warning| F401: `nptyping` imported but unused; consider using
# `importlib.util.find_spec` to test for availability
import nptyping # noqa
except ImportError:
pass
log = get_logger(__name__)
disable_mantracker()
class SharedInt:
'''
Wrapper around a single entry shared memory array which
holds an ``int`` value used as an index counter.
'''
def __init__(
self,
shm: SharedMemory,
) -> None:
self._shm = shm
@property
def value(self) -> int:
return int.from_bytes(self._shm.buf, byteorder)
@value.setter
def value(self, value) -> None:
self._shm.buf[:] = value.to_bytes(self._shm.size, byteorder)
def destroy(self) -> None:
if _USE_POSIX:
# We manually unlink to bypass all the "resource tracker"
# nonsense meant for non-SC systems.
name = self._shm.name
try:
shm_unlink(name)
except FileNotFoundError:
# might be a teardown race here?
log.warning(f'Shm for {name} already unlinked?')
class NDToken(Struct, frozen=True):
'''
Internal represenation of a shared memory ``numpy`` array "token"
which can be used to key and load a system (OS) wide shm entry
and correctly read the array by type signature.
This type is msg safe.
'''
shm_name: str # this servers as a "key" value
shm_first_index_name: str
shm_last_index_name: str
dtype_descr: tuple
size: int # in struct-array index / row terms
# TODO: use nptyping here on dtypes
@property
def dtype(self) -> list[tuple[str, str, tuple[int, ...]]]:
return np.dtype(
list(
map(tuple, self.dtype_descr)
)
).descr
def as_msg(self):
return to_builtins(self)
@classmethod
def from_msg(cls, msg: dict) -> NDToken:
if isinstance(msg, NDToken):
return msg
# TODO: native struct decoding
# return _token_dec.decode(msg)
msg['dtype_descr'] = tuple(map(tuple, msg['dtype_descr']))
return NDToken(**msg)
# _token_dec = msgspec.msgpack.Decoder(NDToken)
# TODO: this api?
# _known_tokens = tractor.ActorVar('_shm_tokens', {})
# _known_tokens = tractor.ContextStack('_known_tokens', )
# _known_tokens = trio.RunVar('shms', {})
# TODO: this should maybe be provided via
# a `.trionics.maybe_open_context()` wrapper factory?
# process-local store of keys to tokens
_known_tokens: dict[str, NDToken] = {}
def get_shm_token(key: str) -> NDToken | None:
'''
Convenience func to check if a token
for the provided key is known by this process.
Returns either the ``numpy`` token or a string for a shared list.
'''
return _known_tokens.get(key)
def _make_token(
key: str,
size: int,
dtype: np.dtype,
) -> NDToken:
'''
Create a serializable token that can be used
to access a shared array.
'''
return NDToken(
shm_name=key,
shm_first_index_name=key + "_first",
shm_last_index_name=key + "_last",
dtype_descr=tuple(np.dtype(dtype).descr),
size=size,
)
class ShmArray:
'''
A shared memory ``numpy.ndarray`` API.
An underlying shared memory buffer is allocated based on
a user specified ``numpy.ndarray``. This fixed size array
can be read and written to by pushing data both onto the "front"
or "back" of a set index range. The indexes for the "first" and
"last" index are themselves stored in shared memory (accessed via
``SharedInt`` interfaces) values such that multiple processes can
interact with the same array using a synchronized-index.
'''
def __init__(
self,
shmarr: np.ndarray,
first: SharedInt,
last: SharedInt,
shm: SharedMemory,
# readonly: bool = True,
) -> None:
self._array = shmarr
# indexes for first and last indices corresponding
# to fille data
self._first = first
self._last = last
self._len = len(shmarr)
self._shm = shm
self._post_init: bool = False
# pushing data does not write the index (aka primary key)
self._write_fields: list[str] | None = None
dtype = shmarr.dtype
if dtype.fields:
self._write_fields = list(shmarr.dtype.fields.keys())[1:]
# TODO: ringbuf api?
@property
def _token(self) -> NDToken:
return NDToken(
shm_name=self._shm.name,
shm_first_index_name=self._first._shm.name,
shm_last_index_name=self._last._shm.name,
dtype_descr=tuple(self._array.dtype.descr),
size=self._len,
)
@property
def token(self) -> dict:
"""Shared memory token that can be serialized and used by
another process to attach to this array.
"""
return self._token.as_msg()
@property
def index(self) -> int:
return self._last.value % self._len
@property
def array(self) -> np.ndarray:
'''
Return an up-to-date ``np.ndarray`` view of the
so-far-written data to the underlying shm buffer.
'''
a = self._array[self._first.value:self._last.value]
# first, last = self._first.value, self._last.value
# a = self._array[first:last]
# TODO: eventually comment this once we've not seen it in the
# wild in a long time..
# XXX: race where first/last indexes cause a reader
# to load an empty array..
if len(a) == 0 and self._post_init:
raise RuntimeError('Empty array race condition hit!?')
# breakpoint()
return a
def ustruct(
self,
fields: Optional[list[str]] = None,
# type that all field values will be cast to
# in the returned view.
common_dtype: np.dtype = float,
) -> np.ndarray:
array = self._array
if fields:
selection = array[fields]
# fcount = len(fields)
else:
selection = array
# fcount = len(array.dtype.fields)
# XXX: manual ``.view()`` attempt that also doesn't work.
# uview = selection.view(
# dtype='<f16',
# ).reshape(-1, 4, order='A')
# assert len(selection) == len(uview)
u = rfn.structured_to_unstructured(
selection,
# dtype=float,
copy=True,
)
# unstruct = np.ndarray(u.shape, dtype=a.dtype, buffer=shm.buf)
# array[:] = a[:]
return u
# return ShmArray(
# shmarr=u,
# first=self._first,
# last=self._last,
# shm=self._shm
# )
def last(
self,
length: int = 1,
) -> np.ndarray:
'''
Return the last ``length``'s worth of ("row") entries from the
array.
'''
return self.array[-length:]
def push(
self,
data: np.ndarray,
field_map: Optional[dict[str, str]] = None,
prepend: bool = False,
update_first: bool = True,
start: int | None = None,
) -> int:
'''
Ring buffer like "push" to append data
into the buffer and return updated "last" index.
NB: no actual ring logic yet to give a "loop around" on overflow
condition, lel.
'''
length = len(data)
if prepend:
index = (start or self._first.value) - length
if index < 0:
raise ValueError(
f'Array size of {self._len} was overrun during prepend.\n'
f'You have passed {abs(index)} too many datums.'
)
else:
index = start if start is not None else self._last.value
end = index + length
if field_map:
src_names, dst_names = zip(*field_map.items())
else:
dst_names = src_names = self._write_fields
try:
self._array[
list(dst_names)
][index:end] = data[list(src_names)][:]
# NOTE: there was a race here between updating
# the first and last indices and when the next reader
# tries to access ``.array`` (which due to the index
# overlap will be empty). Pretty sure we've fixed it now
# but leaving this here as a reminder.
if (
prepend
and update_first
and length
):
assert index < self._first.value
if (
index < self._first.value
and update_first
):
assert prepend, 'prepend=True not passed but index decreased?'
self._first.value = index
elif not prepend:
self._last.value = end
self._post_init = True
return end
except ValueError as err:
if field_map:
raise
# should raise if diff detected
self.diff_err_fields(data)
raise err
def diff_err_fields(
self,
data: np.ndarray,
) -> None:
# reraise with any field discrepancy
our_fields, their_fields = (
set(self._array.dtype.fields),
set(data.dtype.fields),
)
only_in_ours = our_fields - their_fields
only_in_theirs = their_fields - our_fields
if only_in_ours:
raise TypeError(
f"Input array is missing field(s): {only_in_ours}"
)
elif only_in_theirs:
raise TypeError(
f"Input array has unknown field(s): {only_in_theirs}"
)
# TODO: support "silent" prepends that don't update ._first.value?
def prepend(
self,
data: np.ndarray,
) -> int:
end = self.push(data, prepend=True)
assert end
def close(self) -> None:
self._first._shm.close()
self._last._shm.close()
self._shm.close()
def destroy(self) -> None:
if _USE_POSIX:
# We manually unlink to bypass all the "resource tracker"
# nonsense meant for non-SC systems.
shm_unlink(self._shm.name)
self._first.destroy()
self._last.destroy()
def flush(self) -> None:
# TODO: flush to storage backend like markestore?
...
def open_shm_ndarray(
size: int,
key: str | None = None,
dtype: np.dtype | None = None,
append_start_index: int | None = None,
readonly: bool = False,
) -> ShmArray:
'''
Open a memory shared ``numpy`` using the standard library.
This call unlinks (aka permanently destroys) the buffer on teardown
and thus should be used from the parent-most accessor (process).
'''
# create new shared mem segment for which we
# have write permission
a = np.zeros(size, dtype=dtype)
a['index'] = np.arange(len(a))
shm = SharedMemory(
name=key,
create=True,
size=a.nbytes
)
array = np.ndarray(
a.shape,
dtype=a.dtype,
buffer=shm.buf
)
array[:] = a[:]
array.setflags(write=int(not readonly))
token = _make_token(
key=key,
size=size,
dtype=dtype,
)
# create single entry arrays for storing an first and last indices
first = SharedInt(
shm=SharedMemory(
name=token.shm_first_index_name,
create=True,
size=4, # std int
)
)
last = SharedInt(
shm=SharedMemory(
name=token.shm_last_index_name,
create=True,
size=4, # std int
)
)
# Start the "real-time" append-updated (or "pushed-to") section
# after some start index: ``append_start_index``. This allows appending
# from a start point in the array which isn't the 0 index and looks
# something like,
# -------------------------
# | | i
# _________________________
# <-------------> <------->
# history real-time
#
# Once fully "prepended", the history section will leave the
# ``ShmArray._start.value: int = 0`` and the yet-to-be written
# real-time section will start at ``ShmArray.index: int``.
# this sets the index to nearly 2/3rds into the the length of
# the buffer leaving at least a "days worth of second samples"
# for the real-time section.
if append_start_index is None:
append_start_index = round(size * 0.616)
last.value = first.value = append_start_index
shmarr = ShmArray(
array,
first,
last,
shm,
)
assert shmarr._token == token
_known_tokens[key] = shmarr.token
# "unlink" created shm on process teardown by
# pushing teardown calls onto actor context stack
stack = tractor.current_actor().lifetime_stack
stack.callback(shmarr.close)
stack.callback(shmarr.destroy)
return shmarr
def attach_shm_ndarray(
token: tuple[str, str, tuple[str, str]],
readonly: bool = True,
) -> ShmArray:
'''
Attach to an existing shared memory array previously
created by another process using ``open_shared_array``.
No new shared mem is allocated but wrapper types for read/write
access are constructed.
'''
token = NDToken.from_msg(token)
key = token.shm_name
if key in _known_tokens:
assert NDToken.from_msg(_known_tokens[key]) == token, "WTF"
# XXX: ugh, looks like due to the ``shm_open()`` C api we can't
# actually place files in a subdir, see discussion here:
# https://stackoverflow.com/a/11103289
# attach to array buffer and view as per dtype
_err: Optional[Exception] = None
for _ in range(3):
try:
shm = SharedMemory(
name=key,
create=False,
)
break
except OSError as oserr:
_err = oserr
time.sleep(0.1)
else:
if _err:
raise _err
shmarr = np.ndarray(
(token.size,),
dtype=token.dtype,
buffer=shm.buf
)
shmarr.setflags(write=int(not readonly))
first = SharedInt(
shm=SharedMemory(
name=token.shm_first_index_name,
create=False,
size=4, # std int
),
)
last = SharedInt(
shm=SharedMemory(
name=token.shm_last_index_name,
create=False,
size=4, # std int
),
)
# make sure we can read
first.value
sha = ShmArray(
shmarr,
first,
last,
shm,
)
# read test
sha.array
# Stash key -> token knowledge for future queries
# via `maybe_opepn_shm_array()` but only after we know
# we can attach.
if key not in _known_tokens:
_known_tokens[key] = token
# "close" attached shm on actor teardown
tractor.current_actor().lifetime_stack.callback(sha.close)
return sha
def maybe_open_shm_ndarray(
key: str, # unique identifier for segment
size: int,
dtype: np.dtype | None = None,
append_start_index: int = 0,
readonly: bool = True,
) -> tuple[ShmArray, bool]:
'''
Attempt to attach to a shared memory block using a "key" lookup
to registered blocks in the users overall "system" registry
(presumes you don't have the block's explicit token).
This function is meant to solve the problem of discovering whether
a shared array token has been allocated or discovered by the actor
running in **this** process. Systems where multiple actors may seek
to access a common block can use this function to attempt to acquire
a token as discovered by the actors who have previously stored
a "key" -> ``NDToken`` map in an actor local (aka python global)
variable.
If you know the explicit ``NDToken`` for your memory segment instead
use ``attach_shm_array``.
'''
try:
# see if we already know this key
token = _known_tokens[key]
return (
attach_shm_ndarray(
token=token,
readonly=readonly,
),
False, # not newly opened
)
except KeyError:
log.warning(f"Could not find {key} in shms cache")
if dtype:
token = _make_token(
key,
size=size,
dtype=dtype,
)
else:
try:
return (
attach_shm_ndarray(
token=token,
readonly=readonly,
),
False,
)
except FileNotFoundError:
log.warning(f"Could not attach to shm with token {token}")
# This actor does not know about memory
# associated with the provided "key".
# Attempt to open a block and expect
# to fail if a block has been allocated
# on the OS by someone else.
return (
open_shm_ndarray(
key=key,
size=size,
dtype=dtype,
append_start_index=append_start_index,
readonly=readonly,
),
True,
)
class ShmList(ShareableList):
'''
Carbon copy of ``.shared_memory.ShareableList`` with a few
enhancements:
- readonly mode via instance var flag `._readonly: bool`
- ``.__getitem__()`` accepts ``slice`` inputs
- exposes the underlying buffer "name" as a ``.key: str``
'''
def __init__(
self,
sequence: list | None = None,
*,
name: str | None = None,
readonly: bool = True
) -> None:
self._readonly = readonly
self._key = name
return super().__init__(
sequence=sequence,
name=name,
)
@property
def key(self) -> str:
return self._key
@property
def readonly(self) -> bool:
return self._readonly
def __setitem__(
self,
position,
value,
) -> None:
# mimick ``numpy`` error
if self._readonly:
raise ValueError('assignment destination is read-only')
return super().__setitem__(position, value)
def __getitem__(
self,
indexish,
) -> list:
# NOTE: this is a non-writeable view (copy?) of the buffer
# in a new list instance.
if isinstance(indexish, slice):
return list(self)[indexish]
return super().__getitem__(indexish)
# TODO: should we offer a `.array` and `.push()` equivalent
# to the `ShmArray`?
# currently we have the following limitations:
# - can't write slices of input using traditional slice-assign
# syntax due to the ``ShareableList.__setitem__()`` implementation.
# - ``list(shmlist)`` returns a non-mutable copy instead of
# a writeable view which would be handier numpy-style ops.
def open_shm_list(
key: str,
sequence: list | None = None,
size: int = int(2 ** 10),
dtype: float | int | bool | str | bytes | None = float,
readonly: bool = True,
) -> ShmList:
if sequence is None:
default = {
float: 0.,
int: 0,
bool: True,
str: 'doggy',
None: None,
}[dtype]
sequence = [default] * size
shml = ShmList(
sequence=sequence,
name=key,
readonly=readonly,
)
# "close" attached shm on actor teardown
try:
actor = tractor.current_actor()
actor.lifetime_stack.callback(shml.shm.close)
actor.lifetime_stack.callback(shml.shm.unlink)
except RuntimeError:
log.warning('tractor runtime not active, skipping teardown steps')
return shml
def attach_shm_list(
key: str,
readonly: bool = False,
) -> ShmList:
return ShmList(
name=key,
readonly=readonly,
)

View File

@ -0,0 +1,99 @@
# 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/>.
'''
TCP implementation of tractor.ipc._transport.MsgTransport protocol
'''
from __future__ import annotations
import trio
from tractor.msg import MsgCodec
from tractor.log import get_logger
from tractor._addr import TCPAddress
from tractor.ipc._transport import MsgpackTransport
log = get_logger(__name__)
# TODO: typing oddity.. not sure why we have to inherit here, but it
# seems to be an issue with `get_msg_transport()` returning
# a `Type[Protocol]`; probably should make a `mypy` issue?
class MsgpackTCPStream(MsgpackTransport):
'''
A ``trio.SocketStream`` delivering ``msgpack`` formatted data
using the ``msgspec`` codec lib.
'''
address_type = TCPAddress
layer_key: int = 4
@property
def maddr(self) -> str:
host, port = self.raddr.unwrap()
return (
# TODO, use `ipaddress` from stdlib to handle
# first detecting which of `ipv4/6` before
# choosing the routing prefix part.
f'/ipv4/{host}'
f'/{self.address_type.proto_key}/{port}'
# f'/{self.chan.uid[0]}'
# f'/{self.cid}'
# f'/cid={cid_head}..{cid_tail}'
# TODO: ? not use this ^ right ?
)
def connected(self) -> bool:
return self.stream.socket.fileno() != -1
@classmethod
async def connect_to(
cls,
destaddr: TCPAddress,
prefix_size: int = 4,
codec: MsgCodec|None = None,
**kwargs
) -> MsgpackTCPStream:
stream = await trio.open_tcp_stream(
*destaddr.unwrap(),
**kwargs
)
return MsgpackTCPStream(
stream,
prefix_size=prefix_size,
codec=codec
)
@classmethod
def get_stream_addrs(
cls,
stream: trio.SocketStream
) -> tuple[
TCPAddress,
TCPAddress,
]:
# TODO, what types are these?
lsockname = stream.socket.getsockname()
l_sockaddr: tuple[str, int] = tuple(lsockname[:2])
rsockname = stream.socket.getpeername()
r_sockaddr: tuple[str, int] = tuple(rsockname[:2])
return (
TCPAddress.from_addr(l_sockaddr),
TCPAddress.from_addr(r_sockaddr),
)

View File

@ -13,41 +13,32 @@
# 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/>.
'''
typing.Protocol based generic msg API, implement this class to add backends for
tractor.ipc.Channel
"""
Inter-process comms abstractions
"""
'''
from __future__ import annotations
from typing import (
runtime_checkable,
Type,
Protocol,
TypeVar,
ClassVar
)
from collections.abc import (
AsyncGenerator,
AsyncIterator,
)
from contextlib import (
asynccontextmanager as acm,
contextmanager as cm,
)
import platform
from pprint import pformat
import struct
import typing
from typing import (
Any,
Callable,
runtime_checkable,
Protocol,
Type,
TypeVar,
)
import trio
import msgspec
from tricycle import BufferedReceiveStream
import trio
from tractor.log import get_logger
from tractor._exceptions import (
MsgTypeError,
pack_from_raise,
TransportClosed,
_mk_send_mte,
_mk_recv_mte,
@ -59,30 +50,13 @@ from tractor.msg import (
types as msgtypes,
pretty_struct,
)
from tractor._addr import Address
log = get_logger(__name__)
_is_windows = platform.system() == 'Windows'
def get_stream_addrs(
stream: trio.SocketStream
) -> tuple[
tuple[str, int], # local
tuple[str, int], # remote
]:
'''
Return the `trio` streaming transport prot's socket-addrs for
both the local and remote sides as a pair.
'''
# rn, should both be IP sockets
lsockname = stream.socket.getsockname()
rsockname = stream.socket.getpeername()
return (
tuple(lsockname[:2]),
tuple(rsockname[:2]),
)
# (codec, transport)
MsgTransportKey = tuple[str, str]
# from tractor.msg.types import MsgType
@ -92,9 +66,6 @@ def get_stream_addrs(
MsgType = TypeVar('MsgType')
# TODO: break up this mod into a subpkg so we can start adding new
# backends and move this type stuff into a dedicated file.. Bo
#
@runtime_checkable
class MsgTransport(Protocol[MsgType]):
#
@ -102,11 +73,11 @@ 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:
...
address_type: ClassVar[Type[Address]]
codec_key: ClassVar[str]
# XXX: should this instead be called `.sendall()`?
async def send(self, msg: MsgType) -> None:
@ -126,26 +97,57 @@ class MsgTransport(Protocol[MsgType]):
def drain(self) -> AsyncIterator[dict]:
...
@classmethod
def key(cls) -> MsgTransportKey:
return cls.codec_key, cls.address_type.proto_key
@property
def laddr(self) -> tuple[str, int]:
def laddr(self) -> Address:
...
@property
def raddr(self) -> tuple[str, int]:
def raddr(self) -> Address:
...
@property
def maddr(self) -> str:
...
# TODO: typing oddity.. not sure why we have to inherit here, but it
# seems to be an issue with `get_msg_transport()` returning
# a `Type[Protocol]`; probably should make a `mypy` issue?
class MsgpackTCPStream(MsgTransport):
'''
A ``trio.SocketStream`` delivering ``msgpack`` formatted data
using the ``msgspec`` codec lib.
@classmethod
async def connect_to(
cls,
addr: Address,
**kwargs
) -> MsgTransport:
...
'''
layer_key: int = 4
name_key: str = 'tcp'
@classmethod
def get_stream_addrs(
cls,
stream: trio.abc.Stream
) -> tuple[
Address, # local
Address # remote
]:
'''
Return the `trio` streaming transport prot's addrs for both
the local and remote sides as a pair.
'''
...
# TODO, such that all `.raddr`s for each `SocketStream` are
# delivered?
# -[ ] move `.open_listener()` here and internally track the
# listener set, per address?
# def get_peers(
# self,
# ) -> list[Address]:
# ...
class MsgpackTransport(MsgTransport):
# TODO: better naming for this?
# -[ ] check how libp2p does naming for such things?
@ -153,7 +155,7 @@ class MsgpackTCPStream(MsgTransport):
def __init__(
self,
stream: trio.SocketStream,
stream: trio.abc.Stream,
prefix_size: int = 4,
# XXX optionally provided codec pair for `msgspec`:
@ -161,18 +163,14 @@ class MsgpackTCPStream(MsgTransport):
#
# TODO: define this as a `Codec` struct which can be
# overriden dynamically by the application/runtime?
codec: tuple[
Callable[[Any], Any]|None, # coder
Callable[[type, Any], Any]|None, # decoder
]|None = None,
codec: MsgCodec = None,
) -> None:
self.stream = stream
assert self.stream.socket
# should both be IP sockets
self._laddr, self._raddr = get_stream_addrs(stream)
(
self._laddr,
self._raddr,
) = self.get_stream_addrs(stream)
# create read loop instance
self._aiter_pkts = self._iter_packets()
@ -255,8 +253,8 @@ class MsgpackTCPStream(MsgTransport):
raise TransportClosed(
message=(
f'IPC transport already closed by peer\n'
f'x]> {type(trans_err)}\n'
f' |_{self}\n'
f'x)> {type(trans_err)}\n'
f' |_{self}\n'
),
loglevel=loglevel,
) from trans_err
@ -273,8 +271,8 @@ class MsgpackTCPStream(MsgTransport):
raise TransportClosed(
message=(
f'IPC transport already manually closed locally?\n'
f'x]> {type(closure_err)} \n'
f' |_{self}\n'
f'x)> {type(closure_err)} \n'
f' |_{self}\n'
),
loglevel='error',
raise_on_report=(
@ -289,8 +287,8 @@ class MsgpackTCPStream(MsgTransport):
raise TransportClosed(
message=(
f'IPC transport already gracefully closed\n'
f']>\n'
f' |_{self}\n'
f')>\n'
f'|_{self}\n'
),
loglevel='transport',
# cause=??? # handy or no?
@ -424,7 +422,39 @@ class MsgpackTCPStream(MsgTransport):
# supposedly the fastest says,
# https://stackoverflow.com/a/54027962
size: bytes = struct.pack("<I", len(bytes_data))
return await self.stream.send_all(size + bytes_data)
try:
return await self.stream.send_all(size + bytes_data)
except (
trio.BrokenResourceError,
) as trans_err:
loglevel = 'transport'
match trans_err:
case trio.BrokenResourceError() if (
'[Errno 32] Broken pipe' in trans_err.args[0]
# ^XXX, specifc to UDS transport and its,
# well, "speediness".. XD
# |_ likely todo with races related to how fast
# the socket is setup/torn-down on linux
# as it pertains to rando pings from the
# `.discovery` subsys and protos.
):
raise TransportClosed(
message=(
f'IPC transport already closed by peer\n'
f'x)> {type(trans_err)}\n'
f' |_{self}\n'
),
loglevel=loglevel,
) from trans_err
# unless the disconnect condition falls under "a
# normal operation breakage" we usualy console warn
# about it.
case _:
log.exception(
'Transport layer failed for {self.transport!r} ?\n'
)
raise trans_err
# ?TODO? does it help ever to dynamically show this
# frame?
@ -436,15 +466,7 @@ class MsgpackTCPStream(MsgTransport):
# __tracebackhide__: bool = False
# raise
@property
def laddr(self) -> tuple[str, int]:
return self._laddr
@property
def raddr(self) -> tuple[str, int]:
return self._raddr
async def recv(self) -> Any:
async def recv(self) -> msgtypes.MsgType:
return await self._aiter_pkts.asend(None)
async def drain(self) -> AsyncIterator[dict]:
@ -464,357 +486,23 @@ class MsgpackTCPStream(MsgTransport):
def __aiter__(self):
return self._aiter_pkts
def connected(self) -> bool:
return self.stream.socket.fileno() != -1
def get_msg_transport(
key: tuple[str, str],
) -> Type[MsgTransport]:
return {
('msgpack', 'tcp'): MsgpackTCPStream,
}[key]
class Channel:
'''
An inter-process channel for communication between (remote) actors.
Wraps a ``MsgStream``: transport + encoding IPC connection.
Currently we only support ``trio.SocketStream`` for transport
(aka TCP) and the ``msgpack`` interchange format via the ``msgspec``
codec libary.
'''
def __init__(
self,
destaddr: tuple[str, int]|None,
msg_transport_type_key: tuple[str, str] = ('msgpack', 'tcp'),
# TODO: optional reconnection support?
# auto_reconnect: bool = False,
# on_reconnect: typing.Callable[..., typing.Awaitable] = None,
) -> None:
# self._recon_seq = on_reconnect
# self._autorecon = auto_reconnect
self._destaddr = destaddr
self._transport_key = msg_transport_type_key
# Either created in ``.connect()`` or passed in by
# user in ``.from_stream()``.
self._stream: trio.SocketStream|None = None
self._transport: MsgTransport|None = None
# set after handshake - always uid of far end
self.uid: tuple[str, str]|None = None
self._aiter_msgs = self._iter_msgs()
self._exc: Exception|None = None # set if far end actor errors
self._closed: bool = False
# flag set by ``Portal.cancel_actor()`` indicating remote
# (possibly peer) cancellation of the far end actor
# runtime.
self._cancel_called: bool = False
@property
def laddr(self) -> Address:
return self._laddr
@property
def msgstream(self) -> MsgTransport:
log.info(
'`Channel.msgstream` is an old name, use `._transport`'
)
return self._transport
def raddr(self) -> Address:
return self._raddr
@property
def transport(self) -> MsgTransport:
return self._transport
@classmethod
def from_stream(
cls,
stream: trio.SocketStream,
**kwargs,
) -> Channel:
src, dst = get_stream_addrs(stream)
chan = Channel(
destaddr=dst,
**kwargs,
def pformat(self) -> str:
return (
f'<{type(self).__name__}(\n'
f' |_task: {self._task}\n'
f'\n'
f' |_peers: 2\n'
f' laddr: {self._laddr}\n'
f' raddr: {self._raddr}\n'
f')>\n'
)
# set immediately here from provided instance
chan._stream: trio.SocketStream = stream
chan.set_msg_transport(stream)
return chan
def set_msg_transport(
self,
stream: trio.SocketStream,
type_key: tuple[str, str]|None = None,
# XXX optionally provided codec pair for `msgspec`:
# https://jcristharif.com/msgspec/extending.html#mapping-to-from-native-types
codec: MsgCodec|None = None,
) -> MsgTransport:
type_key = (
type_key
or
self._transport_key
)
# get transport type, then
self._transport = get_msg_transport(
type_key
# instantiate an instance of the msg-transport
)(
stream,
codec=codec,
)
return self._transport
@cm
def apply_codec(
self,
codec: MsgCodec,
) -> None:
'''
Temporarily override the underlying IPC msg codec for
dynamic enforcement of messaging schema.
'''
orig: MsgCodec = self._transport.codec
try:
self._transport.codec = codec
yield
finally:
self._transport.codec = orig
# TODO: do a .src/.dst: str for maddrs?
def __repr__(self) -> str:
if not self._transport:
return '<Channel with inactive transport?>'
return repr(
self._transport.stream.socket._sock
).replace( # type: ignore
"socket.socket",
"Channel",
)
@property
def laddr(self) -> tuple[str, int]|None:
return self._transport.laddr if self._transport else None
@property
def raddr(self) -> tuple[str, int]|None:
return self._transport.raddr if self._transport else None
async def connect(
self,
destaddr: tuple[Any, ...] | None = None,
**kwargs
) -> MsgTransport:
if self.connected():
raise RuntimeError("channel is already connected?")
destaddr = destaddr or self._destaddr
assert isinstance(destaddr, tuple)
stream = await trio.open_tcp_stream(
*destaddr,
**kwargs
)
transport = self.set_msg_transport(stream)
log.transport(
f'Opened channel[{type(transport)}]: {self.laddr} -> {self.raddr}'
)
return transport
# TODO: something like,
# `pdbp.hideframe_on(errors=[MsgTypeError])`
# instead of the `try/except` hack we have rn..
# seems like a pretty useful thing to have in general
# along with being able to filter certain stack frame(s / sets)
# possibly based on the current log-level?
async def send(
self,
payload: Any,
hide_tb: bool = False,
) -> None:
'''
Send a coded msg-blob over the transport.
'''
__tracebackhide__: bool = hide_tb
try:
log.transport(
'=> send IPC msg:\n\n'
f'{pformat(payload)}\n'
)
# assert self._transport # but why typing?
await self._transport.send(
payload,
hide_tb=hide_tb,
)
except BaseException as _err:
err = _err # bind for introspection
if not isinstance(_err, MsgTypeError):
# assert err
__tracebackhide__: bool = False
else:
assert err.cid
raise
async def recv(self) -> Any:
assert self._transport
return await self._transport.recv()
# TODO: auto-reconnect features like 0mq/nanomsg?
# -[ ] implement it manually with nods to SC prot
# possibly on multiple transport backends?
# -> seems like that might be re-inventing scalability
# prots tho no?
# try:
# return await self._transport.recv()
# except trio.BrokenResourceError:
# if self._autorecon:
# await self._reconnect()
# return await self.recv()
# raise
async def aclose(self) -> None:
log.transport(
f'Closing channel to {self.uid} '
f'{self.laddr} -> {self.raddr}'
)
assert self._transport
await self._transport.stream.aclose()
self._closed = True
async def __aenter__(self):
await self.connect()
return self
async def __aexit__(self, *args):
await self.aclose(*args)
def __aiter__(self):
return self._aiter_msgs
# ?TODO? run any reconnection sequence?
# -[ ] prolly should be impl-ed as deco-API?
#
# async def _reconnect(self) -> None:
# """Handle connection failures by polling until a reconnect can be
# established.
# """
# down = False
# while True:
# try:
# with trio.move_on_after(3) as cancel_scope:
# await self.connect()
# cancelled = cancel_scope.cancelled_caught
# if cancelled:
# log.transport(
# "Reconnect timed out after 3 seconds, retrying...")
# continue
# else:
# log.transport("Stream connection re-established!")
# # on_recon = self._recon_seq
# # if on_recon:
# # await on_recon(self)
# break
# except (OSError, ConnectionRefusedError):
# if not down:
# down = True
# log.transport(
# f"Connection to {self.raddr} went down, waiting"
# " for re-establishment")
# await trio.sleep(1)
async def _iter_msgs(
self
) -> AsyncGenerator[Any, None]:
'''
Yield `MsgType` IPC msgs decoded and deliverd from
an underlying `MsgTransport` protocol.
This is a streaming routine alo implemented as an async-gen
func (same a `MsgTransport._iter_pkts()`) gets allocated by
a `.__call__()` inside `.__init__()` where it is assigned to
the `._aiter_msgs` attr.
'''
assert self._transport
while True:
try:
async for msg in self._transport:
match msg:
# NOTE: if transport/interchange delivers
# a type error, we pack it with the far
# end peer `Actor.uid` and relay the
# `Error`-msg upward to the `._rpc` stack
# for normal RAE handling.
case MsgTypeError():
yield pack_from_raise(
local_err=msg,
cid=msg.cid,
# XXX we pack it here bc lower
# layers have no notion of an
# actor-id ;)
src_uid=self.uid,
)
case _:
yield msg
except trio.BrokenResourceError:
# if not self._autorecon:
raise
await self.aclose()
# if self._autorecon: # attempt reconnect
# await self._reconnect()
# continue
def connected(self) -> bool:
return self._transport.connected() if self._transport else False
@acm
async def _connect_chan(
host: str,
port: int
) -> typing.AsyncGenerator[Channel, None]:
'''
Create and connect a channel with disconnect on context manager
teardown.
'''
chan = Channel((host, port))
await chan.connect()
yield chan
with trio.CancelScope(shield=True):
await chan.aclose()
__repr__ = __str__ = pformat

View File

@ -0,0 +1,99 @@
# 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/>.
from typing import Type
import trio
import socket
from tractor._addr import Address
from tractor.ipc._transport import (
MsgTransportKey,
MsgTransport
)
from tractor.ipc._tcp import MsgpackTCPStream
from tractor.ipc._uds import MsgpackUDSStream
# manually updated list of all supported msg transport types
_msg_transports = [
MsgpackTCPStream,
MsgpackUDSStream
]
# convert a MsgTransportKey to the corresponding transport type
_key_to_transport: dict[MsgTransportKey, Type[MsgTransport]] = {
cls.key(): cls
for cls in _msg_transports
}
# convert an Address wrapper to its corresponding transport type
_addr_to_transport: dict[Type[Address], Type[MsgTransport]] = {
cls.address_type: cls
for cls in _msg_transports
}
def transport_from_addr(
addr: Address,
codec_key: str = 'msgpack',
) -> Type[MsgTransport]:
'''
Given a destination address and a desired codec, find the
corresponding `MsgTransport` type.
'''
try:
return _addr_to_transport[type(addr)]
except KeyError:
raise NotImplementedError(
f'No known transport for address {repr(addr)}'
)
def transport_from_stream(
stream: trio.abc.Stream,
codec_key: str = 'msgpack'
) -> Type[MsgTransport]:
'''
Given an arbitrary `trio.abc.Stream` and a desired codec,
find the corresponding `MsgTransport` type.
'''
transport = None
if isinstance(stream, trio.SocketStream):
sock: socket.socket = stream.socket
match sock.family:
case socket.AF_INET | socket.AF_INET6:
transport = 'tcp'
case socket.AF_UNIX:
transport = 'uds'
case _:
raise NotImplementedError(
f'Unsupported socket family: {sock.family}'
)
if not transport:
raise NotImplementedError(
f'Could not figure out transport type for stream type {type(stream)}'
)
key = (codec_key, transport)
return _key_to_transport[key]

201
tractor/ipc/_uds.py 100644
View File

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

View File

@ -92,7 +92,7 @@ class StackLevelAdapter(LoggerAdapter):
) -> None:
'''
IPC transport level msg IO; generally anything below
`._ipc.Channel` and friends.
`.ipc.Channel` and friends.
'''
return self.log(5, msg)
@ -285,7 +285,7 @@ def get_logger(
# NOTE: for handling for modules that use ``get_logger(__name__)``
# we make the following stylistic choice:
# - always avoid duplicate project-package token
# in msg output: i.e. tractor.tractor _ipc.py in header
# in msg output: i.e. tractor.tractor.ipc._chan.py in header
# looks ridiculous XD
# - never show the leaf module name in the {name} part
# since in python the {filename} is always this same

View File

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

58
uv.lock
View File

@ -11,6 +11,15 @@ wheels = [
{ url = "https://files.pythonhosted.org/packages/89/aa/ab0f7891a01eeb2d2e338ae8fecbe57fcebea1a24dbb64d45801bfab481d/attrs-24.3.0-py3-none-any.whl", hash = "sha256:ac96cd038792094f438ad1f6ff80837353805ac950cd2aa0e0625ef19850c308", size = 63397 },
]
[[package]]
name = "bidict"
version = "0.23.1"
source = { registry = "https://pypi.org/simple" }
sdist = { url = "https://files.pythonhosted.org/packages/9a/6e/026678aa5a830e07cd9498a05d3e7e650a4f56a42f267a53d22bcda1bdc9/bidict-0.23.1.tar.gz", hash = "sha256:03069d763bc387bbd20e7d49914e75fc4132a41937fa3405417e1a5a2d006d71", size = 29093 }
wheels = [
{ url = "https://files.pythonhosted.org/packages/99/37/e8730c3587a65eb5645d4aba2d27aae48e8003614d6aaf15dda67f702f1f/bidict-0.23.1-py3-none-any.whl", hash = "sha256:5dae8d4d79b552a71cbabc7deb25dfe8ce710b17ff41711e13010ead2abfc3e5", size = 32764 },
]
[[package]]
name = "cffi"
version = "1.17.1"
@ -20,10 +29,38 @@ dependencies = [
]
sdist = { url = "https://files.pythonhosted.org/packages/fc/97/c783634659c2920c3fc70419e3af40972dbaf758daa229a7d6ea6135c90d/cffi-1.17.1.tar.gz", hash = "sha256:1c39c6016c32bc48dd54561950ebd6836e1670f2ae46128f67cf49e789c52824", size = 516621 }
wheels = [
{ url = "https://files.pythonhosted.org/packages/6b/f4/927e3a8899e52a27fa57a48607ff7dc91a9ebe97399b357b85a0c7892e00/cffi-1.17.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:a45e3c6913c5b87b3ff120dcdc03f6131fa0065027d0ed7ee6190736a74cd401", size = 182264 },
{ url = "https://files.pythonhosted.org/packages/6c/f5/6c3a8efe5f503175aaddcbea6ad0d2c96dad6f5abb205750d1b3df44ef29/cffi-1.17.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:30c5e0cb5ae493c04c8b42916e52ca38079f1b235c2f8ae5f4527b963c401caf", size = 178651 },
{ url = "https://files.pythonhosted.org/packages/94/dd/a3f0118e688d1b1a57553da23b16bdade96d2f9bcda4d32e7d2838047ff7/cffi-1.17.1-cp311-cp311-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f75c7ab1f9e4aca5414ed4d8e5c0e303a34f4421f8a0d47a4d019ceff0ab6af4", size = 445259 },
{ url = "https://files.pythonhosted.org/packages/2e/ea/70ce63780f096e16ce8588efe039d3c4f91deb1dc01e9c73a287939c79a6/cffi-1.17.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a1ed2dd2972641495a3ec98445e09766f077aee98a1c896dcb4ad0d303628e41", size = 469200 },
{ url = "https://files.pythonhosted.org/packages/1c/a0/a4fa9f4f781bda074c3ddd57a572b060fa0df7655d2a4247bbe277200146/cffi-1.17.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:46bf43160c1a35f7ec506d254e5c890f3c03648a4dbac12d624e4490a7046cd1", size = 477235 },
{ url = "https://files.pythonhosted.org/packages/62/12/ce8710b5b8affbcdd5c6e367217c242524ad17a02fe5beec3ee339f69f85/cffi-1.17.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a24ed04c8ffd54b0729c07cee15a81d964e6fee0e3d4d342a27b020d22959dc6", size = 459721 },
{ url = "https://files.pythonhosted.org/packages/ff/6b/d45873c5e0242196f042d555526f92aa9e0c32355a1be1ff8c27f077fd37/cffi-1.17.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:610faea79c43e44c71e1ec53a554553fa22321b65fae24889706c0a84d4ad86d", size = 467242 },
{ url = "https://files.pythonhosted.org/packages/1a/52/d9a0e523a572fbccf2955f5abe883cfa8bcc570d7faeee06336fbd50c9fc/cffi-1.17.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:a9b15d491f3ad5d692e11f6b71f7857e7835eb677955c00cc0aefcd0669adaf6", size = 477999 },
{ url = "https://files.pythonhosted.org/packages/44/74/f2a2460684a1a2d00ca799ad880d54652841a780c4c97b87754f660c7603/cffi-1.17.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:de2ea4b5833625383e464549fec1bc395c1bdeeb5f25c4a3a82b5a8c756ec22f", size = 454242 },
{ url = "https://files.pythonhosted.org/packages/f8/4a/34599cac7dfcd888ff54e801afe06a19c17787dfd94495ab0c8d35fe99fb/cffi-1.17.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:fc48c783f9c87e60831201f2cce7f3b2e4846bf4d8728eabe54d60700b318a0b", size = 478604 },
{ url = "https://files.pythonhosted.org/packages/34/33/e1b8a1ba29025adbdcda5fb3a36f94c03d771c1b7b12f726ff7fef2ebe36/cffi-1.17.1-cp311-cp311-win32.whl", hash = "sha256:85a950a4ac9c359340d5963966e3e0a94a676bd6245a4b55bc43949eee26a655", size = 171727 },
{ url = "https://files.pythonhosted.org/packages/3d/97/50228be003bb2802627d28ec0627837ac0bf35c90cf769812056f235b2d1/cffi-1.17.1-cp311-cp311-win_amd64.whl", hash = "sha256:caaf0640ef5f5517f49bc275eca1406b0ffa6aa184892812030f04c2abf589a0", size = 181400 },
{ url = "https://files.pythonhosted.org/packages/5a/84/e94227139ee5fb4d600a7a4927f322e1d4aea6fdc50bd3fca8493caba23f/cffi-1.17.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:805b4371bf7197c329fcb3ead37e710d1bca9da5d583f5073b799d5c5bd1eee4", size = 183178 },
{ url = "https://files.pythonhosted.org/packages/da/ee/fb72c2b48656111c4ef27f0f91da355e130a923473bf5ee75c5643d00cca/cffi-1.17.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:733e99bc2df47476e3848417c5a4540522f234dfd4ef3ab7fafdf555b082ec0c", size = 178840 },
{ url = "https://files.pythonhosted.org/packages/cc/b6/db007700f67d151abadf508cbfd6a1884f57eab90b1bb985c4c8c02b0f28/cffi-1.17.1-cp312-cp312-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1257bdabf294dceb59f5e70c64a3e2f462c30c7ad68092d01bbbfb1c16b1ba36", size = 454803 },
{ url = "https://files.pythonhosted.org/packages/1a/df/f8d151540d8c200eb1c6fba8cd0dfd40904f1b0682ea705c36e6c2e97ab3/cffi-1.17.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:da95af8214998d77a98cc14e3a3bd00aa191526343078b530ceb0bd710fb48a5", size = 478850 },
{ url = "https://files.pythonhosted.org/packages/28/c0/b31116332a547fd2677ae5b78a2ef662dfc8023d67f41b2a83f7c2aa78b1/cffi-1.17.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d63afe322132c194cf832bfec0dc69a99fb9bb6bbd550f161a49e9e855cc78ff", size = 485729 },
{ url = "https://files.pythonhosted.org/packages/91/2b/9a1ddfa5c7f13cab007a2c9cc295b70fbbda7cb10a286aa6810338e60ea1/cffi-1.17.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f79fc4fc25f1c8698ff97788206bb3c2598949bfe0fef03d299eb1b5356ada99", size = 471256 },
{ url = "https://files.pythonhosted.org/packages/b2/d5/da47df7004cb17e4955df6a43d14b3b4ae77737dff8bf7f8f333196717bf/cffi-1.17.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b62ce867176a75d03a665bad002af8e6d54644fad99a3c70905c543130e39d93", size = 479424 },
{ url = "https://files.pythonhosted.org/packages/0b/ac/2a28bcf513e93a219c8a4e8e125534f4f6db03e3179ba1c45e949b76212c/cffi-1.17.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:386c8bf53c502fff58903061338ce4f4950cbdcb23e2902d86c0f722b786bbe3", size = 484568 },
{ url = "https://files.pythonhosted.org/packages/d4/38/ca8a4f639065f14ae0f1d9751e70447a261f1a30fa7547a828ae08142465/cffi-1.17.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:4ceb10419a9adf4460ea14cfd6bc43d08701f0835e979bf821052f1805850fe8", size = 488736 },
{ url = "https://files.pythonhosted.org/packages/86/c5/28b2d6f799ec0bdecf44dced2ec5ed43e0eb63097b0f58c293583b406582/cffi-1.17.1-cp312-cp312-win32.whl", hash = "sha256:a08d7e755f8ed21095a310a693525137cfe756ce62d066e53f502a83dc550f65", size = 172448 },
{ url = "https://files.pythonhosted.org/packages/50/b9/db34c4755a7bd1cb2d1603ac3863f22bcecbd1ba29e5ee841a4bc510b294/cffi-1.17.1-cp312-cp312-win_amd64.whl", hash = "sha256:51392eae71afec0d0c8fb1a53b204dbb3bcabcb3c9b807eedf3e1e6ccf2de903", size = 181976 },
{ url = "https://files.pythonhosted.org/packages/8d/f8/dd6c246b148639254dad4d6803eb6a54e8c85c6e11ec9df2cffa87571dbe/cffi-1.17.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:f3a2b4222ce6b60e2e8b337bb9596923045681d71e5a082783484d845390938e", size = 182989 },
{ url = "https://files.pythonhosted.org/packages/8b/f1/672d303ddf17c24fc83afd712316fda78dc6fce1cd53011b839483e1ecc8/cffi-1.17.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:0984a4925a435b1da406122d4d7968dd861c1385afe3b45ba82b750f229811e2", size = 178802 },
{ url = "https://files.pythonhosted.org/packages/0e/2d/eab2e858a91fdff70533cab61dcff4a1f55ec60425832ddfdc9cd36bc8af/cffi-1.17.1-cp313-cp313-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d01b12eeeb4427d3110de311e1774046ad344f5b1a7403101878976ecd7a10f3", size = 454792 },
{ url = "https://files.pythonhosted.org/packages/75/b2/fbaec7c4455c604e29388d55599b99ebcc250a60050610fadde58932b7ee/cffi-1.17.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:706510fe141c86a69c8ddc029c7910003a17353970cff3b904ff0686a5927683", size = 478893 },
{ url = "https://files.pythonhosted.org/packages/4f/b7/6e4a2162178bf1935c336d4da8a9352cccab4d3a5d7914065490f08c0690/cffi-1.17.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:de55b766c7aa2e2a3092c51e0483d700341182f08e67c63630d5b6f200bb28e5", size = 485810 },
{ url = "https://files.pythonhosted.org/packages/c7/8a/1d0e4a9c26e54746dc08c2c6c037889124d4f59dffd853a659fa545f1b40/cffi-1.17.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c59d6e989d07460165cc5ad3c61f9fd8f1b4796eacbd81cee78957842b834af4", size = 471200 },
{ url = "https://files.pythonhosted.org/packages/26/9f/1aab65a6c0db35f43c4d1b4f580e8df53914310afc10ae0397d29d697af4/cffi-1.17.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dd398dbc6773384a17fe0d3e7eeb8d1a21c2200473ee6806bb5e6a8e62bb73dd", size = 479447 },
{ url = "https://files.pythonhosted.org/packages/5f/e4/fb8b3dd8dc0e98edf1135ff067ae070bb32ef9d509d6cb0f538cd6f7483f/cffi-1.17.1-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:3edc8d958eb099c634dace3c7e16560ae474aa3803a5df240542b305d14e14ed", size = 484358 },
{ url = "https://files.pythonhosted.org/packages/f1/47/d7145bf2dc04684935d57d67dff9d6d795b2ba2796806bb109864be3a151/cffi-1.17.1-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:72e72408cad3d5419375fc87d289076ee319835bdfa2caad331e377589aebba9", size = 488469 },
{ url = "https://files.pythonhosted.org/packages/bf/ee/f94057fa6426481d663b88637a9a10e859e492c73d0384514a17d78ee205/cffi-1.17.1-cp313-cp313-win32.whl", hash = "sha256:e03eab0a8677fa80d646b5ddece1cbeaf556c313dcfac435ba11f107ba117b5d", size = 172475 },
{ url = "https://files.pythonhosted.org/packages/7c/fc/6a8cb64e5f0324877d503c854da15d76c1e50eb722e320b15345c4d0c6de/cffi-1.17.1-cp313-cp313-win_amd64.whl", hash = "sha256:f6a16c31041f09ead72d69f583767292f750d24913dadacf5756b966aacb3f1a", size = 182009 },
]
@ -220,6 +257,21 @@ wheels = [
{ url = "https://files.pythonhosted.org/packages/e4/ea/d836f008d33151c7a1f62caf3d8dd782e4d15f6a43897f64480c2b8de2ad/prompt_toolkit-3.0.50-py3-none-any.whl", hash = "sha256:9b6427eb19e479d98acff65196a307c555eb567989e6d88ebbb1b509d9779198", size = 387816 },
]
[[package]]
name = "psutil"
version = "7.0.0"
source = { registry = "https://pypi.org/simple" }
sdist = { url = "https://files.pythonhosted.org/packages/2a/80/336820c1ad9286a4ded7e845b2eccfcb27851ab8ac6abece774a6ff4d3de/psutil-7.0.0.tar.gz", hash = "sha256:7be9c3eba38beccb6495ea33afd982a44074b78f28c434a1f51cc07fd315c456", size = 497003 }
wheels = [
{ url = "https://files.pythonhosted.org/packages/ed/e6/2d26234410f8b8abdbf891c9da62bee396583f713fb9f3325a4760875d22/psutil-7.0.0-cp36-abi3-macosx_10_9_x86_64.whl", hash = "sha256:101d71dc322e3cffd7cea0650b09b3d08b8e7c4109dd6809fe452dfd00e58b25", size = 238051 },
{ url = "https://files.pythonhosted.org/packages/04/8b/30f930733afe425e3cbfc0e1468a30a18942350c1a8816acfade80c005c4/psutil-7.0.0-cp36-abi3-macosx_11_0_arm64.whl", hash = "sha256:39db632f6bb862eeccf56660871433e111b6ea58f2caea825571951d4b6aa3da", size = 239535 },
{ url = "https://files.pythonhosted.org/packages/2a/ed/d362e84620dd22876b55389248e522338ed1bf134a5edd3b8231d7207f6d/psutil-7.0.0-cp36-abi3-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1fcee592b4c6f146991ca55919ea3d1f8926497a713ed7faaf8225e174581e91", size = 275004 },
{ url = "https://files.pythonhosted.org/packages/bf/b9/b0eb3f3cbcb734d930fdf839431606844a825b23eaf9a6ab371edac8162c/psutil-7.0.0-cp36-abi3-manylinux_2_12_x86_64.manylinux2010_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4b1388a4f6875d7e2aff5c4ca1cc16c545ed41dd8bb596cefea80111db353a34", size = 277986 },
{ url = "https://files.pythonhosted.org/packages/eb/a2/709e0fe2f093556c17fbafda93ac032257242cabcc7ff3369e2cb76a97aa/psutil-7.0.0-cp36-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a5f098451abc2828f7dc6b58d44b532b22f2088f4999a937557b603ce72b1993", size = 279544 },
{ url = "https://files.pythonhosted.org/packages/50/e6/eecf58810b9d12e6427369784efe814a1eec0f492084ce8eb8f4d89d6d61/psutil-7.0.0-cp37-abi3-win32.whl", hash = "sha256:ba3fcef7523064a6c9da440fc4d6bd07da93ac726b5733c29027d7dc95b39d99", size = 241053 },
{ url = "https://files.pythonhosted.org/packages/50/1b/6921afe68c74868b4c9fa424dad3be35b095e16687989ebbb50ce4fceb7c/psutil-7.0.0-cp37-abi3-win_amd64.whl", hash = "sha256:4cf3d4eb1aa9b348dec30105c55cd9b7d4629285735a102beb4441e38db90553", size = 244885 },
]
[[package]]
name = "ptyprocess"
version = "0.7.0"
@ -321,6 +373,8 @@ name = "tractor"
version = "0.1.0a6.dev0"
source = { editable = "." }
dependencies = [
{ name = "bidict" },
{ name = "cffi" },
{ name = "colorlog" },
{ name = "msgspec" },
{ name = "pdbp" },
@ -334,6 +388,7 @@ dev = [
{ name = "greenback" },
{ name = "pexpect" },
{ name = "prompt-toolkit" },
{ name = "psutil" },
{ name = "pyperclip" },
{ name = "pytest" },
{ name = "stackscope" },
@ -342,6 +397,8 @@ dev = [
[package.metadata]
requires-dist = [
{ name = "bidict", specifier = ">=0.23.1" },
{ name = "cffi", specifier = ">=1.17.1" },
{ name = "colorlog", specifier = ">=6.8.2,<7" },
{ name = "msgspec", specifier = ">=0.19.0" },
{ name = "pdbp", specifier = ">=1.6,<2" },
@ -355,6 +412,7 @@ dev = [
{ name = "greenback", specifier = ">=1.2.1,<2" },
{ name = "pexpect", specifier = ">=4.9.0,<5" },
{ name = "prompt-toolkit", specifier = ">=3.0.50" },
{ name = "psutil", specifier = ">=7.0.0" },
{ name = "pyperclip", specifier = ">=1.9.0" },
{ name = "pytest", specifier = ">=8.3.5" },
{ name = "stackscope", specifier = ">=0.2.2,<0.3" },