As per the long outstanding GH issue this starts our rigorous journey
into an attempt at a type-safe, cross-actor SC, IPC protocol Bo
boop -> https://github.com/goodboy/tractor/issues/36
The idea is to "formally" define our SC "shuttle (dialog) protocol" by
specifying a new `.msg.types.Msg` subtype-set which can fully
encapsulate all IPC msg schemas needed in order to accomplish
cross-process SC!
The msg set deviated a little in terms of (type) names from the existing
`dict`-msgs currently used in the runtime impl but, I think the name
changes are much better in terms of explicitly representing the internal
semantics of the actor runtime machinery/subsystems and the
IPC-msg-dialog required for SC enforced RPC.
------ - ------
In cursory, the new formal msgs-spec includes the following msg-subtypes
of a new top-level `Msg` boxing type (that holds the base field schema
for all msgs):
- `Start` to request RPC task scheduling by passing a `FuncSpec` payload
(to replace the currently used `{'cmd': ... }` dict msg impl)
- `StartAck` to allow the RPC task callee-side to report a `IpcCtxSpec`
payload immediately back to the caller (currently responded naively via
a `{'functype': ... }` msg)
- `Started` to deliver the first value from `Context.started()`
(instead of the existing `{'started': ... }`)
- `Yield` to shuttle `MsgStream.send()`-ed values (instead of
our `{'yield': ... }`)
- `Stop` to terminate a `Context.open_stream()` session/block
(over `{'stop': True }`)
- `Return` to deliver the final value from the `Actor.start_remote_task()`
(which is a `{'return': ... }`)
- `Error` to box `RemoteActorError` exceptions via a `.pld: ErrorData`
payload, planned to replace/extend the current `RemoteActorError.msgdata`
mechanism internal to `._exceptions.pack/unpack_error()`
The new `tractor.msg.types` includes all the above msg defs as well an API
for rendering a "payload type specification" using a
`payload_type_spec: Union[Type]` that can be passed to
`msgspec.msgpack.Decoder(type=payload_type_spec)`. This ensures that
(for a subset of the above msg set) `Msg.pld: PayloadT` data is
type-parameterized using `msgspec`'s new `Generic[PayloadT]` field
support and thus enables providing for an API where IPC `Context`
dialogs can strictly define the allowed payload-datatype-set via type
union!
Iow, this is the foundation for supporting `Channel`/`Context`/`MsgStream`
IPC primitives which are type checked/safe as desired in GH issue:
- https://github.com/goodboy/tractor/issues/365
Misc notes on current impl(s) status:
------ - ------
- add a `.msg.types.mk_msg_spec()` which uses the new `msgspec` support
for `class MyStruct[Struct, Generic[T]]` parameterize-able fields and
delivers our boxing SC-msg-(sub)set with the desired `payload_types`
applied to `.pld`:
- https://jcristharif.com/msgspec/supported-types.html#generic-types
- as a note this impl seems to need to use `type.new_class()` dynamic
subtype generation, though i don't really get *why* still.. but
without that the `msgspec.msgpack.Decoder` doesn't seem to reject
`.pld` limited `Msg` subtypes as demonstrated in the new test.
- around this ^ add a `.msg._codec.limit_msg_spec()` cm which exposes
this payload type limiting API such that it can be applied per task
via a `MsgCodec` in app code.
- the orig approach in https://github.com/goodboy/tractor/pull/311 was
the idea of making payload fields `.pld: Raw` wherein we could have
per-field/sub-msg decoders dynamically loaded depending on the
particular application-layer schema in use. I don't want to lose the
idea of this since I think it might be useful for an idea I have about
capability-based-fields(-sharing, maybe using field-subset
encryption?), and as such i've kept the (ostensibly) working impls in
TODO-comments in `.msg._codec` wherein maybe we can add
a `MsgCodec._payload_decs: dict` table for this later on.
|_ also left in the `.msg.types.enc/decmsg()` impls but renamed as
`enc/dec_payload()` (but reworked to not rely on the lifo codec
stack tables; now removed) such that we can prolly move them to
`MsgCodec` methods in the future.
- add an unused `._codec.mk_tagged_union_dec()` helper which was
originally factored out the #311 proto-code but didn't end up working
as desired with the new parameterized generic fields approach (now
in `msg.types.mk_msg_spec()`)
Testing/deps work:
------ - ------
- new `test_limit_msgspec()` which ensures all the `.types` content is
correct but without using the wrapping APIs in `._codec`; i.e. using
a in-line `Decoder` instead of a `MsgCodec`.
- pin us to `msgspec>=0.18.5` which has the needed generic-types support
(which took me way too long yester to figure out when implementing all
this XD)!
Fitting in line with the issues outstanding:
- #36: (msg)spec-ing out our SCIPP (structured-con-inter-proc-prot).
(https://github.com/goodboy/tractor/issues/36)
- #196: adding strictly typed IPC msg dialog schemas, more or less
better described as "dialog/transaction scoped message specs"
using `msgspec`'s tagged unions and custom codecs.
(https://github.com/goodboy/tractor/issues/196)
- #365: using modern static type-annots to drive capability based
messaging and RPC.
(statically https://github.com/goodboy/tractor/issues/365)
This is a first draft of a new API for dynamically overriding IPC msg
codecs for a given interchange lib from any task in the runtime. Right
now we obviously only support `msgspec` but ideally this API holds
general enough to be used for other backends eventually (like
`capnproto`, and apache arrow).
Impl is in a new `tractor.msg._codec` with:
- a new `MsgCodec` type for encapsing `msgspec.msgpack.Encoder/Decoder`
pairs and configuring any custom enc/dec_hooks or typed decoding.
- factory `mk_codec()` for creating new codecs ad-hoc from a task.
- `contextvars` support for a new `trio.Task` scoped
`_ctxvar_MsgCodec: ContextVar[MsgCodec]` named 'msgspec_codec'.
- `apply_codec()` for temporarily modifying the above per task
as needed around `.open_context()` / `.open_stream()` operation.
A new test (suite) in `test_caps_msging.py`:
- verify a parent and its child can enable the same custom codec (in
this case to transmit `NamespacePath`s) with tons of pedantic ctx-vars
checks.
- ToDo: still need to implement #36 msg types in order to be able to get
decodes working (as in `MsgStream.receive()` will deliver an already
created `NamespacePath` obj) since currently all msgs come packed in `dict`-msg
wrapper packets..
-> use the proto from PR #35 to get nested `msgspec.Raw` processing up
and running Bo
It's **almost** there, we're just missing the final translation code to
get from an `asyncio` side task to be able to call
`.devx._debug..wait_for_parent_stdin_hijack()` to do root actor TTY
locking. Then we just need to ensure internals also do the right thing
with `greenback()` for equivalent sync `breakpoint()` style pause
points.
Since i'm deferring this until later, tossing in some xfail tests to
`test_infected_asyncio` with TODOs for the needed implementation as well
as eventual test org.
By "provision" it means we add:
- `greenback` init block to `_run_asyncio_task()` when debug mode is
enabled (but which will currently rte when `asyncio` is detected)
using `.bestow_portal()` around the `asyncio.Task`.
- a call to `_debug.maybe_init_greenback()` in the `run_as_asyncio_guest()`
guest-mode entry point.
- as part of `._debug.Lock.is_main_trio_thread()` whenever the async-lib
is not 'trio' error lock the backend name (which is obvi `'asyncio'`
in this use case).
Now supports use from any `trio` task, any sync thread started with
`trio.to_thread.run_sync()` AND also via `breakpoint()` builtin API!
The only bit missing now is support for `asyncio` tasks when in infected
mode.. Bo
`greenback` setup/API adjustments:
- move `._rpc.maybe_import_gb()` to -> `devx._debug` and factor out the cached
import checking into a sync func whilst placing the async `.ensure_portal()`
bootstrapping into a new async `maybe_init_greenback()`.
- use the new init-er func inside `open_root_actor()` with the output
predicating whether we override the `breakpoint()` hook.
core `devx._debug` implementation deatz:
- make `mk_mpdb()` only return the `pdp.Pdb` subtype instance since
the sigint unshielding func is now accessible from the `Lock`
singleton from anywhere.
- add non-main thread support (at least for `trio.to_thread` use cases)
to our `Lock` with a new `.is_trio_thread()` predicate that delegates
directly to `trio`'s internal version.
- do `Lock.is_trio_thread()` checks inside any methods which require
special provisions when invoked from a non-main `trio` thread:
- `.[un]shield_sigint()` methods since `signal.signal` usage is only
allowed from cpython's main thread.
- `.release()` since `trio.StrictFIFOLock` can only be called from
a `trio` task.
- rework `.pause_from_sync()` itself to directly call `._set_trace()`
and don't bother with `greenback._await()` when we're already calling
it from a `.to_thread.run_sync()` thread, oh and try to use the
thread/task name when setting `Lock.local_task_in_debug`.
- make it an RTE for now if you try to use `.pause_from_sync()` from any
infected-`asyncio` task, but support is (hopefully) coming soon!
For testing we add a new `test_debugger.py::test_pause_from_sync()`
which includes a ctrl-c parametrization around the
`examples/debugging/sync_bp.py` script which includes all currently
supported/working usages:
- `tractor.pause_from_sync()`.
- via `breakpoint()` overload.
- from a `trio.to_thread.run_sync()` spawn.
Use new `RemoteActorError` fields in various assertions particularly
ensuring that an RTE relayed through the spawner from the little_bro
shows up at the client with the right number of entries in the
`.relay_path` and that the error is raised in the client as desired in
the original use case from `modden`'s remote spawn spawn request API
(which was kinda the whole original motivation to finally get all this
multi-actor error relay stuff workin).
Case extensions:
- RTE relayed from little_bro through spawner to client when
`raise_sub_spawn_error_after` is set; in this case test should raise
the relayed and RAE boxed RTE right up to the `trio.run()`.
-> ensure the `rae.src_uid`, `.relay_uid` are set correctly.
-> ensure ctx cancels are no acked.
- use `expect_ctxc()` around root's `tell_little_bro()` usage.
- do `debug_mode` assertions when enabled by test harness in each actor
layer.
- obvi use new `.src_type`/`.boxed_type` for final error propagation
assertions.
More or less just simplifies to not seeing the stream closure errors and
instead expecting KBIs from the simulated user who 'ctl-cs after hang'.
Toss in a little `stuff_hangin_ctlc()` to the script to wrap all that
and always check stream closure before sending the final KBI.
- `trio_typing` is nearly obsolete since `trio >= 0.23`
- `exceptiongroup` is built-in to python 3.11
- `async_generator` primitives have lived in `contextlib` for quite
a while!
Since importing from our top level `conftest.py` is not scaleable
or as "future forward thinking" in terms of:
- LoC-wise (it's only one file),
- prevents "external" (aka non-test) example scripts from importing
content easily,
- seemingly(?) can't be used via abs-import if using
a `[tool.pytest.ini_options]` in a `pyproject.toml` vs.
a `pytest.ini`, see:
https://docs.pytest.org/en/8.0.x/reference/customize.html#pyproject-toml)
=> Go back to having an internal "testing" pkg like `trio` (kinda) does.
Deats:
- move generic top level helpers into pkg-mod including the new
`expect_ctxc()` (which i needed in the advanced faults testing script.
- move `@tractor_test` into `._testing.pytest` sub-mod.
- adjust all the helper imports to be a `from tractor._testing import <..>`
Rework `test_ipc_channel_break_during_stream()` and backing script:
- make test(s) pull `debug_mode` from new fixture (which is now
controlled manually from `--tpdb` flag) and drop the previous
parametrized input.
- update logic in ^ test for "which-side-fails" cases to better match
recently updated/stricter cancel/failure semantics in terms of
`ClosedResouruceError` vs. `EndOfChannel` expectations.
- handle `ExceptionGroup`s with expected embedded errors in test.
- better pendantics around whether to expect a user simulated KBI.
- for `examples/advanced_faults/ipc_failure_during_stream.py` script:
- generalize ipc breakage in new `break_ipc()` with support for diff
internal `trio` methods and a #TODO for future disti frameworks
- only make one sub-actor task break and the other just stream.
- use new `._testing.expect_ctxc()` around ctx block.
- add a bit of exception handling with `print()`s around ctxc (unused
except if 'msg' break method is set) and eoc cases.
- don't break parent side ipc in loop any more then once
after first break, checked via flag var.
- add a `pre_close: bool` flag to control whether
`MsgStreama.aclose()` is called *before* any ipc breakage method.
Still TODO:
- drop `pytest.ini` and add the alt section to `pyproject.py`.
-> currently can't get `--rootdir=` opt to work.. not showing in
console header.
-> ^ also breaks on 'tests' `enable_modules` imports in subactors
during discovery tests?
With the seeming cause that some cases occasionally raise
`ExceptionGroup` instead of a (collapsed out) single error which, in
those cases at least try to check that `.exceptions` has the original
error.
Found exactly why trying this won't work when playing around with
opening workspaces in `modden` using a `Portal.open_context()` back to
the 'bigd' root actor: the RPC machinery only registers one entry in
`Actor._contexts` which will get overwritten by each task's side and
then experience race-based IPC msging errors (eg. rxing `{'started': _}`
on the callee side..). Instead make opening a ctx back to the self-actor
a runtime error describing it as an invalid op.
To match:
- add a new test `test_ctx_with_self_actor()` to the context semantics
suite.
- tried out adding a new `side: str` to the `Actor.get_context()` (and
callers) but ran into not being able to determine the value from in
`._push_result()` where it's needed to figure out which side to push
to.. So, just leaving the commented arg (passing) in the runtime core
for now in case we can come back to trying to make it work, tho i'm
thinking it's not the right hack anyway XD
Such that it's set to whatever `Actor.reg_addrs: list[tuple]` is during
the actor's init-after-spawn guaranteeing each actor has at least the
registry infos from its parent. Ensure we read this if defined over
`_root._default_lo_addrs` in `._discovery` routines, namely
`.find_actor()` since it's the one API normally used without expecting
the runtime's `current_actor()` to be up.
Update the latest inter-peer cancellation test to use the `reg_addr`
fixture (and thus test this new runtime-vars value via `find_actor()`
usage) since it was failing if run *after* the infected `asyncio` suite
due to registry contact failure.
Including mostly tweaking asserts on relayed `ContextCancelled`s and
the new pub ctx properties: `.outcome`, `.maybe_error`, etc. as it
pertains to graceful (absorbed) remote cancellation vs. loud ctxc cases
expected to be raised by any `Portal.cancel_actor()` style teardown.
Start checking a variety internals like `._remote/local_error`,
`._is_self_cancelled()`, `._is_final_result_set()`, `._cancel_msg`
where applicable.
Also factor out the new `expect_ctxc()` checker to our `conftest.py` for
use in other suites.
We don't expect `._scope.cancelled_caught` to be set really ever on
inter-peer cancellation since no ctx is ever cancelling itself, a peer
cancels some other and then bubbles back to all other peers.
Also add `ids: lambda` for `error_during_ctxerr_handling` param to
`test_peer_canceller()`
Buncha subtle details changed mostly to do with when `Context.cancel()`
gets called on "real" remote errors vs. (peer requested) cancellation
and then local side handling of `ContextCancelled`.
Specific changes to make tests pass:
- due to raciness with `sleeper_ctx.result()` raising the ctxc locally
vs. the child-peers receiving similar ctxcs themselves (and then
erroring and propagating back to the root parent), we might not see
`._remote_error` set during the sub-ctx loops (except for the sleeper
itself obvi).
- do not expect `.cancel_called`/`.cancel_caught` to be set on any
sub-ctx since currently `Context.cancel()` is only called non-shielded
and thus is not in invoked when `._scope.cancel()` is called as part
of each root-side ctx ref/block handling the inter-peer ctxc.
- do not expect `Context._scope.cancelled_caught` to be set in most cases
(even the sleeper)
TODO Outstanding adjustments not fixed yet:
-[ ] `_scope.cancelled_caught` checks outside the `.open_context()`
blocks.
Since this was changed as part of overall project wide logging format
updates, and i ended up changing the both the crash and pause `.pdb()`
msgs to include some multi-line-ascii-"stuff", might as well make the
pre-prompt checks in the test suite more flexible to match.
As such, this exposes 2 new constants inside the `.devx._debug` mod:
- `._pause_msg: str` for the pre `tractor.pause()` header emitted via
`log.pdb()` and,
- `._crash_msg: str` for the pre `._post_mortem()` equiv when handling
errors in debug mode.
Adjust the test suite to use these values and thus make us more capable
to absorb changes in the future as well:
- add a new `in_prompt_msg()` predicate, very similar to `assert_before()`
but minus `assert`s which takes in a `parts: list[str]` to match
in the pre-prompt stdout.
- delegate to `in_prompt_msg()` in `assert_before()` since it was mostly
duplicate minus `assert`.
- adjust all previous `<patt> in before` asserts to instead use
`in_prompt_msg()` with separated pre-prompt-header vs. actor-name
`parts`.
- use new `._pause/crash_msg` values in all such calls including any
`assert_before()` cases.
As per a lot of the recent refinements to `Context` cancellation, add
a new test case to replicate the original hang-on-cancel found with
`modden` when using a client actor to spawn a subactor in some other
tree where despite `Context.cancel()` being called the requesting client
would hang on the opened context with the server.
The specific scenario added here is to have,
- root actor spawns 2 children: a client and a spawn server.
- the spawn server opens with a spawn-request serve loop and begins to
wait for the client.
- client spawns and connects to the sibling spawn server, requests to
spawn a sub-actor, the "little bro", connects to it then does some
echo streaming, cancels the request with it's sibling (the spawn
server) which should in turn cancel the root's-grandchild and result
in a cancel-ack back to the client's `.open_context()`.
- root ensures that it can also connect to the grandchild (little bro),
do the same echo streaming, then ensure everything tears down
correctly after cancelling all the children.
More refinements to come here obvi in the specific cancellation
semantics and possibly causes.
Also tweaks the other tests in suite to use the new `Context` properties
recently introduced and similarly updated in the previous patch to the
ctx-semantics suite.
In order to match a very significant and coming-soon patch set to the
IPC `Context` and `Channel` cancellation semantics with significant but
subtle changes to the primitives and runtime logic:
- a new set of `Context` state pub meth APIs for checking exact
inter-actor-linked-task outcomes such as `.outcome`, `.maybe_error`,
and `.cancel_acked`.
- trying to move away from `Context.cancelled_caught` usage since the
semantics from `trio` don't really map well (in terms of cancel
requests and how they result in cancel-scope graceful closure) and
`.cancel_acked: bool` is a better approach for IPC req-resp msging.
- change test usage to access `._scope.cancelled_caught` directly.
- more pedantic ctxc-raising expects around the "type of self
cancellation" and final outcome in ctxc cases:
- `ContextCancelled` is raised by ctx (`Context.result()`) consumer
methods when `Portal.cancel_actor()` is called (since it's an
out-of-band request) despite `Channel._cancel_called` being set.
- also raised by `.open_context().__aexit__()` on close.
- `.outcome` is always `.maybe_error` is always one of
`._local/remote_error`.
Since a bug in the new `MsgStream.aclose()` impl's drain block logic was
triggering an actual inf loop (by not ever canceller the streamer child
actor), make sure we put a loop limit on the `inf_streamer`()` XD
Also add a bit more deats to the test `print()`s in each actor and toss
in `debug_mode` fixture support.
Turns out that py3.11 might be so fast that iterating a EoC-ed
`MsgStream` 1k times is faster then a `Context.cancel()` msg
transmission from a parent actor to it's child (which i guess makes
sense). So tweak the test to delay 5ms between stream async-for iteration
attempts when the stream is detected to be `.closed: bool` (coming in
patch) or `ctx.cancel_called == true`.
Such that with `--tpdb` passed (sub)actors will engage the `pdbp` REPL
automatically and so that we can use the new `stackscope` support when
complex cases hang Bo
Also,
- simplified some type-annots (ns paths),
- doc-ed an inter-peer test func with some ascii msg flows,
- added a bottom #TODO for replicating the scenario i hit in `modden`
where a separate client actor-tree was hanging on cancelling a `bigd`
sub-workspace..
Allows tests (including any `@tractor_test`s) to subscribe to a CLI flag
`--tpdb` (for "tractor python debugger") which the session can provide
to tests which can then proxy the value to `open_root_actor()` (via
`open_nursery()`) when booting the runtime - thus enabling our debug
mode globally to any subscribers B)
This is real handy if you have some failures but can't determine the
root issue without jumping into a `pdbp` REPL inside a (sub-)actor's
spawned-task.
Drop all the nested `@acm` blocks and defunct comments from initial
validations. Add some todos for cases that are still unclear such as
whether the caller / streamer should have `.cancelled_caught == True` in
it's teardown.
We can now make asserts on `.cancelled_caught` and `_remote_error` vs.
`_local_error`. Expect a runtime error when `Context.open_stream()` is
called AFTER `.cancel()` and the remote `ContextCancelled` hasn't
arrived (yet). Adjust to `'itself'` string in self-cancel case.
Tests that appropriate `Context` exit state, the relay of
a `ContextCancelled` error and its `.canceller: tuple[str, str]` value
are set when an inter-peer cancellation happens via an "out of band"
request method (in this case using `Portal.cancel_actor()` and that
cancellation is propagated "horizontally" to other peers. Verify that
any such cancellation scenario which also experiences an "error during
`ContextCancelled` handling" DOES NOT result in that further error being
suppressed and that the user's exception bubbles out of the
`Context.open_context()` block(s) appropriately!
Likely more tests to come as well as some factoring of the teardown
state checks where possible.
Pertains to serious testing the major work landing in #357
Move over relevant test from the "context semantics" test module which
was already verifying peer-caused-`ContextCancelled.canceller: tuple`
error info and propagation during an inter-peer cancellation scenario.
Also begin a more general set of inter-peer cancellation tests starting
with the simplest case where when a peer is cancelled the parent should
NOT get an "muted" `trio.Cancelled` and instead
a `tractor.ContextCancelled` with a `.canceller: tuple` which points to
the sibling actor which requested the peer cancel.
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).
This actually caught further runtime bugs so it's gud i tried..
Add overrun-ignore enabled / disabled cases and error catching for all
of them. More or less this should cover every possible outcome when
it comes to setting `allow_overruns: bool` i hope XD
Turns out stuff was totally broken in these cases because we're either
closing the underlying mem chan too early or not handling the
"allow_overruns" mode's cancellation correctly..
These will verify new changes to the runtime/messaging core which allows
us to adopt an "ignore cancel if requested by us" style handling of
`ContextCancelled` more like how `trio` does with
`trio.Nursery.cancel_scope.cancel()`. We now expect
a `ContextCancelled.canceller: tuple` which is set to the actor uid of
the actor which requested the cancellation which eventually resulted in
the remote error-msg.
Also adds some experimental tweaks to the "backpressure" test which it
turns out is very problematic in coordination with context cancellation
since blocking on the feed mem chan to some task will block the ipc msg
loop and thus handling of cancellation.. More to come to both the test
and core to address this hopefully since right now this test is failing.