WIP: solved the modden client hang..
parent
c35576e196
commit
3e1d033708
|
@ -43,12 +43,17 @@ import warnings
|
||||||
|
|
||||||
import trio
|
import trio
|
||||||
|
|
||||||
|
# from .devx import (
|
||||||
|
# maybe_wait_for_debugger,
|
||||||
|
# pause,
|
||||||
|
# )
|
||||||
from ._exceptions import (
|
from ._exceptions import (
|
||||||
# _raise_from_no_key_in_msg,
|
# _raise_from_no_key_in_msg,
|
||||||
unpack_error,
|
unpack_error,
|
||||||
pack_error,
|
pack_error,
|
||||||
ContextCancelled,
|
ContextCancelled,
|
||||||
# MessagingError,
|
# MessagingError,
|
||||||
|
RemoteActorError,
|
||||||
StreamOverrun,
|
StreamOverrun,
|
||||||
)
|
)
|
||||||
from .log import get_logger
|
from .log import get_logger
|
||||||
|
@ -64,6 +69,164 @@ if TYPE_CHECKING:
|
||||||
log = get_logger(__name__)
|
log = get_logger(__name__)
|
||||||
|
|
||||||
|
|
||||||
|
async def _drain_to_final_msg(
|
||||||
|
ctx: Context,
|
||||||
|
) -> list[dict]:
|
||||||
|
|
||||||
|
# ) -> tuple[
|
||||||
|
# Any|Exception,
|
||||||
|
# list[dict],
|
||||||
|
# ]:
|
||||||
|
raise_overrun: bool = not ctx._allow_overruns
|
||||||
|
|
||||||
|
# wait for a final context result by collecting (but
|
||||||
|
# basically ignoring) any bi-dir-stream msgs still in transit
|
||||||
|
# from the far end.
|
||||||
|
pre_result_drained: list[dict] = []
|
||||||
|
while not ctx._remote_error:
|
||||||
|
try:
|
||||||
|
# NOTE: this REPL usage actually works here dawg! Bo
|
||||||
|
# from .devx._debug import pause
|
||||||
|
# await pause()
|
||||||
|
# if re := ctx._remote_error:
|
||||||
|
# ctx._maybe_raise_remote_err(
|
||||||
|
# re,
|
||||||
|
# # NOTE: obvi we don't care if we
|
||||||
|
# # overran the far end if we're already
|
||||||
|
# # waiting on a final result (msg).
|
||||||
|
# raise_overrun_from_self=raise_overrun,
|
||||||
|
# )
|
||||||
|
|
||||||
|
# TODO: bad idea?
|
||||||
|
# with trio.CancelScope() as res_cs:
|
||||||
|
# ctx._res_scope = res_cs
|
||||||
|
# msg: dict = await ctx._recv_chan.receive()
|
||||||
|
# if res_cs.cancelled_caught:
|
||||||
|
|
||||||
|
# from .devx._debug import pause
|
||||||
|
# await pause()
|
||||||
|
msg: dict = await ctx._recv_chan.receive()
|
||||||
|
ctx._result: Any = msg['return']
|
||||||
|
log.runtime(
|
||||||
|
'Context delivered final result msg:\n'
|
||||||
|
f'{pformat(msg)}'
|
||||||
|
)
|
||||||
|
pre_result_drained.append(msg)
|
||||||
|
# NOTE: we don't need to do this right?
|
||||||
|
# XXX: only close the rx mem chan AFTER
|
||||||
|
# a final result is retreived.
|
||||||
|
# if ctx._recv_chan:
|
||||||
|
# await ctx._recv_chan.aclose()
|
||||||
|
break
|
||||||
|
|
||||||
|
# NOTE: we get here if the far end was
|
||||||
|
# `ContextCancelled` in 2 cases:
|
||||||
|
# 1. we requested the cancellation and thus
|
||||||
|
# SHOULD NOT raise that far end error,
|
||||||
|
# 2. WE DID NOT REQUEST that cancel and thus
|
||||||
|
# SHOULD RAISE HERE!
|
||||||
|
except trio.Cancelled:
|
||||||
|
|
||||||
|
# CASE 2: mask the local cancelled-error(s)
|
||||||
|
# only when we are sure the remote error is
|
||||||
|
# the source cause of this local task's
|
||||||
|
# cancellation.
|
||||||
|
if re := ctx._remote_error:
|
||||||
|
ctx._maybe_raise_remote_err(re)
|
||||||
|
|
||||||
|
# CASE 1: we DID request the cancel we simply
|
||||||
|
# continue to bubble up as normal.
|
||||||
|
raise
|
||||||
|
|
||||||
|
except KeyError:
|
||||||
|
|
||||||
|
if 'yield' in msg:
|
||||||
|
# far end task is still streaming to us so discard
|
||||||
|
log.warning(f'Discarding std "yield"\n{msg}')
|
||||||
|
pre_result_drained.append(msg)
|
||||||
|
continue
|
||||||
|
|
||||||
|
# TODO: work out edge cases here where
|
||||||
|
# a stream is open but the task also calls
|
||||||
|
# this?
|
||||||
|
# -[ ] should be a runtime error if a stream is open
|
||||||
|
# right?
|
||||||
|
elif 'stop' in msg:
|
||||||
|
log.cancel(
|
||||||
|
'Remote stream terminated due to "stop" msg:\n'
|
||||||
|
f'{msg}'
|
||||||
|
)
|
||||||
|
pre_result_drained.append(msg)
|
||||||
|
continue
|
||||||
|
|
||||||
|
# internal error should never get here
|
||||||
|
assert msg.get('cid'), (
|
||||||
|
"Received internal error at portal?"
|
||||||
|
)
|
||||||
|
|
||||||
|
# XXX fallthrough to handle expected error XXX
|
||||||
|
re: Exception|None = ctx._remote_error
|
||||||
|
if re:
|
||||||
|
log.critical(
|
||||||
|
'Remote ctx terminated due to "error" msg:\n'
|
||||||
|
f'{re}'
|
||||||
|
)
|
||||||
|
assert msg is ctx._cancel_msg
|
||||||
|
# NOTE: this solved a super dupe edge case XD
|
||||||
|
# this was THE super duper edge case of:
|
||||||
|
# - local task opens a remote task,
|
||||||
|
# - requests remote cancellation of far end
|
||||||
|
# ctx/tasks,
|
||||||
|
# - needs to wait for the cancel ack msg
|
||||||
|
# (ctxc) or some result in the race case
|
||||||
|
# where the other side's task returns
|
||||||
|
# before the cancel request msg is ever
|
||||||
|
# rxed and processed,
|
||||||
|
# - here this surrounding drain loop (which
|
||||||
|
# iterates all ipc msgs until the ack or
|
||||||
|
# an early result arrives) was NOT exiting
|
||||||
|
# since we are the edge case: local task
|
||||||
|
# does not re-raise any ctxc it receives
|
||||||
|
# IFF **it** was the cancellation
|
||||||
|
# requester..
|
||||||
|
# will raise if necessary, ow break from
|
||||||
|
# loop presuming any error terminates the
|
||||||
|
# context!
|
||||||
|
ctx._maybe_raise_remote_err(
|
||||||
|
re,
|
||||||
|
# NOTE: obvi we don't care if we
|
||||||
|
# overran the far end if we're already
|
||||||
|
# waiting on a final result (msg).
|
||||||
|
# raise_overrun_from_self=False,
|
||||||
|
raise_overrun_from_self=raise_overrun,
|
||||||
|
)
|
||||||
|
|
||||||
|
break # OOOOOF, yeah obvi we need this..
|
||||||
|
|
||||||
|
# XXX we should never really get here
|
||||||
|
# right! since `._deliver_msg()` should
|
||||||
|
# always have detected an {'error': ..}
|
||||||
|
# msg and already called this right!?!
|
||||||
|
elif error := unpack_error(
|
||||||
|
msg=msg,
|
||||||
|
chan=ctx._portal.channel,
|
||||||
|
hide_tb=False,
|
||||||
|
):
|
||||||
|
log.critical('SHOULD NEVER GET HERE!?')
|
||||||
|
assert msg is ctx._cancel_msg
|
||||||
|
assert error.msgdata == ctx._remote_error.msgdata
|
||||||
|
from .devx._debug import pause
|
||||||
|
await pause()
|
||||||
|
ctx._maybe_cancel_and_set_remote_error(error)
|
||||||
|
ctx._maybe_raise_remote_err(error)
|
||||||
|
|
||||||
|
else:
|
||||||
|
# bubble the original src key error
|
||||||
|
raise
|
||||||
|
|
||||||
|
return pre_result_drained
|
||||||
|
|
||||||
|
|
||||||
# TODO: make this a msgspec.Struct!
|
# TODO: make this a msgspec.Struct!
|
||||||
@dataclass
|
@dataclass
|
||||||
class Context:
|
class Context:
|
||||||
|
@ -118,6 +281,7 @@ class Context:
|
||||||
# which is exactly the primitive that allows for
|
# which is exactly the primitive that allows for
|
||||||
# cross-actor-task-supervision and thus SC.
|
# cross-actor-task-supervision and thus SC.
|
||||||
_scope: trio.CancelScope | None = None
|
_scope: trio.CancelScope | None = None
|
||||||
|
# _res_scope: trio.CancelScope|None = None
|
||||||
|
|
||||||
# on a clean exit there should be a final value
|
# on a clean exit there should be a final value
|
||||||
# delivered from the far end "callee" task, so
|
# delivered from the far end "callee" task, so
|
||||||
|
@ -205,6 +369,10 @@ class Context:
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
|
||||||
|
# @property
|
||||||
|
# def is_waiting_result(self) -> bool:
|
||||||
|
# return bool(self._res_scope)
|
||||||
|
|
||||||
@property
|
@property
|
||||||
def side(self) -> str:
|
def side(self) -> str:
|
||||||
'''
|
'''
|
||||||
|
@ -247,7 +415,11 @@ class Context:
|
||||||
await self.chan.send({'yield': data, 'cid': self.cid})
|
await self.chan.send({'yield': data, 'cid': self.cid})
|
||||||
|
|
||||||
async def send_stop(self) -> None:
|
async def send_stop(self) -> None:
|
||||||
await self.chan.send({'stop': True, 'cid': self.cid})
|
# await pause()
|
||||||
|
await self.chan.send({
|
||||||
|
'stop': True,
|
||||||
|
'cid': self.cid
|
||||||
|
})
|
||||||
|
|
||||||
def _maybe_cancel_and_set_remote_error(
|
def _maybe_cancel_and_set_remote_error(
|
||||||
self,
|
self,
|
||||||
|
@ -320,27 +492,37 @@ class Context:
|
||||||
# XXX: set the remote side's error so that after we cancel
|
# XXX: set the remote side's error so that after we cancel
|
||||||
# whatever task is the opener of this context it can raise
|
# whatever task is the opener of this context it can raise
|
||||||
# that error as the reason.
|
# that error as the reason.
|
||||||
|
# if self._remote_error:
|
||||||
|
# return
|
||||||
|
|
||||||
|
# breakpoint()
|
||||||
|
log.cancel(
|
||||||
|
'Setting remote error for ctx \n'
|
||||||
|
f'<= remote ctx uid: {self.chan.uid}\n'
|
||||||
|
f'=>\n{error}'
|
||||||
|
)
|
||||||
self._remote_error: BaseException = error
|
self._remote_error: BaseException = error
|
||||||
|
|
||||||
if (
|
if (
|
||||||
isinstance(error, ContextCancelled)
|
isinstance(error, ContextCancelled)
|
||||||
):
|
):
|
||||||
# always record the cancelling actor's uid since its cancellation
|
|
||||||
# state is linked and we want to know which process was
|
|
||||||
# the cause / requester of the cancellation.
|
|
||||||
self._canceller = error.canceller
|
|
||||||
|
|
||||||
log.cancel(
|
log.cancel(
|
||||||
'Remote task-context was cancelled for '
|
'Remote task-context was cancelled for '
|
||||||
f'actor: {self.chan.uid}\n'
|
f'actor: {self.chan.uid}\n'
|
||||||
f'task: {self.cid}\n'
|
f'task: {self.cid}\n'
|
||||||
f'canceller: {error.canceller}\n'
|
f'canceller: {error.canceller}\n'
|
||||||
)
|
)
|
||||||
|
# always record the cancelling actor's uid since its cancellation
|
||||||
|
# state is linked and we want to know which process was
|
||||||
|
# the cause / requester of the cancellation.
|
||||||
|
# if error.canceller is None:
|
||||||
|
# import pdbp; pdbp.set_trace()
|
||||||
|
|
||||||
|
# breakpoint()
|
||||||
|
self._canceller = error.canceller
|
||||||
|
|
||||||
|
|
||||||
if self._cancel_called:
|
if self._cancel_called:
|
||||||
# from .devx._debug import breakpoint
|
|
||||||
# await breakpoint()
|
|
||||||
|
|
||||||
# this is an expected cancel request response message
|
# this is an expected cancel request response message
|
||||||
# and we **don't need to raise it** in local cancel
|
# and we **don't need to raise it** in local cancel
|
||||||
# scope since it will potentially override a real error.
|
# scope since it will potentially override a real error.
|
||||||
|
@ -348,10 +530,11 @@ class Context:
|
||||||
|
|
||||||
else:
|
else:
|
||||||
log.error(
|
log.error(
|
||||||
f'Remote context error,\n'
|
f'Remote context error:\n'
|
||||||
f'remote actor: {self.chan.uid}\n'
|
f'{error}\n'
|
||||||
f'task: {self.cid}\n'
|
f'{pformat(self)}\n'
|
||||||
f'{error}'
|
# f'remote actor: {self.chan.uid}\n'
|
||||||
|
# f'cid: {self.cid}\n'
|
||||||
)
|
)
|
||||||
self._canceller = self.chan.uid
|
self._canceller = self.chan.uid
|
||||||
|
|
||||||
|
@ -376,9 +559,11 @@ class Context:
|
||||||
self._scope.cancel()
|
self._scope.cancel()
|
||||||
|
|
||||||
# NOTE: this REPL usage actually works here dawg! Bo
|
# NOTE: this REPL usage actually works here dawg! Bo
|
||||||
# from .devx._debug import pause
|
|
||||||
# await pause()
|
# await pause()
|
||||||
|
|
||||||
|
# TODO: maybe we have to use `._res_scope.cancel()` if it
|
||||||
|
# exists?
|
||||||
|
|
||||||
async def cancel(
|
async def cancel(
|
||||||
self,
|
self,
|
||||||
timeout: float = 0.616,
|
timeout: float = 0.616,
|
||||||
|
@ -395,6 +580,8 @@ class Context:
|
||||||
log.cancel(
|
log.cancel(
|
||||||
f'Cancelling {side} side of context to {self.chan.uid}'
|
f'Cancelling {side} side of context to {self.chan.uid}'
|
||||||
)
|
)
|
||||||
|
|
||||||
|
# await pause()
|
||||||
self._cancel_called: bool = True
|
self._cancel_called: bool = True
|
||||||
|
|
||||||
# caller side who entered `Portal.open_context()`
|
# caller side who entered `Portal.open_context()`
|
||||||
|
@ -484,13 +671,11 @@ class Context:
|
||||||
'''
|
'''
|
||||||
actor: Actor = current_actor()
|
actor: Actor = current_actor()
|
||||||
|
|
||||||
# here we create a mem chan that corresponds to the
|
# If the surrounding context has been cancelled by some
|
||||||
# far end caller / callee.
|
# task with a handle to THIS, we error here immediately
|
||||||
|
# since it likely means the surrounding lexical-scope has
|
||||||
# Likewise if the surrounding context has been cancelled we error here
|
# errored, been `trio.Cancelled` or at the least
|
||||||
# since it likely means the surrounding block was exited or
|
# `Context.cancel()` was called by some task.
|
||||||
# killed
|
|
||||||
|
|
||||||
if self._cancel_called:
|
if self._cancel_called:
|
||||||
|
|
||||||
# XXX NOTE: ALWAYS RAISE any remote error here even if
|
# XXX NOTE: ALWAYS RAISE any remote error here even if
|
||||||
|
@ -503,6 +688,11 @@ class Context:
|
||||||
# actually try to stream - a cancel msg was already
|
# actually try to stream - a cancel msg was already
|
||||||
# sent to the other side!
|
# sent to the other side!
|
||||||
if self._remote_error:
|
if self._remote_error:
|
||||||
|
# NOTE: this is diff then calling
|
||||||
|
# `._maybe_raise_from_remote_msg()` specifically
|
||||||
|
# because any task entering this `.open_stream()`
|
||||||
|
# AFTER cancellation has already been requested,
|
||||||
|
# we DO NOT want to absorb any ctxc ACK silently!
|
||||||
raise self._remote_error
|
raise self._remote_error
|
||||||
|
|
||||||
# XXX NOTE: if no `ContextCancelled` has been responded
|
# XXX NOTE: if no `ContextCancelled` has been responded
|
||||||
|
@ -529,7 +719,7 @@ class Context:
|
||||||
# to send a stop from the caller to the callee in the
|
# to send a stop from the caller to the callee in the
|
||||||
# single-direction-stream case you'll get a lookup error
|
# single-direction-stream case you'll get a lookup error
|
||||||
# currently.
|
# currently.
|
||||||
ctx = actor.get_context(
|
ctx: Context = actor.get_context(
|
||||||
self.chan,
|
self.chan,
|
||||||
self.cid,
|
self.cid,
|
||||||
msg_buffer_size=msg_buffer_size,
|
msg_buffer_size=msg_buffer_size,
|
||||||
|
@ -548,6 +738,19 @@ class Context:
|
||||||
'The underlying channel for this stream was already closed!?'
|
'The underlying channel for this stream was already closed!?'
|
||||||
)
|
)
|
||||||
|
|
||||||
|
# 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(
|
async with MsgStream(
|
||||||
ctx=self,
|
ctx=self,
|
||||||
rx_chan=ctx._recv_chan,
|
rx_chan=ctx._recv_chan,
|
||||||
|
@ -567,11 +770,37 @@ class Context:
|
||||||
# await trio.lowlevel.checkpoint()
|
# await trio.lowlevel.checkpoint()
|
||||||
yield stream
|
yield stream
|
||||||
|
|
||||||
# NOTE: Make the stream "one-shot use". On exit,
|
|
||||||
# signal
|
# XXX: (MEGA IMPORTANT) if this is a root opened process we
|
||||||
# ``trio.EndOfChannel``/``StopAsyncIteration`` to
|
# wait for any immediate child in debug before popping the
|
||||||
# the far end.
|
# context from the runtime msg loop otherwise inside
|
||||||
await stream.aclose()
|
# ``Actor._push_result()`` 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()
|
||||||
|
|
||||||
|
# if re := ctx._remote_error:
|
||||||
|
# ctx._maybe_raise_remote_err(
|
||||||
|
# re,
|
||||||
|
# raise_ctxc_from_self_call=True,
|
||||||
|
# )
|
||||||
|
# await trio.lowlevel.checkpoint()
|
||||||
|
|
||||||
finally:
|
finally:
|
||||||
if self._portal:
|
if self._portal:
|
||||||
|
@ -587,7 +816,10 @@ class Context:
|
||||||
def _maybe_raise_remote_err(
|
def _maybe_raise_remote_err(
|
||||||
self,
|
self,
|
||||||
err: Exception,
|
err: Exception,
|
||||||
) -> None:
|
raise_ctxc_from_self_call: bool = False,
|
||||||
|
raise_overrun_from_self: bool = True,
|
||||||
|
|
||||||
|
) -> ContextCancelled|None:
|
||||||
'''
|
'''
|
||||||
Maybe raise a remote error depending on who (which task from
|
Maybe raise a remote error depending on who (which task from
|
||||||
which actor) requested a cancellation (if any).
|
which actor) requested a cancellation (if any).
|
||||||
|
@ -603,13 +835,21 @@ class Context:
|
||||||
# "error"-msg.
|
# "error"-msg.
|
||||||
our_uid: tuple[str, str] = current_actor().uid
|
our_uid: tuple[str, str] = current_actor().uid
|
||||||
if (
|
if (
|
||||||
isinstance(err, ContextCancelled)
|
(not raise_ctxc_from_self_call
|
||||||
|
and isinstance(err, ContextCancelled)
|
||||||
and (
|
and (
|
||||||
self._cancel_called
|
self._cancel_called
|
||||||
or self.chan._cancel_called
|
or self.chan._cancel_called
|
||||||
or self.canceller == our_uid
|
or self.canceller == our_uid
|
||||||
or tuple(err.canceller) == our_uid
|
or tuple(err.canceller) == our_uid)
|
||||||
)
|
)
|
||||||
|
or
|
||||||
|
(not raise_overrun_from_self
|
||||||
|
and isinstance(err, RemoteActorError)
|
||||||
|
and err.msgdata['type_str'] == 'StreamOverrun'
|
||||||
|
and tuple(err.msgdata['sender']) == our_uid
|
||||||
|
)
|
||||||
|
|
||||||
):
|
):
|
||||||
# NOTE: we set the local scope error to any "self
|
# NOTE: we set the local scope error to any "self
|
||||||
# cancellation" error-response thus "absorbing"
|
# cancellation" error-response thus "absorbing"
|
||||||
|
@ -661,77 +901,196 @@ class Context:
|
||||||
assert self._portal, "Context.result() can not be called from callee!"
|
assert self._portal, "Context.result() can not be called from callee!"
|
||||||
assert self._recv_chan
|
assert self._recv_chan
|
||||||
|
|
||||||
if re := self._remote_error:
|
raise_overrun: bool = not self._allow_overruns
|
||||||
return self._maybe_raise_remote_err(re)
|
# if re := self._remote_error:
|
||||||
|
# return self._maybe_raise_remote_err(
|
||||||
|
# re,
|
||||||
|
# # NOTE: obvi we don't care if we
|
||||||
|
# # overran the far end if we're already
|
||||||
|
# # waiting on a final result (msg).
|
||||||
|
# raise_overrun_from_self=raise_overrun,
|
||||||
|
# )
|
||||||
|
|
||||||
|
res_placeholder: int = id(self)
|
||||||
if (
|
if (
|
||||||
self._result == id(self)
|
self._result == res_placeholder
|
||||||
and not self._remote_error
|
and not self._remote_error
|
||||||
and not self._recv_chan._closed # type: ignore
|
and not self._recv_chan._closed # type: ignore
|
||||||
):
|
):
|
||||||
# wait for a final context result consuming
|
|
||||||
# and discarding any bi dir stream msgs still
|
|
||||||
# in transit from the far end.
|
|
||||||
while True:
|
|
||||||
try:
|
|
||||||
msg = await self._recv_chan.receive()
|
|
||||||
self._result: Any = msg['return']
|
|
||||||
|
|
||||||
# NOTE: we don't need to do this right?
|
# wait for a final context result by collecting (but
|
||||||
# XXX: only close the rx mem chan AFTER
|
# basically ignoring) any bi-dir-stream msgs still in transit
|
||||||
# a final result is retreived.
|
# from the far end.
|
||||||
# if self._recv_chan:
|
drained_msgs: list[dict] = await _drain_to_final_msg(ctx=self)
|
||||||
# await self._recv_chan.aclose()
|
log.runtime(
|
||||||
|
'Ctx drained pre-result msgs:\n'
|
||||||
break
|
f'{drained_msgs}'
|
||||||
|
|
||||||
# NOTE: we get here if the far end was
|
|
||||||
# `ContextCancelled` in 2 cases:
|
|
||||||
# 1. we requested the cancellation and thus
|
|
||||||
# SHOULD NOT raise that far end error,
|
|
||||||
# 2. WE DID NOT REQUEST that cancel and thus
|
|
||||||
# SHOULD RAISE HERE!
|
|
||||||
except trio.Cancelled:
|
|
||||||
|
|
||||||
# CASE 2: mask the local cancelled-error(s)
|
|
||||||
# only when we are sure the remote error is the
|
|
||||||
# (likely) source cause of this local runtime
|
|
||||||
# task's cancellation.
|
|
||||||
if re := self._remote_error:
|
|
||||||
self._maybe_raise_remote_err(re)
|
|
||||||
|
|
||||||
# CASE 1: we DID request the cancel we simply
|
|
||||||
# continue to bubble up as normal.
|
|
||||||
raise
|
|
||||||
|
|
||||||
except KeyError: # as msgerr:
|
|
||||||
|
|
||||||
if 'yield' in msg:
|
|
||||||
# far end task is still streaming to us so discard
|
|
||||||
log.warning(f'Discarding stream delivered {msg}')
|
|
||||||
continue
|
|
||||||
|
|
||||||
elif 'stop' in msg:
|
|
||||||
log.debug('Remote stream terminated')
|
|
||||||
continue
|
|
||||||
|
|
||||||
# internal error should never get here
|
|
||||||
assert msg.get('cid'), (
|
|
||||||
"Received internal error at portal?"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
if err:= unpack_error(
|
# TODO: implement via helper func ^^^^
|
||||||
msg,
|
# pre_result_drained: list[dict] = []
|
||||||
self._portal.channel
|
# while not self._remote_error:
|
||||||
): # from msgerr
|
# try:
|
||||||
self._maybe_cancel_and_set_remote_error(err)
|
# # NOTE: this REPL usage actually works here dawg! Bo
|
||||||
self._maybe_raise_remote_err(err)
|
# # from .devx._debug import pause
|
||||||
|
# # await pause()
|
||||||
|
# # if re := self._remote_error:
|
||||||
|
# # self._maybe_raise_remote_err(
|
||||||
|
# # re,
|
||||||
|
# # # NOTE: obvi we don't care if we
|
||||||
|
# # # overran the far end if we're already
|
||||||
|
# # # waiting on a final result (msg).
|
||||||
|
# # raise_overrun_from_self=raise_overrun,
|
||||||
|
# # )
|
||||||
|
|
||||||
else:
|
# # TODO: bad idea?
|
||||||
raise
|
# # with trio.CancelScope() as res_cs:
|
||||||
|
# # self._res_scope = res_cs
|
||||||
|
# # msg: dict = await self._recv_chan.receive()
|
||||||
|
# # if res_cs.cancelled_caught:
|
||||||
|
|
||||||
if re := self._remote_error:
|
# # from .devx._debug import pause
|
||||||
return self._maybe_raise_remote_err(re)
|
# # await pause()
|
||||||
|
# msg: dict = await self._recv_chan.receive()
|
||||||
|
# self._result: Any = msg['return']
|
||||||
|
# log.runtime(
|
||||||
|
# 'Context delivered final result msg:\n'
|
||||||
|
# f'{pformat(msg)}'
|
||||||
|
# )
|
||||||
|
# # NOTE: we don't need to do this right?
|
||||||
|
# # XXX: only close the rx mem chan AFTER
|
||||||
|
# # a final result is retreived.
|
||||||
|
# # if self._recv_chan:
|
||||||
|
# # await self._recv_chan.aclose()
|
||||||
|
# break
|
||||||
|
|
||||||
|
# # NOTE: we get here if the far end was
|
||||||
|
# # `ContextCancelled` in 2 cases:
|
||||||
|
# # 1. we requested the cancellation and thus
|
||||||
|
# # SHOULD NOT raise that far end error,
|
||||||
|
# # 2. WE DID NOT REQUEST that cancel and thus
|
||||||
|
# # SHOULD RAISE HERE!
|
||||||
|
# except trio.Cancelled:
|
||||||
|
|
||||||
|
# # CASE 2: mask the local cancelled-error(s)
|
||||||
|
# # only when we are sure the remote error is
|
||||||
|
# # the source cause of this local task's
|
||||||
|
# # cancellation.
|
||||||
|
# if re := self._remote_error:
|
||||||
|
# self._maybe_raise_remote_err(re)
|
||||||
|
|
||||||
|
# # CASE 1: we DID request the cancel we simply
|
||||||
|
# # continue to bubble up as normal.
|
||||||
|
# raise
|
||||||
|
|
||||||
|
# except KeyError:
|
||||||
|
|
||||||
|
# if 'yield' in msg:
|
||||||
|
# # far end task is still streaming to us so discard
|
||||||
|
# log.warning(f'Discarding std "yield"\n{msg}')
|
||||||
|
# pre_result_drained.append(msg)
|
||||||
|
# continue
|
||||||
|
|
||||||
|
# # TODO: work out edge cases here where
|
||||||
|
# # a stream is open but the task also calls
|
||||||
|
# # this?
|
||||||
|
# # -[ ] should be a runtime error if a stream is open
|
||||||
|
# # right?
|
||||||
|
# elif 'stop' in msg:
|
||||||
|
# log.cancel(
|
||||||
|
# 'Remote stream terminated due to "stop" msg:\n'
|
||||||
|
# f'{msg}'
|
||||||
|
# )
|
||||||
|
# pre_result_drained.append(msg)
|
||||||
|
# continue
|
||||||
|
|
||||||
|
# # internal error should never get here
|
||||||
|
# assert msg.get('cid'), (
|
||||||
|
# "Received internal error at portal?"
|
||||||
|
# )
|
||||||
|
|
||||||
|
# # XXX fallthrough to handle expected error XXX
|
||||||
|
# re: Exception|None = self._remote_error
|
||||||
|
# if re:
|
||||||
|
# log.critical(
|
||||||
|
# 'Remote ctx terminated due to "error" msg:\n'
|
||||||
|
# f'{re}'
|
||||||
|
# )
|
||||||
|
# assert msg is self._cancel_msg
|
||||||
|
# # NOTE: this solved a super dupe edge case XD
|
||||||
|
# # this was THE super duper edge case of:
|
||||||
|
# # - local task opens a remote task,
|
||||||
|
# # - requests remote cancellation of far end
|
||||||
|
# # ctx/tasks,
|
||||||
|
# # - needs to wait for the cancel ack msg
|
||||||
|
# # (ctxc) or some result in the race case
|
||||||
|
# # where the other side's task returns
|
||||||
|
# # before the cancel request msg is ever
|
||||||
|
# # rxed and processed,
|
||||||
|
# # - here this surrounding drain loop (which
|
||||||
|
# # iterates all ipc msgs until the ack or
|
||||||
|
# # an early result arrives) was NOT exiting
|
||||||
|
# # since we are the edge case: local task
|
||||||
|
# # does not re-raise any ctxc it receives
|
||||||
|
# # IFF **it** was the cancellation
|
||||||
|
# # requester..
|
||||||
|
# # will raise if necessary, ow break from
|
||||||
|
# # loop presuming any error terminates the
|
||||||
|
# # context!
|
||||||
|
# self._maybe_raise_remote_err(
|
||||||
|
# re,
|
||||||
|
# # NOTE: obvi we don't care if we
|
||||||
|
# # overran the far end if we're already
|
||||||
|
# # waiting on a final result (msg).
|
||||||
|
# # raise_overrun_from_self=False,
|
||||||
|
# raise_overrun_from_self=raise_overrun,
|
||||||
|
# )
|
||||||
|
|
||||||
|
# break # OOOOOF, yeah obvi we need this..
|
||||||
|
|
||||||
|
# # XXX we should never really get here
|
||||||
|
# # right! since `._deliver_msg()` should
|
||||||
|
# # always have detected an {'error': ..}
|
||||||
|
# # msg and already called this right!?!
|
||||||
|
# elif error := unpack_error(
|
||||||
|
# msg=msg,
|
||||||
|
# chan=self._portal.channel,
|
||||||
|
# hide_tb=False,
|
||||||
|
# ):
|
||||||
|
# log.critical('SHOULD NEVER GET HERE!?')
|
||||||
|
# assert msg is self._cancel_msg
|
||||||
|
# assert error.msgdata == self._remote_error.msgdata
|
||||||
|
# from .devx._debug import pause
|
||||||
|
# await pause()
|
||||||
|
# self._maybe_cancel_and_set_remote_error(error)
|
||||||
|
# self._maybe_raise_remote_err(error)
|
||||||
|
|
||||||
|
# else:
|
||||||
|
# # bubble the original src key error
|
||||||
|
# raise
|
||||||
|
|
||||||
|
if (
|
||||||
|
(re := self._remote_error)
|
||||||
|
and self._result == res_placeholder
|
||||||
|
):
|
||||||
|
maybe_err: Exception|None = self._maybe_raise_remote_err(
|
||||||
|
re,
|
||||||
|
# NOTE: obvi we don't care if we
|
||||||
|
# overran the far end if we're already
|
||||||
|
# waiting on a final result (msg).
|
||||||
|
# raise_overrun_from_self=False,
|
||||||
|
raise_overrun_from_self=(
|
||||||
|
raise_overrun
|
||||||
|
and
|
||||||
|
# only when we ARE NOT the canceller
|
||||||
|
# should we raise overruns, bc ow we're
|
||||||
|
# raising something we know might happen
|
||||||
|
# during cancellation ;)
|
||||||
|
(not self._cancel_called)
|
||||||
|
),
|
||||||
|
)
|
||||||
|
if maybe_err:
|
||||||
|
self._result = maybe_err
|
||||||
|
|
||||||
return self._result
|
return self._result
|
||||||
|
|
||||||
|
@ -779,7 +1138,7 @@ class Context:
|
||||||
while self._overflow_q:
|
while self._overflow_q:
|
||||||
# NOTE: these msgs should never be errors since we always do
|
# NOTE: these msgs should never be errors since we always do
|
||||||
# the check prior to checking if we're in an overrun state
|
# the check prior to checking if we're in an overrun state
|
||||||
# inside ``.deliver_msg()``.
|
# inside ``._deliver_msg()``.
|
||||||
msg = self._overflow_q.popleft()
|
msg = self._overflow_q.popleft()
|
||||||
try:
|
try:
|
||||||
await self._send_chan.send(msg)
|
await self._send_chan.send(msg)
|
||||||
|
@ -830,34 +1189,50 @@ class Context:
|
||||||
messages are eventually sent if possible.
|
messages are eventually sent if possible.
|
||||||
|
|
||||||
'''
|
'''
|
||||||
cid = self.cid
|
cid: str = self.cid
|
||||||
chan = self.chan
|
chan: Channel = self.chan
|
||||||
uid = chan.uid
|
from_uid: tuple[str, str] = chan.uid
|
||||||
send_chan: trio.MemorySendChannel = self._send_chan
|
send_chan: trio.MemorySendChannel = self._send_chan
|
||||||
|
|
||||||
log.runtime(
|
if re := unpack_error(
|
||||||
f"Delivering {msg} from {uid} to caller {cid}"
|
|
||||||
)
|
|
||||||
|
|
||||||
if (
|
|
||||||
msg.get('error') # check for field
|
|
||||||
and (
|
|
||||||
error := unpack_error(
|
|
||||||
msg,
|
msg,
|
||||||
self.chan,
|
self.chan,
|
||||||
)
|
|
||||||
)
|
|
||||||
):
|
):
|
||||||
|
log.error(
|
||||||
|
f'Delivering error-msg from {from_uid} to caller {cid}'
|
||||||
|
f'{re}'
|
||||||
|
)
|
||||||
self._cancel_msg = msg
|
self._cancel_msg = msg
|
||||||
self._maybe_cancel_and_set_remote_error(error)
|
self._maybe_cancel_and_set_remote_error(re)
|
||||||
|
|
||||||
if (
|
# XXX NEVER do this XXX..!!
|
||||||
self._in_overrun
|
# bc if the error is a ctxc and there is a task
|
||||||
):
|
# waiting on `.result()` we need the msg to be sent
|
||||||
|
# over the `send_chan`/`._recv_chan` so that the error
|
||||||
|
# is relayed to that waiter task..
|
||||||
|
# return True
|
||||||
|
#
|
||||||
|
# XXX ALSO NO!! XXX
|
||||||
|
# if self._remote_error:
|
||||||
|
# self._maybe_raise_remote_err(error)
|
||||||
|
|
||||||
|
if self._in_overrun:
|
||||||
|
log.warning(
|
||||||
|
f'Capturing overrun-msg from {from_uid} to caller {cid}'
|
||||||
|
f'{msg}'
|
||||||
|
)
|
||||||
self._overflow_q.append(msg)
|
self._overflow_q.append(msg)
|
||||||
return False
|
return False
|
||||||
|
|
||||||
try:
|
try:
|
||||||
|
log.runtime(
|
||||||
|
f'Delivering IPC `Context` msg:\n'
|
||||||
|
f'<= {from_uid}\n'
|
||||||
|
f'=> caller: {cid}\n'
|
||||||
|
f'{msg}'
|
||||||
|
)
|
||||||
|
# from .devx._debug import pause
|
||||||
|
# await pause()
|
||||||
send_chan.send_nowait(msg)
|
send_chan.send_nowait(msg)
|
||||||
return True
|
return True
|
||||||
# if an error is deteced we should always
|
# if an error is deteced we should always
|
||||||
|
@ -890,7 +1265,8 @@ class Context:
|
||||||
lines = [
|
lines = [
|
||||||
f'OVERRUN on actor-task context {cid}@{local_uid}!\n'
|
f'OVERRUN on actor-task context {cid}@{local_uid}!\n'
|
||||||
# TODO: put remote task name here if possible?
|
# TODO: put remote task name here if possible?
|
||||||
f'remote sender actor: {uid}',
|
f'sender: {from_uid}',
|
||||||
|
f'msg: {msg}',
|
||||||
# TODO: put task func name here and maybe an arrow
|
# TODO: put task func name here and maybe an arrow
|
||||||
# from sender to overrunner?
|
# from sender to overrunner?
|
||||||
# f'local task {self.func_name}'
|
# f'local task {self.func_name}'
|
||||||
|
@ -926,11 +1302,19 @@ class Context:
|
||||||
# anything different.
|
# anything different.
|
||||||
return False
|
return False
|
||||||
else:
|
else:
|
||||||
|
# raise local overrun and immediately pack as IPC
|
||||||
|
# msg for far end.
|
||||||
try:
|
try:
|
||||||
raise StreamOverrun(text)
|
raise StreamOverrun(
|
||||||
|
text,
|
||||||
|
sender=from_uid,
|
||||||
|
)
|
||||||
except StreamOverrun as err:
|
except StreamOverrun as err:
|
||||||
err_msg = pack_error(err)
|
err_msg: dict[str, dict] = pack_error(
|
||||||
err_msg['cid'] = cid
|
err,
|
||||||
|
cid=cid,
|
||||||
|
)
|
||||||
|
# err_msg['cid']: str = cid
|
||||||
try:
|
try:
|
||||||
await chan.send(err_msg)
|
await chan.send(err_msg)
|
||||||
except trio.BrokenResourceError:
|
except trio.BrokenResourceError:
|
||||||
|
|
|
@ -39,7 +39,15 @@ import trio
|
||||||
from async_generator import asynccontextmanager
|
from async_generator import asynccontextmanager
|
||||||
|
|
||||||
from .trionics import maybe_open_nursery
|
from .trionics import maybe_open_nursery
|
||||||
from ._state import current_actor
|
from .devx import (
|
||||||
|
# acquire_debug_lock,
|
||||||
|
# pause,
|
||||||
|
maybe_wait_for_debugger,
|
||||||
|
)
|
||||||
|
from ._state import (
|
||||||
|
current_actor,
|
||||||
|
debug_mode,
|
||||||
|
)
|
||||||
from ._ipc import Channel
|
from ._ipc import Channel
|
||||||
from .log import get_logger
|
from .log import get_logger
|
||||||
from .msg import NamespacePath
|
from .msg import NamespacePath
|
||||||
|
@ -48,6 +56,7 @@ from ._exceptions import (
|
||||||
unpack_error,
|
unpack_error,
|
||||||
NoResult,
|
NoResult,
|
||||||
ContextCancelled,
|
ContextCancelled,
|
||||||
|
RemoteActorError,
|
||||||
)
|
)
|
||||||
from ._context import (
|
from ._context import (
|
||||||
Context,
|
Context,
|
||||||
|
@ -55,7 +64,6 @@ from ._context import (
|
||||||
from ._streaming import (
|
from ._streaming import (
|
||||||
MsgStream,
|
MsgStream,
|
||||||
)
|
)
|
||||||
from .devx._debug import maybe_wait_for_debugger
|
|
||||||
|
|
||||||
|
|
||||||
log = get_logger(__name__)
|
log = get_logger(__name__)
|
||||||
|
@ -469,7 +477,6 @@ class Portal:
|
||||||
ctx._started_called: bool = True
|
ctx._started_called: bool = True
|
||||||
|
|
||||||
except KeyError as src_error:
|
except KeyError as src_error:
|
||||||
|
|
||||||
_raise_from_no_key_in_msg(
|
_raise_from_no_key_in_msg(
|
||||||
ctx=ctx,
|
ctx=ctx,
|
||||||
msg=msg,
|
msg=msg,
|
||||||
|
@ -494,6 +501,33 @@ class Portal:
|
||||||
# in enter tuple.
|
# in enter tuple.
|
||||||
yield ctx, first
|
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!
|
||||||
|
# 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
|
||||||
|
|
||||||
# when in allow_overruns mode there may be
|
# when in allow_overruns mode there may be
|
||||||
# lingering overflow sender tasks remaining?
|
# lingering overflow sender tasks remaining?
|
||||||
if nurse.child_tasks:
|
if nurse.child_tasks:
|
||||||
|
@ -539,7 +573,7 @@ class Portal:
|
||||||
# `.canceller: tuple[str, str]` to be same value as
|
# `.canceller: tuple[str, str]` to be same value as
|
||||||
# caught here in a `ContextCancelled.canceller`.
|
# caught here in a `ContextCancelled.canceller`.
|
||||||
#
|
#
|
||||||
# Again, there are 2 cases:
|
# AGAIN to restate the above, there are 2 cases:
|
||||||
#
|
#
|
||||||
# 1-some other context opened in this `.open_context()`
|
# 1-some other context opened in this `.open_context()`
|
||||||
# block cancelled due to a self or peer cancellation
|
# block cancelled due to a self or peer cancellation
|
||||||
|
@ -555,6 +589,16 @@ class Portal:
|
||||||
except ContextCancelled as ctxc:
|
except ContextCancelled as ctxc:
|
||||||
scope_err = ctxc
|
scope_err = ctxc
|
||||||
|
|
||||||
|
# XXX TODO XXX: FIX THIS debug_mode BUGGGG!!!
|
||||||
|
# using this code and then resuming the REPL will
|
||||||
|
# cause a SIGINT-ignoring HANG!
|
||||||
|
# -> prolly due to a stale debug lock entry..
|
||||||
|
# -[ ] USE `.stackscope` to demonstrate that (possibly
|
||||||
|
# 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
|
# CASE 2: context was cancelled by local task calling
|
||||||
# `.cancel()`, we don't raise and the exit block should
|
# `.cancel()`, we don't raise and the exit block should
|
||||||
# exit silently.
|
# exit silently.
|
||||||
|
@ -562,18 +606,23 @@ class Portal:
|
||||||
ctx._cancel_called
|
ctx._cancel_called
|
||||||
and (
|
and (
|
||||||
ctxc is ctx._remote_error
|
ctxc is ctx._remote_error
|
||||||
or
|
# ctxc.msgdata == ctx._remote_error.msgdata
|
||||||
ctxc.canceller is self.canceller
|
|
||||||
|
# TODO: uhh `Portal.canceller` ain't a thangg
|
||||||
|
# dawg? (was `self.canceller` before?!?)
|
||||||
|
and
|
||||||
|
ctxc.canceller == self.actor.uid
|
||||||
)
|
)
|
||||||
):
|
):
|
||||||
log.debug(
|
log.cancel(
|
||||||
f'Context {ctx} cancelled gracefully with:\n'
|
f'Context (cid=[{ctx.cid[-6:]}..] cancelled gracefully with:\n'
|
||||||
f'{ctxc}'
|
f'{ctxc}'
|
||||||
)
|
)
|
||||||
# CASE 1: this context was never cancelled via a local
|
# CASE 1: this context was never cancelled via a local
|
||||||
# task (tree) having called `Context.cancel()`, raise
|
# 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!
|
||||||
else:
|
else:
|
||||||
|
# await pause()
|
||||||
raise
|
raise
|
||||||
|
|
||||||
# the above `._scope` can be cancelled due to:
|
# the above `._scope` can be cancelled due to:
|
||||||
|
@ -602,8 +651,8 @@ class Portal:
|
||||||
trio.Cancelled, # NOTE: NOT from inside the ctx._scope
|
trio.Cancelled, # NOTE: NOT from inside the ctx._scope
|
||||||
KeyboardInterrupt,
|
KeyboardInterrupt,
|
||||||
|
|
||||||
) as err:
|
) as caller_err:
|
||||||
scope_err = err
|
scope_err = caller_err
|
||||||
|
|
||||||
# XXX: ALWAYS request the context to CANCEL ON any ERROR.
|
# XXX: ALWAYS request the context to CANCEL ON any ERROR.
|
||||||
# NOTE: `Context.cancel()` is conversely NEVER CALLED in
|
# NOTE: `Context.cancel()` is conversely NEVER CALLED in
|
||||||
|
@ -611,11 +660,26 @@ class Portal:
|
||||||
# handled in the block above!
|
# handled in the block above!
|
||||||
log.cancel(
|
log.cancel(
|
||||||
'Context cancelled for task due to\n'
|
'Context cancelled for task due to\n'
|
||||||
f'{err}\n'
|
f'{caller_err}\n'
|
||||||
'Sending cancel request..\n'
|
'Sending cancel request..\n'
|
||||||
f'task:{cid}\n'
|
f'task:{cid}\n'
|
||||||
f'actor:{uid}'
|
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()
|
||||||
|
log.pdb(
|
||||||
|
'Acquired debug lock! '
|
||||||
|
'Calling `ctx.cancel()`!'
|
||||||
|
)
|
||||||
|
|
||||||
try:
|
try:
|
||||||
await ctx.cancel()
|
await ctx.cancel()
|
||||||
except trio.BrokenResourceError:
|
except trio.BrokenResourceError:
|
||||||
|
@ -629,6 +693,33 @@ class Portal:
|
||||||
|
|
||||||
# no local scope error, the "clean exit with a result" case.
|
# no local scope error, the "clean exit with a result" case.
|
||||||
else:
|
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():
|
if ctx.chan.connected():
|
||||||
log.info(
|
log.info(
|
||||||
'Waiting on final context-task result for\n'
|
'Waiting on final context-task result for\n'
|
||||||
|
@ -645,13 +736,8 @@ class Portal:
|
||||||
# As per `Context._deliver_msg()`, that error IS
|
# As per `Context._deliver_msg()`, that error IS
|
||||||
# ALWAYS SET any time "callee" side fails and causes "caller
|
# ALWAYS SET any time "callee" side fails and causes "caller
|
||||||
# side" cancellation via a `ContextCancelled` here.
|
# side" cancellation via a `ContextCancelled` here.
|
||||||
# result = await ctx.result()
|
|
||||||
try:
|
try:
|
||||||
result = await ctx.result()
|
result_or_err: Exception|Any = await ctx.result()
|
||||||
log.runtime(
|
|
||||||
f'Context {fn_name} returned value from callee:\n'
|
|
||||||
f'`{result}`'
|
|
||||||
)
|
|
||||||
except BaseException as berr:
|
except BaseException as berr:
|
||||||
# on normal teardown, if we get some error
|
# on normal teardown, if we get some error
|
||||||
# raised in `Context.result()` we still want to
|
# raised in `Context.result()` we still want to
|
||||||
|
@ -663,7 +749,48 @@ class Portal:
|
||||||
scope_err = berr
|
scope_err = berr
|
||||||
raise
|
raise
|
||||||
|
|
||||||
|
# an exception type boxed in a `RemoteActorError`
|
||||||
|
# is returned (meaning it was obvi not raised).
|
||||||
|
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},
|
||||||
|
ContextCancelled(),
|
||||||
|
):
|
||||||
|
log.cancel(tbstr)
|
||||||
|
|
||||||
|
case (
|
||||||
|
{'tb_str': tbstr},
|
||||||
|
RemoteActorError(),
|
||||||
|
):
|
||||||
|
log.exception(
|
||||||
|
f'Context `{fn_name}` remotely errored:\n'
|
||||||
|
f'`{tbstr}`'
|
||||||
|
)
|
||||||
|
case (None, _):
|
||||||
|
log.runtime(
|
||||||
|
f'Context {fn_name} returned value from callee:\n'
|
||||||
|
f'`{result_or_err}`'
|
||||||
|
)
|
||||||
|
|
||||||
finally:
|
finally:
|
||||||
|
# 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._push_result()`` 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()
|
||||||
|
|
||||||
# though it should be impossible for any tasks
|
# though it should be impossible for any tasks
|
||||||
# operating *in* this scope to have survived
|
# operating *in* this scope to have survived
|
||||||
# we tear down the runtime feeder chan last
|
# we tear down the runtime feeder chan last
|
||||||
|
@ -708,6 +835,10 @@ class Portal:
|
||||||
# out any exception group or legit (remote) ctx
|
# out any exception group or legit (remote) ctx
|
||||||
# error that sourced from the remote task or its
|
# error that sourced from the remote task or its
|
||||||
# runtime.
|
# runtime.
|
||||||
|
#
|
||||||
|
# NOTE: further, this should be the only place the
|
||||||
|
# underlying feeder channel is
|
||||||
|
# once-and-only-CLOSED!
|
||||||
with trio.CancelScope(shield=True):
|
with trio.CancelScope(shield=True):
|
||||||
await ctx._recv_chan.aclose()
|
await ctx._recv_chan.aclose()
|
||||||
|
|
||||||
|
@ -737,18 +868,11 @@ class Portal:
|
||||||
f'actor:{uid}'
|
f'actor:{uid}'
|
||||||
)
|
)
|
||||||
|
|
||||||
# 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._push_result()`` 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()
|
|
||||||
|
|
||||||
# FINALLY, remove the context from runtime tracking and
|
# FINALLY, remove the context from runtime tracking and
|
||||||
# exit!
|
# exit!
|
||||||
|
log.runtime(
|
||||||
|
f'Exiting context opened with {ctx.chan.uid}'
|
||||||
|
)
|
||||||
self.actor._contexts.pop(
|
self.actor._contexts.pop(
|
||||||
(self.channel.uid, ctx.cid),
|
(self.channel.uid, ctx.cid),
|
||||||
None,
|
None,
|
||||||
|
|
|
@ -21,8 +21,9 @@ The machinery and types behind ``Context.open_stream()``
|
||||||
|
|
||||||
'''
|
'''
|
||||||
from __future__ import annotations
|
from __future__ import annotations
|
||||||
import inspect
|
|
||||||
from contextlib import asynccontextmanager as acm
|
from contextlib import asynccontextmanager as acm
|
||||||
|
import inspect
|
||||||
|
from pprint import pformat
|
||||||
from typing import (
|
from typing import (
|
||||||
Any,
|
Any,
|
||||||
Callable,
|
Callable,
|
||||||
|
@ -35,6 +36,7 @@ import trio
|
||||||
|
|
||||||
from ._exceptions import (
|
from ._exceptions import (
|
||||||
_raise_from_no_key_in_msg,
|
_raise_from_no_key_in_msg,
|
||||||
|
ContextCancelled,
|
||||||
)
|
)
|
||||||
from .log import get_logger
|
from .log import get_logger
|
||||||
from .trionics import (
|
from .trionics import (
|
||||||
|
@ -84,8 +86,8 @@ class MsgStream(trio.abc.Channel):
|
||||||
self._broadcaster = _broadcaster
|
self._broadcaster = _broadcaster
|
||||||
|
|
||||||
# flag to denote end of stream
|
# flag to denote end of stream
|
||||||
self._eoc: bool = False
|
self._eoc: bool|trio.EndOfChannel = False
|
||||||
self._closed: bool = False
|
self._closed: bool|trio.ClosedResourceError = False
|
||||||
|
|
||||||
# delegate directly to underlying mem channel
|
# delegate directly to underlying mem channel
|
||||||
def receive_nowait(self):
|
def receive_nowait(self):
|
||||||
|
@ -93,6 +95,9 @@ class MsgStream(trio.abc.Channel):
|
||||||
try:
|
try:
|
||||||
return msg['yield']
|
return msg['yield']
|
||||||
except KeyError as kerr:
|
except KeyError as kerr:
|
||||||
|
# if 'return' in msg:
|
||||||
|
# return msg
|
||||||
|
|
||||||
_raise_from_no_key_in_msg(
|
_raise_from_no_key_in_msg(
|
||||||
ctx=self._ctx,
|
ctx=self._ctx,
|
||||||
msg=msg,
|
msg=msg,
|
||||||
|
@ -122,16 +127,26 @@ class MsgStream(trio.abc.Channel):
|
||||||
# see ``.aclose()`` for notes on the old behaviour prior to
|
# see ``.aclose()`` for notes on the old behaviour prior to
|
||||||
# introducing this
|
# introducing this
|
||||||
if self._eoc:
|
if self._eoc:
|
||||||
raise trio.EndOfChannel
|
raise self._eoc
|
||||||
|
# raise trio.EndOfChannel
|
||||||
|
|
||||||
if self._closed:
|
if self._closed:
|
||||||
raise trio.ClosedResourceError('This stream was closed')
|
raise self._closed
|
||||||
|
# raise trio.ClosedResourceError(
|
||||||
|
# 'This stream was already closed'
|
||||||
|
# )
|
||||||
|
|
||||||
|
src_err: Exception|None = None
|
||||||
|
try:
|
||||||
try:
|
try:
|
||||||
msg = await self._rx_chan.receive()
|
msg = await self._rx_chan.receive()
|
||||||
return msg['yield']
|
return msg['yield']
|
||||||
|
|
||||||
except KeyError as kerr:
|
except KeyError as kerr:
|
||||||
|
src_err = kerr
|
||||||
|
|
||||||
|
# NOTE: may raise any of the below error types
|
||||||
|
# includg EoC when a 'stop' msg is found.
|
||||||
_raise_from_no_key_in_msg(
|
_raise_from_no_key_in_msg(
|
||||||
ctx=self._ctx,
|
ctx=self._ctx,
|
||||||
msg=msg,
|
msg=msg,
|
||||||
|
@ -141,11 +156,14 @@ class MsgStream(trio.abc.Channel):
|
||||||
stream=self,
|
stream=self,
|
||||||
)
|
)
|
||||||
|
|
||||||
except (
|
|
||||||
trio.ClosedResourceError, # by self._rx_chan
|
|
||||||
trio.EndOfChannel, # by self._rx_chan or `stop` msg from far end
|
|
||||||
):
|
|
||||||
# XXX: we close the stream on any of these error conditions:
|
# XXX: we close the stream on any of these error conditions:
|
||||||
|
except (
|
||||||
|
# trio.ClosedResourceError, # by self._rx_chan
|
||||||
|
trio.EndOfChannel, # by self._rx_chan or `stop` msg from far end
|
||||||
|
) as eoc:
|
||||||
|
src_err = eoc
|
||||||
|
self._eoc = eoc
|
||||||
|
# await trio.sleep(1)
|
||||||
|
|
||||||
# a ``ClosedResourceError`` indicates that the internal
|
# a ``ClosedResourceError`` indicates that the internal
|
||||||
# feeder memory receive channel was closed likely by the
|
# feeder memory receive channel was closed likely by the
|
||||||
|
@ -168,14 +186,53 @@ class MsgStream(trio.abc.Channel):
|
||||||
# closing this stream and not flushing a final value to
|
# closing this stream and not flushing a final value to
|
||||||
# remaining (clone) consumers who may not have been
|
# remaining (clone) consumers who may not have been
|
||||||
# scheduled to receive it yet.
|
# scheduled to receive it yet.
|
||||||
|
# try:
|
||||||
|
# maybe_err_msg_or_res: dict = self._rx_chan.receive_nowait()
|
||||||
|
# if maybe_err_msg_or_res:
|
||||||
|
# log.warning(
|
||||||
|
# 'Discarding un-processed msg:\n'
|
||||||
|
# f'{maybe_err_msg_or_res}'
|
||||||
|
# )
|
||||||
|
# except trio.WouldBlock:
|
||||||
|
# # no queued msgs that might be another remote
|
||||||
|
# # error, so just raise the original EoC
|
||||||
|
# pass
|
||||||
|
|
||||||
|
# raise eoc
|
||||||
|
|
||||||
|
except trio.ClosedResourceError as cre: # by self._rx_chan
|
||||||
|
src_err = cre
|
||||||
|
log.warning(
|
||||||
|
'`Context._rx_chan` was already closed?'
|
||||||
|
)
|
||||||
|
self._closed = cre
|
||||||
|
|
||||||
# when the send is closed we assume the stream has
|
# when the send is closed we assume the stream has
|
||||||
# terminated and signal this local iterator to stop
|
# terminated and signal this local iterator to stop
|
||||||
await self.aclose()
|
drained: list[Exception|dict] = await self.aclose()
|
||||||
|
if drained:
|
||||||
|
log.warning(
|
||||||
|
'Drained context msgs during closure:\n'
|
||||||
|
f'{drained}'
|
||||||
|
)
|
||||||
|
# TODO: pass these to the `._ctx._drained_msgs: deque`
|
||||||
|
# and then iterate them as part of any `.result()` call?
|
||||||
|
|
||||||
raise # propagate
|
# NOTE XXX: if the context was cancelled or remote-errored
|
||||||
|
# but we received the stream close msg first, we
|
||||||
|
# probably want to instead raise the remote error
|
||||||
|
# over the end-of-stream connection error since likely
|
||||||
|
# the remote error was the source cause?
|
||||||
|
ctx: Context = self._ctx
|
||||||
|
if re := ctx._remote_error:
|
||||||
|
ctx._maybe_raise_remote_err(
|
||||||
|
re,
|
||||||
|
raise_ctxc_from_self_call=True,
|
||||||
|
)
|
||||||
|
|
||||||
async def aclose(self):
|
raise src_err # propagate
|
||||||
|
|
||||||
|
async def aclose(self) -> list[Exception|dict]:
|
||||||
'''
|
'''
|
||||||
Cancel associated remote actor task and local memory channel on
|
Cancel associated remote actor task and local memory channel on
|
||||||
close.
|
close.
|
||||||
|
@ -185,15 +242,55 @@ class MsgStream(trio.abc.Channel):
|
||||||
# https://trio.readthedocs.io/en/stable/reference-io.html#trio.abc.AsyncResource.aclose
|
# https://trio.readthedocs.io/en/stable/reference-io.html#trio.abc.AsyncResource.aclose
|
||||||
rx_chan = self._rx_chan
|
rx_chan = self._rx_chan
|
||||||
|
|
||||||
if rx_chan._closed:
|
if (
|
||||||
log.cancel(f"{self} is already closed")
|
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'
|
||||||
|
)
|
||||||
|
|
||||||
# this stream has already been closed so silently succeed as
|
# this stream has already been closed so silently succeed as
|
||||||
# per ``trio.AsyncResource`` semantics.
|
# per ``trio.AsyncResource`` semantics.
|
||||||
# https://trio.readthedocs.io/en/stable/reference-io.html#trio.abc.AsyncResource.aclose
|
# https://trio.readthedocs.io/en/stable/reference-io.html#trio.abc.AsyncResource.aclose
|
||||||
return
|
return []
|
||||||
|
|
||||||
self._eoc = True
|
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(
|
||||||
|
'Drained un-processed stream msg:\n'
|
||||||
|
f'{pformat(maybe_final_msg)}'
|
||||||
|
)
|
||||||
|
# TODO: inject into parent `Context` buf?
|
||||||
|
drained.append(maybe_final_msg)
|
||||||
|
|
||||||
|
except trio.WouldBlock as be:
|
||||||
|
drained.append(be)
|
||||||
|
break
|
||||||
|
|
||||||
|
except trio.EndOfChannel as eoc:
|
||||||
|
drained.append(eoc)
|
||||||
|
# caught_eoc = True
|
||||||
|
self._eoc: bool = eoc
|
||||||
|
break
|
||||||
|
|
||||||
|
except ContextCancelled as ctxc:
|
||||||
|
log.cancel(
|
||||||
|
'Context was cancelled during stream closure:\n'
|
||||||
|
f'canceller: {ctxc.canceller}\n'
|
||||||
|
f'{pformat(ctxc.msgdata)}'
|
||||||
|
)
|
||||||
|
break
|
||||||
|
|
||||||
# NOTE: this is super subtle IPC messaging stuff:
|
# NOTE: this is super subtle IPC messaging stuff:
|
||||||
# Relay stop iteration to far end **iff** we're
|
# Relay stop iteration to far end **iff** we're
|
||||||
|
@ -224,26 +321,33 @@ class MsgStream(trio.abc.Channel):
|
||||||
except (
|
except (
|
||||||
trio.BrokenResourceError,
|
trio.BrokenResourceError,
|
||||||
trio.ClosedResourceError
|
trio.ClosedResourceError
|
||||||
):
|
) as re:
|
||||||
# the underlying channel may already have been pulled
|
# the underlying channel may already have been pulled
|
||||||
# in which case our stop message is meaningless since
|
# in which case our stop message is meaningless since
|
||||||
# it can't traverse the transport.
|
# it can't traverse the transport.
|
||||||
ctx = self._ctx
|
|
||||||
log.warning(
|
log.warning(
|
||||||
f'Stream was already destroyed?\n'
|
f'Stream was already destroyed?\n'
|
||||||
f'actor: {ctx.chan.uid}\n'
|
f'actor: {ctx.chan.uid}\n'
|
||||||
f'ctx id: {ctx.cid}'
|
f'ctx id: {ctx.cid}'
|
||||||
)
|
)
|
||||||
|
drained.append(re)
|
||||||
|
self._closed = re
|
||||||
|
|
||||||
self._closed = True
|
# if caught_eoc:
|
||||||
|
# # from .devx import _debug
|
||||||
|
# # await _debug.pause()
|
||||||
|
# with trio.CancelScope(shield=True):
|
||||||
|
# await rx_chan.aclose()
|
||||||
|
|
||||||
# Do we close the local mem chan ``self._rx_chan`` ??!?
|
# self._eoc: bool = caught_eoc
|
||||||
|
|
||||||
# NO, DEFINITELY NOT if we're a bi-dir ``MsgStream``!
|
# ?XXX WAIT, why do we not close the local mem chan `._rx_chan` XXX?
|
||||||
# BECAUSE this same core-msg-loop mem recv-chan is used to deliver
|
# => NO, DEFINITELY NOT! <=
|
||||||
# the potential final result from the surrounding inter-actor
|
# if we're a bi-dir ``MsgStream`` BECAUSE this same
|
||||||
# `Context` so we don't want to close it until that context has
|
# core-msg-loop mem recv-chan is used to deliver the
|
||||||
# run to completion.
|
# potential final result from the surrounding inter-actor
|
||||||
|
# `Context` so we don't want to close it until that
|
||||||
|
# context has run to completion.
|
||||||
|
|
||||||
# XXX: Notes on old behaviour:
|
# XXX: Notes on old behaviour:
|
||||||
# await rx_chan.aclose()
|
# await rx_chan.aclose()
|
||||||
|
@ -272,6 +376,8 @@ class MsgStream(trio.abc.Channel):
|
||||||
# runtime's closure of ``rx_chan`` in the case where we may
|
# runtime's closure of ``rx_chan`` in the case where we may
|
||||||
# still need to consume msgs that are "in transit" from the far
|
# still need to consume msgs that are "in transit" from the far
|
||||||
# end (eg. for ``Context.result()``).
|
# end (eg. for ``Context.result()``).
|
||||||
|
# self._closed = True
|
||||||
|
return drained
|
||||||
|
|
||||||
@acm
|
@acm
|
||||||
async def subscribe(
|
async def subscribe(
|
||||||
|
@ -337,9 +443,13 @@ class MsgStream(trio.abc.Channel):
|
||||||
raise self._ctx._remote_error # from None
|
raise self._ctx._remote_error # from None
|
||||||
|
|
||||||
if self._closed:
|
if self._closed:
|
||||||
raise trio.ClosedResourceError('This stream was already closed')
|
raise self._closed
|
||||||
|
# raise trio.ClosedResourceError('This stream was already closed')
|
||||||
|
|
||||||
await self._ctx.chan.send({'yield': data, 'cid': self._ctx.cid})
|
await self._ctx.chan.send({
|
||||||
|
'yield': data,
|
||||||
|
'cid': self._ctx.cid,
|
||||||
|
})
|
||||||
|
|
||||||
|
|
||||||
def stream(func: Callable) -> Callable:
|
def stream(func: Callable) -> Callable:
|
||||||
|
|
Loading…
Reference in New Issue