forked from goodboy/tractor
Drop `msg` kwarg from `Context.cancel()`
Well first off, turns out it's never used and generally speaking doesn't seem to help much with "runtime hacking/debugging"; why would we need to "fabricate" a msg when `.cancel()` is called to self-cancel? Also (and since `._maybe_cancel_and_set_remote_error()` now takes an `error: BaseException` as input and thus expects error-msg unpacking prior to being called), we now manually set `Context._cancel_msg: dict` just prior to any remote error assignment - so any case where we would have fabbed a "cancel msg" near calling `.cancel()`, just do the manual assign. In this vein some other subtle changes: - obviously don't set `._cancel_msg` in `.cancel()` since it's no longer an input. - generally do walrus-style `error := unpack_error()` before applying and setting remote error-msg state. - always raise any `._remote_error` in `.result()` instead of returning the exception instance and check before AND after the underlying mem chan read. - add notes/todos around `raise self._remote_error from None` masking of (runtime) errors in `._maybe_raise_remote_err()` and use it inside `.result()` since we had the inverse duplicate logic there anyway.. Further, this adds and extends a ton of (internal) interface docs and details comments around the `Context` API including many subtleties pertaining to calling `._maybe_cancel_and_set_remote_error()`.multihomed
parent
e4a6223256
commit
534e5d150d
|
@ -86,30 +86,51 @@ class Context:
|
||||||
|
|
||||||
'''
|
'''
|
||||||
chan: Channel
|
chan: Channel
|
||||||
cid: str
|
cid: str # "context id", more or less a unique linked-task-pair id
|
||||||
|
|
||||||
# these are the "feeder" channels for delivering
|
# the "feeder" channels for delivering message values to the
|
||||||
# message values to the local task from the runtime
|
# local task from the runtime's msg processing loop.
|
||||||
# msg processing loop.
|
|
||||||
_recv_chan: trio.MemoryReceiveChannel
|
_recv_chan: trio.MemoryReceiveChannel
|
||||||
_send_chan: trio.MemorySendChannel
|
_send_chan: trio.MemorySendChannel
|
||||||
|
|
||||||
|
# the "invocation type" of the far end task-entry-point
|
||||||
|
# function, normally matching a logic block inside
|
||||||
|
# `._runtime.invoke()`.
|
||||||
_remote_func_type: str | None = None
|
_remote_func_type: str | None = None
|
||||||
|
|
||||||
# only set on the caller side
|
# NOTE: (for now) only set (a portal) on the caller side since
|
||||||
_portal: Portal | None = None # type: ignore # noqa
|
# the callee doesn't generally need a ref to one and should
|
||||||
|
# normally need to explicitly ask for handle to its peer if
|
||||||
|
# more the the `Context` is needed?
|
||||||
|
_portal: Portal | None = None
|
||||||
|
|
||||||
|
# NOTE: each side of the context has its own cancel scope
|
||||||
|
# which is exactly the primitive that allows for
|
||||||
|
# cross-actor-task-supervision and thus SC.
|
||||||
|
_scope: trio.CancelScope | None = None
|
||||||
_result: Any | int = None
|
_result: Any | int = None
|
||||||
_remote_error: BaseException | None = None
|
_remote_error: BaseException | None = None
|
||||||
|
|
||||||
# cancellation state
|
# cancellation state
|
||||||
_cancel_called: bool = False # did WE cancel the far end?
|
_cancel_called: bool = False # did WE cancel the far end?
|
||||||
_cancelled_remote: tuple[str, str] | None = None
|
_cancelled_remote: tuple[str, str] | None = None
|
||||||
_cancel_msg: str | None = None
|
|
||||||
_scope: trio.CancelScope | None = None
|
|
||||||
|
|
||||||
# NOTE: this is set by the `.devx._debug` machinery
|
# NOTE: we try to ensure assignment of a "cancel msg" since
|
||||||
# to indicate whether code in `._runtime` should handle
|
# there's always going to be an "underlying reason" that any
|
||||||
# cancelled context crashes in the pdbp REPL.
|
# context was closed due to either a remote side error or
|
||||||
|
# a call to `.cancel()` which triggers `ContextCancelled`.
|
||||||
|
_cancel_msg: str | dict | None = None
|
||||||
|
|
||||||
|
# NOTE: this state var used by the runtime to determine if the
|
||||||
|
# `pdbp` REPL is allowed to engage on contexts terminated via
|
||||||
|
# a `ContextCancelled` due to a call to `.cancel()` triggering
|
||||||
|
# "graceful closure" on either side:
|
||||||
|
# - `._runtime._invoke()` will check this flag before engaging
|
||||||
|
# the crash handler REPL in such cases where the "callee"
|
||||||
|
# raises the cancellation,
|
||||||
|
# - `.devx._debug.lock_tty_for_child()` will set it to `False` if
|
||||||
|
# the global tty-lock has been configured to filter out some
|
||||||
|
# actors from being able to acquire the debugger lock.
|
||||||
_enter_debugger_on_cancel: bool = True
|
_enter_debugger_on_cancel: bool = True
|
||||||
|
|
||||||
@property
|
@property
|
||||||
|
@ -177,36 +198,71 @@ class Context:
|
||||||
|
|
||||||
async def _maybe_cancel_and_set_remote_error(
|
async def _maybe_cancel_and_set_remote_error(
|
||||||
self,
|
self,
|
||||||
error_msg: dict[str, Any],
|
error: BaseException,
|
||||||
|
|
||||||
) -> None:
|
) -> None:
|
||||||
'''
|
'''
|
||||||
(Maybe) unpack and raise a msg error into the local scope
|
(Maybe) cancel this local scope due to a received remote
|
||||||
nursery for this context.
|
error (normally via an IPC msg) which the actor runtime
|
||||||
|
routes to this context.
|
||||||
|
|
||||||
Acts as a form of "relay" for a remote error raised
|
Acts as a form of "relay" for a remote error raised in the
|
||||||
in the corresponding remote callee task.
|
corresponding remote task's `Context` wherein the next time
|
||||||
|
the local task exectutes a checkpoint, a `trio.Cancelled`
|
||||||
|
will be raised and depending on the type and source of the
|
||||||
|
original remote error, and whether or not the local task
|
||||||
|
called `.cancel()` itself prior, an equivalent
|
||||||
|
`ContextCancelled` or `RemoteActorError` wrapping the
|
||||||
|
remote error may be raised here by any of,
|
||||||
|
|
||||||
|
- `Portal.open_context()`
|
||||||
|
- `Portal.result()`
|
||||||
|
- `Context.open_stream()`
|
||||||
|
- `Context.result()`
|
||||||
|
|
||||||
|
when called/closed by actor local task(s).
|
||||||
|
|
||||||
|
NOTEs & TODOs:
|
||||||
|
- It is expected that the caller has previously unwrapped
|
||||||
|
the remote error using a call to `unpack_error()` and
|
||||||
|
provides that output exception value as the input
|
||||||
|
`error` argument here.
|
||||||
|
- If this is an error message from a context opened by
|
||||||
|
`Portal.open_context()` we want to interrupt any
|
||||||
|
ongoing local tasks operating within that `Context`'s
|
||||||
|
cancel-scope so as to be notified ASAP of the remote
|
||||||
|
error and engage any caller handling (eg. for
|
||||||
|
cross-process task supervision).
|
||||||
|
- In some cases we may want to raise the remote error
|
||||||
|
immediately since there is no guarantee the locally
|
||||||
|
operating task(s) will attempt to execute a checkpoint
|
||||||
|
any time soon; in such cases there are 2 possible
|
||||||
|
approaches depending on the current task's work and
|
||||||
|
wrapping "thread" type:
|
||||||
|
|
||||||
|
- `trio`-native-and-graceful: only ever wait for tasks
|
||||||
|
to exec a next `trio.lowlevel.checkpoint()` assuming
|
||||||
|
that any such task must do so to interact with the
|
||||||
|
actor runtime and IPC interfaces.
|
||||||
|
|
||||||
|
- (NOT IMPLEMENTED) system-level-aggressive: maybe we
|
||||||
|
could eventually interrupt sync code (invoked using
|
||||||
|
`trio.to_thread` or some other adapter layer) with
|
||||||
|
a signal (a custom unix one for example?
|
||||||
|
https://stackoverflow.com/a/5744185) depending on the
|
||||||
|
task's wrapping thread-type such that long running
|
||||||
|
sync code should never cause the delay of actor
|
||||||
|
supervision tasks such as cancellation and respawn
|
||||||
|
logic.
|
||||||
|
|
||||||
'''
|
'''
|
||||||
# If this is an error message from a context opened by
|
# XXX: currently this should only be used when
|
||||||
# ``Portal.open_context()`` we want to interrupt any ongoing
|
# `Portal.open_context()` has been opened since it's
|
||||||
# (child) tasks within that context to be notified of the remote
|
# assumed that other portal APIs like,
|
||||||
# error relayed here.
|
# - `Portal.run()`,
|
||||||
#
|
# - `ActorNursery.run_in_actor()`
|
||||||
# The reason we may want to raise the remote error immediately
|
# do their own error checking at their own call points and
|
||||||
# is that there is no guarantee the associated local task(s)
|
# result processing.
|
||||||
# will attempt to read from any locally opened stream any time
|
|
||||||
# soon.
|
|
||||||
#
|
|
||||||
# NOTE: this only applies when
|
|
||||||
# ``Portal.open_context()`` has been called since it is assumed
|
|
||||||
# (currently) that other portal APIs (``Portal.run()``,
|
|
||||||
# ``.run_in_actor()``) do their own error checking at the point
|
|
||||||
# of the call and result processing.
|
|
||||||
error = unpack_error(
|
|
||||||
error_msg,
|
|
||||||
self.chan,
|
|
||||||
)
|
|
||||||
|
|
||||||
# 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
|
||||||
|
@ -236,35 +292,25 @@ class Context:
|
||||||
else:
|
else:
|
||||||
log.error(
|
log.error(
|
||||||
f'Remote context error for {self.chan.uid}:{self.cid}:\n'
|
f'Remote context error for {self.chan.uid}:{self.cid}:\n'
|
||||||
f'{error_msg["error"]["tb_str"]}'
|
f'{error}'
|
||||||
)
|
)
|
||||||
# TODO: tempted to **not** do this by-reraising in a
|
# TODO: tempted to **not** do this by-reraising in a
|
||||||
# nursery and instead cancel a surrounding scope, detect
|
# nursery and instead cancel a surrounding scope, detect
|
||||||
# the cancellation, then lookup the error that was set?
|
# the cancellation, then lookup the error that was set?
|
||||||
# YES! this is way better and simpler!
|
# YES! this is way better and simpler!
|
||||||
if (
|
if self._scope:
|
||||||
self._scope
|
|
||||||
):
|
|
||||||
# from trio.testing import wait_all_tasks_blocked
|
# from trio.testing import wait_all_tasks_blocked
|
||||||
# await wait_all_tasks_blocked()
|
# await wait_all_tasks_blocked()
|
||||||
# self._cancelled_remote = self.chan.uid
|
# self._cancelled_remote = self.chan.uid
|
||||||
self._scope.cancel()
|
self._scope.cancel()
|
||||||
|
|
||||||
# NOTE: this usage actually works here B)
|
# this REPL usage actually works here BD
|
||||||
# from .devx._debug import breakpoint
|
# from .devx._debug import pause
|
||||||
# await breakpoint()
|
# await pause()
|
||||||
|
|
||||||
# XXX: this will break early callee results sending
|
|
||||||
# since when `.result()` is finally called, this
|
|
||||||
# chan will be closed..
|
|
||||||
# if self._recv_chan:
|
|
||||||
# await self._recv_chan.aclose()
|
|
||||||
|
|
||||||
async def cancel(
|
async def cancel(
|
||||||
self,
|
self,
|
||||||
msg: str | None = None,
|
|
||||||
timeout: float = 0.616,
|
timeout: float = 0.616,
|
||||||
# timeout: float = 1000,
|
|
||||||
|
|
||||||
) -> None:
|
) -> None:
|
||||||
'''
|
'''
|
||||||
|
@ -274,15 +320,12 @@ class Context:
|
||||||
Timeout quickly in an attempt to sidestep 2-generals...
|
Timeout quickly in an attempt to sidestep 2-generals...
|
||||||
|
|
||||||
'''
|
'''
|
||||||
side = 'caller' if self._portal else 'callee'
|
side: str = 'caller' if self._portal else 'callee'
|
||||||
if msg:
|
log.cancel(
|
||||||
assert side == 'callee', 'Only callee side can provide cancel msg'
|
f'Cancelling {side} side of context to {self.chan.uid}'
|
||||||
|
)
|
||||||
|
|
||||||
log.cancel(f'Cancelling {side} side of context to {self.chan.uid}')
|
self._cancel_called: bool = True
|
||||||
|
|
||||||
self._cancel_called = True
|
|
||||||
# await devx._debug.breakpoint()
|
|
||||||
# breakpoint()
|
|
||||||
|
|
||||||
if side == 'caller':
|
if side == 'caller':
|
||||||
if not self._portal:
|
if not self._portal:
|
||||||
|
@ -290,12 +333,13 @@ class Context:
|
||||||
"No portal found, this is likely a callee side context"
|
"No portal found, this is likely a callee side context"
|
||||||
)
|
)
|
||||||
|
|
||||||
cid = self.cid
|
cid: str = self.cid
|
||||||
with trio.move_on_after(timeout) as cs:
|
with trio.move_on_after(timeout) as cs:
|
||||||
cs.shield = True
|
cs.shield = True
|
||||||
log.cancel(
|
log.cancel(
|
||||||
f"Cancelling stream {cid} to "
|
f'Cancelling stream {cid} to '
|
||||||
f"{self._portal.channel.uid}")
|
f'{self._portal.channel.uid}'
|
||||||
|
)
|
||||||
|
|
||||||
# NOTE: we're telling the far end actor to cancel a task
|
# NOTE: we're telling the far end actor to cancel a task
|
||||||
# corresponding to *this actor*. The far end local channel
|
# corresponding to *this actor*. The far end local channel
|
||||||
|
@ -314,17 +358,17 @@ class Context:
|
||||||
# if not self._portal.channel.connected():
|
# if not self._portal.channel.connected():
|
||||||
if not self.chan.connected():
|
if not self.chan.connected():
|
||||||
log.cancel(
|
log.cancel(
|
||||||
"May have failed to cancel remote task "
|
'May have failed to cancel remote task '
|
||||||
f"{cid} for {self._portal.channel.uid}")
|
f'{cid} for {self._portal.channel.uid}'
|
||||||
|
)
|
||||||
else:
|
else:
|
||||||
log.cancel(
|
log.cancel(
|
||||||
"Timed out on cancelling remote task "
|
'Timed out on cancel request of remote task '
|
||||||
f"{cid} for {self._portal.channel.uid}")
|
f'{cid} for {self._portal.channel.uid}'
|
||||||
|
)
|
||||||
|
|
||||||
# callee side remote task
|
# callee side remote task
|
||||||
else:
|
else:
|
||||||
self._cancel_msg = msg
|
|
||||||
|
|
||||||
# TODO: should we have an explicit cancel message
|
# TODO: should we have an explicit cancel message
|
||||||
# or is relaying the local `trio.Cancelled` as an
|
# or is relaying the local `trio.Cancelled` as an
|
||||||
# {'error': trio.Cancelled, cid: "blah"} enough?
|
# {'error': trio.Cancelled, cid: "blah"} enough?
|
||||||
|
@ -335,7 +379,6 @@ class Context:
|
||||||
|
|
||||||
@acm
|
@acm
|
||||||
async def open_stream(
|
async def open_stream(
|
||||||
|
|
||||||
self,
|
self,
|
||||||
allow_overruns: bool | None = False,
|
allow_overruns: bool | None = False,
|
||||||
msg_buffer_size: int | None = None,
|
msg_buffer_size: int | None = None,
|
||||||
|
@ -354,10 +397,10 @@ class Context:
|
||||||
``Portal.open_context()``. In the future this may change but
|
``Portal.open_context()``. In the future this may change but
|
||||||
currently there seems to be no obvious reason to support
|
currently there seems to be no obvious reason to support
|
||||||
"re-opening":
|
"re-opening":
|
||||||
- pausing a stream can be done with a message.
|
- pausing a stream can be done with a message.
|
||||||
- task errors will normally require a restart of the entire
|
- task errors will normally require a restart of the entire
|
||||||
scope of the inter-actor task context due to the nature of
|
scope of the inter-actor task context due to the nature of
|
||||||
``trio``'s cancellation system.
|
``trio``'s cancellation system.
|
||||||
|
|
||||||
'''
|
'''
|
||||||
actor = current_actor()
|
actor = current_actor()
|
||||||
|
@ -439,18 +482,19 @@ class Context:
|
||||||
self,
|
self,
|
||||||
err: Exception,
|
err: Exception,
|
||||||
) -> None:
|
) -> None:
|
||||||
|
'''
|
||||||
|
Maybe raise a remote error depending on who (which task from
|
||||||
|
which actor) requested a cancellation (if any).
|
||||||
|
|
||||||
|
'''
|
||||||
# NOTE: whenever the context's "opener" side (task) **is**
|
# NOTE: whenever the context's "opener" side (task) **is**
|
||||||
# the side which requested the cancellation (likekly via
|
# the side which requested the cancellation (likekly via
|
||||||
# ``Context.cancel()``), we don't want to re-raise that
|
# ``Context.cancel()``), we don't want to re-raise that
|
||||||
# cancellation signal locally (would be akin to
|
# cancellation signal locally (would be akin to
|
||||||
# a ``trio.Nursery`` nursery raising ``trio.Cancelled``
|
# a ``trio.Nursery`` nursery raising ``trio.Cancelled``
|
||||||
# whenever ``CancelScope.cancel()`` was called) and instead
|
# whenever ``CancelScope.cancel()`` was called) and
|
||||||
# silently reap the expected cancellation "error"-msg.
|
# instead silently reap the expected cancellation
|
||||||
# if 'pikerd' in err.msgdata['tb_str']:
|
# "error"-msg.
|
||||||
# # from . import _debug
|
|
||||||
# # await _debug.breakpoint()
|
|
||||||
# breakpoint()
|
|
||||||
|
|
||||||
if (
|
if (
|
||||||
isinstance(err, ContextCancelled)
|
isinstance(err, ContextCancelled)
|
||||||
and (
|
and (
|
||||||
|
@ -461,7 +505,18 @@ class Context:
|
||||||
):
|
):
|
||||||
return err
|
return err
|
||||||
|
|
||||||
raise err # from None
|
# NOTE: currently we are masking underlying runtime errors
|
||||||
|
# which are often superfluous to user handler code. not
|
||||||
|
# sure if this is still needed / desired for all operation?
|
||||||
|
# TODO: maybe we can only NOT mask if:
|
||||||
|
# - [ ] debug mode is enabled or,
|
||||||
|
# - [ ] a certain log level is set?
|
||||||
|
# - [ ] consider using `.with_traceback()` to filter out
|
||||||
|
# runtime frames from the tb explicitly?
|
||||||
|
# https://docs.python.org/3/reference/simple_stmts.html#the-raise-statement
|
||||||
|
# https://stackoverflow.com/a/24752607
|
||||||
|
__tracebackhide__: bool = True
|
||||||
|
raise err from None
|
||||||
|
|
||||||
async def result(self) -> Any | Exception:
|
async def result(self) -> Any | Exception:
|
||||||
'''
|
'''
|
||||||
|
@ -489,12 +544,12 @@ class Context:
|
||||||
of the remote cancellation.
|
of the remote cancellation.
|
||||||
|
|
||||||
'''
|
'''
|
||||||
|
__tracebackhide__: bool = True
|
||||||
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:
|
if re := self._remote_error:
|
||||||
self._maybe_raise_remote_err(re)
|
return self._maybe_raise_remote_err(re)
|
||||||
return re
|
|
||||||
|
|
||||||
if (
|
if (
|
||||||
self._result == id(self)
|
self._result == id(self)
|
||||||
|
@ -505,8 +560,8 @@ class Context:
|
||||||
# and discarding any bi dir stream msgs still
|
# and discarding any bi dir stream msgs still
|
||||||
# in transit from the far end.
|
# in transit from the far end.
|
||||||
while True:
|
while True:
|
||||||
msg = await self._recv_chan.receive()
|
|
||||||
try:
|
try:
|
||||||
|
msg = await self._recv_chan.receive()
|
||||||
self._result: Any = msg['return']
|
self._result: Any = msg['return']
|
||||||
|
|
||||||
# NOTE: we don't need to do this right?
|
# NOTE: we don't need to do this right?
|
||||||
|
@ -519,17 +574,22 @@ class Context:
|
||||||
|
|
||||||
# NOTE: we get here if the far end was
|
# NOTE: we get here if the far end was
|
||||||
# `ContextCancelled` in 2 cases:
|
# `ContextCancelled` in 2 cases:
|
||||||
# - we requested the cancellation and thus
|
# 1. we requested the cancellation and thus
|
||||||
# SHOULD NOT raise that far end error,
|
# SHOULD NOT raise that far end error,
|
||||||
# - WE DID NOT REQUEST that cancel and thus
|
# 2. WE DID NOT REQUEST that cancel and thus
|
||||||
# SHOULD RAISE HERE!
|
# SHOULD RAISE HERE!
|
||||||
except trio.Cancelled:
|
except trio.Cancelled:
|
||||||
if not self._cancel_called:
|
|
||||||
raise self._remote_error
|
# CASE 2: mask the local cancelled-error(s)
|
||||||
else:
|
# only when we are sure the remote error is the
|
||||||
# if we DID request the cancel we simply
|
# (likely) source cause of this local runtime
|
||||||
# continue as normal.
|
# task's cancellation.
|
||||||
raise
|
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:
|
except KeyError: # as msgerr:
|
||||||
|
|
||||||
|
@ -544,7 +604,8 @@ class Context:
|
||||||
|
|
||||||
# internal error should never get here
|
# internal error should never get here
|
||||||
assert msg.get('cid'), (
|
assert msg.get('cid'), (
|
||||||
"Received internal error at portal?")
|
"Received internal error at portal?"
|
||||||
|
)
|
||||||
|
|
||||||
err = unpack_error(
|
err = unpack_error(
|
||||||
msg,
|
msg,
|
||||||
|
@ -554,7 +615,10 @@ class Context:
|
||||||
err = self._maybe_raise_remote_err(err)
|
err = self._maybe_raise_remote_err(err)
|
||||||
self._remote_error = err
|
self._remote_error = err
|
||||||
|
|
||||||
return self._remote_error or self._result
|
if re := self._remote_error:
|
||||||
|
return self._maybe_raise_remote_err(re)
|
||||||
|
|
||||||
|
return self._result
|
||||||
|
|
||||||
async def started(
|
async def started(
|
||||||
self,
|
self,
|
||||||
|
@ -563,7 +627,7 @@ class Context:
|
||||||
) -> None:
|
) -> None:
|
||||||
'''
|
'''
|
||||||
Indicate to calling actor's task that this linked context
|
Indicate to calling actor's task that this linked context
|
||||||
has started and send ``value`` to the other side.
|
has started and send ``value`` to the other side via IPC.
|
||||||
|
|
||||||
On the calling side ``value`` is the second item delivered
|
On the calling side ``value`` is the second item delivered
|
||||||
in the tuple returned by ``Portal.open_context()``.
|
in the tuple returned by ``Portal.open_context()``.
|
||||||
|
@ -571,19 +635,17 @@ class Context:
|
||||||
'''
|
'''
|
||||||
if self._portal:
|
if self._portal:
|
||||||
raise RuntimeError(
|
raise RuntimeError(
|
||||||
f"Caller side context {self} can not call started!")
|
f'Caller side context {self} can not call started!'
|
||||||
|
)
|
||||||
|
|
||||||
elif self._started_called:
|
elif self._started_called:
|
||||||
raise RuntimeError(
|
raise RuntimeError(
|
||||||
f"called 'started' twice on context with {self.chan.uid}")
|
f'called `.started()` twice on context with {self.chan.uid}'
|
||||||
|
)
|
||||||
|
|
||||||
await self.chan.send({'started': value, 'cid': self.cid})
|
await self.chan.send({'started': value, 'cid': self.cid})
|
||||||
self._started_called = True
|
self._started_called = True
|
||||||
|
|
||||||
# TODO: do we need a restart api?
|
|
||||||
# async def restart(self) -> None:
|
|
||||||
# pass
|
|
||||||
|
|
||||||
async def _drain_overflows(
|
async def _drain_overflows(
|
||||||
self,
|
self,
|
||||||
) -> None:
|
) -> None:
|
||||||
|
@ -638,10 +700,21 @@ class Context:
|
||||||
self,
|
self,
|
||||||
msg: dict,
|
msg: dict,
|
||||||
|
|
||||||
draining: bool = False,
|
# draining: bool = False,
|
||||||
|
|
||||||
) -> bool:
|
) -> bool:
|
||||||
|
'''
|
||||||
|
Deliver an IPC msg received from a transport-channel to
|
||||||
|
this context's underlying mem chan for handling by
|
||||||
|
user operating tasks; deliver a bool indicating whether the
|
||||||
|
msg was immediately sent.
|
||||||
|
|
||||||
|
If `._allow_overruns == True` (maybe) append the msg to an
|
||||||
|
"overflow queue" and start a "drainer task" (inside the
|
||||||
|
`._scope_nursery: trio.Nursery`) which ensures that such
|
||||||
|
messages are eventually sent if possible.
|
||||||
|
|
||||||
|
'''
|
||||||
cid = self.cid
|
cid = self.cid
|
||||||
chan = self.chan
|
chan = self.chan
|
||||||
uid = chan.uid
|
uid = chan.uid
|
||||||
|
@ -652,8 +725,12 @@ class Context:
|
||||||
)
|
)
|
||||||
|
|
||||||
error = msg.get('error')
|
error = msg.get('error')
|
||||||
if error:
|
if error := unpack_error(
|
||||||
await self._maybe_cancel_and_set_remote_error(msg)
|
msg,
|
||||||
|
self.chan,
|
||||||
|
):
|
||||||
|
self._cancel_msg = msg
|
||||||
|
await self._maybe_cancel_and_set_remote_error(error)
|
||||||
|
|
||||||
if (
|
if (
|
||||||
self._in_overrun
|
self._in_overrun
|
||||||
|
@ -685,6 +762,7 @@ class Context:
|
||||||
# the sender; the main motivation is that using bp can block the
|
# the sender; the main motivation is that using bp can block the
|
||||||
# msg handling loop which calls into this method!
|
# msg handling loop which calls into this method!
|
||||||
except trio.WouldBlock:
|
except trio.WouldBlock:
|
||||||
|
|
||||||
# XXX: always push an error even if the local
|
# XXX: always push an error even if the local
|
||||||
# receiver is in overrun state.
|
# receiver is in overrun state.
|
||||||
# await self._maybe_cancel_and_set_remote_error(msg)
|
# await self._maybe_cancel_and_set_remote_error(msg)
|
||||||
|
|
Loading…
Reference in New Issue