Compare commits

...

4 Commits

Author SHA1 Message Date
Tyler Goodlet f0342d6ae3 Move `Context.open_stream()` impl to `._streaming`
Exactly like how it's organized for `Portal.open_context()`, put the
main streaming API `@acm` with the `MsgStream` code and bind the method
to the new module func.

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

Also Move the commented TODO proto for a codec hook factory from
`.types` to `._codec` where it prolly better fits and update some msg
related todo/questions.
2024-05-31 12:03:18 -04:00
Tyler Goodlet d802c8aa90 Woops, set `post_mortem=False` by default again! 2024-05-30 18:33:25 -04:00
6 changed files with 320 additions and 435 deletions

View File

@ -315,7 +315,7 @@ def test_basic_payload_spec(
f"match type-spec: `{msg_type_str}.pld: PldMsg|NoneType`",
],
# only for debug
post_mortem=True,
# post_mortem=True,
),
p.open_context(
child,

View File

@ -86,7 +86,10 @@ from .msg import (
from ._ipc import (
Channel,
)
from ._streaming import MsgStream
from ._streaming import (
MsgStream,
open_stream_from_ctx,
)
from ._state import (
current_actor,
debug_mode,
@ -978,198 +981,6 @@ class Context:
assert self._scope
self._scope.cancel()
# TODO? should we move this to `._streaming` much like we
# moved `Portal.open_context()`'s def to this mod?
@acm
async def open_stream(
self,
allow_overruns: bool|None = False,
msg_buffer_size: int|None = None,
) -> AsyncGenerator[MsgStream, None]:
'''
Open a ``MsgStream``, a bi-directional stream connected to the
cross-actor (far end) task for this ``Context``.
This context manager must be entered on both the caller and
callee for the stream to logically be considered "connected".
A ``MsgStream`` is currently "one-shot" use, meaning if you
close it you can not "re-open" it for streaming and instead you
must re-establish a new surrounding ``Context`` using
``Portal.open_context()``. In the future this may change but
currently there seems to be no obvious reason to support
"re-opening":
- pausing a stream can be done with a message.
- task errors will normally require a restart of the entire
scope of the inter-actor task context due to the nature of
``trio``'s cancellation system.
'''
actor: Actor = self._actor
# If the surrounding context has been cancelled by some
# task with a handle to THIS, we error here immediately
# since it likely means the surrounding lexical-scope has
# errored, been `trio.Cancelled` or at the least
# `Context.cancel()` was called by some task.
if self._cancel_called:
# XXX NOTE: ALWAYS RAISE any remote error here even if
# it's an expected `ContextCancelled` due to a local
# task having called `.cancel()`!
#
# WHY: we expect the error to always bubble up to the
# surrounding `Portal.open_context()` call and be
# absorbed there (silently) and we DO NOT want to
# actually try to stream - a cancel msg was already
# sent to the other side!
self.maybe_raise(
raise_ctxc_from_self_call=True,
)
# NOTE: this is diff then calling
# `._maybe_raise_remote_err()` specifically
# because we want to raise a ctxc on any task entering this `.open_stream()`
# AFTER cancellation was already been requested,
# we DO NOT want to absorb any ctxc ACK silently!
# if self._remote_error:
# raise self._remote_error
# XXX NOTE: if no `ContextCancelled` has been responded
# back from the other side (yet), we raise a different
# runtime error indicating that this task's usage of
# `Context.cancel()` and then `.open_stream()` is WRONG!
task: str = trio.lowlevel.current_task().name
raise RuntimeError(
'Stream opened after `Context.cancel()` called..?\n'
f'task: {actor.uid[0]}:{task}\n'
f'{self}'
)
if (
not self._portal
and not self._started_called
):
raise RuntimeError(
'Context.started()` must be called before opening a stream'
)
# NOTE: in one way streaming this only happens on the
# parent-ctx-task side (on the side that calls
# `Actor.start_remote_task()`) so if you try to send
# a stop from the caller to the callee in the
# single-direction-stream case you'll get a lookup error
# currently.
ctx: Context = actor.get_context(
chan=self.chan,
cid=self.cid,
nsf=self._nsf,
# side=self.side,
msg_buffer_size=msg_buffer_size,
allow_overruns=allow_overruns,
)
ctx._allow_overruns: bool = allow_overruns
assert ctx is self
# XXX: If the underlying channel feeder receive mem chan has
# been closed then likely client code has already exited
# a ``.open_stream()`` block prior or there was some other
# unanticipated error or cancellation from ``trio``.
if ctx._rx_chan._closed:
raise trio.ClosedResourceError(
'The underlying channel for this stream was already closed!\n'
)
# NOTE: implicitly this will call `MsgStream.aclose()` on
# `.__aexit__()` due to stream's parent `Channel` type!
#
# XXX NOTE XXX: ensures the stream is "one-shot use",
# which specifically means that on exit,
# - signal ``trio.EndOfChannel``/``StopAsyncIteration`` to
# the far end indicating that the caller exited
# the streaming context purposefully by letting
# the exit block exec.
# - this is diff from the cancel/error case where
# a cancel request from this side or an error
# should be sent to the far end indicating the
# stream WAS NOT just closed normally/gracefully.
async with MsgStream(
ctx=self,
rx_chan=ctx._rx_chan,
) as stream:
# NOTE: we track all existing streams per portal for
# the purposes of attempting graceful closes on runtime
# cancel requests.
if self._portal:
self._portal._streams.add(stream)
try:
self._stream_opened: bool = True
self._stream = stream
# XXX: do we need this?
# ensure we aren't cancelled before yielding the stream
# await trio.lowlevel.checkpoint()
yield stream
# XXX: (MEGA IMPORTANT) if this is a root opened process we
# wait for any immediate child in debug before popping the
# context from the runtime msg loop otherwise inside
# ``Actor._deliver_ctx_payload()`` the msg will be discarded and in
# the case where that msg is global debugger unlock (via
# a "stop" msg for a stream), this can result in a deadlock
# where the root is waiting on the lock to clear but the
# child has already cleared it and clobbered IPC.
#
# await maybe_wait_for_debugger()
# XXX TODO: pretty sure this isn't needed (see
# note above this block) AND will result in
# a double `.send_stop()` call. The only reason to
# put it here would be to due with "order" in
# terms of raising any remote error (as per
# directly below) or bc the stream's
# `.__aexit__()` block might not get run
# (doubtful)? Either way if we did put this back
# in we also need a state var to avoid the double
# stop-msg send..
#
# await stream.aclose()
# NOTE: absorb and do not raise any
# EoC received from the other side such that
# it is not raised inside the surrounding
# context block's scope!
except trio.EndOfChannel as eoc:
if (
eoc
and
stream.closed
):
# sanity, can remove?
assert eoc is stream._eoc
log.warning(
'Stream was terminated by EoC\n\n'
# NOTE: won't show the error <Type> but
# does show txt followed by IPC msg.
f'{str(eoc)}\n'
)
finally:
if self._portal:
try:
self._portal._streams.remove(stream)
except KeyError:
log.warning(
f'Stream was already destroyed?\n'
f'actor: {self.chan.uid}\n'
f'ctx id: {self.cid}'
)
# TODO: replace all the `._maybe_raise_remote_err()` usage
# with instances of this!!
def maybe_raise(
@ -1178,6 +989,14 @@ class Context:
**kwargs,
) -> Exception|None:
'''
Check for for a remote error delivered by the runtime from
our peer (task); if set immediately raise.
This is a convenience wrapper for
`._maybe_raise_remote_err(self._remote_error)`.
'''
__tracebackhide__: bool = hide_tb
if re := self._remote_error:
return self._maybe_raise_remote_err(
@ -1290,8 +1109,7 @@ class Context:
raise remote_error
# TODO: change to `.wait_for_result()`?
async def result(
async def wait_for_result(
self,
hide_tb: bool = True,
@ -1380,18 +1198,27 @@ class Context:
(not self._cancel_called)
)
)
# TODO: eventually make `.outcome: Outcome` and thus return
# `self.outcome.unwrap()` here!
return self.outcome
# TODO: switch this with above!
# -[ ] should be named `.wait_for_outcome()` and instead do
# a `.outcome.Outcome.unwrap()` ?
#
# @property
# def result(self) -> Any|None:
# if self._final_result_is_set():
# return self._result
# raise RuntimeError('No result is available!')
async def result(
self,
*args,
**kwargs,
) -> Any|Exception:
log.warning(
'`Context.result()` is DEPRECATED!\n'
'Use `Context.[no]wait_for_result()` instead!\n'
)
return await self.wait_for_result(
*args,
**kwargs,
)
@property
def maybe_error(self) -> BaseException|None:
@ -1447,6 +1274,9 @@ class Context:
return self._result is not Unresolved
# def get_result_nowait(self) -> Any|None:
# def get_outcome_nowait(self) -> Any|None:
# def recv_result_nowait(self) -> Any|None:
# def receive_outcome_nowait(self) -> Any|None:
# TODO: use `outcome.Outcome` here instead?
@property
def outcome(self) -> (
@ -1476,7 +1306,6 @@ class Context:
def has_outcome(self) -> bool:
return bool(self.maybe_error) or self._final_result_is_set()
# @property
def repr_outcome(
self,
show_error_fields: bool = False,
@ -1498,7 +1327,8 @@ class Context:
# just deliver the type name.
if (
(reprol := getattr(merr, 'reprol', False))
and show_error_fields
and
show_error_fields
):
return reprol()
@ -1515,10 +1345,6 @@ class Context:
repr(merr)
)
# just the type name
# else: # but wen?
# return type(merr).__name__
# for all other errors show their regular output
return (
str(merr)
@ -1572,7 +1398,7 @@ class Context:
_, # any non-unresolved value
None,
) if self._final_result_is_set():
status = 'returned'
status = 'result-returned'
# normal operation but still in a pre-`Return`-result
# dialog phase
@ -1940,6 +1766,11 @@ class Context:
# ow, indicate unable to deliver by default
return False
# NOTE: similar to `Portal.open_context()`, this impl is found in
# the `._streaming`` mod to make reading/groking the details
# simpler code-org-wise.
open_stream = open_stream_from_ctx
# TODO: exception tb masking by using a manual
# `.__aexit__()`/.__aenter__()` pair on a type?

View File

@ -739,37 +739,24 @@ async def _invoke(
cid,
))
logmeth: Callable = log.runtime
merr: Exception|None = ctx.maybe_error
(
res_type_str,
res_str,
) = (
('error', f'{type(merr)}',) if merr
else (
'result',
f'`{repr(ctx.outcome)}`',
)
)
descr_str: str = 'with final result `{repr(ctx.outcome)}`'
message: str = (
f'IPC context terminated with a final {res_type_str}\n\n'
f'{ctx}'
f'IPC context terminated {descr_str}\n\n'
)
if merr:
from tractor import RemoteActorError
if not isinstance(merr, RemoteActorError):
fmt_merr: str = (
f'\n{merr!r}\n'
# f'{merr.args[0]!r}\n'
descr_str: str = (
f'with ctx having {ctx.repr_state!r}\n'
f'{ctx.repr_outcome()}\n'
)
if isinstance(merr, ContextCancelled):
logmeth: Callable = log.runtime
else:
fmt_merr = f'\n{merr!r}'
log.error(
message
+
fmt_merr
)
else:
log.runtime(message)
logmeth: Callable = log.error
message += f'\n{merr!r}\n'
logmeth(message)
async def try_ship_error_to_remote(

View File

@ -26,6 +26,7 @@ import inspect
from pprint import pformat
from typing import (
Any,
AsyncGenerator,
Callable,
AsyncIterator,
TYPE_CHECKING,
@ -51,6 +52,7 @@ from tractor.msg import (
)
if TYPE_CHECKING:
from ._runtime import Actor
from ._context import Context
from ._ipc import Channel
@ -550,6 +552,213 @@ class MsgStream(trio.abc.Channel):
# ...
@acm
async def open_stream_from_ctx(
ctx: Context,
allow_overruns: bool|None = False,
msg_buffer_size: int|None = None,
) -> AsyncGenerator[MsgStream, None]:
'''
Open a `MsgStream`, a bi-directional msg transport dialog
connected to the cross-actor peer task for an IPC `Context`.
This context manager must be entered in both the "parent" (task
which entered `Portal.open_context()`) and "child" (RPC task
which is decorated by `@context`) tasks for the stream to
logically be considered "open"; if one side begins sending to an
un-opened peer, depending on policy config, msgs will either be
queued until the other side opens and/or a `StreamOverrun` will
(eventually) be raised.
------ - ------
Runtime semantics design:
A `MsgStream` session adheres to "one-shot use" semantics,
meaning if you close the scope it **can not** be "re-opened".
Instead you must re-establish a new surrounding RPC `Context`
(RTC: remote task context?) using `Portal.open_context()`.
In the future this *design choice* may need to be changed but
currently there seems to be no obvious reason to support such
semantics..
- "pausing a stream" can be supported with a message implemented
by the `tractor` application dev.
- any remote error will normally require a restart of the entire
`trio.Task`'s scope due to the nature of `trio`'s cancellation
(`CancelScope`) system and semantics (level triggered).
'''
actor: Actor = ctx._actor
# If the surrounding context has been cancelled by some
# task with a handle to THIS, we error here immediately
# since it likely means the surrounding lexical-scope has
# errored, been `trio.Cancelled` or at the least
# `Context.cancel()` was called by some task.
if ctx._cancel_called:
# XXX NOTE: ALWAYS RAISE any remote error here even if
# it's an expected `ContextCancelled` due to a local
# task having called `.cancel()`!
#
# WHY: we expect the error to always bubble up to the
# surrounding `Portal.open_context()` call and be
# absorbed there (silently) and we DO NOT want to
# actually try to stream - a cancel msg was already
# sent to the other side!
ctx.maybe_raise(
raise_ctxc_from_self_call=True,
)
# NOTE: this is diff then calling
# `._maybe_raise_remote_err()` specifically
# because we want to raise a ctxc on any task entering this `.open_stream()`
# AFTER cancellation was already been requested,
# we DO NOT want to absorb any ctxc ACK silently!
# if ctx._remote_error:
# raise ctx._remote_error
# XXX NOTE: if no `ContextCancelled` has been responded
# back from the other side (yet), we raise a different
# runtime error indicating that this task's usage of
# `Context.cancel()` and then `.open_stream()` is WRONG!
task: str = trio.lowlevel.current_task().name
raise RuntimeError(
'Stream opened after `Context.cancel()` called..?\n'
f'task: {actor.uid[0]}:{task}\n'
f'{ctx}'
)
if (
not ctx._portal
and not ctx._started_called
):
raise RuntimeError(
'Context.started()` must be called before opening a stream'
)
# NOTE: in one way streaming this only happens on the
# parent-ctx-task side (on the side that calls
# `Actor.start_remote_task()`) so if you try to send
# a stop from the caller to the callee in the
# single-direction-stream case you'll get a lookup error
# currently.
ctx: Context = actor.get_context(
chan=ctx.chan,
cid=ctx.cid,
nsf=ctx._nsf,
# side=ctx.side,
msg_buffer_size=msg_buffer_size,
allow_overruns=allow_overruns,
)
ctx._allow_overruns: bool = allow_overruns
assert ctx is ctx
# XXX: If the underlying channel feeder receive mem chan has
# been closed then likely client code has already exited
# a ``.open_stream()`` block prior or there was some other
# unanticipated error or cancellation from ``trio``.
if ctx._rx_chan._closed:
raise trio.ClosedResourceError(
'The underlying channel for this stream was already closed!\n'
)
# NOTE: implicitly this will call `MsgStream.aclose()` on
# `.__aexit__()` due to stream's parent `Channel` type!
#
# XXX NOTE XXX: ensures the stream is "one-shot use",
# which specifically means that on exit,
# - signal ``trio.EndOfChannel``/``StopAsyncIteration`` to
# the far end indicating that the caller exited
# the streaming context purposefully by letting
# the exit block exec.
# - this is diff from the cancel/error case where
# a cancel request from this side or an error
# should be sent to the far end indicating the
# stream WAS NOT just closed normally/gracefully.
async with MsgStream(
ctx=ctx,
rx_chan=ctx._rx_chan,
) as stream:
# NOTE: we track all existing streams per portal for
# the purposes of attempting graceful closes on runtime
# cancel requests.
if ctx._portal:
ctx._portal._streams.add(stream)
try:
ctx._stream_opened: bool = True
ctx._stream = stream
# XXX: do we need this?
# ensure we aren't cancelled before yielding the stream
# await trio.lowlevel.checkpoint()
yield stream
# XXX: (MEGA IMPORTANT) if this is a root opened process we
# wait for any immediate child in debug before popping the
# context from the runtime msg loop otherwise inside
# ``Actor._deliver_ctx_payload()`` the msg will be discarded and in
# the case where that msg is global debugger unlock (via
# a "stop" msg for a stream), this can result in a deadlock
# where the root is waiting on the lock to clear but the
# child has already cleared it and clobbered IPC.
#
# await maybe_wait_for_debugger()
# XXX TODO: pretty sure this isn't needed (see
# note above this block) AND will result in
# a double `.send_stop()` call. The only reason to
# put it here would be to due with "order" in
# terms of raising any remote error (as per
# directly below) or bc the stream's
# `.__aexit__()` block might not get run
# (doubtful)? Either way if we did put this back
# in we also need a state var to avoid the double
# stop-msg send..
#
# await stream.aclose()
# NOTE: absorb and do not raise any
# EoC received from the other side such that
# it is not raised inside the surrounding
# context block's scope!
except trio.EndOfChannel as eoc:
if (
eoc
and
stream.closed
):
# sanity, can remove?
assert eoc is stream._eoc
log.warning(
'Stream was terminated by EoC\n\n'
# NOTE: won't show the error <Type> but
# does show txt followed by IPC msg.
f'{str(eoc)}\n'
)
finally:
if ctx._portal:
try:
ctx._portal._streams.remove(stream)
except KeyError:
log.warning(
f'Stream was already destroyed?\n'
f'actor: {ctx.chan.uid}\n'
f'ctx id: {ctx.cid}'
)
def stream(func: Callable) -> Callable:
'''
Mark an async function as a streaming routine with ``@stream``.

View File

@ -52,10 +52,6 @@ from msgspec import (
msgpack,
Raw,
)
# from trio.lowlevel import (
# RunVar,
# RunVarToken,
# )
# TODO: see notes below from @mikenerone..
# from tricycle import TreeVar
@ -368,160 +364,16 @@ class MsgCodec(Struct):
# https://jcristharif.com/msgspec/usage.html#typed-decoding
return self._dec.decode(msg)
# TODO: a sub-decoder system as well?
# payload_msg_specs: Union[Type[Struct]] = Any
# see related comments in `.msg.types`
# _payload_decs: (
# dict[
# str,
# msgpack.Decoder,
# ]
# |None
# ) = None
# OR
# ) = {
# # pre-seed decoders for std-py-type-set for use when
# # `MsgType.pld == None|Any`.
# None: msgpack.Decoder(Any),
# Any: msgpack.Decoder(Any),
# }
#
# -[ ] do we still want to try and support the sub-decoder with
# `.Raw` technique in the case that the `Generic` approach gives
# future grief?
#
# -[ ] <NEW-ISSUE-FOR-ThIS-HERE>
# -> https://jcristharif.com/msgspec/api.html#raw
#
#def mk_pld_subdec(
# self,
# payload_types: Union[Type[Struct]],
#) -> msgpack.Decoder:
# # TODO: sub-decoder suppor for `.pld: Raw`?
# # => see similar notes inside `.msg.types`..
# #
# # not sure we'll end up needing this though it might have
# # unforeseen advantages in terms of enabling encrypted
# # appliciation layer (only) payloads?
# #
# # register sub-payload decoders to load `.pld: Raw`
# # decoded `Msg`-packets using a dynamic lookup (table)
# # instead of a pre-defined msg-spec via `Generic`
# # parameterization.
# #
# (
# tags,
# payload_dec,
# ) = mk_tagged_union_dec(
# tagged_structs=list(payload_types.__args__),
# )
# # register sub-decoders by tag
# subdecs: dict[str, msgpack.Decoder]|None = self._payload_decs
# for name in tags:
# subdecs.setdefault(
# name,
# payload_dec,
# )
# return payload_dec
# sub-decoders for retreiving embedded
# payload data and decoding to a sender
# side defined (struct) type.
# def dec_payload(
# codec: MsgCodec,
# msg: Msg,
# ) -> Any|Struct:
# msg: PayloadMsg = codec.dec.decode(msg)
# payload_tag: str = msg.header.payload_tag
# payload_dec: msgpack.Decoder = codec._payload_decs[payload_tag]
# return payload_dec.decode(msg.pld)
# def enc_payload(
# codec: MsgCodec,
# payload: Any,
# cid: str,
# ) -> bytes:
# # tag_field: str|None = None
# plbytes = codec.enc.encode(payload)
# if b'msg_type' in plbytes:
# assert isinstance(payload, Struct)
# # tag_field: str = type(payload).__name__
# payload = msgspec.Raw(plbytes)
# msg = Msg(
# cid=cid,
# pld=payload,
# # Header(
# # payload_tag=tag_field,
# # # dialog_id,
# # ),
# )
# return codec.enc.encode(msg)
# TODO: sub-decoded `Raw` fields?
# -[ ] see `MsgCodec._payload_decs` notes
# [x] TODO: a sub-decoder system as well? => No!
#
# XXX if we wanted something more complex then field name str-keys
# we might need a header field type to describe the lookup sys?
# class Header(Struct, tag=True):
# '''
# A msg header which defines payload properties
# '''
# payload_tag: str|None = None
#def mk_tagged_union_dec(
# tagged_structs: list[Struct],
#) -> tuple[
# list[str],
# msgpack.Decoder,
#]:
# '''
# Create a `msgpack.Decoder` for an input `list[msgspec.Struct]`
# and return a `list[str]` of each struct's `tag_field: str` value
# which can be used to "map to" the initialized dec.
# '''
# # See "tagged unions" docs:
# # https://jcristharif.com/msgspec/structs.html#tagged-unions
# # "The quickest way to enable tagged unions is to set tag=True when
# # defining every struct type in the union. In this case tag_field
# # defaults to "type", and tag defaults to the struct class name
# # (e.g. "Get")."
# first: Struct = tagged_structs[0]
# types_union: Union[Type[Struct]] = Union[
# first
# ]|Any
# tags: list[str] = [first.__name__]
# for struct in tagged_structs[1:]:
# types_union |= struct
# tags.append(
# getattr(
# struct,
# struct.__struct_config__.tag_field,
# struct.__name__,
# )
# )
# dec = msgpack.Decoder(types_union)
# return (
# tags,
# dec,
# )
# -[x] do we still want to try and support the sub-decoder with
# `.Raw` technique in the case that the `Generic` approach gives
# future grief?
# => NO, since we went with the `PldRx` approach instead B)
#
# IF however you want to see the code that was staged for this
# from wayyy back, see the pure removal commit.
def mk_codec(
@ -644,10 +496,6 @@ _def_tractor_codec: MsgCodec = mk_codec(
# 3. We similarly set the pending values for the child nurseries
# of the *current* task.
#
# TODO: STOP USING THIS, since it's basically a global and won't
# allow sub-IPC-ctxs to limit the msg-spec however desired..
# _ctxvar_MsgCodec: MsgCodec = RunVar(
_ctxvar_MsgCodec: ContextVar[MsgCodec] = ContextVar(
'msgspec_codec',
default=_def_tractor_codec,
@ -782,3 +630,31 @@ def limit_msg_spec(
# # import pdbp; pdbp.set_trace()
# assert ext_codec.pld_spec == extended_spec
# yield ext_codec
# TODO: make something similar to this inside `._codec` such that
# user can just pass a type table of some sort?
# -[ ] we would need to decode all msgs to `pretty_struct.Struct`
# and then call `.to_dict()` on them?
# -[x] we're going to need to re-impl all the stuff changed in the
# runtime port such that it can handle dicts or `Msg`s?
#
# def mk_dict_msg_codec_hooks() -> tuple[Callable, Callable]:
# '''
# Deliver a `enc_hook()`/`dec_hook()` pair which does
# manual convertion from our above native `Msg` set
# to `dict` equivalent (wire msgs) in order to keep legacy compat
# with the original runtime implementation.
#
# Note: this is is/was primarly used while moving the core
# runtime over to using native `Msg`-struct types wherein we
# start with the send side emitting without loading
# a typed-decoder and then later flipping the switch over to
# load to the native struct types once all runtime usage has
# been adjusted appropriately.
#
# '''
# return (
# # enc_to_dict,
# dec_from_dict,
# )

View File

@ -26,7 +26,6 @@ from __future__ import annotations
import types
from typing import (
Any,
# Callable,
Generic,
Literal,
Type,
@ -161,7 +160,6 @@ class SpawnSpec(
bind_addrs: list[tuple[str, int]]
# TODO: caps based RPC support in the payload?
#
# -[ ] integration with our ``enable_modules: list[str]`` caps sys.
@ -314,8 +312,9 @@ class Started(
pld: PayloadT|Raw
# TODO: instead of using our existing `Start`
# for this (as we did with the original `{'cmd': ..}` style)
# TODO: cancel request dedicated msg?
# -[ ] instead of using our existing `Start`?
#
# class Cancel:
# cid: str
@ -477,12 +476,16 @@ def from_dict_msg(
)
return msgT(**dict_msg)
# TODO: should be make a msg version of `ContextCancelled?`
# and/or with a scope field or a full `ActorCancelled`?
# TODO: should be make a set of cancel msgs?
# -[ ] a version of `ContextCancelled`?
# |_ and/or with a scope field?
# -[ ] or, a full `ActorCancelled`?
#
# class Cancelled(MsgType):
# cid: str
# TODO what about overruns?
#
# -[ ] what about overruns?
#
# class Overrun(MsgType):
# cid: str
@ -564,10 +567,17 @@ def mk_msg_spec(
Create a payload-(data-)type-parameterized IPC message specification.
Allows generating IPC msg types from the above builtin set
with a payload (field) restricted data-type via the `Msg.pld:
PayloadT` type var. This allows runtime-task contexts to use
the python type system to limit/filter payload values as
determined by the input `payload_type_union: Union[Type]`.
with a payload (field) restricted data-type, the `Msg.pld: PayloadT`.
This allows runtime-task contexts to use the python type system
to limit/filter payload values as determined by the input
`payload_type_union: Union[Type]`.
Notes: originally multiple approaches for constructing the
type-union passed to `msgspec` were attempted as selected via the
`spec_build_method`, but it turns out only the defaul method
'indexed_generics' seems to work reliably in all use cases. As
such, the others will likely be removed in the near future.
'''
submsg_types: list[MsgType] = Msg.__subclasses__()
@ -707,31 +717,3 @@ def mk_msg_spec(
+
ipc_msg_types,
)
# TODO: make something similar to this inside `._codec` such that
# user can just pass a type table of some sort?
# -[ ] we would need to decode all msgs to `pretty_struct.Struct`
# and then call `.to_dict()` on them?
# -[ ] we're going to need to re-impl all the stuff changed in the
# runtime port such that it can handle dicts or `Msg`s?
#
# def mk_dict_msg_codec_hooks() -> tuple[Callable, Callable]:
# '''
# Deliver a `enc_hook()`/`dec_hook()` pair which does
# manual convertion from our above native `Msg` set
# to `dict` equivalent (wire msgs) in order to keep legacy compat
# with the original runtime implementation.
#
# Note: this is is/was primarly used while moving the core
# runtime over to using native `Msg`-struct types wherein we
# start with the send side emitting without loading
# a typed-decoder and then later flipping the switch over to
# load to the native struct types once all runtime usage has
# been adjusted appropriately.
#
# '''
# return (
# # enc_to_dict,
# dec_from_dict,
# )