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()`
It's been on the todo for a while and I've given up trying to properly
hide the `trio.CancelScope.__exit__()` frame for now instead opting to
just `log.pdb()` a big apology XD
Users can obvi still just not use the flag and wrap `tractor.pause()` in
their own cs block if they want to avoid having to hit `'up'` in the pdb
REPL if needed in a cancelled task-scope.
Impl deatz:
- factor orig `.pause()` impl into new `._pause()` so that we can more tersely
wrap the original content depending on `shield: bool` input; only open
the cancel-scope when shield is set to avoid aforemented extra strack
frame annoyance.
- pass through `shield` to underlying `_pause` and `debug_func()` so we
can actually know when so log our apology.
- add a buncha notes to new `.pause()` wrapper regarding the inability
to hide the cancel-scope `.__exit__()`, inluding that overriding the
code in `trio._core._run.CancelScope` doesn't seem to solve the issue
either..
Unrelated `maybe_wait_for_debugger()` tweaks:
- don't read `Lock.global_actor_in_debug` more then needed, rename local
read var to `in_debug` (since it can also hold the root actor uid, not
just sub-actors).
- shield the `await debug_complete.wait()` since ideally we avoid the
root cancellation child-actors in debug even when the root calls this
func in a cancelled scope.
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.
Since eventually we want to implement all other RPC "func types" as
contexts underneath this starts the rework to move all the other cases
into a separate func not only to simplify the main `._invoke()` body but
also as a reminder of the intention to do it XD
Details of re-factor:
- add a new `._invoke_non_context()` which just moves all the old blocks
for non-context handling to a single def.
- factor what was basically just the `finally:` block handler (doing all
the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()`
with that content packed into the post-`yield` (also with a `hide_tb:
bool` flag added of course).
* include a `debug_kbis: bool` for when needed.
- since the `@context` block is the only type left in the main
`_invoke()` body, de-dent it so it's more grok-able B)
Obviously this patch also includes a few improvements regarding
context-cancellation-semantics (for the `context` RPC case) on the
callee side in order to match previous changes to the `Context` api:
- always setting any ctxc as the `Context._local_error`.
- using the new convenience `.maybe_raise()` topically (for now).
- avoiding any previous reliance on `Context.cancelled_caught` for
anything public of meaning.
Further included is more logging content updates:
- being pedantic in `.cancel()` msgs about whether termination is caused
by error or ctxc.
- optional `._invoke()` traceback hiding via a `hide_tb: bool`.
- simpler log headers throughout instead leveraging new `.__repr__()` on
primitives.
- buncha `<= <actor-uid>` sent some message emissions.
- simplified handshake statuses reporting.
Other subsys api changes we need to match:
- change to `Channel.transport`.
- avoiding any `local_nursery: ActorNursery` waiting when the
`._implicit_runtime_started` is set.
And yes, lotsa more comments for #TODOs dawg.. since there's always
somethin!
In the case where the callee side delivers us a ctxc with `.canceller`
set we can presume that remote cancellation already has taken place and
thus we don't need to do the normal call-`Context.cancel()`-on-error
step. Further, in the case where we do call it also handle any
`trio.CloseResourceError` gracefully with a `.warning()`.
Also, originally I had added a post-`yield`-maybe-raise to attempt
handling any remote ctxc the same as for the local case (i.e. raised
from `yield` line) wherein if we get a remote ctxc the same handler
branch-path would trigger, thus avoiding different behaviour in that
case. I ended up masking it out (but can't member why.. ) as it seems
the normal `.result()` call and its internal handling gets the same
behaviour? I've left in the heavily commented code in case it ends up
being the better way to go; likely making the move to having a single
code in both cases is better even if it is just a matter of deciding
whether to swallow the ctxc or not in the `.cancel_acked` case.
Further teensie improvements:
- obvi improve/simplify log msg contents as in prior patches.
- use the new `maybe_wait_for_debugger(header_msg: str)` if/when waiting
to exit in debug mode.
- another `hide_tb: bool` frame hider flag.
- rando type-annot updates of course :)
Spanning from the pub API, to instance `repr()` customization (for
logging/REPL content), to the impl details around the notion of a "final
outcome" and surrounding IPC msg draining mechanics during teardown.
A few API and field updates:
- new `.cancel_acked: bool` to replace what we were mostly using
`.cancelled_caught: bool` for but, for purposes of better mapping the
semantics of remote cancellation of parallel executing tasks; it's set
only when `.cancel_called` is set and a ctxc arrives with
a `.canceller` field set to the current actor uid indicating we
requested and received acknowledgement from the other side's task
that is cancelled gracefully.
- strongly document and delegate (and prolly eventually remove as a pub
attr) the `.cancelled_caught` property entirely to the underlying
`._scope: trio.CancelScope`; the `trio` semantics don't really map
well to the "parallel with IPC msging" case in the sense that for
us it breaks the concept of the ctx/scope closure having "caught"
something instead of having "received" a msg that the other side has
"acknowledged" (i.e. which for us is the completion of cancellation).
- new `.__repr__()`/`.__str__()` format that tries to tersely yet
comprehensively as possible display everything you need to know about
the 3 main layers of an SC-linked-IPC-context:
* ipc: the transport + runtime layers net-addressing and prot info.
* rpc: the specific linked caller-callee task signature details
including task and msg-stream instances.
* state: current execution and final outcome state of the task pair.
* a teensie extra `.repr_rpc` for a condensed rpc signature.
- new `.dst_maddr` to get a `libp2p` style "multi-address" (though right
now it's just showing the transport layers so maybe we should move to
to our `Channel`?)
- new public instance-var fields supporting more granular remote
cancellation/result/error state:
* `.maybe_error: Exception|None` for any final (remote) error/ctxc
which computes logic on the values of `._remote_error`/`._local_error`
to determine the "final error" (if any) on termination.
* `.outcome` to the final error or result (or `None` if un-terminated)
* `.repr_outcome()` for a console/logging friendly version of the
final result or error as needed for the `.__str__()`.
- new private interface bits to support all of ^:
* a new "no result yet" sentinel value, `Unresolved`, using a module
level class singleton that `._result` is set too (instead of
`id(self)`) to both determine if and present when no final result
from the callee has-yet-been/was delivered (ever).
=> really we should get rid of `.result()` and change it to
`.wait_for_result()` (or something)u
* `_final_result_is_set()` predicate to avoid waiting for an already
delivered result.
* `._maybe_raise()` proto-impl that we should use to replace all the
`if re:` blocks it can XD
* new `._stream: MsgStream|None` for when a stream is opened to aid
with the state repr mentioned above.
Tweaks to the termination drain loop `_drain_to_final_msg()`:
- obviously (obvi) use all the changes above when determining whether or
not a "final outcome" has arrived and thus breaking from the loop ;)
* like the `.outcome` `.maybe_error` and `._final_ctx_is_set()` in
the `while` pred expression.
- drop the `_recv_chan.receive_nowait()` + guard logic since it seems
with all the surrounding (and coming soon) changes to
`Portal.open_context()` using all the new API stuff (mentioned in
first bullet set above) we never hit the case of inf-block?
Oh right and obviously a ton of (hopefully improved) logging msg content
changes, commented code removal and detailed comment-docs strewn about!
After some deep logging improvements to many parts of `._runtime`,
I realized a silly detail where we are always waiting on any opened
`local_nursery: ActorNursery` to signal exit from
`Actor._stream_handler()` even in the case of being an implicitly opened
root actor (`open_root_actor()` wasn't called by user/app code) via
`._supervise.open_nursery()`..
So, to address this add a `ActorNursery._implicit_runtime_started: bool`
that can be set and then checked to avoid doing the unnecessary
`.exited.wait()` (and any subsequent warn logging on an exit timeout) in
that special but most common case XD
Matching with other subsys log format refinements, improve readability
and simplicity of the actor-nursery supervisory log msgs, including:
- simplify and/or remove any content that more or less duplicates msg
content found in emissions from lower-level primitives and sub-systems
(like `._runtime`, `_context`, `_portal` etc.).
- add a specific `._open_and_supervise_one_cancels_all_nursery()`
handler block for `ContextCancelled` to log with `.cancel()` level
noting that the case is a "remote cancellation".
- put the nursery-exit and actor-tree shutdown status into a single msg
in the `implicit_runtime` case.
- rename `.soft_wait()` -> `.soft_kill()`
- rename `.do_hard_kill()` -> `.hard_kill()`
- adjust any `trio.Process.__repr__()` log msg contents to have the
little tree branch prefix: `'|_'`
Our remote error box types `RemoteActorError`, `ContextCancelled` and
`StreamOverrun` needed a console display makeover particularly for
logging content and `repr()` in higher level primitives like `Context`.
This adds a more "dramatic" str-representation to showcase the
underlying boxed traceback content more sensationally (via ascii-art
emphasis) as well as support a more terse `.reprol()` (representation
for one-line) format that can be used for types that track remote
errors/cancels like with `Context._remote_error`.
Impl deats:
- change `RemoteActorError.__repr__()` formatting to show (sub-type
specific) `.msgdata` fields in a multi-line format (similar to our new
`.msg.types.Struct` style) followed by some ascii accented delimiter
lines to emphasize any `.msgdata["tb_str"]` packed by the remote
- for rme and subtypes allow picking the specifically relevant fields
via a type defined `.reprol_fields: list[str]` and pick for each
subtype:
|_ `RemoteActorError.src_actor_uid`
|_ `ContextCancelled.canceller`
|_ `StreamOverrun.sender`
- add `.reprol()` to show a `repr()`-on-one-line formatted string that
can be used by other multi-line-field-`repr()` styled composite types
as needed in (high level) logging info.
- toss in some mod level `_body_fields: list[str]` for summary of such
fields (if needed).
- add some new rae (remote-actor-error) props:
- `.type` around a newly named `.boxed_type`
- `.type_str: str`
- `.tb_str: str`
Hit a reallly weird bug in the `._runtime` IPC msg handling loop where
it seems that by `str.format()`-ing a `Channel` before initializing it
would put the `._MsgTransport._agen()` in an already started state
causing an irrecoverable core startup failure..
I presume it's something to do with delegating to the
`MsgpackTCPStream.__repr__()` and, something something.. the
`.set_msg_transport(stream)` getting called to too early such that
`.msgstream.__init__()` is called thus init-ing the `._agen()` before
necessary? I'm sure there's a design lesson to be learned in here
somewhere XD
This was discovered while trying to add more "fancy" logging throughout
said core for the purposes of cobbling together an init attempt at
libp2p style multi-address representations for our IPC primitives. Thus
I also tinker here with adding some new fields to `MsgpackTCPStream`:
- `layer_key`: int = 4
- `name_key`: str = 'tcp'
- `codec_key`: str = 'msgpack'
Anyway, just changed it so that if `.msgstream` ain't set then we just
return a little "null repr" `str` value thinger.
Also renames `Channel.msgstream` internally to `._transport` with
appropriate pub `@property`s added such that everything else won't break
;p
Also drops `Optional` typing vis-a-vi modern union syntax B)
Obviously we can't deterministic-ally call `.load_ref()` (since you'd
have to point to an `id()` or something and presume a particular
py-runtime + virt-mem space for it to exist?) but it at least helps with
the `str` formatting for logging purposes (like `._cancel_rpc_tasks()`)
when `repr`-ing ctxs and their specific "rpc signatures".
Maybe in the future getting this working at least for singleton types
per process (like `Actor` XD ) will be a thing we can support and make
some sense of.. Bo
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`.
Allow callers to stick in a header to the `.pdb()` level emitted msg(s)
such that any "waiting status" content is only shown if the caller
actually get's blocked waiting for the debug lock; use it inside the
`._spawn` sub-process reaper call.
Also, return early if `Lock.global_actor_in_debug == None` and thus
only enter the poll loop when actually needed, consequently raise
if we fall through the loop without acquisition.
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.
Besides improving a bunch more log msg contents similarly as before this
changes the cancel method signatures slightly with different arg names:
for `.cancel()`:
- instead of `requesting_uid: str` take in a `req_chan: Channel`
since we can always just read its `.uid: tuple` for logging and
further we can then offer the `chan=None` case indicating a
"self cancel" (since there's no "requesting channel").
- the semantics of "requesting" here better indicate that the IPC connection
is an IPC peer and further (eventually) will allow permission checking
against given peers for cancellation requests.
- when `chan==None` we also define a meth-internal `requester_type: str`
differently for logging content :)
- add much more detailed `.cancel()` content around the requester, its
type, and any debugger related locking steps.
for `._cancel_task()`:
- change the `chan` arg to `parent_chan: Channel` since "parent"
correctly indicates that the channel is the parent of the locally
spawned rpc task to cancel; in fact no other chan should be able to
cancel tasks parented/spawned by other channels obvi!
- also add more extensive meth-internal `.cancel()` logging with a #TODO
around showing only the "relevant/lasest" `Context` state vars in such
logging content.
for `.cancel_rpc_tasks()`:
- shorten `requesting_uid` -> `req_uid`.
- add `parent_chan: Channel` to be similar as above in `._cancel_task()`
(since it's internally delegated to anyway) which replaces the prior
`only_chan` and use it to filter to only tasks spawned by this channel
(thus as their "parent") as before.
- instead of `if tasks:` to enter, invert and `return` early on
`if not tasks`, for less indentation B)
- add WIP str-repr format (for `.cancel()` emissions) to show
a multi-address (maddr) + task func (via the new `Context._nsf`) and
report all cancel task targets with it a "tree"; include #TODO to
finalize and implement some utils for all this!
To match ensure we adjust `process_messages()` self/`Actor` cancel
handling blocks to provide the new `kwargs` (now with `dict`-merge
syntax) to `._invoke()`.
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 you see the children entries prior to exit instead of the
prior somewhat detail/use-less logging. Also, rename all `anursery` vars
to just `an` as is the convention in most examples.
Since that's what we're now doing in `MsgStream._eoc` internal
assignments (coming in future patch), do the same in this exception
re-raise-helper and include more extensive doc string detailing all
the msg-type-to-raised-error cases. Also expose a `hide_tb: bool` like
we have already in `unpack_error()`.
As similarly improved in other parts of the runtime, adds much more
pedantic (`.cancel()`) logging content to indicate the src of remote
cancellation request particularly for `Actor.cancel()` and
`._cancel_task()` cases prior to `._invoke()` task scheduling. Also add
detailed case comments and much more info to the
"request-to-cancel-already-terminated-RPC-task" log emission to include
the `Channel` and `Context.cid` deats.
This helped me find the src of a race condition causing a test to fail
where a callee ctx task was returning a result *before* an expected
`ctx.cancel()` request arrived B). Adding much more pedantic
`.cancel()` msg contents around the requester's deats should ensure
these cases are much easier to detect going forward!
Also, simplify the `._invoke()` final result/error log msg to only put
*one of either* the final error or returned result above the `Context`
pprint.
The only case where we can't is in `Portal.run_from_ns()` usage (since we
pass a path with `self:<Actor.meth>`) and because `.to_tuple()`
internally uses `.load_ref()` which will of course fail on such a path..
So or now impl as,
- mk `Actor.start_remote_task()` take a `nsf: NamespacePath` but also
offer a `load_nsf: bool = False` such that by default we bypass ref
loading (maybe this is fine for perf long run as well?) for the
`Actor`/'self:'` case mentioned above.
- mk `.get_context()` take an instance `nsf` obvi.
More logging msg format tweaks:
- change msg-flow related content to show the `Context._nsf`, which,
right, is coming follow up commit..
- bunch more `.runtime()` format updates to show `msg: dict` contents
and internal primitives with trailing `'\n'` for easier reading.
- report import loading `stackscope` in subactors.
When entered by the root actor avoid excessive polling cycles by,
- blocking on the `Lock.no_remote_has_tty: trio.Event` and breaking
*immediately* when set (though we should really also lock
it from the root right?) to avoid extra loops..
- shielding the `await trio.sleep(poll_delay)` call to avoid any local
cancellation causing the (presumably root-actor task) caller to move
on (possibly to cancel its children) and instead to continue
poll-blocking until the lock is actually released by its user.
- `break` the poll loop immediately if no remote locker is detected.
- use `.pdb()` level for reporting lock state changes.
Also add a #TODO to handle calls by non-root actors as it pertains to
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..
If `stackscope` is importable and debug_mode is enabled then we by
default call and report `.devx.enable_stack_on_sig()` is set B)
This makes debugging unexpected (SIGINT ignoring) hangs a cinch!
Given i just similarly revamped a buncha `._runtime` log msg formatting,
might as well do something similar inside the spawning machinery such
that groking teardown sequences of each supervising task is much more
sane XD
Mostly this includes doing similar `'<field>: <value>\n'` multi-line
formatting when reporting various subproc supervision steps as well as
showing a detailed `trio.Process.__repr__()` as appropriate.
Also adds a detailed #TODO according to the needs of #320 for which
we're going to need some internal mechanism for intermediary parent
actors to determine if a given debug tty locker (sub-actor) is one of
*their* (transitive) children and thus stall the normal
cancellation/teardown sequence until that locker is complete.
Can be optionally enabled via a new `enable_stack_on_sig()` which will
swap in the SIGUSR1 handler. Much thanks to @oremanj for writing this
amazing project, it's thus far helped me fix some very subtle hangs
inside our new IPC-context cancellation machinery that would have
otherwise taken much more manual pdb-ing and hair pulling XD
Full credit for `dump_task_tree()` goes to the original project author
with some minor tweaks as was handed to me via the trio-general matrix
room B)
Slight changes from orig version:
- use a `log.pdb()` emission to pprint to console
- toss in an ex sh CLI cmd to trigger the dump from another terminal
using `kill` + `pgrep`.
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.
As part of solving some final edge cases todo with inter-peer remote
cancellation (particularly a remote cancel from a separate actor
tree-client hanging on the request side in `modden`..) I needed less
dense, more line-delimited log msg formats when understanding ipc
channel and context cancels from console logging; this adds a ton of
that to:
- `._invoke()` which now does,
- better formatting of `Context`-task info as multi-line
`'<field>: <value>\n'` messages,
- use of `trio.Task` (from `.lowlevel.current_task()` for full
rpc-func namespace-path info,
- better "msg flow annotations" with `<=` for understanding
`ContextCancelled` flow.
- `Actor._stream_handler()` where in we break down IPC peers reporting
better as multi-line `|_<Channel>` log msgs instead of all jammed on
one line..
- `._ipc.Channel.send()` use `pformat()` for repr of packet.
Also tweak some optional deps imports for debug mode:
- add `maybe_import_gb()` for attempting to import `greenback`.
- maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed.
Add a further stale-debugger-lock guard before removal:
- read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly
`maybe_wait_for_debugger()` when the child-user is known to have
a live process in our tree.
- only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when
the disconnected channel maps to the `Lock.global_actor_in_debug`,
though not sure this is correct yet?
Started adding missing type annots in sections that were modified.