Compare commits

..

23 Commits

Author SHA1 Message Date
Tyler Goodlet e696caf810 Mega-refactor on `._invoke()` targeting `@context`s
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!
2024-03-02 19:26:40 -05:00
Tyler Goodlet ed10632d97 Avoid `ctx.cancel()` after ctxc rxed in `.open_context()`
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 :)
2024-03-02 17:18:55 -05:00
Tyler Goodlet 299429a278 Deep `Context` refinements
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!
2024-03-01 22:37:32 -05:00
Tyler Goodlet 28fefe4ffe Make stream draining status logs `.debug()` level 2024-03-01 19:27:10 -05:00
Tyler Goodlet 08a6a51cb8 Add `._implicit_runtime_started` mark, better logs
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.
2024-03-01 15:44:01 -05:00
Tyler Goodlet 50465d4b34 Spawn naming and log format tweaks
- 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: `'|_'`
2024-03-01 11:37:23 -05:00
Tyler Goodlet 4f69af872c Add field-first subproca `.info()` to `._entry` 2024-02-29 20:01:39 -05:00
Tyler Goodlet 9bc6a61c93 Add "fancier" remote-error `.__repr__()`-ing
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`
2024-02-29 18:56:31 -05:00
Tyler Goodlet 23aa97692e Fix `Channel.__repr__()` safety, renames to `._transport`
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)
2024-02-29 18:37:04 -05:00
Tyler Goodlet 1e5810e56c Make `NamespacePath` kinda support methods..
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
2024-02-29 17:37:02 -05:00
Tyler Goodlet b54cb6682c Add #TODO for generating func-sig type-annots as `str` for pprinting 2024-02-29 17:21:43 -05:00
Tyler Goodlet 3ed309f019 Add test for `modden` sub-spawner-server hangs on cancel
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.
2024-02-29 15:45:55 -05:00
Tyler Goodlet d08aeaeafe Make `@context`-cancelled tests more pedantic
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`.
2024-02-28 19:25:27 -05:00
Tyler Goodlet c6ee4e5dc1 Add a `pytest.ini` config 2024-02-22 20:37:12 -05:00
Tyler Goodlet ad5eee5666 WIP final impl of ctx-cancellation-semantics 2024-02-22 18:33:18 -05:00
Tyler Goodlet fc72d75061 Support `maybe_wait_for_debugger(header_msg: str)`
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.
2024-02-22 15:08:10 -05:00
Tyler Goodlet de1843dc84 Few more log msg tweaks in runtime 2024-02-22 15:06:39 -05:00
Tyler Goodlet 930d498841 Call `actor.cancel(None)` from root to avoid mismatch with (any future) meth sig changes 2024-02-22 14:45:08 -05:00
Tyler Goodlet 5ea112699d Tweak broadcast fanout test to never inf loop
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.
2024-02-22 14:41:28 -05:00
Tyler Goodlet e244747bc3 Add note that maybe `Context._eoc` should be set by caller? 2024-02-22 14:22:45 -05:00
Tyler Goodlet 5a09ccf459 Tweak `Actor` cancel method signatures
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()`.
2024-02-22 14:22:08 -05:00
Tyler Goodlet ce1bcf6d36 Fix overruns test to avoid return-beats-ctxc race
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`.
2024-02-21 13:53:25 -05:00
Tyler Goodlet 28ba5e5435 Add `pformat()` of `ActorNursery._children` to logging
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.
2024-02-21 13:21:28 -05:00
18 changed files with 3010 additions and 1311 deletions

8
pytest.ini 100644
View File

@ -0,0 +1,8 @@
# vim: ft=ini
# pytest.ini for tractor
[pytest]
# don't show frickin captured logs AGAIN in the report..
addopts = --show-capture='no'
log_cli = false
; minversion = 6.0

View File

@ -298,44 +298,69 @@ async def inf_streamer(
async with (
ctx.open_stream() as stream,
trio.open_nursery() as n,
trio.open_nursery() as tn,
):
async def bail_on_sentinel():
async def close_stream_on_sentinel():
async for msg in stream:
if msg == 'done':
print(
'streamer RXed "done" sentinel msg!\n'
'CLOSING `MsgStream`!'
)
await stream.aclose()
else:
print(f'streamer received {msg}')
else:
print('streamer exited recv loop')
# start termination detector
n.start_soon(bail_on_sentinel)
tn.start_soon(close_stream_on_sentinel)
for val in itertools.count():
cap: int = 10000 # so that we don't spin forever when bug..
for val in range(cap):
try:
print(f'streamer sending {val}')
await stream.send(val)
except trio.ClosedResourceError:
if val > cap:
raise RuntimeError(
'Streamer never cancelled by setinel?'
)
await trio.sleep(0.001)
# close out the stream gracefully
except trio.ClosedResourceError:
print('msgstream closed on streamer side!')
assert stream.closed
break
else:
raise RuntimeError(
'Streamer not cancelled before finished sending?'
)
print('terminating streamer')
print('streamer exited .open_streamer() block')
def test_local_task_fanout_from_stream():
def test_local_task_fanout_from_stream(
debug_mode: bool,
):
'''
Single stream with multiple local consumer tasks using the
``MsgStream.subscribe()` api.
Ensure all tasks receive all values after stream completes sending.
Ensure all tasks receive all values after stream completes
sending.
'''
consumers = 22
consumers: int = 22
async def main():
counts = Counter()
async with tractor.open_nursery() as tn:
p = await tn.start_actor(
async with tractor.open_nursery(
debug_mode=debug_mode,
) as tn:
p: tractor.Portal = await tn.start_actor(
'inf_streamer',
enable_modules=[__name__],
)
@ -343,7 +368,6 @@ def test_local_task_fanout_from_stream():
p.open_context(inf_streamer) as (ctx, _),
ctx.open_stream() as stream,
):
async def pull_and_count(name: str):
# name = trio.lowlevel.current_task().name
async with stream.subscribe() as recver:
@ -352,7 +376,7 @@ def test_local_task_fanout_from_stream():
tractor.trionics.BroadcastReceiver
)
async for val in recver:
# print(f'{name}: {val}')
print(f'bx {name} rx: {val}')
counts[name] += 1
print(f'{name} bcaster ended')
@ -362,10 +386,14 @@ def test_local_task_fanout_from_stream():
with trio.fail_after(3):
async with trio.open_nursery() as nurse:
for i in range(consumers):
nurse.start_soon(pull_and_count, i)
nurse.start_soon(
pull_and_count,
i,
)
# delay to let bcast consumers pull msgs
await trio.sleep(0.5)
print('\nterminating')
print('terminating nursery of bcast rxer consumers!')
await stream.send('done')
print('closed stream connection')

View File

@ -48,11 +48,13 @@ async def do_nuthin():
ids=['no_args', 'unexpected_args'],
)
def test_remote_error(reg_addr, args_err):
"""Verify an error raised in a subactor that is propagated
'''
Verify an error raised in a subactor that is propagated
to the parent nursery, contains the underlying boxed builtin
error type info and causes cancellation and reraising all the
way up the stack.
"""
'''
args, errtype = args_err
async def main():
@ -65,7 +67,9 @@ def test_remote_error(reg_addr, args_err):
# an exception group outside the nursery since the error
# here and the far end task error are one in the same?
portal = await nursery.run_in_actor(
assert_err, name='errorer', **args
assert_err,
name='errorer',
**args
)
# get result(s) from main task

View File

@ -5,9 +5,10 @@ Verify the we raise errors when streams are opened prior to
sync-opening a ``tractor.Context`` beforehand.
'''
# from contextlib import asynccontextmanager as acm
from contextlib import asynccontextmanager as acm
from itertools import count
import platform
from pprint import pformat
from typing import (
Callable,
)
@ -249,6 +250,17 @@ def test_simple_context(
trio.run(main)
@acm
async def expect_ctxc(yay: bool) -> None:
if yay:
try:
yield
except ContextCancelled:
return
else:
yield
@pytest.mark.parametrize(
'callee_returns_early',
[True, False],
@ -279,23 +291,60 @@ def test_caller_cancels(
async def check_canceller(
ctx: Context,
) -> None:
# should not raise yet return the remote
# context cancelled error.
actor: Actor = current_actor()
uid: tuple = actor.uid
if (
cancel_method == 'portal'
and not callee_returns_early
):
try:
res = await ctx.result()
assert 0, 'Portal cancel should raise!'
except ContextCancelled as ctxc:
assert ctx.chan._cancel_called
assert ctxc.canceller == uid
assert ctxc is ctx.maybe_error
# NOTE: should not ever raise even in the `ctx`
# case since self-cancellation should swallow the ctxc
# silently!
else:
res = await ctx.result()
# we actually get a result
if callee_returns_early:
assert res == 'yo'
assert ctx.outcome is res
assert ctx.maybe_error is None
else:
err = res
err: Exception = ctx.outcome
assert isinstance(err, ContextCancelled)
assert (
tuple(err.canceller)
==
current_actor().uid
uid
)
assert (
err
is ctx.maybe_error
is ctx._remote_error
)
if le := ctx._local_error:
assert err is le
# else:
# TODO: what should this be then?
# not defined until block closes right?
#
# await tractor.pause()
# assert ctx._local_error is None
async def main():
async with tractor.open_nursery(
debug_mode=debug_mode,
) as an:
@ -305,11 +354,16 @@ def test_caller_cancels(
)
timeout = 0.5 if not callee_returns_early else 2
with trio.fail_after(timeout):
async with portal.open_context(
async with (
expect_ctxc(yay=cancel_method == 'portal'),
portal.open_context(
simple_setup_teardown,
data=10,
block_forever=not callee_returns_early,
) as (ctx, sent):
) as (ctx, sent),
):
if callee_returns_early:
# ensure we block long enough before sending
@ -331,6 +385,16 @@ def test_caller_cancels(
if cancel_method != 'portal':
await portal.cancel_actor()
# since the `.cancel_actor()` call just above
# will cause the `.open_context().__aexit__()` raise
# a ctxc which should in turn cause `ctx._scope` to
# catch any cancellation?
if (
not callee_returns_early
and cancel_method == 'portal'
):
assert ctx._scope.cancelled_caught
trio.run(main)
@ -433,7 +497,6 @@ async def test_callee_closes_ctx_after_stream_open(
@tractor.context
async def expect_cancelled(
ctx: Context,
) -> None:
@ -453,7 +516,7 @@ async def expect_cancelled(
raise
else:
assert 0, "Wasn't cancelled!?"
assert 0, "callee wasn't cancelled !?"
@pytest.mark.parametrize(
@ -472,8 +535,8 @@ async def test_caller_closes_ctx_after_callee_opens_stream(
async with tractor.open_nursery(
debug_mode=debug_mode,
) as an:
root: Actor = current_actor()
root: Actor = current_actor()
portal = await an.start_actor(
'ctx_cancelled',
enable_modules=[__name__],
@ -486,11 +549,13 @@ async def test_caller_closes_ctx_after_callee_opens_stream(
await portal.run(assert_state, value=True)
# call cancel explicitly
# call `ctx.cancel()` explicitly
if use_ctx_cancel_method:
await ctx.cancel()
# NOTE: means the local side `ctx._scope` will
# have been cancelled by an ctxc ack and thus
# `._scope.cancelled_caught` should be set.
try:
async with ctx.open_stream() as stream:
async for msg in stream:
@ -519,20 +584,35 @@ async def test_caller_closes_ctx_after_callee_opens_stream(
assert portal.channel.connected()
# ctx is closed here
await portal.run(assert_state, value=False)
await portal.run(
assert_state,
value=False,
)
else:
try:
with trio.fail_after(0.2):
await ctx.result()
assert 0, "Callee should have blocked!?"
except trio.TooSlowError:
# NO-OP -> since already called above
await ctx.cancel()
# local scope should have absorbed the cancellation
assert ctx.cancelled_caught
assert ctx._remote_error is ctx._local_error
# NOTE: local scope should have absorbed the cancellation since
# in this case we call `ctx.cancel()` and the local
# `._scope` gets `.cancel_called` on the ctxc ack.
if use_ctx_cancel_method:
assert ctx._scope.cancelled_caught
# rxed ctxc response from far end
assert ctx.cancel_acked
assert (
ctx._remote_error
is ctx._local_error
is ctx.maybe_error
is ctx.outcome
)
try:
async with ctx.open_stream() as stream:
@ -815,7 +895,10 @@ async def echo_back_sequence(
# NOTE: ensure that if the caller is expecting to cancel this task
# that we stay echoing much longer then they are so we don't
# return early instead of receive the cancel msg.
total_batches: int = 1000 if wait_for_cancel else 6
total_batches: int = (
1000 if wait_for_cancel
else 6
)
await ctx.started()
# await tractor.breakpoint()
@ -834,8 +917,23 @@ async def echo_back_sequence(
)
seq = list(seq) # bleh, msgpack sometimes ain't decoded right
for _ in range(total_batches):
for i in range(total_batches):
print(f'starting new stream batch {i} iter in child')
batch = []
# EoC case, delay a little instead of hot
# iter-stopping (since apparently py3.11+ can do that
# faster then a ctxc can be sent) on the async for
# loop when child was requested to ctxc.
if (
stream.closed
or
ctx.cancel_called
):
print('child stream already closed!?!')
await trio.sleep(0.05)
continue
async for msg in stream:
batch.append(msg)
if batch == seq:
@ -846,15 +944,18 @@ async def echo_back_sequence(
print('callee waiting on next')
print(f'callee echoing back latest batch\n{batch}')
for msg in batch:
print(f'callee sending {msg}')
print(f'callee sending msg\n{msg}')
await stream.send(msg)
print(
'EXITING CALLEEE:\n'
f'{ctx.canceller}'
)
try:
return 'yo'
finally:
print(
'exiting callee with context:\n'
f'{pformat(ctx)}\n'
)
@pytest.mark.parametrize(
@ -916,8 +1017,8 @@ def test_maybe_allow_overruns_stream(
wait_for_cancel=cancel_ctx,
be_slow=(slow_side == 'child'),
allow_overruns_side=allow_overruns_side,
) as (ctx, sent):
) as (ctx, sent):
assert sent is None
async with ctx.open_stream(
@ -945,10 +1046,10 @@ def test_maybe_allow_overruns_stream(
if cancel_ctx:
# cancel the remote task
print('sending root side cancel')
print('Requesting `ctx.cancel()` in parent!')
await ctx.cancel()
res = await ctx.result()
res: str|ContextCancelled = await ctx.result()
if cancel_ctx:
assert isinstance(res, ContextCancelled)

View File

@ -10,6 +10,9 @@ import pytest
import trio
import tractor
from tractor import ( # typing
Actor,
current_actor,
open_nursery,
Portal,
Context,
ContextCancelled,
@ -238,19 +241,23 @@ async def stream_from_peer(
# caller peer should not be the cancel requester
assert not ctx.cancel_called
assert not ctx.cancel_acked
# XXX can never be true since `._invoke` only
# XXX can NEVER BE TRUE since `._invoke` only
# sets this AFTER the nursery block this task
# was started in, exits.
assert not ctx.cancelled_caught
assert not ctx._scope.cancelled_caught
# we never requested cancellation
# we never requested cancellation, it was the 'canceller'
# peer.
assert not peer_ctx.cancel_called
assert not peer_ctx.cancel_acked
# the `.open_context()` exit definitely caught
# a cancellation in the internal `Context._scope` since
# likely the runtime called `_deliver_msg()` after
# receiving the remote error from the streaming task.
assert peer_ctx.cancelled_caught
assert not peer_ctx._scope.cancelled_caught
# TODO / NOTE `.canceller` won't have been set yet
# here because that machinery is inside
@ -259,6 +266,8 @@ async def stream_from_peer(
# checkpoint) that cancellation was due to
# a remote, we COULD assert this here..see,
# https://github.com/goodboy/tractor/issues/368
#
# assert 'canceller' in ctx.canceller
# root/parent actor task should NEVER HAVE cancelled us!
assert not ctx.canceller
@ -356,8 +365,7 @@ def test_peer_canceller(
'just_caller', # but i just met her?
enable_modules=[__name__],
)
root = tractor.current_actor()
root: Actor = current_actor()
try:
async with (
@ -395,8 +403,8 @@ def test_peer_canceller(
# not request the sleeper cancellation ;)
except ContextCancelled as ctxerr:
print(
'CAUGHT REMOTE CONTEXT CANCEL FOM\n'
f'{ctxerr}'
'CAUGHT REMOTE CONTEXT CANCEL\n\n'
f'{ctxerr}\n'
)
# canceller and caller peers should not
@ -404,6 +412,9 @@ def test_peer_canceller(
assert canceller_ctx.canceller is None
assert caller_ctx.canceller is None
# we were not the actor, our peer was
assert not sleeper_ctx.cancel_acked
assert ctxerr.canceller[0] == 'canceller'
# XXX NOTE XXX: since THIS `ContextCancelled`
@ -411,11 +422,13 @@ def test_peer_canceller(
# `sleeper.open_context().__aexit__()` this
# value is not yet set, however outside this
# block it should be.
assert not sleeper_ctx.cancelled_caught
assert not sleeper_ctx._scope.cancelled_caught
# CASE_1: error-during-ctxc-handling,
if error_during_ctxerr_handling:
raise RuntimeError('Simulated error during teardown')
# CASE_2: standard teardown inside in `.open_context()` block
raise
# XXX SHOULD NEVER EVER GET HERE XXX
@ -436,7 +449,6 @@ def test_peer_canceller(
else:
pytest.fail(
'did not rx ctxc ?!?\n\n'
f'{ctxs}\n'
)
@ -447,21 +459,48 @@ def test_peer_canceller(
_loc_err = loc_err
# NOTE: the main state to check on `Context` is:
# - `.cancelled_caught` (maps to nursery cs)
# - `.cancel_called` (bool of whether this side
# requested)
# - `.cancel_acked` (bool of whether a ctxc
# response was received due to cancel req).
# - `.maybe_error` (highest prio error to raise
# locally)
# - `.outcome` (final error or result value)
# - `.canceller` (uid of cancel-causing actor-task)
# - `._remote_error` (any `RemoteActorError`
# instance from other side of context)
# - `._local_error` (any error caught inside the
# `.open_context()` block).
#
# XXX: Deprecated and internal only
# - `.cancelled_caught` (maps to nursery cs)
# - now just use `._scope.cancelled_caught`
# since it maps to the internal (maps to nursery cs)
#
# TODO: are we really planning to use this tho?
# - `._cancel_msg` (any msg that caused the
# cancel)
# CASE: error raised during handling of
# `ContextCancelled` inside `.open_context()`
# block
# CASE_1: error-during-ctxc-handling,
# - far end cancels due to peer 'canceller',
# - `ContextCancelled` relayed to this scope,
# - inside `.open_context()` ctxc is caught and
# a rte raised instead
#
# => block should raise the rte but all peers
# should be cancelled by US.
#
if error_during_ctxerr_handling:
assert isinstance(loc_err, RuntimeError)
print(f'_loc_err: {_loc_err}\n')
# assert sleeper_ctx._local_error is _loc_err
# assert sleeper_ctx._local_error is _loc_err
assert not (
loc_err
is sleeper_ctx.maybe_error
is sleeper_ctx.outcome
is sleeper_ctx._remote_error
)
# NOTE: this root actor task should have
# called `Context.cancel()` on the
@ -495,7 +534,25 @@ def test_peer_canceller(
root.uid
)
# CASE: standard teardown inside in `.open_context()` block
# since the sleeper errors while handling a
# peer-cancelled (by ctxc) scenario, we expect
# that the `.open_context()` block DOES call
# `.cancel() (despite in this test case it
# being unecessary).
assert (
sleeper_ctx.cancel_called
and
not sleeper_ctx.cancel_acked
)
# CASE_2: standard teardown inside in `.open_context()` block
# - far end cancels due to peer 'canceller',
# - `ContextCancelled` relayed to this scope and
# raised locally without any raise-during-handle,
#
# => inside `.open_context()` ctxc is raised and
# propagated
#
else:
assert isinstance(loc_err, ContextCancelled)
assert loc_err.canceller == sleeper_ctx.canceller
@ -509,24 +566,42 @@ def test_peer_canceller(
# the sleeper's remote error is the error bubbled
# out of the context-stack above!
re = sleeper_ctx._remote_error
assert re is loc_err
re = sleeper_ctx.outcome
assert (
re is loc_err
is sleeper_ctx.maybe_error
is sleeper_ctx._remote_error
)
for ctx in ctxs:
re: BaseException|None = ctx._remote_error
assert re
re: BaseException|None = ctx.outcome
assert (
re and
(
re is ctx.maybe_error
is ctx._remote_error
)
)
le: trio.MultiError = ctx._local_error
assert (
le
and ctx._local_error
)
# root doesn't cancel sleeper since it's
# cancelled by its peer.
if ctx is sleeper_ctx:
assert not ctx.cancel_called
assert not ctx.cancel_acked
# since sleeper_ctx.result() IS called
# above we should have (silently)
# absorbed the corresponding
# `ContextCancelled` for it and thus
# the logic inside `.cancelled_caught`
# should trigger!
assert ctx.cancelled_caught
assert ctx._scope.cancelled_caught
elif ctx is caller_ctx:
# since its context was remotely
@ -535,15 +610,33 @@ def test_peer_canceller(
# done by the peer and also we never
assert ctx.cancel_called
# TODO: figure out the details of
# this..
# TODO: figure out the details of this..?
# if you look the `._local_error` here
# is a multi of ctxc + 2 Cancelleds?
# assert not ctx.cancelled_caught
elif ctx is canceller_ctx:
# XXX NOTE XXX: ONLY the canceller
# will get a self-cancelled outcome
# whilst everyone else gets
# a peer-caused cancellation!
#
# TODO: really we should avoid calling
# .cancel() whenever an interpeer
# cancel takes place since each
# reception of a ctxc
assert (
ctx.cancel_called
and ctx.cancel_acked
)
assert not ctx._scope.cancelled_caught
else:
assert ctx.cancel_called
assert not ctx.cancelled_caught
pytest.fail(
'Uhh wut ctx is this?\n'
f'{ctx}\n'
)
# TODO: do we even need this flag?
# -> each context should have received
@ -559,14 +652,24 @@ def test_peer_canceller(
# `Context.cancel()` SHOULD NOT have been
# called inside
# `Portal.open_context().__aexit__()`.
assert not sleeper_ctx.cancel_called
assert not (
sleeper_ctx.cancel_called
or
sleeper_ctx.cancel_acked
)
# XXX NOTE XXX: and see matching comment above but,
# this flag is set only AFTER the `.open_context()`
# has exited and should be set in both outcomes
# including the case where ctx-cancel handling
# itself errors.
assert sleeper_ctx.cancelled_caught
# the `._scope` is only set by `trio` AFTER the
# `.open_context()` block has exited and should be
# set in both outcomes including the case where
# ctx-cancel handling itself errors.
assert sleeper_ctx._scope.cancelled_caught
assert _loc_err is sleeper_ctx._local_error
assert (
sleeper_ctx.outcome
is sleeper_ctx.maybe_error
is sleeper_ctx._remote_error
)
raise # always to ensure teardown
@ -582,12 +685,315 @@ def test_peer_canceller(
assert excinfo.value.canceller[0] == 'canceller'
def test_client_tree_spawns_and_cancels_service_subactor():
...
# TODO: test for the modden `mod wks open piker` bug!
@tractor.context
async def basic_echo_server(
ctx: Context,
peer_name: str = 'stepbro',
) -> None:
'''
Just the simplest `MsgStream` echo server which resays what
you told it but with its uid in front ;)
'''
actor: Actor = tractor.current_actor()
uid: tuple = actor.uid
await ctx.started(uid)
async with ctx.open_stream() as ipc:
async for msg in ipc:
# repack msg pair with our uid
# as first element.
(
client_uid,
i,
) = msg
resp: tuple = (
uid,
i,
)
# OOF! looks like my runtime-error is causing a lockup
# assert 0
await ipc.send(resp)
@tractor.context
async def serve_subactors(
ctx: Context,
peer_name: str,
) -> None:
async with open_nursery() as an:
await ctx.started(peer_name)
async with ctx.open_stream() as reqs:
async for msg in reqs:
peer_name: str = msg
peer: Portal = await an.start_actor(
name=peer_name,
enable_modules=[__name__],
)
print(
'Spawning new subactor\n'
f'{peer_name}\n'
f'|_{peer}\n'
)
await reqs.send((
peer.chan.uid,
peer.chan.raddr,
))
print('Spawner exiting spawn serve loop!')
@tractor.context
async def client_req_subactor(
ctx: Context,
peer_name: str,
# used to simulate a user causing an error to be raised
# directly in thread (like a KBI) to better replicate the
# case where a `modden` CLI client would hang afer requesting
# a `Context.cancel()` to `bigd`'s wks spawner.
reraise_on_cancel: str|None = None,
) -> None:
# TODO: other cases to do with sub lifetimes:
# -[ ] test that we can have the server spawn a sub
# that lives longer then ctx with this client.
# -[ ] test that
# open ctx with peer spawn server and ask it to spawn a little
# bro which we'll then connect and stream with.
async with (
tractor.find_actor(
name='spawn_server',
raise_on_none=True,
# TODO: we should be isolating this from other runs!
# => ideally so we can eventually use something like
# `pytest-xdist` Bo
# registry_addrs=bigd._reg_addrs,
) as spawner,
spawner.open_context(
serve_subactors,
peer_name=peer_name,
) as (spawner_ctx, first),
):
assert first == peer_name
await ctx.started(
'yup i had brudder',
)
async with spawner_ctx.open_stream() as reqs:
# send single spawn request to the server
await reqs.send(peer_name)
with trio.fail_after(3):
(
sub_uid,
sub_raddr,
) = await reqs.receive()
await tell_little_bro(
actor_name=sub_uid[0],
caller='client',
)
# TODO: test different scope-layers of
# cancellation?
# with trio.CancelScope() as cs:
try:
await trio.sleep_forever()
# TODO: would be super nice to have a special injected
# cancel type here (maybe just our ctxc) but using
# some native mechanism in `trio` :p
except (
trio.Cancelled
) as err:
_err = err
if reraise_on_cancel:
errtype = globals()['__builtins__'][reraise_on_cancel]
assert errtype
to_reraise: BaseException = errtype()
print(f'client re-raising on cancel: {repr(to_reraise)}')
raise err
raise
# if cs.cancelled_caught:
# print('client handling expected KBI!')
# await ctx.
# await trio.sleep(
# await tractor.pause()
# await spawner_ctx.cancel()
# cancel spawned sub-actor directly?
# await sub_ctx.cancel()
# maybe cancel runtime?
# await sub.cancel_actor()
async def tell_little_bro(
actor_name: str,
caller: str = ''
):
# contact target actor, do a stream dialog.
async with (
tractor.wait_for_actor(
name=actor_name
) as lb,
lb.open_context(
basic_echo_server,
) as (sub_ctx, first),
sub_ctx.open_stream(
basic_echo_server,
) as echo_ipc,
):
actor: Actor = current_actor()
uid: tuple = actor.uid
for i in range(100):
msg: tuple = (
uid,
i,
)
await echo_ipc.send(msg)
resp = await echo_ipc.receive()
print(
f'{caller} => {actor_name}: {msg}\n'
f'{caller} <= {actor_name}: {resp}\n'
)
(
sub_uid,
_i,
) = resp
assert sub_uid != uid
assert _i == i
@pytest.mark.parametrize(
'raise_client_error',
[None, 'KeyboardInterrupt'],
)
def test_peer_spawns_and_cancels_service_subactor(
debug_mode: bool,
raise_client_error: str,
):
# NOTE: this tests for the modden `mod wks open piker` bug
# discovered as part of implementing workspace ctx
# open-.pause()-ctx.cancel() as part of the CLI..
# -> start actor-tree (server) that offers sub-actor spawns via
# context API
# -> start another full actor-tree (client) which requests to the first to
# spawn over its `@context` ep / api.
# -> client actor cancels the context and should exit gracefully
# and the server's spawned child should cancel and terminate!
peer_name: str = 'little_bro'
async def main():
async with tractor.open_nursery(
# NOTE: to halt the peer tasks on ctxc, uncomment this.
debug_mode=debug_mode,
) as an:
server: Portal = await an.start_actor(
(server_name := 'spawn_server'),
enable_modules=[__name__],
)
print(f'Spawned `{server_name}`')
client: Portal = await an.start_actor(
client_name := 'client',
enable_modules=[__name__],
)
print(f'Spawned `{client_name}`')
try:
async with (
server.open_context(
serve_subactors,
peer_name=peer_name,
) as (spawn_ctx, first),
client.open_context(
client_req_subactor,
peer_name=peer_name,
reraise_on_cancel=raise_client_error,
) as (client_ctx, client_says),
):
print(
f'Server says: {first}\n'
f'Client says: {client_says}\n'
)
# attach to client-requested-to-spawn
# (grandchild of this root actor) "little_bro"
# and ensure we can also use it as an echo
# server.
async with tractor.wait_for_actor(
name=peer_name,
) as sub:
assert sub
print(
'Sub-spawn came online\n'
f'portal: {sub}\n'
f'.uid: {sub.actor.uid}\n'
f'chan.raddr: {sub.chan.raddr}\n'
)
await tell_little_bro(
actor_name=peer_name,
caller='root',
)
# signal client to raise a KBI
await client_ctx.cancel()
print('root cancelled client, checking that sub-spawn is down')
async with tractor.find_actor(
name=peer_name,
) as sub:
assert not sub
print('root cancelling server/client sub-actors')
# await tractor.pause()
res = await client_ctx.result(hide_tb=False)
assert isinstance(res, ContextCancelled)
assert client_ctx.cancel_acked
assert res.canceller == current_actor().uid
await spawn_ctx.cancel()
# await server.cancel_actor()
# since we called `.cancel_actor()`, `.cancel_ack`
# will not be set on the ctx bc `ctx.cancel()` was not
# called directly fot this confext.
except ContextCancelled as ctxc:
print('caught ctxc from contexts!')
assert ctxc.canceller == current_actor().uid
assert ctxc is spawn_ctx.outcome
assert ctxc is spawn_ctx.maybe_error
raise
# assert spawn_ctx.cancel_acked
assert spawn_ctx.cancel_acked
assert client_ctx.cancel_acked
await client.cancel_actor()
await server.cancel_actor()
# WOA WOA WOA! we need this to close..!!!??
# that's super bad XD
# TODO: why isn't this working!?!?
# we're now outside the `.open_context()` block so
# the internal `Context._scope: CancelScope` should be
# gracefully "closed" ;)
# assert spawn_ctx.cancelled_caught
trio.run(main)

File diff suppressed because it is too large Load Diff

View File

@ -106,25 +106,25 @@ def _trio_main(
Entry point for a `trio_run_in_process` subactor.
'''
log.info(f"Started new trio process for {actor.uid}")
if actor.loglevel is not None:
log.info(
f"Setting loglevel for {actor.uid} to {actor.loglevel}")
get_console_log(actor.loglevel)
log.info(
f"Started {actor.uid}")
_state._current_actor = actor
log.debug(f"parent_addr is {parent_addr}")
trio_main = partial(
async_main,
actor,
parent_addr=parent_addr
)
if actor.loglevel is not None:
get_console_log(actor.loglevel)
import os
log.info(
'Started new trio process:\n'
f'|_{actor}\n'
f' uid: {actor.uid}\n'
f' pid: {os.getpid()}\n'
f' parent_addr: {parent_addr}\n'
f' loglevel: {actor.loglevel}\n'
)
try:
if infect_asyncio:
actor._infected_aio = True
@ -133,7 +133,7 @@ def _trio_main(
trio.run(trio_main)
except KeyboardInterrupt:
log.cancel(
f'Actor@{actor.uid} received KBI'
f'@{actor.uid} received KBI'
)
finally:

View File

@ -27,6 +27,7 @@ from typing import (
Type,
TYPE_CHECKING,
)
import textwrap
import traceback
import exceptiongroup as eg
@ -37,8 +38,9 @@ from .log import get_logger
if TYPE_CHECKING:
from ._context import Context
from ._stream import MsgStream
from .log import StackLevelAdapter
from ._stream import MsgStream
from ._ipc import Channel
log = get_logger('tractor')
@ -49,6 +51,25 @@ class ActorFailure(Exception):
"General actor failure"
class InternalError(RuntimeError):
'''
Entirely unexpected internal machinery error indicating
a completely invalid state or interface.
'''
_body_fields: list[str] = [
'src_actor_uid',
'canceller',
'sender',
]
_msgdata_keys: list[str] = [
'type_str',
] + _body_fields
# TODO: rename to just `RemoteError`?
class RemoteActorError(Exception):
'''
@ -60,6 +81,10 @@ class RemoteActorError(Exception):
a special "error" IPC msg sent by some remote actor-runtime.
'''
reprol_fields: list[str] = [
'src_actor_uid',
]
def __init__(
self,
message: str,
@ -77,23 +102,82 @@ class RemoteActorError(Exception):
# - .remote_type
# also pertains to our long long oustanding issue XD
# https://github.com/goodboy/tractor/issues/5
self.type: str = suberror_type
self.boxed_type: str = suberror_type
self.msgdata: dict[str, Any] = msgdata
@property
def type(self) -> str:
return self.boxed_type
@property
def type_str(self) -> str:
return str(type(self.boxed_type).__name__)
@property
def src_actor_uid(self) -> tuple[str, str]|None:
return self.msgdata.get('src_actor_uid')
def __repr__(self) -> str:
@property
def tb_str(
self,
indent: str = ' '*3,
) -> str:
if remote_tb := self.msgdata.get('tb_str'):
pformat(remote_tb)
return (
f'{type(self).__name__}(\n'
f'msgdata={pformat(self.msgdata)}\n'
')'
return textwrap.indent(
remote_tb,
prefix=indent,
)
return super().__repr__()
return ''
def reprol(self) -> str:
'''
Represent this error for "one line" display, like in
a field of our `Context.__repr__()` output.
'''
_repr: str = f'{type(self).__name__}('
for key in self.reprol_fields:
val: Any|None = self.msgdata.get(key)
if val:
_repr += f'{key}={repr(val)} '
return _repr
def __repr__(self) -> str:
fields: str = ''
for key in _body_fields:
val: str|None = self.msgdata.get(key)
if val:
fields += f'{key}={val}\n'
fields: str = textwrap.indent(
fields,
# prefix=' '*2,
prefix=' |_',
)
indent: str = ''*1
body: str = (
f'{fields}'
f' |\n'
f' ------ - ------\n\n'
f'{self.tb_str}\n'
f' ------ - ------\n'
f' _|\n'
)
# f'|\n'
# f' |\n'
if indent:
body: str = textwrap.indent(
body,
prefix=indent,
)
return (
f'<{type(self).__name__}(\n'
f'{body}'
')>'
)
# TODO: local recontruction of remote exception deats
# def unbox(self) -> BaseException:
@ -102,8 +186,9 @@ class RemoteActorError(Exception):
class InternalActorError(RemoteActorError):
'''
Remote internal ``tractor`` error indicating
failure of some primitive or machinery.
(Remote) internal `tractor` error indicating failure of some
primitive, machinery state or lowlevel task that should never
occur.
'''
@ -114,6 +199,9 @@ class ContextCancelled(RemoteActorError):
``Portal.cancel_actor()`` or ``Context.cancel()``.
'''
reprol_fields: list[str] = [
'canceller',
]
@property
def canceller(self) -> tuple[str, str]|None:
'''
@ -145,6 +233,9 @@ class ContextCancelled(RemoteActorError):
f'{self}'
)
# to make `.__repr__()` work uniformly
# src_actor_uid = canceller
class TransportClosed(trio.ClosedResourceError):
"Underlying channel transport was closed prior to use"
@ -166,6 +257,9 @@ class StreamOverrun(
RemoteActorError,
trio.TooSlowError,
):
reprol_fields: list[str] = [
'sender',
]
'''
This stream was overrun by sender
@ -213,6 +307,7 @@ def pack_error(
] = {
'tb_str': tb_str,
'type_str': type(exc).__name__,
'boxed_type': type(exc).__name__,
'src_actor_uid': current_actor().uid,
}
@ -238,8 +333,8 @@ def unpack_error(
msg: dict[str, Any],
chan=None,
err_type=RemoteActorError,
chan: Channel|None = None,
box_type: RemoteActorError = RemoteActorError,
hide_tb: bool = True,
@ -264,12 +359,15 @@ def unpack_error(
# retrieve the remote error's msg encoded details
tb_str: str = error_dict.get('tb_str', '')
message: str = f'{chan.uid}\n' + tb_str
type_name: str = error_dict['type_str']
type_name: str = (
error_dict.get('type_str')
or error_dict['boxed_type']
)
suberror_type: Type[BaseException] = Exception
if type_name == 'ContextCancelled':
err_type = ContextCancelled
suberror_type = err_type
box_type = ContextCancelled
suberror_type = box_type
else: # try to lookup a suitable local error type
for ns in [
@ -285,7 +383,7 @@ def unpack_error(
):
break
exc = err_type(
exc = box_type(
message,
suberror_type=suberror_type,
@ -371,6 +469,8 @@ def _raise_from_no_key_in_msg(
) from None
# `MsgStream` termination msg.
# TODO: does it make more sense to pack
# the stream._eoc outside this in the calleer always?
elif (
msg.get('stop')
or (

View File

@ -30,7 +30,6 @@ import typing
from typing import (
Any,
runtime_checkable,
Optional,
Protocol,
Type,
TypeVar,
@ -113,6 +112,13 @@ class MsgpackTCPStream(MsgTransport):
using the ``msgspec`` codec lib.
'''
layer_key: int = 4
name_key: str = 'tcp'
# TODO: better naming for this?
# -[ ] check how libp2p does naming for such things?
codec_key: str = 'msgpack'
def __init__(
self,
stream: trio.SocketStream,
@ -268,7 +274,7 @@ class Channel:
def __init__(
self,
destaddr: Optional[tuple[str, int]],
destaddr: tuple[str, int]|None,
msg_transport_type_key: tuple[str, str] = ('msgpack', 'tcp'),
@ -286,14 +292,14 @@ class Channel:
# Either created in ``.connect()`` or passed in by
# user in ``.from_stream()``.
self._stream: Optional[trio.SocketStream] = None
self.msgstream: Optional[MsgTransport] = None
self._stream: trio.SocketStream|None = None
self._transport: MsgTransport|None = None
# set after handshake - always uid of far end
self.uid: Optional[tuple[str, str]] = None
self.uid: tuple[str, str]|None = None
self._agen = self._aiter_recv()
self._exc: Optional[Exception] = None # set if far end actor errors
self._exc: Exception|None = None # set if far end actor errors
self._closed: bool = False
# flag set by ``Portal.cancel_actor()`` indicating remote
@ -301,6 +307,15 @@ class Channel:
# runtime.
self._cancel_called: bool = False
@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,
@ -310,40 +325,44 @@ class Channel:
) -> Channel:
src, dst = get_stream_addrs(stream)
chan = Channel(destaddr=dst, **kwargs)
chan = Channel(
destaddr=dst,
**kwargs,
)
# set immediately here from provided instance
chan._stream = stream
chan._stream: trio.SocketStream = stream
chan.set_msg_transport(stream)
return chan
def set_msg_transport(
self,
stream: trio.SocketStream,
type_key: Optional[tuple[str, str]] = None,
type_key: tuple[str, str]|None = None,
) -> MsgTransport:
type_key = type_key or self._transport_key
self.msgstream = get_msg_transport(type_key)(stream)
return self.msgstream
self._transport = get_msg_transport(type_key)(stream)
return self._transport
def __repr__(self) -> str:
if self.msgstream:
if not self._transport:
return '<Channel with inactive transport?>'
return repr(
self.msgstream.stream.socket._sock
self._transport.stream.socket._sock
).replace( # type: ignore
"socket.socket",
"Channel",
)
return object.__repr__(self)
@property
def laddr(self) -> Optional[tuple[str, int]]:
return self.msgstream.laddr if self.msgstream else None
def laddr(self) -> tuple[str, int]|None:
return self._transport.laddr if self._transport else None
@property
def raddr(self) -> Optional[tuple[str, int]]:
return self.msgstream.raddr if self.msgstream else None
def raddr(self) -> tuple[str, int]|None:
return self._transport.raddr if self._transport else None
async def connect(
self,
@ -362,12 +381,12 @@ class Channel:
*destaddr,
**kwargs
)
msgstream = self.set_msg_transport(stream)
transport = self.set_msg_transport(stream)
log.transport(
f'Opened channel[{type(msgstream)}]: {self.laddr} -> {self.raddr}'
f'Opened channel[{type(transport)}]: {self.laddr} -> {self.raddr}'
)
return msgstream
return transport
async def send(self, item: Any) -> None:
@ -375,16 +394,16 @@ class Channel:
'=> send IPC msg:\n\n'
f'{pformat(item)}\n'
) # type: ignore
assert self.msgstream
assert self._transport
await self.msgstream.send(item)
await self._transport.send(item)
async def recv(self) -> Any:
assert self.msgstream
return await self.msgstream.recv()
assert self._transport
return await self._transport.recv()
# try:
# return await self.msgstream.recv()
# return await self._transport.recv()
# except trio.BrokenResourceError:
# if self._autorecon:
# await self._reconnect()
@ -397,8 +416,8 @@ class Channel:
f'Closing channel to {self.uid} '
f'{self.laddr} -> {self.raddr}'
)
assert self.msgstream
await self.msgstream.stream.aclose()
assert self._transport
await self._transport.stream.aclose()
self._closed = True
async def __aenter__(self):
@ -449,16 +468,16 @@ class Channel:
Async iterate items from underlying stream.
'''
assert self.msgstream
assert self._transport
while True:
try:
async for item in self.msgstream:
async for item in self._transport:
yield item
# sent = yield item
# if sent is not None:
# # optimization, passing None through all the
# # time is pointless
# await self.msgstream.send(sent)
# await self._transport.send(sent)
except trio.BrokenResourceError:
# if not self._autorecon:
@ -471,7 +490,7 @@ class Channel:
# continue
def connected(self) -> bool:
return self.msgstream.connected() if self.msgstream else False
return self._transport.connected() if self._transport else False
@asynccontextmanager

View File

@ -27,8 +27,9 @@ from __future__ import annotations
import importlib
import inspect
from typing import (
Any, Optional,
Callable, AsyncGenerator,
Any,
Callable,
AsyncGenerator,
Type,
)
from functools import partial
@ -52,6 +53,7 @@ from ._ipc import Channel
from .log import get_logger
from .msg import NamespacePath
from ._exceptions import (
InternalError,
_raise_from_no_key_in_msg,
unpack_error,
NoResult,
@ -69,18 +71,35 @@ from ._streaming import (
log = get_logger(__name__)
# TODO: rename to `unwrap_result()` and use
# `._raise_from_no_key_in_msg()` (after tweak to
# accept a `chan: Channel` arg) in key block!
def _unwrap_msg(
msg: dict[str, Any],
channel: Channel
channel: Channel,
hide_tb: bool = True,
) -> Any:
__tracebackhide__ = True
'''
Unwrap a final result from a `{return: <Any>}` IPC msg.
'''
__tracebackhide__: bool = hide_tb
try:
return msg['return']
except KeyError as ke:
# internal error should never get here
assert msg.get('cid'), "Received internal error at portal?"
raise unpack_error(msg, channel) from ke
assert msg.get('cid'), (
"Received internal error at portal?"
)
raise unpack_error(
msg,
channel
) from ke
class Portal:
@ -107,9 +126,9 @@ class Portal:
cancel_timeout: float = 0.5
def __init__(self, channel: Channel) -> None:
self.channel = channel
self.chan = channel
# during the portal's lifetime
self._result_msg: Optional[dict] = None
self._result_msg: dict|None = None
# When set to a ``Context`` (when _submit_for_result is called)
# it is expected that ``result()`` will be awaited at some
@ -118,6 +137,18 @@ class Portal:
self._streams: set[MsgStream] = set()
self.actor = current_actor()
@property
def channel(self) -> Channel:
'''
Proxy to legacy attr name..
Consider the shorter `Portal.chan` instead of `.channel` ;)
'''
log.debug(
'Consider the shorter `Portal.chan` instead of `.channel` ;)'
)
return self.chan
async def _submit_for_result(
self,
ns: str,
@ -125,14 +156,14 @@ class Portal:
**kwargs
) -> None:
assert self._expect_result is None, \
assert self._expect_result is None, (
"A pending main result has already been submitted"
)
self._expect_result = await self.actor.start_remote_task(
self.channel,
ns,
func,
kwargs
nsf=NamespacePath(f'{ns}:{func}'),
kwargs=kwargs
)
async def _return_once(
@ -142,7 +173,7 @@ class Portal:
) -> dict[str, Any]:
assert ctx._remote_func_type == 'asyncfunc' # single response
msg = await ctx._recv_chan.receive()
msg: dict = await ctx._recv_chan.receive()
return msg
async def result(self) -> Any:
@ -173,7 +204,10 @@ class Portal:
self._expect_result
)
return _unwrap_msg(self._result_msg, self.channel)
return _unwrap_msg(
self._result_msg,
self.channel,
)
async def _cancel_streams(self):
# terminate all locally running async generator
@ -215,26 +249,33 @@ class Portal:
purpose.
'''
if not self.channel.connected():
log.cancel("This channel is already closed can't cancel")
chan: Channel = self.channel
if not chan.connected():
log.runtime(
'This channel is already closed, skipping cancel request..'
)
return False
reminfo: str = (
f'{self.channel.uid}\n'
f' |_{chan}\n'
)
log.cancel(
f"Sending actor cancel request to {self.channel.uid} on "
f"{self.channel}")
self.channel._cancel_called = True
f'Sending runtime `.cancel()` request to peer\n\n'
f'{reminfo}'
)
self.channel._cancel_called: bool = True
try:
# send cancel cmd - might not get response
# XXX: sure would be nice to make this work with
# a proper shield
with trio.move_on_after(
timeout
or self.cancel_timeout
or
self.cancel_timeout
) as cs:
cs.shield = True
cs.shield: bool = True
await self.run_from_ns(
'self',
'cancel',
@ -242,7 +283,12 @@ class Portal:
return True
if cs.cancelled_caught:
log.cancel(f"May have failed to cancel {self.channel.uid}")
# may timeout and we never get an ack (obvi racy)
# but that doesn't mean it wasn't cancelled.
log.debug(
'May have failed to cancel peer?\n'
f'{reminfo}'
)
# if we get here some weird cancellation case happened
return False
@ -251,9 +297,11 @@ class Portal:
trio.ClosedResourceError,
trio.BrokenResourceError,
):
log.cancel(
f"{self.channel} for {self.channel.uid} was already "
"closed or broken?")
log.debug(
'IPC chan for actor already closed or broken?\n\n'
f'{self.channel.uid}\n'
f' |_{self.channel}\n'
)
return False
async def run_from_ns(
@ -274,25 +322,31 @@ class Portal:
A special namespace `self` can be used to invoke `Actor`
instance methods in the remote runtime. Currently this
should only be used solely for ``tractor`` runtime
internals.
should only ever be used for `Actor` (method) runtime
internals!
'''
nsf = NamespacePath(
f'{namespace_path}:{function_name}'
)
ctx = await self.actor.start_remote_task(
self.channel,
namespace_path,
function_name,
kwargs,
chan=self.channel,
nsf=nsf,
kwargs=kwargs,
)
ctx._portal = self
msg = await self._return_once(ctx)
return _unwrap_msg(msg, self.channel)
return _unwrap_msg(
msg,
self.channel,
)
async def run(
self,
func: str,
fn_name: Optional[str] = None,
fn_name: str|None = None,
**kwargs
) -> Any:
'''
Submit a remote function to be scheduled and run by actor, in
@ -311,8 +365,9 @@ class Portal:
DeprecationWarning,
stacklevel=2,
)
fn_mod_path = func
fn_mod_path: str = func
assert isinstance(fn_name, str)
nsf = NamespacePath(f'{fn_mod_path}:{fn_name}')
else: # function reference was passed directly
if (
@ -325,13 +380,12 @@ class Portal:
raise TypeError(
f'{func} must be a non-streaming async function!')
fn_mod_path, fn_name = NamespacePath.from_ref(func).to_tuple()
nsf = NamespacePath.from_ref(func)
ctx = await self.actor.start_remote_task(
self.channel,
fn_mod_path,
fn_name,
kwargs,
nsf=nsf,
kwargs=kwargs,
)
ctx._portal = self
return _unwrap_msg(
@ -355,15 +409,10 @@ class Portal:
raise TypeError(
f'{async_gen_func} must be an async generator function!')
fn_mod_path, fn_name = NamespacePath.from_ref(
async_gen_func
).to_tuple()
ctx = await self.actor.start_remote_task(
ctx: Context = await self.actor.start_remote_task(
self.channel,
fn_mod_path,
fn_name,
kwargs
nsf=NamespacePath.from_ref(async_gen_func),
kwargs=kwargs,
)
ctx._portal = self
@ -373,7 +422,8 @@ class Portal:
try:
# deliver receive only stream
async with MsgStream(
ctx, ctx._recv_chan,
ctx=ctx,
rx_chan=ctx._recv_chan,
) as rchan:
self._streams.add(rchan)
yield rchan
@ -400,12 +450,25 @@ class Portal:
# await recv_chan.aclose()
self._streams.remove(rchan)
# TODO: move this impl to `._context` mod and
# instead just bind it here as a method so that the logic
# for ctx stuff stays all in one place (instead of frickin
# having to open this file in tandem every gd time!!! XD)
#
@asynccontextmanager
async def open_context(
self,
func: Callable,
allow_overruns: bool = False,
# TODO: if we set this the wrapping `@acm` body will
# still be shown (awkwardly) on pdb REPL entry. Ideally
# we can similarly annotate that frame to NOT show?
hide_tb: bool = False,
# proxied to RPC
**kwargs,
) -> AsyncGenerator[tuple[Context, Any], None]:
@ -438,6 +501,8 @@ class Portal:
collection. See ``tractor.Context`` for more details.
'''
__tracebackhide__: bool = hide_tb
# conduct target func method structural checks
if not inspect.iscoroutinefunction(func) and (
getattr(func, '_tractor_contex_function', False)
@ -448,13 +513,12 @@ class Portal:
# TODO: i think from here onward should probably
# just be factored into an `@acm` inside a new
# a new `_context.py` mod.
fn_mod_path, fn_name = NamespacePath.from_ref(func).to_tuple()
nsf = NamespacePath.from_ref(func)
ctx = await self.actor.start_remote_task(
ctx: Context = await self.actor.start_remote_task(
self.channel,
fn_mod_path,
fn_name,
kwargs,
nsf=nsf,
kwargs=kwargs,
# NOTE: it's imporant to expose this since you might
# get the case where the parent who opened the context does
@ -492,8 +556,11 @@ class Portal:
# placeholder for any exception raised in the runtime
# or by user tasks which cause this context's closure.
scope_err: BaseException|None = None
ctxc_from_callee: ContextCancelled|None = None
try:
async with trio.open_nursery() as nurse:
# NOTE: used to start overrun queuing tasks
ctx._scope_nursery: trio.Nursery = nurse
ctx._scope: trio.CancelScope = nurse.cancel_scope
@ -501,14 +568,26 @@ class Portal:
# in enter tuple.
yield ctx, first
# between the caller exiting and arriving here the
# far end may have sent a ctxc-msg or other error,
# so check for it here immediately and maybe raise
# so as to engage the ctxc handling block below!
# ??TODO??: do we still want to consider this or is
# the `else:` block handling via a `.result()`
# call below enough??
# -[ ] pretty sure `.result()` internals do the
# same as our ctxc handler below so it ended up
# being same (repeated?) behaviour, but ideally we
# wouldn't have that duplication either by somehow
# factoring the `.result()` handler impl in a way
# that we can re-use it around the `yield` ^ here
# or vice versa?
#
# NOTE: between the caller exiting and arriving
# here the far end may have sent a ctxc-msg or
# other error, so check for it here immediately
# and maybe raise so as to engage the ctxc
# handling block below!
#
# if re := ctx._remote_error:
# maybe_ctxc: ContextCancelled|None = ctx._maybe_raise_remote_err(
# re,
# # TODO: do we want this to always raise?
# # - means that on self-ctxc, if/when the
# # block is exited before the msg arrives
@ -526,7 +605,7 @@ class Portal:
# # block?
# raise_ctxc_from_self_call=True,
# )
# assert maybe_ctxc
# ctxc_from_callee = maybe_ctxc
# when in allow_overruns mode there may be
# lingering overflow sender tasks remaining?
@ -538,13 +617,18 @@ class Portal:
not ctx._allow_overruns
or len(nurse.child_tasks) > 1
):
raise RuntimeError(
raise InternalError(
'Context has sub-tasks but is '
'not in `allow_overruns=True` mode!?'
)
# ensure cancel of all overflow sender tasks
# started in the ctx nursery.
# ensure we cancel all overflow sender
# tasks started in the nursery when
# `._allow_overruns == True`.
#
# NOTE: this means `._scope.cancelled_caught`
# will prolly be set! not sure if that's
# non-ideal or not ???
ctx._scope.cancel()
# XXX NOTE XXX: maybe shield against
@ -557,14 +641,15 @@ class Portal:
# of a `Context`. In both cases any `ContextCancelled`
# raised in this scope-block came from a transport msg
# relayed from some remote-actor-task which our runtime set
# as a `Context._remote_error`
# as to `Context._remote_error`
#
# the CASES:
#
# - if that context IS THE SAME ONE that called
# `Context.cancel()`, we want to absorb the error
# silently and let this `.open_context()` block to exit
# without raising.
# without raising, ideally eventually receiving the ctxc
# ack msg thus resulting in `ctx.cancel_acked == True`.
#
# - if it is from some OTHER context (we did NOT call
# `.cancel()`), we want to re-RAISE IT whilst also
@ -588,6 +673,7 @@ class Portal:
# `Nursery.cancel_scope.cancel()`)
except ContextCancelled as ctxc:
scope_err = ctxc
ctxc_from_callee = ctxc
# XXX TODO XXX: FIX THIS debug_mode BUGGGG!!!
# using this code and then resuming the REPL will
@ -597,6 +683,7 @@ class Portal:
# documenting it as a definittive example of
# debugging the tractor-runtime itself using it's
# own `.devx.` tooling!
#
# await pause()
# CASE 2: context was cancelled by local task calling
@ -604,15 +691,10 @@ class Portal:
# exit silently.
if (
ctx._cancel_called
and (
and
ctxc is ctx._remote_error
# ctxc.msgdata == ctx._remote_error.msgdata
# TODO: uhh `Portal.canceller` ain't a thangg
# dawg? (was `self.canceller` before?!?)
and
ctxc.canceller == self.actor.uid
)
):
log.cancel(
f'Context (cid=[{ctx.cid[-6:]}..] cancelled gracefully with:\n'
@ -620,9 +702,9 @@ class Portal:
)
# CASE 1: this context was never cancelled via a local
# task (tree) having called `Context.cancel()`, raise
# the error since it was caused by someone else!
# the error since it was caused by someone else
# -> probably a remote peer!
else:
# await pause()
raise
# the above `._scope` can be cancelled due to:
@ -635,19 +717,29 @@ class Portal:
# CASE 3: standard local error in this caller/yieldee
Exception,
# CASES 1 & 2: normally manifested as
# a `Context._scope_nursery` raised
# CASES 1 & 2: can manifest as a `ctx._scope_nursery`
# exception-group of,
#
# 1.-`trio.Cancelled`s, since
# `._scope.cancel()` will have been called and any
# `ContextCancelled` absorbed and thus NOT RAISED in
# any `Context._maybe_raise_remote_err()`,
# `._scope.cancel()` will have been called
# (transitively by the runtime calling
# `._deliver_msg()`) and any `ContextCancelled`
# eventually absorbed and thus absorbed/supressed in
# any `Context._maybe_raise_remote_err()` call.
#
# 2.-`BaseExceptionGroup[ContextCancelled | RemoteActorError]`
# from any error raised in the "callee" side with
# a group only raised if there was any more then one
# task started here in the "caller" in the
# `yield`-ed to task.
BaseExceptionGroup, # since overrun handler tasks may have been spawned
# from any error delivered from the "callee" side
# AND a group-exc is only raised if there was > 1
# tasks started *here* in the "caller" / opener
# block. If any one of those tasks calls
# `.result()` or `MsgStream.receive()`
# `._maybe_raise_remote_err()` will be transitively
# called and the remote error raised causing all
# tasks to be cancelled.
# NOTE: ^ this case always can happen if any
# overrun handler tasks were spawned!
BaseExceptionGroup,
trio.Cancelled, # NOTE: NOT from inside the ctx._scope
KeyboardInterrupt,
@ -657,32 +749,38 @@ class Portal:
# XXX: ALWAYS request the context to CANCEL ON any ERROR.
# NOTE: `Context.cancel()` is conversely NEVER CALLED in
# the `ContextCancelled` "self cancellation absorbed" case
# handled in the block above!
# handled in the block above ^^^ !!
log.cancel(
'Context cancelled for task due to\n'
'Context terminated due to\n\n'
f'{caller_err}\n'
'Sending cancel request..\n'
f'task:{cid}\n'
f'actor:{uid}'
)
if debug_mode():
log.pdb(
'Delaying `ctx.cancel()` until debug lock '
'acquired..'
)
# async with acquire_debug_lock(self.actor.uid):
# pass
# TODO: factor ^ into below for non-root cases?
await maybe_wait_for_debugger()
was_acquired: bool = await maybe_wait_for_debugger(
header_msg=(
'Delaying `ctx.cancel()` until debug lock '
'acquired..\n'
),
)
if was_acquired:
log.pdb(
'Acquired debug lock! '
'Calling `ctx.cancel()`!'
'Calling `ctx.cancel()`!\n'
)
# we don't need to cancel the callee if it already
# told us it's cancelled ;p
if ctxc_from_callee is None:
try:
await ctx.cancel()
except trio.BrokenResourceError:
except (
trio.BrokenResourceError,
trio.ClosedResourceError,
):
log.warning(
'IPC connection for context is broken?\n'
f'task:{cid}\n'
@ -693,38 +791,11 @@ class Portal:
# no local scope error, the "clean exit with a result" case.
else:
# between the caller exiting and arriving here the
# far end may have sent a ctxc-msg or other error,
# so check for it here immediately and maybe raise
# so as to engage the ctxc handling block below!
# if re := ctx._remote_error:
# maybe_ctxc: ContextCancelled|None = ctx._maybe_raise_remote_err(
# re,
# # TODO: do we want this to always raise?
# # - means that on self-ctxc, if/when the
# # block is exited before the msg arrives
# # but then the msg during __exit__
# # calling we may not activate the
# # ctxc-handler block below? should we
# # be?
# # - if there's a remote error that arrives
# # after the child has exited, we won't
# # handle until the `finally:` block
# # where `.result()` is always called,
# # again in which case we handle it
# # differently then in the handler block
# # that would normally engage from THIS
# # block?
# raise_ctxc_from_self_call=True,
# )
# assert maybe_ctxc
if ctx.chan.connected():
log.info(
'Waiting on final context-task result for\n'
f'task: {cid}\n'
f'actor: {uid}'
log.runtime(
'Waiting on final context result for\n'
f'peer: {uid}\n'
f'|_{ctx._task}\n'
)
# XXX NOTE XXX: the below call to
# `Context.result()` will ALWAYS raise
@ -749,16 +820,18 @@ class Portal:
scope_err = berr
raise
# yes! this worx Bp
# from .devx import _debug
# await _debug.pause()
# an exception type boxed in a `RemoteActorError`
# is returned (meaning it was obvi not raised).
# is returned (meaning it was obvi not raised)
# that we want to log-report on.
msgdata: str|None = getattr(
result_or_err,
'msgdata',
None
)
# yes! this worx Bp
# from .devx import _debug
# await _debug.pause()
match (msgdata, result_or_err):
case (
{'tb_str': tbstr},
@ -771,13 +844,19 @@ class Portal:
RemoteActorError(),
):
log.exception(
f'Context `{fn_name}` remotely errored:\n'
f'`{tbstr}`'
'Context remotely errored!\n'
f'<= peer: {uid}\n'
f' |_ {nsf}()\n\n'
f'{tbstr}'
)
case (None, _):
log.runtime(
f'Context {fn_name} returned value from callee:\n'
f'`{result_or_err}`'
'Context returned final result from callee task:\n'
f'<= peer: {uid}\n'
f' |_ {nsf}()\n\n'
f'`{result_or_err}`\n'
)
finally:
@ -795,9 +874,8 @@ class Portal:
# operating *in* this scope to have survived
# we tear down the runtime feeder chan last
# to avoid premature stream clobbers.
rxchan: trio.ReceiveChannel = ctx._recv_chan
if (
rxchan
(rxchan := ctx._recv_chan)
# maybe TODO: yes i know the below check is
# touching `trio` memchan internals..BUT, there are
@ -853,28 +931,38 @@ class Portal:
etype: Type[BaseException] = type(scope_err)
# CASE 2
if ctx._cancel_called:
if (
ctx._cancel_called
and ctx.cancel_acked
):
log.cancel(
f'Context {fn_name} cancelled by caller with\n'
f'{etype}'
'Context cancelled by caller task\n'
f'|_{ctx._task}\n\n'
f'{repr(scope_err)}\n'
)
# TODO: should we add a `._cancel_req_received`
# flag to determine if the callee manually called
# `ctx.cancel()`?
# -[ ] going to need a cid check no?
# CASE 1
else:
log.cancel(
f'Context cancelled by callee with {etype}\n'
f'target: `{fn_name}`\n'
f'task:{cid}\n'
f'actor:{uid}'
f'Context terminated due to local scope error:\n'
f'{etype.__name__}\n'
)
# FINALLY, remove the context from runtime tracking and
# exit!
log.runtime(
f'Exiting context opened with {ctx.chan.uid}'
'Removing IPC ctx opened with peer\n'
f'{uid}\n'
f'|_{ctx}\n'
)
self.actor._contexts.pop(
(self.channel.uid, ctx.cid),
(uid, cid),
None,
)
@ -911,7 +999,7 @@ class LocalPortal:
async def open_portal(
channel: Channel,
nursery: Optional[trio.Nursery] = None,
nursery: trio.Nursery|None = None,
start_msg_loop: bool = True,
shield: bool = False,
@ -936,7 +1024,7 @@ async def open_portal(
if channel.uid is None:
await actor._do_handshake(channel)
msg_loop_cs: Optional[trio.CancelScope] = None
msg_loop_cs: trio.CancelScope|None = None
if start_msg_loop:
from ._runtime import process_messages
msg_loop_cs = await nursery.start(

View File

@ -326,7 +326,7 @@ async def open_root_actor(
not entered
and not is_multi_cancelled(err)
):
logger.exception("Root actor crashed:")
logger.exception('Root actor crashed:\n')
# ALWAYS re-raise any error bubbled up from the
# runtime!
@ -343,9 +343,7 @@ async def open_root_actor(
# tempn.start_soon(an.exited.wait)
logger.cancel("Shutting down root actor")
await actor.cancel(
requesting_uid=actor.uid,
)
await actor.cancel(None) # self cancel
finally:
_state._current_actor = None

File diff suppressed because it is too large Load Diff

View File

@ -196,16 +196,16 @@ async def cancel_on_completion(
result: Any|Exception = await exhaust_portal(portal, actor)
if isinstance(result, Exception):
errors[actor.uid]: Exception = result
log.warning(
'Cancelling subactor due to error:\n'
f'uid: {portal.channel.uid}\n'
log.cancel(
'Cancelling subactor runtime due to error:\n\n'
f'Portal.cancel_actor() => {portal.channel.uid}\n\n'
f'error: {result}\n'
)
else:
log.runtime(
'Cancelling subactor gracefully:\n'
f'uid: {portal.channel.uid}\n'
'Cancelling subactor gracefully:\n\n'
f'Portal.cancel_actor() => {portal.channel.uid}\n\n'
f'result: {result}\n'
)
@ -213,7 +213,7 @@ async def cancel_on_completion(
await portal.cancel_actor()
async def do_hard_kill(
async def hard_kill(
proc: trio.Process,
terminate_after: int = 3,
@ -288,7 +288,7 @@ async def do_hard_kill(
proc.kill()
async def soft_wait(
async def soft_kill(
proc: ProcessType,
wait_func: Callable[
@ -299,17 +299,20 @@ async def soft_wait(
) -> None:
'''
Wait for proc termination but **dont' yet** teardown
std-streams (since it will clobber any ongoing pdb REPL
session). This is our "soft" (and thus itself cancellable)
join/reap on an actor-runtime-in-process.
Wait for proc termination but **don't yet** teardown
std-streams since it will clobber any ongoing pdb REPL
session.
This is our "soft"/graceful, and thus itself also cancellable,
join/reap on an actor-runtime-in-process shutdown; it is
**not** the same as a "hard kill" via an OS signal (for that
see `.hard_kill()`).
'''
uid: tuple[str, str] = portal.channel.uid
try:
log.cancel(
'Soft waiting on sub-actor proc:\n'
f'uid: {uid}\n'
'Soft killing sub-actor via `Portal.cancel_actor()`\n'
f'|_{proc}\n'
)
# wait on sub-proc to signal termination
@ -326,8 +329,9 @@ async def soft_wait(
async def cancel_on_proc_deth():
'''
"Cancel the (actor) cancel" request if we detect
that that the underlying sub-process terminated.
"Cancel-the-cancel" request: if we detect that the
underlying sub-process exited prior to
a `Portal.cancel_actor()` call completing .
'''
await wait_func(proc)
@ -444,14 +448,17 @@ async def trio_proc(
try:
# TODO: needs ``trio_typing`` patch?
proc = await trio.lowlevel.open_process(spawn_cmd)
log.runtime(f"Started {proc}")
log.runtime(
'Started new sub-proc\n'
f'|_{proc}\n'
)
# wait for actor to spawn and connect back to us
# channel should have handshake completed by the
# local actor by the time we get a ref to it
event, chan = await actor_nursery._actor.wait_for_peer(
subactor.uid)
subactor.uid
)
except trio.Cancelled:
cancelled_during_spawn = True
@ -512,7 +519,7 @@ async def trio_proc(
# This is a "soft" (cancellable) join/reap which
# will remote cancel the actor on a ``trio.Cancelled``
# condition.
await soft_wait(
await soft_kill(
proc,
trio.Process.wait,
portal
@ -541,13 +548,14 @@ async def trio_proc(
with trio.move_on_after(0.5):
await proc.wait()
log.pdb(
'Delaying subproc reaper while debugger locked..'
)
await maybe_wait_for_debugger(
child_in_debug=_runtime_vars.get(
'_debug_mode', False
),
header_msg=(
'Delaying subproc reaper while debugger locked..\n'
),
# TODO: need a diff value then default?
# poll_steps=9999999,
)
@ -573,7 +581,7 @@ async def trio_proc(
if proc.poll() is None:
log.cancel(f"Attempting to hard kill {proc}")
await do_hard_kill(proc)
await hard_kill(proc)
log.debug(f"Joined {proc}")
else:
@ -717,7 +725,7 @@ async def mp_proc(
# This is a "soft" (cancellable) join/reap which
# will remote cancel the actor on a ``trio.Cancelled``
# condition.
await soft_wait(
await soft_kill(
proc,
proc_waiter,
portal

View File

@ -95,9 +95,6 @@ class MsgStream(trio.abc.Channel):
try:
return msg['yield']
except KeyError as kerr:
# if 'return' in msg:
# return msg
_raise_from_no_key_in_msg(
ctx=self._ctx,
msg=msg,
@ -128,13 +125,9 @@ class MsgStream(trio.abc.Channel):
# introducing this
if self._eoc:
raise self._eoc
# raise trio.EndOfChannel
if self._closed:
raise self._closed
# raise trio.ClosedResourceError(
# 'This stream was already closed'
# )
src_err: Exception|None = None
try:
@ -143,6 +136,7 @@ class MsgStream(trio.abc.Channel):
return msg['yield']
except KeyError as kerr:
# log.exception('GOT KEYERROR')
src_err = kerr
# NOTE: may raise any of the below error types
@ -161,9 +155,9 @@ class MsgStream(trio.abc.Channel):
# trio.ClosedResourceError, # by self._rx_chan
trio.EndOfChannel, # by self._rx_chan or `stop` msg from far end
) as eoc:
# log.exception('GOT EOC')
src_err = eoc
self._eoc = eoc
# await trio.sleep(1)
# a ``ClosedResourceError`` indicates that the internal
# feeder memory receive channel was closed likely by the
@ -201,6 +195,7 @@ class MsgStream(trio.abc.Channel):
# raise eoc
except trio.ClosedResourceError as cre: # by self._rx_chan
# log.exception('GOT CRE')
src_err = cre
log.warning(
'`Context._rx_chan` was already closed?'
@ -211,6 +206,8 @@ class MsgStream(trio.abc.Channel):
# terminated and signal this local iterator to stop
drained: list[Exception|dict] = await self.aclose()
if drained:
# from .devx import pause
# await pause()
log.warning(
'Drained context msgs during closure:\n'
f'{drained}'
@ -237,54 +234,64 @@ class MsgStream(trio.abc.Channel):
Cancel associated remote actor task and local memory channel on
close.
Notes:
- REMEMBER that this is also called by `.__aexit__()` so
careful consideration must be made to handle whatever
internal stsate is mutated, particuarly in terms of
draining IPC msgs!
- more or less we try to maintain adherance to trio's `.aclose()` semantics:
https://trio.readthedocs.io/en/stable/reference-io.html#trio.abc.AsyncResource.aclose
'''
# XXX: keep proper adherance to trio's `.aclose()` semantics:
# https://trio.readthedocs.io/en/stable/reference-io.html#trio.abc.AsyncResource.aclose
rx_chan = self._rx_chan
if (
rx_chan._closed
or
self._closed
):
log.cancel(
f'`MsgStream` is already closed\n'
f'.cid: {self._ctx.cid}\n'
f'._rx_chan`: {rx_chan}\n'
f'._eoc: {self._eoc}\n'
f'._closed: {self._eoc}\n'
)
# rx_chan = self._rx_chan
# XXX NOTE XXX
# it's SUPER IMPORTANT that we ensure we don't DOUBLE
# DRAIN msgs on closure so avoid getting stuck handing on
# the `._rx_chan` since we call this method on
# `.__aexit__()` as well!!!
# => SO ENSURE WE CATCH ALL TERMINATION STATES in this
# block including the EoC..
if self.closed:
# this stream has already been closed so silently succeed as
# per ``trio.AsyncResource`` semantics.
# https://trio.readthedocs.io/en/stable/reference-io.html#trio.abc.AsyncResource.aclose
return []
ctx: Context = self._ctx
# caught_eoc: bool = False
drained: list[Exception|dict] = []
while not drained:
try:
maybe_final_msg = self.receive_nowait()
if maybe_final_msg:
log.cancel(
log.debug(
'Drained un-processed stream msg:\n'
f'{pformat(maybe_final_msg)}'
)
# TODO: inject into parent `Context` buf?
drained.append(maybe_final_msg)
# NOTE: we only need these handlers due to the
# `.receive_nowait()` call above which may re-raise
# one of these errors on a msg key error!
except trio.WouldBlock as be:
drained.append(be)
break
except trio.EndOfChannel as eoc:
self._eoc: Exception = eoc
drained.append(eoc)
# caught_eoc = True
self._eoc: bool = eoc
break
except trio.ClosedResourceError as cre:
self._closed = cre
drained.append(cre)
break
except ContextCancelled as ctxc:
# log.exception('GOT CTXC')
log.cancel(
'Context was cancelled during stream closure:\n'
f'canceller: {ctxc.canceller}\n'
@ -339,8 +346,11 @@ class MsgStream(trio.abc.Channel):
# with trio.CancelScope(shield=True):
# await rx_chan.aclose()
# self._eoc: bool = caught_eoc
if not self._eoc:
self._eoc: bool = trio.EndOfChannel(
f'Context stream closed by {self._ctx.side}\n'
f'|_{self}\n'
)
# ?XXX WAIT, why do we not close the local mem chan `._rx_chan` XXX?
# => NO, DEFINITELY NOT! <=
# if we're a bi-dir ``MsgStream`` BECAUSE this same
@ -379,6 +389,26 @@ class MsgStream(trio.abc.Channel):
# self._closed = True
return drained
@property
def closed(self) -> bool:
if (
(rxc := self._rx_chan._closed)
or
(_closed := self._closed)
or
(_eoc := self._eoc)
):
log.runtime(
f'`MsgStream` is already closed\n'
f'{self}\n'
f' |_cid: {self._ctx.cid}\n'
f' |_rx_chan._closed: {type(rxc)} = {rxc}\n'
f' |_closed: {type(_closed)} = {_closed}\n'
f' |_eoc: {type(_eoc)} = {_eoc}'
)
return True
return False
@acm
async def subscribe(
self,

View File

@ -21,6 +21,7 @@
from contextlib import asynccontextmanager as acm
from functools import partial
import inspect
from pprint import pformat
from typing import TYPE_CHECKING
import typing
import warnings
@ -33,7 +34,10 @@ from ._state import current_actor, is_main_process
from .log import get_logger, get_loglevel
from ._runtime import Actor
from ._portal import Portal
from ._exceptions import is_multi_cancelled
from ._exceptions import (
is_multi_cancelled,
ContextCancelled,
)
from ._root import open_root_actor
from . import _state
from . import _spawn
@ -103,6 +107,14 @@ class ActorNursery:
self.errors = errors
self.exited = trio.Event()
# NOTE: when no explicit call is made to
# `.open_root_actor()` by application code,
# `.open_nursery()` will implicitly call it to start the
# actor-tree runtime. In this case we mark ourselves as
# such so that runtime components can be aware for logging
# and syncing purposes to any actor opened nurseries.
self._implicit_runtime_started: bool = False
async def start_actor(
self,
name: str,
@ -189,14 +201,16 @@ class ActorNursery:
**kwargs, # explicit args to ``fn``
) -> Portal:
"""Spawn a new actor, run a lone task, then terminate the actor and
'''
Spawn a new actor, run a lone task, then terminate the actor and
return its result.
Actors spawned using this method are kept alive at nursery teardown
until the task spawned by executing ``fn`` completes at which point
the actor is terminated.
"""
mod_path = fn.__module__
'''
mod_path: str = fn.__module__
if name is None:
# use the explicit function name if not provided
@ -231,7 +245,11 @@ class ActorNursery:
)
return portal
async def cancel(self, hard_kill: bool = False) -> None:
async def cancel(
self,
hard_kill: bool = False,
) -> None:
'''
Cancel this nursery by instructing each subactor to cancel
itself and wait for all subactors to terminate.
@ -242,10 +260,13 @@ class ActorNursery:
'''
self.cancelled = True
log.cancel(f"Cancelling nursery in {self._actor.uid}")
# TODO: impl a repr for spawn more compact
# then `._children`..
children: dict = self._children
child_count: int = len(children)
msg: str = f'Cancelling actor nursery with {child_count} children\n'
with trio.move_on_after(3) as cs:
async with trio.open_nursery() as nursery:
async with trio.open_nursery() as tn:
subactor: Actor
proc: trio.Process
@ -254,7 +275,7 @@ class ActorNursery:
subactor,
proc,
portal,
) in self._children.values():
) in children.values():
# TODO: are we ever even going to use this or
# is the spawning backend responsible for such
@ -266,12 +287,13 @@ class ActorNursery:
if portal is None: # actor hasn't fully spawned yet
event = self._actor._peer_connected[subactor.uid]
log.warning(
f"{subactor.uid} wasn't finished spawning?")
f"{subactor.uid} never 't finished spawning?"
)
await event.wait()
# channel/portal should now be up
_, _, portal = self._children[subactor.uid]
_, _, portal = children[subactor.uid]
# XXX should be impossible to get here
# unless method was called from within
@ -288,13 +310,15 @@ class ActorNursery:
# spawn cancel tasks for each sub-actor
assert portal
if portal.channel.connected():
nursery.start_soon(portal.cancel_actor)
tn.start_soon(portal.cancel_actor)
log.cancel(msg)
# if we cancelled the cancel (we hung cancelling remote actors)
# then hard kill all sub-processes
if cs.cancelled_caught:
log.error(
f'Failed to cancel {self}\nHard killing process tree!'
f'Failed to cancel {self}?\n'
'Hard killing underlying subprocess tree!\n'
)
subactor: Actor
proc: trio.Process
@ -303,7 +327,7 @@ class ActorNursery:
subactor,
proc,
portal,
) in self._children.values():
) in children.values():
log.warning(f"Hard killing process {proc}")
proc.terminate()
@ -343,7 +367,7 @@ async def _open_and_supervise_one_cancels_all_nursery(
# the above "daemon actor" nursery will be notified.
async with trio.open_nursery() as ria_nursery:
anursery = ActorNursery(
an = ActorNursery(
actor,
ria_nursery,
da_nursery,
@ -352,16 +376,16 @@ async def _open_and_supervise_one_cancels_all_nursery(
try:
# spawning of actors happens in the caller's scope
# after we yield upwards
yield anursery
yield an
# When we didn't error in the caller's scope,
# signal all process-monitor-tasks to conduct
# the "hard join phase".
log.runtime(
f"Waiting on subactors {anursery._children} "
"to complete"
'Waiting on subactors to complete:\n'
f'{pformat(an._children)}\n'
)
anursery._join_procs.set()
an._join_procs.set()
except BaseException as inner_err:
errors[actor.uid] = inner_err
@ -373,34 +397,47 @@ async def _open_and_supervise_one_cancels_all_nursery(
# Instead try to wait for pdb to be released before
# tearing down.
await maybe_wait_for_debugger(
child_in_debug=anursery._at_least_one_child_in_debug
child_in_debug=an._at_least_one_child_in_debug
)
# if the caller's scope errored then we activate our
# one-cancels-all supervisor strategy (don't
# worry more are coming).
anursery._join_procs.set()
an._join_procs.set()
# XXX: hypothetically an error could be
# raised and then a cancel signal shows up
# XXX NOTE XXX: hypothetically an error could
# be raised and then a cancel signal shows up
# slightly after in which case the `else:`
# block here might not complete? For now,
# shield both.
with trio.CancelScope(shield=True):
etype = type(inner_err)
etype: type = type(inner_err)
if etype in (
trio.Cancelled,
KeyboardInterrupt
KeyboardInterrupt,
) or (
is_multi_cancelled(inner_err)
):
log.cancel(
f"Nursery for {current_actor().uid} "
f"was cancelled with {etype}")
f'Actor-nursery cancelled by {etype}\n\n'
f'{current_actor().uid}\n'
f' |_{an}\n\n'
# TODO: show tb str?
# f'{tb_str}'
)
elif etype in {
ContextCancelled,
}:
log.cancel(
'Actor-nursery caught remote cancellation\n\n'
f'{inner_err.tb_str}'
)
else:
log.exception(
f"Nursery for {current_actor().uid} "
"errored with:"
'Nursery errored with:\n'
# TODO: same thing as in
# `._invoke()` to compute how to
@ -413,7 +450,7 @@ async def _open_and_supervise_one_cancels_all_nursery(
)
# cancel all subactors
await anursery.cancel()
await an.cancel()
# ria_nursery scope end
@ -434,18 +471,22 @@ async def _open_and_supervise_one_cancels_all_nursery(
# XXX: yet another guard before allowing the cancel
# sequence in case a (single) child is in debug.
await maybe_wait_for_debugger(
child_in_debug=anursery._at_least_one_child_in_debug
child_in_debug=an._at_least_one_child_in_debug
)
# If actor-local error was raised while waiting on
# ".run_in_actor()" actors then we also want to cancel all
# remaining sub-actors (due to our lone strategy:
# one-cancels-all).
log.cancel(f"Nursery cancelling due to {err}")
if anursery._children:
if an._children:
log.cancel(
'Actor-nursery cancelling due error type:\n'
f'{err}\n'
)
with trio.CancelScope(shield=True):
await anursery.cancel()
await an.cancel()
raise
finally:
# No errors were raised while awaiting ".run_in_actor()"
# actors but those actors may have returned remote errors as
@ -454,9 +495,9 @@ async def _open_and_supervise_one_cancels_all_nursery(
# collected in ``errors`` so cancel all actors, summarize
# all errors and re-raise.
if errors:
if anursery._children:
if an._children:
with trio.CancelScope(shield=True):
await anursery.cancel()
await an.cancel()
# use `BaseExceptionGroup` as needed
if len(errors) > 1:
@ -491,7 +532,7 @@ async def open_nursery(
which cancellation scopes correspond to each spawned subactor set.
'''
implicit_runtime = False
implicit_runtime: bool = False
actor = current_actor(err_on_no_runtime=False)
@ -503,7 +544,7 @@ async def open_nursery(
log.info("Starting actor runtime!")
# mark us for teardown on exit
implicit_runtime = True
implicit_runtime: bool = True
async with open_root_actor(**kwargs) as actor:
assert actor is current_actor()
@ -511,24 +552,42 @@ async def open_nursery(
try:
async with _open_and_supervise_one_cancels_all_nursery(
actor
) as anursery:
yield anursery
) as an:
# NOTE: mark this nursery as having
# implicitly started the root actor so
# that `._runtime` machinery can avoid
# certain teardown synchronization
# blocking/waits and any associated (warn)
# logging when it's known that this
# nursery shouldn't be exited before the
# root actor is.
an._implicit_runtime_started = True
yield an
finally:
anursery.exited.set()
# XXX: this event will be set after the root actor
# runtime is already torn down, so we want to
# avoid any blocking on it.
an.exited.set()
else: # sub-nursery case
try:
async with _open_and_supervise_one_cancels_all_nursery(
actor
) as anursery:
yield anursery
) as an:
yield an
finally:
anursery.exited.set()
an.exited.set()
finally:
log.debug("Nursery teardown complete")
msg: str = (
'Actor-nursery exited\n'
f'|_{an}\n\n'
)
# shutdown runtime if it was started
if implicit_runtime:
log.info("Shutting down actor tree")
msg += '=> Shutting down actor runtime <=\n'
log.info(msg)

View File

@ -999,6 +999,8 @@ async def maybe_wait_for_debugger(
poll_delay: float = 0.1,
child_in_debug: bool = False,
header_msg: str = '',
) -> None:
if (
@ -1007,6 +1009,8 @@ async def maybe_wait_for_debugger(
):
return
msg: str = header_msg
if (
is_root_process()
):
@ -1016,13 +1020,13 @@ async def maybe_wait_for_debugger(
# will make the pdb repl unusable.
# Instead try to wait for pdb to be released before
# tearing down.
sub_in_debug: tuple[str, str] | None = None
for istep in range(poll_steps):
sub_in_debug: tuple[str, str]|None = Lock.global_actor_in_debug
debug_complete: trio.Event|None = Lock.no_remote_has_tty
if sub_in_debug := Lock.global_actor_in_debug:
log.pdb(
f'Lock in use by {sub_in_debug}'
msg += (
'Debug `Lock` in use by subactor\n'
f'|_{sub_in_debug}\n'
)
# TODO: could this make things more deterministic?
# wait to see if a sub-actor task will be
@ -1030,34 +1034,45 @@ async def maybe_wait_for_debugger(
# tick?
# XXX => but it doesn't seem to work..
# await trio.testing.wait_all_tasks_blocked(cushion=0)
else:
log.pdb(
msg
+
'Root immediately acquired debug TTY LOCK'
)
return
for istep in range(poll_steps):
debug_complete: trio.Event|None = Lock.no_remote_has_tty
if (
debug_complete
and not debug_complete.is_set()
and sub_in_debug is not None
):
log.pdb(
'Root has errored but pdb is in use by child\n'
'Waiting on tty lock to release..\n'
f'uid: {sub_in_debug}\n'
msg
+
'Root is waiting on tty lock to release..\n'
)
await debug_complete.wait()
log.pdb(
f'Child subactor released debug lock!\n'
f'uid: {sub_in_debug}\n'
f'Child subactor released debug lock:'
f'|_{sub_in_debug}\n'
)
if debug_complete.is_set():
break
# is no subactor locking debugger currently?
elif (
if (
sub_in_debug is None
and (
debug_complete is None
or sub_in_debug is None
or debug_complete.is_set()
)
):
log.pdb(
'Root acquired debug TTY LOCK from child\n'
f'uid: {sub_in_debug}'
msg
+
'Root acquired tty lock!'
)
break
@ -1073,8 +1088,14 @@ async def maybe_wait_for_debugger(
with trio.CancelScope(shield=True):
await trio.sleep(poll_delay)
continue
# fallthrough on failure to acquire..
else:
log.pdb('Root acquired debug TTY LOCK')
raise RuntimeError(
msg
+
'Root actor failed to acquire debug lock?'
)
# else:
# # TODO: non-root call for #320?

View File

@ -43,17 +43,24 @@ IPC-compat cross-mem-boundary object pointer.
# - https://github.com/msgpack/msgpack-python#packingunpacking-of-custom-data-type
from __future__ import annotations
from inspect import isfunction
from inspect import (
isfunction,
ismethod,
)
from pkgutil import resolve_name
class NamespacePath(str):
'''
A serializeable description of a (function) Python object
location described by the target's module path and namespace
key meant as a message-native "packet" to allows actors to
point-and-load objects by an absolute ``str`` (and thus
serializable) reference.
A serializeable `str`-subtype implementing a "namespace
pointer" to any Python object reference (like a function)
using the same format as the built-in `pkgutil.resolve_name()`
system.
A value describes a target's module-path and namespace-key
separated by a ':' and thus can be easily used as
a IPC-message-native reference-type allowing memory isolated
actors to point-and-load objects via a minimal `str` value.
'''
_ref: object | type | None = None
@ -81,13 +88,23 @@ class NamespacePath(str):
'''
if (
isinstance(ref, object)
and not isfunction(ref)
isfunction(ref)
):
name: str = type(ref).__name__
else:
name: str = getattr(ref, '__name__')
elif ismethod(ref):
# build out the path manually i guess..?
# TODO: better way?
name: str = '.'.join([
type(ref.__self__).__name__,
ref.__func__.__name__,
])
else: # object or other?
# isinstance(ref, object)
# and not isfunction(ref)
name: str = type(ref).__name__
# fully qualified namespace path, tuple.
fqnp: tuple[str, str] = (
ref.__module__,

View File

@ -35,6 +35,24 @@ from msgspec import (
structs,
)
# TODO: auto-gen type sig for input func both for
# type-msgs and logging of RPC tasks?
# taken and modified from:
# https://stackoverflow.com/a/57110117
# import inspect
# from typing import List
# def my_function(input_1: str, input_2: int) -> list[int]:
# pass
# def types_of(func):
# specs = inspect.getfullargspec(func)
# return_type = specs.annotations['return']
# input_types = [t.__name__ for s, t in specs.annotations.items() if s != 'return']
# return f'{func.__name__}({": ".join(input_types)}) -> {return_type}'
# types_of(my_function)
class DiffDump(UserList):
'''
@ -161,6 +179,7 @@ class Struct(
# https://docs.python.org/3.11/library/pprint.html#pprint.saferepr
val_str: str = saferepr(v)
# TODO: LOLOL use `textwrap.indent()` instead dawwwwwg!
obj_str += (field_ws + f'{k}: {typ_name} = {val_str},\n')
return (