Commit Graph

1887 Commits (b56352b0e4951d73be2b94af1b692bd3bd98aec3)

Author SHA1 Message Date
Tyler Goodlet b56352b0e4 Quieter `Stop` handling on ctx result capture
In the `drain_to_final_msg()` impl, since a stream terminating
gracefully requires this msg, there's really no reason to `log.cancel()`
about it; go `.runtime()` level instead since we're trying de-noise
under "normal operation".

Also,
- passthrough `hide_tb` to taskc-handler's `ctx.maybe_raise()` call.
- raise `MessagingError` for the `MsgType` unmatched `case _:`.
- detail the doc string motivation a little more.
2024-07-03 22:42:32 -04:00
Tyler Goodlet 9be821a5cf More failed REPL-lock-request refinements
In `lock_stdio_for_peer()` better internal-error handling/reporting:
- only `Lock._blocked.remove(ctx.cid)` if that same cid was added on
  entry to avoid needless key-errors.
- drop all `Lock.release(force: bool)` usage remnants.
- if `req_ctx.cancel()` fails mention it with `ctx_err.add_note()`.
- add more explicit internal-failed-request log messaging via a new
  `fail_reason: str`.
- use and use new `x)<=\n|_` annots in any failure logging.

Other cleanups/niceties:
- drop `force: bool` flag entirely from the `Lock.release()`.
- use more supervisor-op-annots in `.pdb()` logging
  with both `_pause/crash_msg: str` instead of double '|' lines when
  `.pdb()`-reported from `._set_trace()`/`._post_mortem()`.
2024-07-02 17:06:50 -04:00
Tyler Goodlet b46400a86f Use `._entry` proto-ed "lifetime ops" in logging
As per a WIP scribbled out TODO in `._entry.nest_from_op()`, change
a bunch of "supervisor/lifetime mgmt ops" related log messages to
contain some supervisor-annotation "headers" in an effort to give
a terser "visual indication" of how some execution/scope/storage
primitive entity (like an actor/task/ctx/connection) is being operated
on (like, opening/started/closed/cancelled/erroring) from a "supervisor
action" POV.

Also tweak a bunch more emissions to lower levels to reduce noise around
normal inter-actor operations like process and IPC ctx supervision.
2024-07-02 16:31:58 -04:00
Tyler Goodlet 02812b9f51 Reraise RAEs in `MsgStream.receive()`; truncate tbs
To avoid showing lowlevel details of exception handling around the
underlying call to `return await self._ctx._pld_rx.recv_pld(ipc=self)`,
any time a `RemoteActorError` is unpacked (an raised locally) we re-raise
it directly from the captured `src_err` captured so as to present to
the user/app caller-code an exception raised directly from the `.receive()`
frame. This simplifies traceback call-stacks for any `log.exception()`
or `pdb`-REPL output filtering out the lower `PldRx` frames by default.
2024-07-02 16:31:15 -04:00
Tyler Goodlet 3c5816c977 Add `Portal.chan` property, to wrap `._chan` attr 2024-07-02 15:53:33 -04:00
Tyler Goodlet af3745684c More formal `TransportClosed` reporting/raising
Since it was all ad-hoc defined inside
`._ipc.MsgpackTCPStream._iter_pkts()` more or less, this starts
formalizing a way for particular transport backends to indicate whether
a disconnect condition should be re-raised in the RPC msg loop and if
not what log level to report it at (if any).

Based on our lone transport currently we try to suppress any logging
noise from ephemeral connections expected during normal actor
interaction and discovery subsys ops:
- any short lived discovery related TCP connects are only logged as
  `.transport()` level.
- both `.error()` and raise on any underlying `trio.ClosedResource`
  cause since that normally means some task touched transport layer
  internals that it shouldn't have.
- do a `.warning()` on anything else unexpected.

Impl deats:
- extend the `._exceptions.TransportClosed` to accept an input log
  level, raise-on-report toggle and custom reporting & raising via a new
  `.report_n_maybe_raise()` method.
- construct the TCs with inputs per case in (the newly named) `._iter_pkts().
- call ^ this method from the `TransportClosed` handler block inside the
  RPC msg loop thus delegating reporting levels and/or raising to the
  backend's per-case TC instantiating.

Related `._ipc` changes:
- mask out all the `MsgpackTCPStream._codec` debug helper stuff and drop
  any lingering cruft from the initial proto-ing of msg-codecs.
- rename some attrs/methods:
  |_`MsgpackTCPStream._iter_packets()` -> `._iter_pkts()` and
    `._agen` -> `_aiter_pkts`.
  |_`Channel._aiter_recv()` -> `._aiter_msgs()` and
    `._agen` -> `_aiter_msgs`.
- add `hide_tb: bool` support to `Channel.send()` and only show the
  frame on non-MTEs.
2024-07-02 12:21:26 -04:00
Tyler Goodlet 3907cba68e Refine some `.trionics` docs and logging
- allow passing and report the lib name (`trio` or `tractor`) from
  `maybe_open_nursery()`.
- use `.runtime()` level when reporting `_Cache`-hits in
  `maybe_open_context()`.
- tidy up some doc strings.
2024-06-28 19:28:12 -04:00
Tyler Goodlet e3d59964af Woops, set `.cancel()` level in custom levels table.. 2024-06-28 19:27:13 -04:00
Tyler Goodlet ba83bab776 Todo a test for sync-pausing from non-main-root-tasks 2024-06-28 19:26:35 -04:00
Tyler Goodlet 18d440c207 (Re)type annot some tests
- For the (still not finished) `test_caps_based_msging`, switch to
  using the new `PayloadMsg`.
- add `testdir` fixture type.
2024-06-28 19:24:03 -04:00
Tyler Goodlet edac717613 Use `msgspec.Struct.__repr__()` failover impl
In case the struct doesn't import a field type (which will cause the
`.pformat()` to raise) just report the issue and try to fall back to the
original `repr()` version.
2024-06-28 19:17:05 -04:00
Tyler Goodlet 7e93b81a83 Don't use pretty struct stuff in `._invoke`
It's too fragile to put in side core RPC machinery since
`msgspec.Struct` defs can fail if a field type can't be
looked up at creation time (like can easily happen if you
conditionally import using `if TYPE_CHECKING:`)

Also,
- rename `cs` to `rpc_ctx_cs: CancelScope` since it's literally
  the wrapping RPC `Context._scope`.
- report self cancellation via `explain: str` and add tail case for
  "unknown cause".
- put a ?TODO? around what to do about KBIs if a context is opened
  from an `infected_aio`-actor task.
- similar to our nursery and portal add TODO list for moving all
  `_invoke_non_context()` content out the RPC core and instead implement
  them as `.hilevel` endpoint helpers (maybe as decorators?)which under
  neath define `@context`-funcs.
2024-06-28 19:06:17 -04:00
Tyler Goodlet 4fbd469c33 Update `._entry` actor status log
Log-report the different types of actor exit conditions including cancel
via KBI, error or normal return with varying levels depending on case.

Also, start proto-ing out this weird ascii-syntax idea for describing
conc system states and implement the first bit in a `nest_from_op()`
log-message fmter that joins and indents an obj `repr()` with
a tree-like `'>)\n|_'` header.
2024-06-28 18:45:52 -04:00
Tyler Goodlet cb90f3e6ba Update `MsgTypeError` content matching to latest 2024-06-28 14:46:29 -04:00
Tyler Goodlet 5e009a8229 Further formalize `greenback` integration
Since we more or less require it for `tractor.pause_from_sync()` this
refines enable toggles and their relay down the actor tree as well as
more explicit logging around init and activation.

Tweaks summary:
- `.info()` report the module if discovered during root boot.
- use a `._state._runtime_vars['use_greenback']: bool` activation flag
  inside `Actor._from_parent()` to determine if the sub should try to
  use it and set to `False` if mod-loading fails / not installed.
- expose `maybe_init_greenback()` from `.devx` sugpkg.
- comment out RTE in `._pause()` for now since we already have it in
  `.pause_from_sync()`.
- always `.exception()` on `maybe_init_greenback()` import errors to
  clarify the underlying failure deats.
- always explicitly report if `._state._runtime_vars['use_greenback']`
  was NOT set when `.pause_from_sync()` is called.

Other `._runtime.async_main()` adjustments:
- combine the "internal error call ur parents" message and the failed
  registry contact status into one new `err_report: str`.
- drop the final exception handler's call to
  `Actor.lifetime_stack.close()` since we're already doing it in the
  `finally:` block and the earlier call has no currently known benefit.
- only report on the `.lifetime_stack()` callbacks if any are detected
  as registered.
2024-06-28 14:45:45 -04:00
Tyler Goodlet b72a025d0f Always reset `._state._ctxvar_Context` to prior
Not sure how I forgot this but, obviously it's correct context-var
semantics to revert the current IPC `Context` (set in the latest
`.open_context()` block) such that any prior instance is reset..

This ensures the sanity `assert`s pass inside
`.msg._ops.maybe_limit_plds()` and just in general ensures for any task
that the last opened `Context` is the one returned from
`current_ipc_ctx()`.
2024-06-28 12:59:31 -04:00
Tyler Goodlet 5739e79645 Use `delay=0` in pump loop..
Turns out it does work XD

Prior presumption was from before I had the fute poll-loop so makes
sense we needed more then one sched-tick's worth of context switch vs.
now we can just keep looping-n-pumping as fast possible until the
guest-run's main task completes.

Also,
- minimize the preface commentary (as per todo) now that we have tests
  codifying all the edge cases :finger_crossed:
- parameter-ize the pump-loop-cycle delay and default it to 0.
2024-06-27 19:27:59 -04:00
Tyler Goodlet 2ac999cc3c Prep for legacy RPC API factor-n-remove
This change is adding commentary about the upcoming API removal and
simplification of nursery + portal internals; no actual code changes are
included.

The plan to (re)move the old RPC methods:
- `ActorNursery.run_in_actor()`
- `Portal.run()`
- `Portal.run_from_ns()`

and any related impl internals out of each conc-primitive and instead
into something like a `.hilevel.rpc` set of APIs which then are all
implemented using the newer and more lowlevel `Context`/`MsgStream`
primitives instead Bo

Further,
- formally deprecate the `Portal.result()` meth for
  `.wait_for_result()`.
- only `log.info()` about runtime shutdown in the implicit root case.
2024-06-27 16:25:46 -04:00
Tyler Goodlet 9f9b0b17dc Add a `Context.portal`, more cancel tooing
Might as well add a public maybe-getter for use on the "parent" side
since it can be handy to check out-of-band cancellation conditions (like
from `Portal.cancel_actor()`).

Buncha bitty tweaks for more easily debugging cancel conditions:
- add a `@.cancel_called.setter` for hooking into `.cancel_called = True`
  being set in hard to decipher "who cancelled us" scenarios.
- use a new `self_ctxc: bool` var in `.cancel()` to capture the output
  state from `._is_self_cancelled(remote_error)` at call time so it can
  be compared against the measured value at crash-time (when REPL-ing it
  can often have already changed due to runtime teardown sequencing vs.
  the crash handler hook entry).
- proxy `hide_tb` to `.drain_to_final_msg()` from `.wait_for_result()`.
- use `remote_error.sender` attr directly instead of through
  `RAE.msgdata: dict` lookup.
- change var name `our_uid` -> `peer_uid`; it's not "ours"..

Other various docs/comment updates:
- extend the main class doc to include some other name ideas.
- change over all remaining `.result()` refs to `.wait_for_result()`.
- doc more details on how we want `.outcome` to eventually signature.
2024-06-26 16:26:18 -04:00
Tyler Goodlet 9133f42b07 Solve our abandonment issues..
To make the recent set of tests pass this (hopefully) finally solves all
`asyncio` embedded `trio` guest-run abandonment by ensuring we "pump the
event loop" until the guest-run future is fully complete.

Accomplished via simple poll loop of the form `while not
trio_done_fut.done(): await asyncio.sleep(.1)` in the `aio_main()`
task's exception teardown sequence. The loop does a naive 10ms
"pump-via-sleep & poll" for the `trio` side to complete before finally
exiting (and presumably raising) from the SIGINT cancellation.

Other related cleanups and refinements:
- use `asyncio.Task.result()` inside `cancel_trio()` since it also
  inline-raises any exception outcome and we can also log-report the
  result in non-error cases.
- comment out buncha not-sure-we-need-it stuff in `cancel_trio()`.
- remove the botched `AsyncioCancelled(CancelledError):` idea obvi XD
- comment `greenback` init for now in `aio_main()` since (pretty sure)
  we don't ever want to actually REPL in that specific func-as-task?
- always capture any `fute_err: BaseException` from the `main_outcome:
  Outcome` delivered by the `trio` side guest-run task.
- add and raise a new super noisy `AsyncioRuntimeTranslationError`
  whenever we detect that the guest-run `trio_done_fut` has not
  completed before task exit; should avoid abandonment issues ever
  happening again without knowing!
2024-06-26 13:48:36 -04:00
Tyler Goodlet 268bd0d8ec Demo-abandonment on shielded `trio`-side work
Finally this reproduces the issue as it (originally?) exhibited inside
`piker` where the `Actor.lifetime_stack` wasn't closed in cases where
during `infected_aio`-actor cancellation/shutdown `trio` side tasks
which are doing shielded (teardown) work are NOT being watched/waited on
from the `aio_main()` task-closure inside `run_as_asyncio_guest()`!

This is then the root cause of the guest-run being abandoned since if
our `aio_main()` task-closure doesn't know it should allow the run to
finish, it's going to call `loop.close()` eventually resulting in the
`GeneratorExit` thrown into `trio._core._run.unrolled_run()`..

So, this extends the `test_sigint_closes_lifetime_stack()` suite to
include cases for such shielded `trio`-task ops:
- add a new `trio_side_is_shielded: bool` which will toggle whether to
  add a shielded 0.5s `trio.sleep()` loop to `manage_file()` which
  should outlive the `asyncio` event-loop shutdown sequence and result
  in an abandoned guest-run and thus a leaked file.
- parametrize the existing suite with this case resulting in a total 16
  test set B)

This patch demonstrates the problem with our `aio_main()` task-closure
impl via the now 4 failing tests, a fix is coming in a follow up commit!
2024-06-26 12:01:36 -04:00
Tyler Goodlet 4f1db1ff52 Lel, revert `AsyncioCancelled` inherit, module..
Turns out it somehow breaks our `to_asyncio` error relay since obvi
`asyncio`'s runtime seems to specially handle it (prolly via
`isinstance()` ?) and it caused our
`test_aio_cancelled_from_aio_causes_trio_cancelled()` to hang..
Further, obvi `unpack_error()` won't be able to find the type def if not
kept inside `._exceptions`..

So given all that, revert the change/move as well as:
- tweak the aio-from-aio cancel test to timeout.
- do `trio.sleep()` conc with any bg aio task by moving out nursery
  block.
- add a `send_sigint_to: str` parameter to
  `test_sigint_closes_lifetime_stack()` such that we test the SIGINT
  being relayed to just the parent or the child.
2024-06-25 23:47:14 -04:00
Tyler Goodlet a870df68c0 Hack `asyncio` to not abandon a guest-mode run?
Took me a while to figure out what the heck was going on but, turns out
`asyncio` changed their SIGINT handling in 3.11 as per:

https://docs.python.org/3/library/asyncio-runner.html#handling-keyboard-interruption

I'm not entirely sure if it's the 3.11 changes or possibly wtv further
updates were made in 3.12  but more or less due to the way
our current main task was written the `trio` guest-run was getting
abandoned on SIGINTs sent from the OS to the infected child proc..

Note that much of the bug and soln cases are layed out in very detailed
comment-notes both in the new test and `run_as_asyncio_guest()`, right
above the final "fix" lines.

Add new `test_infected_aio.test_sigint_closes_lifetime_stack()` test suite
which reliably triggers all abandonment issues with multiple cases
of different parent behaviour post-sending-SIGINT-to-child:
 1. briefly sleep then raise a KBI in the parent which was originally
    demonstrating the file leak not being cleaned up by `Actor.lifetime_stack.close()`
    and simulates a ctl-c from the console (relayed in tandem by
    the OS to the parent and child processes).
 2. do `Context.wait_for_result()` on the child context which would
    hang and timeout since the actor runtime would never complete and
    thus never relay a `ContextCancelled`.
 3. both with and without running a `asyncio` task in the `manage_file`
    child actor; originally it seemed that with an aio task scheduled in
    the child actor the guest-run abandonment always was the "loud" case
    where there seemed to be some actor teardown but with tbs from
    python failing to gracefully exit the `trio` runtime..

The (seemingly working) "fix" required 2 lines of code to be run inside
a `asyncio.CancelledError` handler around the call to `await trio_done_fut`:
- `Actor.cancel_soon()` which schedules the actor runtime to cancel on
  the next `trio` runner cycle and results in a "self cancellation" of
  the actor.
- "pumping the `asyncio` event loop" with a non-0 `.sleep(0.1)` XD
 |_ seems that a "shielded" pump with some actual `delay: float >= 0`
   did the trick to get `asyncio` to allow the `trio` runner/loop to
   fully complete its guest-run without abandonment.

Other supporting changes:
- move `._exceptions.AsyncioCancelled`, our renamed
  `asyncio.CancelledError` error-sub-type-wrapper, to `.to_asyncio` and make
  it derive from `CancelledError` so as to be sure when raised by our
  `asyncio` x-> `trio` exception relay machinery that `asyncio` is
  getting the specific type it expects during cancellation.
- do "summary status" style logging in `run_as_asyncio_guest()` wherein
  we compile the eventual `startup_msg: str` emitted just before waiting
  on the `trio_done_fut`.
- shield-wait with `out: Outcome = await asyncio.shield(trio_done_fut)`
  even though it seems to do nothing in the SIGINT handling case..(I
  presume it might help avoid abandonment in a `asyncio.Task.cancel()`
  case maybe?)
2024-06-24 16:10:23 -04:00
Tyler Goodlet 3d12a7e005 Flip `infected_asyncio` status msg to `.runtime()` 2024-06-18 18:14:58 -04:00
Tyler Goodlet 9292d73b40 Avoid actor-nursery-exit warns on registrees
Since a local-actor-nursery-parented subactor might also use the root as
its registry, we need to avoid warning when short lived IPC `Channel`
connections establish and then disconnect (quickly, bc the apparently
the subactor isn't re-using an already cached parente-peer<->child conn
as you'd expect efficiency..) since such cases currently considered
normal operation of our super shoddy/naive "discovery sys" XD

As such, (un)guard the whole local-actor-nursery OR channel-draining
waiting blocks with the additional `or Actor._cancel_called` branch
since really we should also be waiting on the parent nurse to exit (at
least, for sure and always) when the local `Actor` indeed has been
"globally" cancelled-called. Further add separate timeout warnings for
channel-draining vs. local-actor-nursery-exit waiting since they are
technically orthogonal cases (at least, afaik).

Also,
- adjust the `Actor._stream_handler()` connection status log-emit to
  `.runtime()`, especially to reduce noise around the aforementioned
  ephemeral registree connection-requests.
- if we do wait on a local actor-nurse to exit, report its `._children`
  table (which should help figure out going forward how useful the
  warning is, if at all).
2024-06-18 16:10:36 -04:00
Tyler Goodlet 83d69fe395 Change `_Cache` reuse emit to `.runtime()` 2024-06-18 14:40:26 -04:00
Tyler Goodlet 72df312e71 Expand `PayloadMsg` doc-str 2024-06-18 09:57:10 -04:00
Tyler Goodlet 711f639fc5 Break `_mk_msg_type_err()` into recv/send side funcs
Name them `_mk_send_mte()`/`_mk_recv_mte()` and change the runtime to
call each appropriately depending on location/usage.

Also add some dynamic call-frame "unhide" blocks such that when we
expect raised MTE from the aboves calls but we get a different
unexpected error from the runtime, we ensure the call stack downward is
shown in tbs/pdb.
|_ ideally in the longer run we come up with a fancier dynamic sys for
   this, prolly something in `.devx._frame_stack`?
2024-06-17 13:12:16 -04:00
Tyler Goodlet 8477919fc9 Don't pass `ipc_msg` for send side MTEs
Just pass `_bad_msg` such that it get's injected to `.msgdata` since
with a send-side `MsgTypeError` we don't have a remote `._ipc_msg:
Error` per say to include.
2024-06-17 10:32:50 -04:00
Tyler Goodlet 872feef24b Add note about using `@acm` as decorator in 3.10 2024-06-17 10:32:38 -04:00
Tyler Goodlet affc210033 Update pld-rx limiting test(s) to use deco input
The tests only use one input spec (conveniently) so there's not much to
change in the logic,
- only pass the `maybe_msg_spec` to the child-side decorator and obvi
  drop the surrounding `msgops.limit_plds()` block in the child.
- tweak a few `MsgDec` asserts, mostly dropping the
  `msg._ops._def_any_spec` state checks since the child-side won't have
  any pre pld-spec state given the runtime now applies the `pld_spec`
  before running the task's func body.
  - also allowed dropping the `finally:` which did a similar check
    outside the `.limit_plds()` block.
2024-06-17 09:24:03 -04:00
Tyler Goodlet 04bd111037 Proxy through `dec_hook` in `.limit_plds()` APIs 2024-06-17 09:23:31 -04:00
Tyler Goodlet a0ee0cc713 Port debug request ep to use `@context(pld_spec)`
Namely passing the `.__pld_spec__` directly to the
`lock_stdio_for_peer()` decorator B)

Also, allows dropping `apply_debug_pldec()` (which was a todo) and
removing a `lock_stdio_for_peer()` indent level.
2024-06-17 09:01:13 -04:00
Tyler Goodlet 5449bd5673 Offer a `@context(pld_spec=<TypeAlias>)` API
Instead of the WIP/prototyped `Portal.open_context()` offering
a `pld_spec` input arg, this changes to a proper decorator API for
specifying the "payload spec" on `@context` endpoints.

The impl change details actually cover 2-birds:
- monkey patch decorated functions with a new
  `._tractor_context_meta: dict[str, Any]` and insert any provided input
  `@context` kwargs: `_pld_spec`, `enc_hook`, `enc_hook`.
- use `inspect.get_annotations()` to scan for a `func` arg
  type-annotated with `tractor.Context` and use the name of that arg as
  the RPC task-side injected `Context`, thus injecting the needed arg
  by type instead of by name (a longstanding TODO); raise a type-error
  when not found.
- pull the `pld_spec` from the `._tractor_context_meta` attr both in the
  `.open_context()` parent-side and child-side `._invoke()`-cation of
  the RPC task and use the `msg._ops.maybe_limit_plds()` API to apply it
  internally in the runtime for each case.
2024-06-16 23:12:53 -04:00
Tyler Goodlet e6d4ec43b9 Log tbs from non-RAE `._invoke()`-RPC-task errors
`RemoteActorError`s show this by default in their `.__repr__()`, and we
obvi capture and embed the src traceback in an `Error` msg prior to
transit, but for logging it's also handy to see the tb of any set
`Context._remote_error` on console especially when trying to decipher
remote error details at their origin actor. Also improve the log message
description using `ctx.repr_state` and show any `ctx.outcome`.
2024-06-14 15:49:30 -04:00
Tyler Goodlet 418c6907fd Add `enable_stack_on_sig: bool` for `stackscope` toggle 2024-06-14 15:37:57 -04:00
Tyler Goodlet d528e7ab4d Add `@context(pld_spec=<TypeAlias>)` TODO list
Longer run we don't want `tractor` app devs having to call
`msg._ops.limit_plds()` from every child endpoint.. so this starts
a list of decorator API ideas and obviously ties in with an ideal final
API design that will come with py3.13 and typed funcs. Obviously this
is directly fueled by,

- https://github.com/goodboy/tractor/issues/365

Other,
- type with direct `trio.lowlevel.Task` import.
- use `log.exception()` to show tbs for all error-terminations in
  `.open_context()` (for now) and always explicitly mention the `.side`.
2024-06-14 15:37:07 -04:00
Tyler Goodlet 7a89b59a3f Bleh, make `log.devx()` level less then cancel but > `.runtime()` 2024-06-11 20:45:41 -04:00
Tyler Goodlet 7d4cd8944c Use `_debug._sync_pause_from_builtin()` as `breakpoint()` override 2024-06-10 19:16:21 -04:00
Tyler Goodlet a6058d14ae Use new `._debug._repl_fail_msg` inside `test_pause_from_sync` 2024-06-10 17:57:43 -04:00
Tyler Goodlet 43a8cf4be1 Make big TODO: for `devx._debug` refinements
Hopefully would make grok-ing this fairly sophisticated sub-sys possible
for any up-and-coming `tractor` hacker

XD

A lot of internal API and re-org ideas I discovered/realized as part of
finishing the `__pld_spec__` and multi-threaded support. Particularly
better isolation between root-actor vs subactor task APIs and generally
less globally-state-ful stuff like `DebugStatus` and `Lock` method APIs
would likely make a lot of the hard to follow edge cases more clear?
2024-06-10 17:46:10 -04:00
Tyler Goodlet 6534a363a5 First proto: multi-threaded synced `pdb`-REPLs
Functionally working for multi-threaded (via cpython threads spawned
from `to_trio.to_thread.run_sync()`) alongside subactors, tested (for
now) only with threads started inside the root actor (which seemed to
have the most issues in terms of the impl and special cases..) using the
new `tractor.pause_from_sync()` API!

Main implementation changes to `.pause_from_sync()`
------ - ------
- from the root actor, we need to ensure bg thread case is handled
  *specially* since no IPC is used to request the TTY stdio mutex and
  `Lock` (API) usage is conducted entirely from a local task or thread;
  dedicated `Lock` usage for the root-actor already is branched inside
  `._pause()` and needs similar handling from a root bg-thread:
 |_for the special case of a root bg thread we need to
   `trio`-main-thread schedule a bg task inside a new
   `_pause_from_bg_root_thread()`. The new task needs to implement most
   of what was is handled inside `._pause()` manually, mostly because in
   this root-actor-bg-thread case we have 2 constraints:
   1. to enter `PdbREPL.interaction()` **from the bg thread** directly,
   2. the task that `Lock._debug_lock.acquire()`s has to be the same
      that calls `.release() (a `trio.FIFOLock` constraint)
 |_impl deats of this `_pause_from_bg_root_thread()` include:
   - (for now) calling `._pause()` to acquire the `Lock._debug_lock`.
   - setting its own `DebugStatus.repl_release`.
   - calling `.DebugStatus.shield_sigint()` to ensure the root's
     main thread  uses the right handler when the bg one is REPL-ing.
   - wait manually on the `.repl_release()` to be set by the thread's
     dedicated `PdbREPL` exit.
   - manually calling `Lock.release()` from the **same task** that
     acquired it.
- expect calls to `._pause()` to deliver a `tuple[Task, PdbREPL]` such
  that we always get the handle both to any newly created REPl instance
  and the (maybe) the scheduled bg task within which is runs.
- add a single `message: str` style to `log.devx()` based on branching
  style for logging.
- ensure both `DebugStatus.repl` and `.repl_task` are set **just
  before** calling `._set_trace()` to ensure the correct `Task|Thread`
  is set when the REPL is finally entered from sync code.
- add a wrapping caller `_sync_pause_from_builtin()` which passes in the
  new `called_from_builtin=True` to indicate `breakpoint()` caller
  usage, obvi pass in `api_frame`.

Changes to `._pause()` in support of ^
------ - ------
- `TaskStatus.started()` and return the `tuple[Task, PdbREPL]` to
  callers / starters.
- only call `DebugStatus.shield_sigint()` when no `repl` passed bc some
  callers (like bg threads) may need to apply it at some specific point
  themselves.
- tweak some asserts for the `debug_func == None` / non-`trio`-thread
  case.
- add a mod-level `_repl_fail_msg: str` to be used when there's an
  internal `._pause()` failure for testing, easier to pexpect match.
- more comprehensive logging for the root-actor branched case to
  (attempt to) indicate any of the 3 cases:
  - remote ctx from subactor has the `Lock`,
  - already existing root task or thread has it or,
  - some kinda stale `.locked()` situation where the root has the lock
    but we don't know why.
- for root usage, revert to always `await Lock._debug_lock.acquire()`-ing
  despite `called_from_sync` since `.pause_from_sync()` was reworked to
  instead handle the special bg thread case in the new
  `_pause_from_bg_root_thread()` task.
- always do `return _enter_repl_sync(debug_func)`.
- try to report any `repl_task: Task|Thread` set by the caller
  (particularly for the bg thread cases) as being the thread or task
  `._pause()` was called "on behalf of"

Changes to `DebugStatus`/`Lock` in support of ^
------ - ------
- only call `Lock.release()` from `DebugStatus.set_[quit/continue]()`
  when called from the main `trio` thread and always call
  `DebugStatus.release()` **after** to ensure `.repl_released()` is set
  **after** `._debug_lock.release()`.
- only call `.repl_release.set()` from `trio` thread otherwise use
  `.from_thread.run()`.
- much more refinements in `Lock.release()` for threading cases:
  - return `bool` to indicate whether lock was released by caller.
  - mask (in prep to drop) `_pause()` usage of
    `Lock.release.force=True)` since forcing a release can't ever avoid
    the RTE from `trio`.. same task **must** acquire/release.
  - don't allow usage from non-`trio`-main-threads, ever; there's no
    point since the same-task-needs-to-manage-`FIFOLock` constraint.
  - much more detailed logging using `message`-building-style for all
    caller (edge) cases.
   |_ use a `we_released: bool` to determine failed-to-release edge
      cases which can happen if called from bg threads, ensure we
      `log.exception()` on any incorrect usage resulting in  release
      failure.
   |_ complain loudly if the release fails and some other task/thread
      still holds the lock.
   |_ be explicit about "who" (which task or thread) the release is "on
      behalf of" by reading `DebugStatus.repl_task` since the caller
      isn't the REPL operator in many sync cases.
  - more or less drop `force` support, as mentioned above.
  - ensure we unset `._owned_by_root` if the caller is a root task.

Other misc
------ - ------
- rename `lock_tty_for_child()` -> `lock_stdio_for_peer()`.
- rejig `Lock.repr()` to show lock and event stats.
- stage `Lock.stats` and `.owner` methods in prep for doing a singleton
  instance and `@property`s.
2024-06-10 17:45:52 -04:00
Tyler Goodlet 30d60379c1 Drop thread logging to make `log.pdb()` patts match in test 2024-06-07 22:35:59 -04:00
Tyler Goodlet 408a74784e Catch `.pause_from_sync()` in root bg thread bugs!
Originally discovered as while using `tractor.pause_from_sync()`
from the `i3ipc` client running in a bg-thread that uses `asyncio`
inside `modden`.

Turns out we definitely aren't correctly handling `.pause_from_sync()`
from the root actor when called from a `trio.to_thread.run_sync()`
bg thread:
- root-actor bg threads which can't `Lock._debug_lock.acquire()` since
  they aren't in `trio.Task`s.
- even if scheduled via `.to_thread.run_sync(_debug._pause)` the
  acquirer won't be the task/thread which calls `Lock.release()` from
  `PdbREPL` hooks; this results in a RTE raised by `trio`..
- multiple threads will step on each other's stdio since cpython's GIL
  seems to ctx switch threads on every input from the user to the REPL
  loop..

Reproduce via reworking our example and test so that they catch and fail
for all edge cases:
- rework the `/examples/debugging/sync_bp.py` example to demonstrate the
  above issues, namely the stdio clobbering in the REPL when multiple
  threads and/or a subactor try to debug simultaneously.
  |_ run one thread using a task nursery to ensure it runs conc with the
     nursery's parent task.
  |_ ensure the bg threads run conc a subactor usage of
     `.pause_from_sync()`.
  |_ gravely detail all the special cases inside a TODO comment.
  |_ add some control flags to `sync_pause()` helper and don't use
     `breakpoint()` by default.
- extend and adjust `test_debugger.test_pause_from_sync` to match (and
  thus currently fail) by ensuring exclusive `PdbREPL` attachment when
  the 2 bg root-actor threads are concurrently interacting alongside the
  subactor:
  |_ should only see one of the `_pause_msg` logs at a time for either
     one of the threads or the subactor.
  |_ ensure each attaches (in no particular order) before expecting the
     script to exit.

Impl adjustments to `.devx._debug`:
- drop `Lock.repl`, no longer used.
- add `Lock._owned_by_root: bool` for the `.ctx_in_debug == None`
  root-actor-task active case.
- always `log.exception()` for any `._debug_lock.release()` ownership
  RTE emitted by `trio`, like we used to..
- add special `Lock.release()` log message for the stale lock but
  `._owned_by_root == True` case; oh yeah and actually
  `log.devx(message)`..
- rename `Lock.acquire()` -> `.acquire_for_ctx()` since it's only ever
  used from subactor IPC usage; well that and for local root-task
  usage we should prolly add a `.acquire_from_root_task()`?
- buncha `._pause()` impl improvements:
 |_ type `._pause()`'s `debug_func` as a `partial` as well.
 |_ offer `called_from_sync: bool` and `called_from_bg_thread: bool`
    for the special case handling when called from `.pause_from_sync()`
 |_ only set `DebugStatus.repl/repl_task` when `debug_func != None`
   (OW ensure the `.repl_task` is not the current one).
 |_ handle error logging even when `debug_func is None`..
 |_ lotsa detailed commentary around root-actor-bg-thread special cases.
- when `._set_trace(hide_tb=False)` do `pdbp.set_trace(frame=currentframe())`
  so the `._debug` internal frames are always included.
- by default always hide tracebacks for `.pause[_from_sync]()` internals.
- improve `.pause_from_sync()` to avoid root-bg-thread crashes:
 |_ pass new `called_from_xxx_` flags and ensure `DebugStatus.repl_task`
    is actually set to the `threading.current_thread()` when needed.
 |_ manually call `Lock._debug_lock.acquire_nowait()` for the non-bg
    thread case.
 |_ TODO: still need to implement the bg-thread case using a bg
    `trio.Task`-in-thread with an `trio.Event` set by thread REPL exit.
2024-06-06 16:56:30 -04:00
Tyler Goodlet f0342d6ae3 Move `Context.open_stream()` impl to `._streaming`
Exactly like how it's organized for `Portal.open_context()`, put the
main streaming API `@acm` with the `MsgStream` code and bind the method
to the new module func.

Other,
- rename `Context.result()` -> `.wait_for_result()` to better match the
  blocking semantics and rebind `.result()` as deprecated.
- add doc-str for `Context.maybe_raise()`.
2024-05-31 17:32:11 -04:00
Tyler Goodlet 21f633a900 Use `Context` repr APIs for RPC outcome logs
Delegate to the new `.repr_state: str` and adjust log level based on
error vs. cancel vs. result.
2024-05-31 14:40:55 -04:00
Tyler Goodlet 4a270f85ca Drop sub-decoder proto-cruft from `.msg._codec`
It ended up getting necessarily implemented as the `PldRx` though at
a different layer and won't be needed as part of `MsgCodec` most likely,
though this original idea did provide the source of inspiration for how
things work now!

Also Move the commented TODO proto for a codec hook factory from
`.types` to `._codec` where it prolly better fits and update some msg
related todo/questions.
2024-05-31 12:03:18 -04:00
Tyler Goodlet d802c8aa90 Woops, set `post_mortem=False` by default again! 2024-05-30 18:33:25 -04:00
Tyler Goodlet 8ea0f08386 Finally, officially support shielded REPL-ing!
It's been a long time prepped and now finally implemented!

Offer a `shield: bool` argument from our async `._debug` APIs:
- `await tractor.pause(shield=True)`,
- `await tractor.post_mortem(shield=True)`

^-These-^ can now be used inside cancelled `trio.CancelScope`s,
something very handy when introspecting complex (distributed) system
tear/shut-downs particularly under remote error or (inter-peer)
cancellation conditions B)

Thanks to previous prepping in a prior attempt and various patches from
the rigorous rework of `.devx._debug` internals around typed msg specs,
there ain't much that was needed!

Impl deats
- obvi passthrough `shield` from the public API endpoints (was already
  done from a prior attempt).
- put ad-hoc internal `with trio.CancelScope(shield=shield):` around all
  checkpoints inside `._pause()` for both the root-process and subactor
  case branches.

Add a fairly rigorous example, `examples/debugging/shielded_pause.py`
with a wrapping `pexpect` test, `test_debugger.test_shield_pause()` and
ensure it covers as many cases as i can think of offhand:

- multiple `.pause()` entries in a loop despite parent scope
  cancellation in a subactor RPC task which itself spawns a sub-task.
- a `trio.Nursery.parent_task` which raises, is handled and
  tries to enter and unshielded `.post_mortem()`, which of course
  internally raises `Cancelled` in a `._pause()` checkpoint, so we catch
  the `Cancelled` again and then debug the debugger's internal
  cancellation with specific checks for the particular raising
  checkpoint-LOC.
- do ^- the latter -^ for both subactor and root cases to ensure we
  can debug `._pause()` itself when it tries to REPL engage from
  a cancelled task scope Bo
2024-05-30 17:52:24 -04:00
Tyler Goodlet 13ea500a44 Rename `PldRx.dec_msg()` -> `.decode_pld()`
Keep the old alias, but i think it's better form to use longer names for
internal public APIs and this name better reflects the functionality:
decoding and returning a `PayloadMsg.pld` field.
2024-05-30 16:09:59 -04:00