tractor/tractor/_runtime.py

2637 lines
90 KiB
Python
Raw Normal View History

Re-license code base for distribution under AGPL This commit obviously denotes a re-license of all applicable parts of the code base. Acknowledgement of this change was completed in #274 by the majority of the current set of contributors. From here henceforth all changes will be AGPL licensed and distributed. This is purely an effort to maintain the same copy-left policy whilst closing the (perceived) SaaS loophole the GPL allows for. It is merely for this loophole: to avoid code hiding by any potential "network providers" who are attempting to use the project to make a profit without either compensating the authors or re-distributing their changes. I thought quite a bit about this change and can't see a reason not to close the SaaS loophole in our current license. We still are (hard) copy-left and I plan to keep the code base this way for a couple reasons: - The code base produces income/profit through parent projects and is demonstrably of high value. - I believe firms should not get free lunch for the sake of "contributions from their employees" or "usage as a service" which I have found to be a dubious argument at best. - If a firm who intends to profit from the code base wants to use it they can propose a secondary commercial license to purchase with the proceeds going to the project's authors under some form of well defined contract. - Many successful projects like Qt use this model; I see no reason it can't work in this case until such a time as the authors feel it should be loosened. There has been detailed discussion in #103 on licensing alternatives. The main point of this AGPL change is to protect the code base for the time being from exploitation while it grows and as we move into the next phase of development which will include extension into the multi-host distributed software space.
2021-12-13 18:08:32 +00:00
# tractor: structured concurrent "actors".
# Copyright 2018-eternity Tyler Goodlet.
# This program is free software: you can redistribute it and/or modify
# it under the terms of the GNU Affero General Public License as published by
# the Free Software Foundation, either version 3 of the License, or
# (at your option) any later version.
# This program is distributed in the hope that it will be useful,
# but WITHOUT ANY WARRANTY; without even the implied warranty of
# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
# GNU Affero General Public License for more details.
# You should have received a copy of the GNU Affero General Public License
# along with this program. If not, see <https://www.gnu.org/licenses/>.
2018-07-14 20:09:05 +00:00
"""
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
The fundamental core machinery implementing every "actor" including
the process-local (python-interpreter global) `Actor` state-type
primitive(s), RPC-in-task scheduling, and IPC connectivity and
low-level transport msg handling.
2018-07-14 20:09:05 +00:00
"""
from __future__ import annotations
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
from contextlib import (
ExitStack,
asynccontextmanager as acm,
)
2018-07-14 20:09:05 +00:00
from collections import defaultdict
from functools import partial
from itertools import chain
import importlib
import importlib.util
import inspect
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
from pprint import pformat
import signal
import sys
2022-02-14 15:37:31 +00:00
from typing import (
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
Any,
2022-02-14 15:37:31 +00:00
Callable,
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
Union,
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
Coroutine,
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
TYPE_CHECKING,
2022-02-14 15:37:31 +00:00
)
import uuid
from types import ModuleType
import os
import warnings
2018-07-14 20:09:05 +00:00
from async_generator import aclosing
from exceptiongroup import BaseExceptionGroup
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
import trio
from trio import (
CancelScope,
)
from trio_typing import (
Nursery,
TaskStatus,
)
2018-07-14 20:09:05 +00:00
from .msg import NamespacePath
from ._ipc import Channel
from ._context import (
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
mk_context,
Context,
)
from .log import get_logger
from ._exceptions import (
pack_error,
unpack_error,
2020-12-25 20:20:58 +00:00
ModuleNotExposed,
is_multi_cancelled,
ContextCancelled,
TransportClosed,
)
from . import _debug
from ._discovery import get_arbiter
from ._portal import Portal
2018-07-14 20:09:05 +00:00
from . import _state
from . import _mp_fixup_main
2018-07-14 20:09:05 +00:00
2022-02-14 15:37:31 +00:00
if TYPE_CHECKING:
from ._supervise import ActorNursery
2018-07-14 20:09:05 +00:00
log = get_logger('tractor')
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
_gb_mod: ModuleType|None|False = None
async def maybe_import_gb():
global _gb_mod
if _gb_mod is False:
return
try:
import greenback
_gb_mod = greenback
await greenback.ensure_portal()
except ModuleNotFoundError:
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
log.debug(
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
'`greenback` is not installed.\n'
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
'No sync debug support!\n'
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
)
_gb_mod = False
2018-07-14 20:09:05 +00:00
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
async def _invoke_non_context(
actor: Actor,
cancel_scope: CancelScope,
ctx: Context,
cid: str,
chan: Channel,
func: Callable,
coro: Coroutine,
kwargs: dict[str, Any],
treat_as_gen: bool,
is_rpc: bool,
task_status: TaskStatus[
Context | BaseException
] = trio.TASK_STATUS_IGNORED,
):
# TODO: can we unify this with the `context=True` impl below?
if inspect.isasyncgen(coro):
await chan.send({'functype': 'asyncgen', 'cid': cid})
# XXX: massive gotcha! If the containing scope
# is cancelled and we execute the below line,
# any ``ActorNursery.__aexit__()`` WON'T be
# triggered in the underlying async gen! So we
# have to properly handle the closing (aclosing)
# of the async gen in order to be sure the cancel
# is propagated!
with cancel_scope as cs:
ctx._scope = cs
task_status.started(ctx)
async with aclosing(coro) as agen:
async for item in agen:
# TODO: can we send values back in here?
# it's gonna require a `while True:` and
# some non-blocking way to retrieve new `asend()`
# values from the channel:
# to_send = await chan.recv_nowait()
# if to_send is not None:
# to_yield = await coro.asend(to_send)
await chan.send({'yield': item, 'cid': cid})
log.runtime(f"Finished iterating {coro}")
# TODO: we should really support a proper
# `StopAsyncIteration` system here for returning a final
# value if desired
await chan.send({'stop': True, 'cid': cid})
# one way @stream func that gets treated like an async gen
# TODO: can we unify this with the `context=True` impl below?
elif treat_as_gen:
await chan.send({'functype': 'asyncgen', 'cid': cid})
# XXX: the async-func may spawn further tasks which push
# back values like an async-generator would but must
# manualy construct the response dict-packet-responses as
# above
with cancel_scope as cs:
ctx._scope = cs
task_status.started(ctx)
await coro
if not cs.cancelled_caught:
# task was not cancelled so we can instruct the
# far end async gen to tear down
await chan.send({'stop': True, 'cid': cid})
else:
# regular async function/method
# XXX: possibly just a scheduled `Actor._cancel_task()`
# from a remote request to cancel some `Context`.
# ------ - ------
# TODO: ideally we unify this with the above `context=True`
# block such that for any remote invocation ftype, we
# always invoke the far end RPC task scheduling the same
# way: using the linked IPC context machinery.
failed_resp: bool = False
try:
await chan.send({
'functype': 'asyncfunc',
'cid': cid
})
except (
trio.ClosedResourceError,
trio.BrokenResourceError,
BrokenPipeError,
) as ipc_err:
failed_resp = True
if is_rpc:
raise
else:
# TODO: should this be an `.exception()` call?
log.warning(
f'Failed to respond to non-rpc request: {func}\n'
f'{ipc_err}'
)
with cancel_scope as cs:
ctx._scope: CancelScope = cs
task_status.started(ctx)
result = await coro
fname: str = func.__name__
log.runtime(
'RPC complete:\n'
f'task: {ctx._task}\n'
f'|_cid={ctx.cid}\n'
f'|_{fname}() -> {pformat(result)}\n'
)
# NOTE: only send result if we know IPC isn't down
if (
not failed_resp
and chan.connected()
):
try:
await chan.send(
{'return': result,
'cid': cid}
)
except (
BrokenPipeError,
trio.BrokenResourceError,
):
log.warning(
'Failed to return result:\n'
f'{func}@{actor.uid}\n'
f'remote chan: {chan.uid}'
)
@acm
async def _errors_relayed_via_ipc(
actor: Actor,
chan: Channel,
ctx: Context,
is_rpc: bool,
hide_tb: bool = False,
debug_kbis: bool = False,
task_status: TaskStatus[
Context | BaseException
] = trio.TASK_STATUS_IGNORED,
) -> None:
__tracebackhide__: bool = hide_tb # TODO: use hide_tb here?
try:
yield # run RPC invoke body
# box and ship RPC errors for wire-transit via
# the task's requesting parent IPC-channel.
except (
Exception,
BaseExceptionGroup,
KeyboardInterrupt,
) as err:
# always hide this frame from debug REPL if the crash
# originated from an rpc task and we DID NOT fail due to
# an IPC transport error!
if (
is_rpc
and chan.connected()
):
__tracebackhide__: bool = hide_tb
if not is_multi_cancelled(err):
# TODO: maybe we'll want different "levels" of debugging
# eventualy such as ('app', 'supervisory', 'runtime') ?
# if not isinstance(err, trio.ClosedResourceError) and (
# if not is_multi_cancelled(err) and (
entered_debug: bool = False
if (
(
not isinstance(err, ContextCancelled)
or (
isinstance(err, ContextCancelled)
and ctx._cancel_called
# if the root blocks the debugger lock request from a child
# we will get a remote-cancelled condition.
and ctx._enter_debugger_on_cancel
)
)
and
(
not isinstance(err, KeyboardInterrupt)
or (
isinstance(err, KeyboardInterrupt)
and debug_kbis
)
)
):
# await _debug.pause()
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# XXX QUESTION XXX: is there any case where we'll
# want to debug IPC disconnects as a default?
# => I can't think of a reason that inspecting this
# type of failure will be useful for respawns or
# recovery logic - the only case is some kind of
# strange bug in our transport layer itself? Going
# to keep this open ended for now.
entered_debug = await _debug._maybe_enter_pm(err)
if not entered_debug:
log.exception('Actor crashed:\n')
# always ship errors back to caller
err_msg: dict[str, dict] = pack_error(
err,
# tb=tb, # TODO: special tb fmting?
cid=ctx.cid,
)
# NOTE: the src actor should always be packed into the
# error.. but how should we verify this?
# assert err_msg['src_actor_uid']
# if not err_msg['error'].get('src_actor_uid'):
# import pdbp; pdbp.set_trace()
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
if is_rpc:
try:
await chan.send(err_msg)
# TODO: tests for this scenario:
# - RPC caller closes connection before getting a response
# should **not** crash this actor..
except (
trio.ClosedResourceError,
trio.BrokenResourceError,
BrokenPipeError,
) as ipc_err:
# if we can't propagate the error that's a big boo boo
log.exception(
f"Failed to ship error to caller @ {chan.uid} !?\n"
f'{ipc_err}'
)
# error is probably from above coro running code *not from
# the target rpc invocation since a scope was never
# allocated around the coroutine await.
if ctx._scope is None:
# we don't ever raise directly here to allow the
# msg-loop-scheduler to continue running for this
# channel.
task_status.started(err)
# always reraise KBIs so they propagate at the sys-process
# level.
if isinstance(err, KeyboardInterrupt):
raise
# RPC task bookeeping
finally:
try:
ctx, func, is_complete = actor._rpc_tasks.pop(
(chan, ctx.cid)
)
is_complete.set()
except KeyError:
if is_rpc:
# If we're cancelled before the task returns then the
# cancel scope will not have been inserted yet
log.warning(
'RPC task likely errored or cancelled before start?'
f'|_{ctx._task}\n'
f' >> {ctx.repr_rpc}\n'
)
else:
log.cancel(
'Failed to de-alloc internal runtime cancel task?\n'
f'|_{ctx._task}\n'
f' >> {ctx.repr_rpc}\n'
)
finally:
if not actor._rpc_tasks:
log.runtime("All RPC tasks have completed")
actor._ongoing_rpc_tasks.set()
2018-07-14 20:09:05 +00:00
async def _invoke(
actor: 'Actor',
cid: str,
chan: Channel,
2022-02-14 15:37:31 +00:00
func: Callable,
2021-12-02 17:34:27 +00:00
kwargs: dict[str, Any],
2021-10-08 22:20:08 +00:00
is_rpc: bool = True,
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
hide_tb: bool = True,
2021-01-14 23:18:44 +00:00
task_status: TaskStatus[
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
Union[Context, BaseException]
2021-01-14 23:18:44 +00:00
] = trio.TASK_STATUS_IGNORED,
2018-07-14 20:09:05 +00:00
):
'''
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
Schedule a `trio` task-as-func and deliver result(s) over
connected IPC channel.
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
This is the core "RPC" `trio.Task` scheduling machinery used to start every
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
remotely invoked function, normally in `Actor._service_n: Nursery`.
'''
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
__tracebackhide__: bool = hide_tb
treat_as_gen: bool = False
# possibly a traceback (not sure what typing is for this..)
tb = None
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
cancel_scope = CancelScope()
# activated cancel scope ref
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
cs: CancelScope|None = None
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
ctx = actor.get_context(
chan=chan,
cid=cid,
nsf=NamespacePath.from_ref(func),
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
# We shouldn't ever need to pass this through right?
# it's up to the soon-to-be called rpc task to
# open the stream with this option.
# allow_overruns=True,
)
context: bool = False
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
# TODO: deprecate this style..
if getattr(func, '_tractor_stream_function', False):
# handle decorated ``@tractor.stream`` async functions
sig = inspect.signature(func)
params = sig.parameters
# compat with old api
kwargs['ctx'] = ctx
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
treat_as_gen = True
if 'ctx' in params:
warnings.warn(
"`@tractor.stream decorated funcs should now declare "
"a `stream` arg, `ctx` is now designated for use with "
"@tractor.context",
DeprecationWarning,
stacklevel=2,
)
elif 'stream' in params:
assert 'stream' in params
kwargs['stream'] = ctx
elif getattr(func, '_tractor_context_function', False):
# handle decorated ``@tractor.context`` async function
kwargs['ctx'] = ctx
context = True
# errors raised inside this block are propgated back to caller
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
async with _errors_relayed_via_ipc(
actor,
chan,
ctx,
is_rpc,
hide_tb=hide_tb,
task_status=task_status,
):
if not (
inspect.isasyncgenfunction(func) or
inspect.iscoroutinefunction(func)
):
raise TypeError(f'{func} must be an async function!')
2018-07-14 20:09:05 +00:00
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# init coroutine with `kwargs` to immediately catch any
# type-sig errors.
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
try:
coro = func(**kwargs)
except TypeError:
raise
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# TODO: implement all these cases in terms of the
# `Context` one!
if not context:
await _invoke_non_context(
actor,
cancel_scope,
ctx,
cid,
chan,
func,
coro,
kwargs,
treat_as_gen,
is_rpc,
task_status,
)
# below is only for `@context` funcs
return
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
# our most general case: a remote SC-transitive,
# IPC-linked, cross-actor-task "context"
# ------ - ------
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
# TODO: every other "func type" should be implemented from
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
# a special case of this impl eventually!
# -[ ] streaming funcs should instead of being async-for
# handled directly here wrapped in
# a async-with-open_stream() closure that does the
# normal thing you'd expect a far end streaming context
# to (if written by the app-dev).
# -[ ] one off async funcs can literally just be called
# here and awaited directly, possibly just with a small
# wrapper that calls `Context.started()` and then does
# the `await coro()`?
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# a "context" endpoint type is the most general and
# "least sugary" type of RPC ep with support for
# bi-dir streaming B)
await chan.send({
'functype': 'context',
'cid': cid
})
# TODO: should we also use an `.open_context()` equiv
# for this callee side by factoring the impl from
# `Portal.open_context()` into a common helper?
#
# NOTE: there are many different ctx state details
# in a callee side instance according to current impl:
# - `.cancelled_caught` can never be `True`.
# -> the below scope is never exposed to the
# `@context` marked RPC function.
# - `._portal` is never set.
try:
async with trio.open_nursery() as tn:
ctx._scope_nursery = tn
ctx._scope = tn.cancel_scope
task_status.started(ctx)
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# TODO: should would be nice to have our
# `TaskMngr` nursery here!
res: Any = await coro
ctx._result = res
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# deliver final result to caller side.
await chan.send({
'return': res,
'cid': cid
})
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# NOTE: this happens IFF `ctx._scope.cancel()` is
# called by any of,
# - *this* callee task manually calling `ctx.cancel()`.
# - the runtime calling `ctx._deliver_msg()` which
# itself calls `ctx._maybe_cancel_and_set_remote_error()`
# which cancels the scope presuming the input error
# is not a `.cancel_acked` pleaser.
# - currently a never-should-happen-fallthrough case
# inside ._context._drain_to_final_msg()`..
# # TODO: remove this ^ right?
if ctx._scope.cancelled_caught:
our_uid: tuple = actor.uid
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
# first check for and raise any remote error
# before raising any context cancelled case
# so that real remote errors don't get masked as
# ``ContextCancelled``s.
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
if re := ctx._remote_error:
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
ctx._maybe_raise_remote_err(re)
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
cs: CancelScope = ctx._scope
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
if cs.cancel_called:
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
canceller: tuple = ctx.canceller
msg: str = (
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
'actor was cancelled by '
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
)
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
# NOTE / TODO: if we end up having
# ``Actor._cancel_task()`` call
# ``Context.cancel()`` directly, we're going to
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
# need to change this logic branch since it
# will always enter..
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
if ctx._cancel_called:
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
# TODO: test for this!!!!!
canceller: tuple = our_uid
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
msg += 'itself '
Support debug-lock blocking, use on no-more IPC This is a lingering debugger locking race case we needed to handle: - child crashes acquires TTY lock in root and attaches to `pdb` - child IPC goes down such that all channels to the root are broken / non-functional. - root is stuck thinking the child is still in debug even though it can't be contacted and the child actor machinery hasn't been cancelled by its parent. - root get's stuck in deadlock with child since it won't send a cancel request until the child is finished debugging, but the child can't unlock the debugger bc IPC is down. To avoid this scenario add debug lock blocking list via `._debug.Lock._blocked: set[tuple]` which holds actor uids for any actor that is detected by the root as having no transport channel connections with said root (of which at least one should exist if this sub-actor at some point acquired the debug lock). The root consequently checks this list for any actor that tries to (re)acquire the lock and blocks with a `ContextCancelled`. When a debug condition is tested in `._runtime._invoke` the context's `._enter_debugger_on_cancel` which is set to `False` if the actor is on the block list in which case the post-mortem entry is skipped. Further this adds a root-locking-task side cancel scope to `Lock._root_local_task_cs_in_debug` which can be cancelled by the root runtime when a stale lock is detected after all IPC channels for the actor have been torn down. NOTE: right now we're NOT doing this since it seems to cause test failures likely due because it may cause pre-mature cancellation and maybe needs a bit more experimenting?
2022-10-11 19:22:19 +00:00
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
# if the channel which spawned the ctx is the
# one that cancelled it then we report that, vs.
# it being some other random actor that for ex.
# some actor who calls `Portal.cancel_actor()`
# and by side-effect cancels this ctx.
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
elif canceller == ctx.chan.uid:
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
msg += 'its caller'
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
else:
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
msg += 'a remote peer'
div_chars: str = '------ - ------'
div_offset: int = (
round(len(msg)/2)+1
+
round(len(div_chars)/2)+1
)
div_str: str = (
'\n'
+
' '*div_offset
+
f'{div_chars}\n'
)
msg += (
div_str +
f'<= canceller: {canceller}\n'
f'=> uid: {our_uid}\n'
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
f' |_{ctx._task}()'
# TODO: instead just show the
# ctx.__str__() here?
# -[ ] textwrap.indent() it correctly!
# -[ ] BUT we need to wait until
# the state is filled out before emitting
# this msg right ow its kinda empty? bleh..
#
# f' |_{ctx}'
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
)
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
# task-contex was either cancelled by request using
# ``Portal.cancel_actor()`` or ``Context.cancel()``
# on the far end, or it was cancelled by the local
# (callee) task, so relay this cancel signal to the
# other side.
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
ctxc = ContextCancelled(
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
msg,
suberror_type=trio.Cancelled,
canceller=canceller,
)
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# assign local error so that the `.outcome`
# resolves to an error for both reporting and
# state checks.
ctx._local_error = ctxc
raise ctxc
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# XXX: do we ever trigger this block any more?
except (
BaseExceptionGroup,
trio.Cancelled,
BaseException,
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
) as scope_error:
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# always set this (callee) side's exception as the
# local error on the context
ctx._local_error: BaseException = scope_error
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# if a remote error was set then likely the
# exception group was raised due to that, so
# and we instead raise that error immediately!
ctx.maybe_raise()
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# maybe TODO: pack in come kinda
# `trio.Cancelled.__traceback__` here so they can be
# unwrapped and displayed on the caller side? no se..
raise
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# `@context` entrypoint task bookeeping.
# i.e. only pop the context tracking if used ;)
finally:
assert chan.uid
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# don't pop the local context until we know the
# associated child isn't in debug any more
await _debug.maybe_wait_for_debugger()
ctx: Context = actor._contexts.pop(
(chan.uid, cid)
)
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
merr: Exception|None = ctx.maybe_error
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
(
res_type_str,
res_str,
) = (
('error', f'{type(merr)}',)
if merr
else (
'result',
f'`{repr(ctx.outcome)}`',
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
)
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
)
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
log.cancel(
f'IPC context terminated with a final {res_type_str}\n\n'
f'{ctx}\n'
)
2018-07-14 20:09:05 +00:00
def _get_mod_abspath(module):
return os.path.abspath(module.__file__)
async def try_ship_error_to_parent(
2021-12-02 17:34:27 +00:00
channel: Channel,
err: Union[Exception, BaseExceptionGroup],
) -> None:
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
'''
Box, pack and encode a local runtime(-internal) exception for
an IPC channel `.send()` with transport/network failures and
local cancellation ignored but logged as critical(ly bad).
'''
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
with CancelScope(shield=True):
try:
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
await channel.send(
# NOTE: normally only used for internal runtime errors
# so ship to peer actor without a cid.
pack_error(err)
)
except (
trio.ClosedResourceError,
trio.BrokenResourceError,
):
# in SC terms this is one of the worst things that can
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
# happen and provides for a 2-general's dilemma..
log.critical(
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
f'Failed to ship error to parent '
f'{channel.uid}, IPC transport failure!'
)
2018-07-14 20:09:05 +00:00
class Actor:
'''
The fundamental "runtime" concurrency primitive.
An *actor* is the combination of a regular Python process executing
a ``trio`` task tree, communicating with other actors through
"memory boundary portals" - which provide a native async API around
IPC transport "channels" which themselves encapsulate various
(swappable) network protocols.
2023-08-18 14:10:36 +00:00
Each "actor" is ``trio.run()`` scheduled "runtime" composed of
many concurrent tasks in a single thread. The "runtime" tasks
conduct a slew of low(er) level functions to make it possible
for message passing between actors as well as the ability to
create new actors (aka new "runtimes" in new processes which
are supervised via a nursery construct). Each task which sends
messages to a task in a "peer" (not necessarily a parent-child,
depth hierarchy) is able to do so via an "address", which maps
IPC connections across memory boundaries, and a task request id
which allows for per-actor tasks to send and receive messages
to specific peer-actor tasks with which there is an ongoing
RPC/IPC dialog.
2018-07-14 20:09:05 +00:00
'''
# ugh, we need to get rid of this and replace with a "registry" sys
# https://github.com/goodboy/tractor/issues/216
2019-12-10 05:55:03 +00:00
is_arbiter: bool = False
Disable msg stream backpressure by default Half of portal API usage requires a 1 message response (`.run()`, `.run_in_actor()`) and the streaming APIs should probably be explicitly enabled for backpressure if desired by the user. This makes more sense in (psuedo) realtime systems where it's better to notify on a block then freeze without notice. Make this default behaviour with a new error to be raised: `tractor._exceptions.StreamOverrun` when a sender overruns a stream by the default size (2**6 for now). The old behavior can be enabled with `Context.open_stream(backpressure=True)` but now with warning log messages when there are overruns. Add task-linked-context error propagation using a "nursery raising" technique such that if either end of context linked pair of tasks errors, that error can be relayed to other side and raised as a form of interrupt at the receiving task's next `trio` checkpoint. This enables reliable error relay without expecting the (error) receiving task to call an API which would raise the remote exception (which it might never currently if using `tractor.MsgStream` APIs). Further internal implementation details: - define the default msg buffer size as `Actor.msg_buffer_size` - expose a `msg_buffer_size: int` kwarg from `Actor.get_context()` - maybe raise aforementioned context errors using `Context._maybe_error_from_remote_msg()` inside `Actor._push_result()` - support optional backpressure on a stream when pushing messages in `Actor._push_result()` - in `_invote()` handle multierrors raised from a `@tractor.context` entrypoint as being potentially caused by a relayed error from the remote caller task, if `Context._error` has been set then raise that error inside the `RemoteActorError` that will be relayed back to that caller more or less proxying through the source side error back to its origin.
2021-12-06 00:31:41 +00:00
msg_buffer_size: int = 2**6
2019-12-10 05:55:03 +00:00
2022-08-03 19:29:34 +00:00
# nursery placeholders filled in by `async_main()` after fork
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
_root_n: Nursery | None = None
_service_n: Nursery | None = None
_server_n: Nursery | None = None
2018-07-14 20:09:05 +00:00
# Information about `__main__` from parent
2021-12-02 17:34:27 +00:00
_parent_main_data: dict[str, str]
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
_parent_chan_cs: CancelScope | None = None
# syncs for setup/teardown sequences
2023-05-25 20:00:05 +00:00
_server_down: trio.Event | None = None
# user toggled crash handling (including monkey-patched in
# `trio.open_nursery()` via `.trionics._supervisor` B)
_debug_mode: bool = False
# if started on ``asycio`` running ``trio`` in guest mode
_infected_aio: bool = False
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
# _ans: dict[
# tuple[str, str],
# list[ActorNursery],
# ] = {}
# Process-global stack closed at end on actor runtime teardown.
# NOTE: this is currently an undocumented public api.
lifetime_stack: ExitStack = ExitStack()
2018-07-14 20:09:05 +00:00
def __init__(
self,
name: str,
*,
enable_modules: list[str] = [],
uid: str | None = None,
loglevel: str | None = None,
2023-05-25 20:00:05 +00:00
arbiter_addr: tuple[str, int] | None = None,
spawn_method: str | None = None
2018-08-31 21:16:24 +00:00
) -> None:
2022-08-03 20:09:16 +00:00
'''
This constructor is called in the parent actor **before** the spawning
phase (aka before a new process is executed).
2022-08-03 20:09:16 +00:00
'''
2018-07-14 20:09:05 +00:00
self.name = name
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
self.uid = (
name,
uid or str(uuid.uuid4())
)
self._cancel_complete = trio.Event()
self._cancel_called_by_remote: tuple[str, tuple] | None = None
self._cancel_called: bool = False
# retreive and store parent `__main__` data which
# will be passed to children
self._parent_main_data = _mp_fixup_main._mp_figure_out_main()
2020-07-30 14:41:58 +00:00
# always include debugging tools module
2021-01-05 13:28:06 +00:00
enable_modules.append('tractor._debug')
2020-07-30 14:41:58 +00:00
mods = {}
2021-01-05 13:28:06 +00:00
for name in enable_modules:
mod = importlib.import_module(name)
mods[name] = _get_mod_abspath(mod)
2021-01-05 13:28:06 +00:00
self.enable_modules = mods
2021-12-02 17:34:27 +00:00
self._mods: dict[str, ModuleType] = {}
2018-07-14 20:09:05 +00:00
self.loglevel = loglevel
2021-07-01 18:52:52 +00:00
self._arb_addr: tuple[str, int] | None = (
str(arbiter_addr[0]),
int(arbiter_addr[1])
) if arbiter_addr else None
2018-07-14 20:09:05 +00:00
# marked by the process spawning backend at startup
# will be None for the parent most process started manually
# by the user (currently called the "arbiter")
self._spawn_method = spawn_method
2018-08-31 21:16:24 +00:00
self._peers: defaultdict = defaultdict(list)
self._peer_connected: dict = {}
2018-07-14 20:09:05 +00:00
self._no_more_peers = trio.Event()
self._no_more_peers.set()
self._ongoing_rpc_tasks = trio.Event()
self._ongoing_rpc_tasks.set()
# (chan, cid) -> (cancel_scope, func)
2021-12-02 17:34:27 +00:00
self._rpc_tasks: dict[
tuple[Channel, str],
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
tuple[Context, Callable, trio.Event]
2018-08-31 21:16:24 +00:00
] = {}
# map {actor uids -> Context}
self._contexts: dict[
tuple[tuple[str, str], str],
Context
2019-12-10 05:55:03 +00:00
] = {}
self._listeners: list[trio.abc.Listener] = []
2023-05-25 20:00:05 +00:00
self._parent_chan: Channel | None = None
self._forkserver_info: tuple | None = None
self._actoruid2nursery: dict[
tuple[str, str],
ActorNursery | None,
] = {} # type: ignore # noqa
2018-07-14 20:09:05 +00:00
2018-08-26 17:12:29 +00:00
async def wait_for_peer(
self, uid: tuple[str, str]
) -> tuple[trio.Event, Channel]:
2022-08-03 20:09:16 +00:00
'''
Wait for a connection back from a spawned actor with a given
2018-07-14 20:09:05 +00:00
``uid``.
2022-08-03 20:09:16 +00:00
'''
log.runtime(f"Waiting for peer {uid} to connect")
2018-07-14 20:09:05 +00:00
event = self._peer_connected.setdefault(uid, trio.Event())
await event.wait()
log.runtime(f"{uid} successfully connected back to us")
2018-07-14 20:09:05 +00:00
return event, self._peers[uid][-1]
def load_modules(self) -> None:
2022-08-03 20:09:16 +00:00
'''
Load allowed RPC modules locally (after fork).
Since this actor may be spawned on a different machine from
the original nursery we need to try and load the local module
code (if it exists).
2022-08-03 20:09:16 +00:00
'''
2019-11-26 14:23:37 +00:00
try:
if self._spawn_method == 'trio':
parent_data = self._parent_main_data
if 'init_main_from_name' in parent_data:
_mp_fixup_main._fixup_main_from_name(
parent_data['init_main_from_name'])
elif 'init_main_from_path' in parent_data:
_mp_fixup_main._fixup_main_from_path(
parent_data['init_main_from_path'])
2021-01-05 13:28:06 +00:00
for modpath, filepath in self.enable_modules.items():
# XXX append the allowed module to the python path which
# should allow for relative (at least downward) imports.
sys.path.append(os.path.dirname(filepath))
log.runtime(f"Attempting to import {modpath}@{filepath}")
2020-02-09 06:05:52 +00:00
mod = importlib.import_module(modpath)
self._mods[modpath] = mod
if modpath == '__main__':
self._mods['__mp_main__'] = mod
2021-11-07 22:05:40 +00:00
2019-11-26 14:23:37 +00:00
except ModuleNotFoundError:
# it is expected the corresponding `ModuleNotExposed` error
# will be raised later
log.error(f"Failed to import {modpath} in {self.name}")
2019-11-26 14:23:37 +00:00
raise
2018-07-14 20:09:05 +00:00
def _get_rpc_func(self, ns, funcname):
try:
return getattr(self._mods[ns], funcname)
except KeyError as err:
2020-10-16 02:49:12 +00:00
mne = ModuleNotExposed(*err.args)
if ns == '__main__':
modpath = '__name__'
else:
modpath = f"'{ns}'"
msg = (
"\n\nMake sure you exposed the target module, `{ns}`, "
"using:\n"
"ActorNursery.start_actor(<name>, enable_modules=[{mod}])"
).format(
ns=ns,
mod=modpath,
)
2020-10-16 02:49:12 +00:00
mne.msg += msg
2020-10-16 02:49:12 +00:00
raise mne
2018-07-14 20:09:05 +00:00
async def _stream_handler(
2018-07-14 20:09:05 +00:00
self,
stream: trio.SocketStream,
) -> None:
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
'''
Entry point for new inbound connections to the channel server.
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
'''
self._no_more_peers = trio.Event() # unset by making new
chan = Channel.from_stream(stream)
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
their_uid: tuple[str, str]|None = chan.uid
2018-07-14 20:09:05 +00:00
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
con_msg: str = ''
if their_uid:
# NOTE: `.uid` is only set after first contact
con_msg = (
'IPC Re-connection from already known peer? '
)
else:
con_msg = (
'New IPC connection to us '
)
con_msg += (
f'<= @{chan.raddr}\n'
f'|_{chan}\n'
# f' |_@{chan.raddr}\n\n'
)
2018-07-14 20:09:05 +00:00
# send/receive initial handshake response
try:
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
uid: tuple|None = await self._do_handshake(chan)
except (
# we need this for ``msgspec`` for some reason?
# for now, it's been put in the stream backend.
# trio.BrokenResourceError,
# trio.ClosedResourceError,
TransportClosed,
):
# XXX: This may propagate up from ``Channel._aiter_recv()``
# and ``MsgpackStream._inter_packets()`` on a read from the
# stream particularly when the runtime is first starting up
# inside ``open_root_actor()`` where there is a check for
# a bound listener on the "arbiter" addr. the reset will be
# because the handshake was never meant took place.
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
log.warning(
con_msg
+
' -> But failed to handshake? Ignoring..\n'
)
2018-07-14 20:09:05 +00:00
return
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
con_msg += (
f' -> Handshake with actor `{uid[0]}[{uid[1][-6:]}]` complete\n'
)
# IPC connection tracking for both peers and new children:
# - if this is a new channel to a locally spawned
# sub-actor there will be a spawn wait even registered
# by a call to `.wait_for_peer()`.
# - if a peer is connecting no such event will exit.
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
event: trio.Event|None = self._peer_connected.pop(
uid,
None,
)
2018-07-14 20:09:05 +00:00
if event:
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
con_msg += (
' -> Waking subactor spawn waiters: '
f'{event.statistics().tasks_waiting}\n'
f' -> Registered IPC chan for child actor {uid}@{chan.raddr}\n'
# f' {event}\n'
# f' |{event.statistics()}\n'
)
# wake tasks waiting on this IPC-transport "connect-back"
2018-07-14 20:09:05 +00:00
event.set()
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
else:
con_msg += (
f' -> Registered IPC chan for peer actor {uid}@{chan.raddr}\n'
) # type: ignore
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
chans: list[Channel] = self._peers[uid]
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# if chans:
# # TODO: re-use channels for new connections instead
# # of always new ones?
# # => will require changing all the discovery funcs..
2021-06-30 17:47:07 +00:00
2018-07-14 20:09:05 +00:00
# append new channel
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
# TODO: can we just use list-ref directly?
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
chans.append(chan)
log.runtime(con_msg)
2018-07-14 20:09:05 +00:00
# Begin channel management - respond to remote requests and
# process received reponses.
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
disconnected: bool = False
2018-07-14 20:09:05 +00:00
try:
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
disconnected: bool = await process_messages(
self,
chan,
)
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
except trio.Cancelled:
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
log.cancel(
'IPC transport msg loop was cancelled for \n'
f'|_{chan}\n'
)
raise
2018-07-14 20:09:05 +00:00
finally:
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
local_nursery: (
ActorNursery|None
) = self._actoruid2nursery.get(uid)
# This is set in ``Portal.cancel_actor()``. So if
# the peer was cancelled we try to wait for them
# to tear down their side of the connection before
# moving on with closing our own side.
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
if local_nursery:
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
if chan._cancel_called:
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
log.cancel(
'Waiting on cancel request to peer\n'
f'`Portal.cancel_actor()` => {chan.uid}\n'
)
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
# XXX: this is a soft wait on the channel (and its
# underlying transport protocol) to close from the
# remote peer side since we presume that any channel
# which is mapped to a sub-actor (i.e. it's managed by
# one of our local nurseries) has a message is sent to
# the peer likely by this actor (which is now in
# a cancelled condition) when the local runtime here is
# now cancelled while (presumably) in the middle of msg
# loop processing.
Disable msg stream backpressure by default Half of portal API usage requires a 1 message response (`.run()`, `.run_in_actor()`) and the streaming APIs should probably be explicitly enabled for backpressure if desired by the user. This makes more sense in (psuedo) realtime systems where it's better to notify on a block then freeze without notice. Make this default behaviour with a new error to be raised: `tractor._exceptions.StreamOverrun` when a sender overruns a stream by the default size (2**6 for now). The old behavior can be enabled with `Context.open_stream(backpressure=True)` but now with warning log messages when there are overruns. Add task-linked-context error propagation using a "nursery raising" technique such that if either end of context linked pair of tasks errors, that error can be relayed to other side and raised as a form of interrupt at the receiving task's next `trio` checkpoint. This enables reliable error relay without expecting the (error) receiving task to call an API which would raise the remote exception (which it might never currently if using `tractor.MsgStream` APIs). Further internal implementation details: - define the default msg buffer size as `Actor.msg_buffer_size` - expose a `msg_buffer_size: int` kwarg from `Actor.get_context()` - maybe raise aforementioned context errors using `Context._maybe_error_from_remote_msg()` inside `Actor._push_result()` - support optional backpressure on a stream when pushing messages in `Actor._push_result()` - in `_invote()` handle multierrors raised from a `@tractor.context` entrypoint as being potentially caused by a relayed error from the remote caller task, if `Context._error` has been set then raise that error inside the `RemoteActorError` that will be relayed back to that caller more or less proxying through the source side error back to its origin.
2021-12-06 00:31:41 +00:00
with trio.move_on_after(0.5) as cs:
cs.shield = True
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# attempt to wait for the far end to close the
# channel and bail after timeout (a 2-generals
# problem on closure).
assert chan.transport
async for msg in chan.transport.drain():
# try to deliver any lingering msgs
# before we destroy the channel.
# This accomplishes deterministic
# ``Portal.cancel_actor()`` cancellation by
# making sure any RPC response to that call is
# delivered the local calling task.
# TODO: factor this into a helper?
log.warning(
2024-02-22 20:06:39 +00:00
'Draining msg from disconnected peer\n'
f'{chan.uid}\n'
f'|_{chan}\n'
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
f' |_{chan.transport}\n\n'
2024-02-22 20:06:39 +00:00
f'{pformat(msg)}\n'
)
cid = msg.get('cid')
if cid:
# deliver response to local caller/waiter
await self._push_result(
chan,
cid,
msg,
)
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# NOTE: when no call to `open_root_actor()` was
# made, we implicitly make that call inside
# the first `.open_nursery()`, in this case we
# can assume that we are the root actor and do
# not have to wait for the nursery-enterer to
# exit before shutting down the actor runtime.
#
# see matching note inside `._supervise.open_nursery()`
if not local_nursery._implicit_runtime_started:
log.runtime(
'Waiting on local actor nursery to exit..\n'
f'|_{local_nursery}\n'
)
await local_nursery.exited.wait()
if (
cs.cancelled_caught
and not local_nursery._implicit_runtime_started
):
log.warning(
'Failed to exit local actor nursery?\n'
f'|_{local_nursery}\n'
)
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# await _debug.pause()
if disconnected:
# if the transport died and this actor is still
# registered within a local nursery, we report that the
# IPC layer may have failed unexpectedly since it may be
# the cause of other downstream errors.
entry = local_nursery._children.get(uid)
if entry:
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
proc: trio.Process
_, proc, _ = entry
poll = getattr(proc, 'poll', None)
if poll and poll() is None:
log.cancel(
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
f'Peer IPC broke but subproc is alive?\n\n'
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
f'<=x {chan.uid}@{chan.raddr}\n'
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
f' |_{proc}\n'
)
# ``Channel`` teardown and closure sequence
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
# drop ref to channel so it can be gc-ed and disconnected
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
log.runtime(
f'Disconnected IPC channel:\n'
f'uid: {chan.uid}\n'
f'|_{pformat(chan)}\n'
)
2018-07-14 20:09:05 +00:00
chans.remove(chan)
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# TODO: do we need to be this pedantic?
2018-07-14 20:09:05 +00:00
if not chans:
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
log.runtime(
f'No more channels with {chan.uid}'
)
self._peers.pop(uid, None)
2018-07-14 20:09:05 +00:00
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
peers_str: str = ''
for uid, chans in self._peers.items():
peers_str += (
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
f'|_ uid: {uid}\n'
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
)
for i, chan in enumerate(chans):
peers_str += (
f' |_[{i}] {pformat(chan)}\n'
)
log.runtime(
f'Remaining IPC {len(self._peers)} peers:\n'
+ peers_str
)
# No more channels to other actors (at all) registered
# as connected.
if not self._peers:
log.runtime("Signalling no more peer channel connections")
self._no_more_peers.set()
Support debug-lock blocking, use on no-more IPC This is a lingering debugger locking race case we needed to handle: - child crashes acquires TTY lock in root and attaches to `pdb` - child IPC goes down such that all channels to the root are broken / non-functional. - root is stuck thinking the child is still in debug even though it can't be contacted and the child actor machinery hasn't been cancelled by its parent. - root get's stuck in deadlock with child since it won't send a cancel request until the child is finished debugging, but the child can't unlock the debugger bc IPC is down. To avoid this scenario add debug lock blocking list via `._debug.Lock._blocked: set[tuple]` which holds actor uids for any actor that is detected by the root as having no transport channel connections with said root (of which at least one should exist if this sub-actor at some point acquired the debug lock). The root consequently checks this list for any actor that tries to (re)acquire the lock and blocks with a `ContextCancelled`. When a debug condition is tested in `._runtime._invoke` the context's `._enter_debugger_on_cancel` which is set to `False` if the actor is on the block list in which case the post-mortem entry is skipped. Further this adds a root-locking-task side cancel scope to `Lock._root_local_task_cs_in_debug` which can be cancelled by the root runtime when a stale lock is detected after all IPC channels for the actor have been torn down. NOTE: right now we're NOT doing this since it seems to cause test failures likely due because it may cause pre-mature cancellation and maybe needs a bit more experimenting?
2022-10-11 19:22:19 +00:00
# NOTE: block this actor from acquiring the
# debugger-TTY-lock since we have no way to know if we
# cancelled it and further there is no way to ensure the
# lock will be released if acquired due to having no
# more active IPC channels.
if _state.is_root_process():
Support debug-lock blocking, use on no-more IPC This is a lingering debugger locking race case we needed to handle: - child crashes acquires TTY lock in root and attaches to `pdb` - child IPC goes down such that all channels to the root are broken / non-functional. - root is stuck thinking the child is still in debug even though it can't be contacted and the child actor machinery hasn't been cancelled by its parent. - root get's stuck in deadlock with child since it won't send a cancel request until the child is finished debugging, but the child can't unlock the debugger bc IPC is down. To avoid this scenario add debug lock blocking list via `._debug.Lock._blocked: set[tuple]` which holds actor uids for any actor that is detected by the root as having no transport channel connections with said root (of which at least one should exist if this sub-actor at some point acquired the debug lock). The root consequently checks this list for any actor that tries to (re)acquire the lock and blocks with a `ContextCancelled`. When a debug condition is tested in `._runtime._invoke` the context's `._enter_debugger_on_cancel` which is set to `False` if the actor is on the block list in which case the post-mortem entry is skipped. Further this adds a root-locking-task side cancel scope to `Lock._root_local_task_cs_in_debug` which can be cancelled by the root runtime when a stale lock is detected after all IPC channels for the actor have been torn down. NOTE: right now we're NOT doing this since it seems to cause test failures likely due because it may cause pre-mature cancellation and maybe needs a bit more experimenting?
2022-10-11 19:22:19 +00:00
pdb_lock = _debug.Lock
pdb_lock._blocked.add(uid)
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
# TODO: NEEEDS TO BE TESTED!
# actually, no idea if this ever even enters.. XD
pdb_user_uid: tuple = pdb_lock.global_actor_in_debug
if (
pdb_user_uid
and local_nursery
):
entry: tuple|None = local_nursery._children.get(pdb_user_uid)
if entry:
proc: trio.Process
_, proc, _ = entry
if (
(poll := getattr(proc, 'poll', None))
and poll() is None
):
log.cancel(
'Root actor reports no-more-peers, BUT '
'a DISCONNECTED child still has the debug '
'lock!\n'
f'root uid: {self.uid}\n'
f'last disconnected child uid: {uid}\n'
f'locking child uid: {pdb_user_uid}\n'
)
await _debug.maybe_wait_for_debugger(
child_in_debug=True
)
# TODO: just bc a child's transport dropped
# doesn't mean it's not still using the pdb
# REPL! so,
# -[ ] ideally we can check out child proc
# tree to ensure that its alive (and
# actually using the REPL) before we cancel
# it's lock acquire by doing the below!
# -[ ] create a way to read the tree of each actor's
# grandchildren such that when an
# intermediary parent is cancelled but their
# child has locked the tty, the grandparent
# will not allow the parent to cancel or
# zombie reap the child! see open issue:
# - https://github.com/goodboy/tractor/issues/320
# ------ - ------
Support debug-lock blocking, use on no-more IPC This is a lingering debugger locking race case we needed to handle: - child crashes acquires TTY lock in root and attaches to `pdb` - child IPC goes down such that all channels to the root are broken / non-functional. - root is stuck thinking the child is still in debug even though it can't be contacted and the child actor machinery hasn't been cancelled by its parent. - root get's stuck in deadlock with child since it won't send a cancel request until the child is finished debugging, but the child can't unlock the debugger bc IPC is down. To avoid this scenario add debug lock blocking list via `._debug.Lock._blocked: set[tuple]` which holds actor uids for any actor that is detected by the root as having no transport channel connections with said root (of which at least one should exist if this sub-actor at some point acquired the debug lock). The root consequently checks this list for any actor that tries to (re)acquire the lock and blocks with a `ContextCancelled`. When a debug condition is tested in `._runtime._invoke` the context's `._enter_debugger_on_cancel` which is set to `False` if the actor is on the block list in which case the post-mortem entry is skipped. Further this adds a root-locking-task side cancel scope to `Lock._root_local_task_cs_in_debug` which can be cancelled by the root runtime when a stale lock is detected after all IPC channels for the actor have been torn down. NOTE: right now we're NOT doing this since it seems to cause test failures likely due because it may cause pre-mature cancellation and maybe needs a bit more experimenting?
2022-10-11 19:22:19 +00:00
# if a now stale local task has the TTY lock still
# we cancel it to allow servicing other requests for
# the lock.
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
db_cs: trio.CancelScope|None = pdb_lock._root_local_task_cs_in_debug
Support debug-lock blocking, use on no-more IPC This is a lingering debugger locking race case we needed to handle: - child crashes acquires TTY lock in root and attaches to `pdb` - child IPC goes down such that all channels to the root are broken / non-functional. - root is stuck thinking the child is still in debug even though it can't be contacted and the child actor machinery hasn't been cancelled by its parent. - root get's stuck in deadlock with child since it won't send a cancel request until the child is finished debugging, but the child can't unlock the debugger bc IPC is down. To avoid this scenario add debug lock blocking list via `._debug.Lock._blocked: set[tuple]` which holds actor uids for any actor that is detected by the root as having no transport channel connections with said root (of which at least one should exist if this sub-actor at some point acquired the debug lock). The root consequently checks this list for any actor that tries to (re)acquire the lock and blocks with a `ContextCancelled`. When a debug condition is tested in `._runtime._invoke` the context's `._enter_debugger_on_cancel` which is set to `False` if the actor is on the block list in which case the post-mortem entry is skipped. Further this adds a root-locking-task side cancel scope to `Lock._root_local_task_cs_in_debug` which can be cancelled by the root runtime when a stale lock is detected after all IPC channels for the actor have been torn down. NOTE: right now we're NOT doing this since it seems to cause test failures likely due because it may cause pre-mature cancellation and maybe needs a bit more experimenting?
2022-10-11 19:22:19 +00:00
if (
db_cs
and not db_cs.cancel_called
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
and uid == pdb_user_uid
Support debug-lock blocking, use on no-more IPC This is a lingering debugger locking race case we needed to handle: - child crashes acquires TTY lock in root and attaches to `pdb` - child IPC goes down such that all channels to the root are broken / non-functional. - root is stuck thinking the child is still in debug even though it can't be contacted and the child actor machinery hasn't been cancelled by its parent. - root get's stuck in deadlock with child since it won't send a cancel request until the child is finished debugging, but the child can't unlock the debugger bc IPC is down. To avoid this scenario add debug lock blocking list via `._debug.Lock._blocked: set[tuple]` which holds actor uids for any actor that is detected by the root as having no transport channel connections with said root (of which at least one should exist if this sub-actor at some point acquired the debug lock). The root consequently checks this list for any actor that tries to (re)acquire the lock and blocks with a `ContextCancelled`. When a debug condition is tested in `._runtime._invoke` the context's `._enter_debugger_on_cancel` which is set to `False` if the actor is on the block list in which case the post-mortem entry is skipped. Further this adds a root-locking-task side cancel scope to `Lock._root_local_task_cs_in_debug` which can be cancelled by the root runtime when a stale lock is detected after all IPC channels for the actor have been torn down. NOTE: right now we're NOT doing this since it seems to cause test failures likely due because it may cause pre-mature cancellation and maybe needs a bit more experimenting?
2022-10-11 19:22:19 +00:00
):
log.warning(
f'STALE DEBUG LOCK DETECTED FOR {uid}'
)
# TODO: figure out why this breaks tests..
db_cs.cancel()
2018-07-14 20:09:05 +00:00
# XXX: is this necessary (GC should do it)?
2024-02-19 19:41:03 +00:00
# XXX WARNING XXX
# Be AWARE OF THE INDENT LEVEL HERE
# -> ONLY ENTER THIS BLOCK WHEN ._peers IS
# EMPTY!!!!
if (
not self._peers
and chan.connected()
):
# if the channel is still connected it may mean the far
# end has not closed and we may have gotten here due to
# an error and so we should at least try to terminate
# the channel from this end gracefully.
log.runtime(
'Terminating channel with `None` setinel msg\n'
f'|_{chan}\n'
)
try:
# send a msg loop terminate sentinel
await chan.send(None)
# XXX: do we want this?
# causes "[104] connection reset by peer" on other end
# await chan.aclose()
except trio.BrokenResourceError:
log.runtime(f"Channel {chan.uid} was already closed")
2018-07-14 20:09:05 +00:00
2019-02-15 21:23:58 +00:00
async def _push_result(
self,
2019-02-16 19:05:03 +00:00
chan: Channel,
cid: str,
2021-12-02 17:34:27 +00:00
msg: dict[str, Any],
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
) -> None|bool:
Disable msg stream backpressure by default Half of portal API usage requires a 1 message response (`.run()`, `.run_in_actor()`) and the streaming APIs should probably be explicitly enabled for backpressure if desired by the user. This makes more sense in (psuedo) realtime systems where it's better to notify on a block then freeze without notice. Make this default behaviour with a new error to be raised: `tractor._exceptions.StreamOverrun` when a sender overruns a stream by the default size (2**6 for now). The old behavior can be enabled with `Context.open_stream(backpressure=True)` but now with warning log messages when there are overruns. Add task-linked-context error propagation using a "nursery raising" technique such that if either end of context linked pair of tasks errors, that error can be relayed to other side and raised as a form of interrupt at the receiving task's next `trio` checkpoint. This enables reliable error relay without expecting the (error) receiving task to call an API which would raise the remote exception (which it might never currently if using `tractor.MsgStream` APIs). Further internal implementation details: - define the default msg buffer size as `Actor.msg_buffer_size` - expose a `msg_buffer_size: int` kwarg from `Actor.get_context()` - maybe raise aforementioned context errors using `Context._maybe_error_from_remote_msg()` inside `Actor._push_result()` - support optional backpressure on a stream when pushing messages in `Actor._push_result()` - in `_invote()` handle multierrors raised from a `@tractor.context` entrypoint as being potentially caused by a relayed error from the remote caller task, if `Context._error` has been set then raise that error inside the `RemoteActorError` that will be relayed back to that caller more or less proxying through the source side error back to its origin.
2021-12-06 00:31:41 +00:00
'''
Push an RPC result to the local consumer's queue.
'''
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
uid: tuple[str, str] = chan.uid
assert uid, f"`chan.uid` can't be {uid}"
try:
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
ctx: Context = self._contexts[(uid, cid)]
except KeyError:
log.warning(
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
'Ignoring invalid IPC ctx msg!\n\n'
f'<= sender: {uid}\n'
f'=> cid: {cid}\n\n'
f'{msg}\n'
)
return
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
return await ctx._deliver_msg(msg)
2019-02-15 21:23:58 +00:00
def get_context(
2018-08-31 21:16:24 +00:00
self,
chan: Channel,
cid: str,
nsf: NamespacePath,
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
msg_buffer_size: int | None = None,
allow_overruns: bool = False,
) -> Context:
'''
Look up or create a new inter-actor-task-IPC-linked task
"context" which encapsulates the local task's scheduling
enviroment including a ``trio`` cancel scope, a pair of IPC
messaging "feeder" channels, and an RPC id unique to the
task-as-function invocation.
'''
actor_uid = chan.uid
assert actor_uid
2019-02-15 21:23:58 +00:00
try:
ctx = self._contexts[(actor_uid, cid)]
log.runtime(
f'Retreived cached IPC ctx for\n'
f'peer: {chan.uid}\n'
f'cid:{cid}\n'
)
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
ctx._allow_overruns = allow_overruns
Disable msg stream backpressure by default Half of portal API usage requires a 1 message response (`.run()`, `.run_in_actor()`) and the streaming APIs should probably be explicitly enabled for backpressure if desired by the user. This makes more sense in (psuedo) realtime systems where it's better to notify on a block then freeze without notice. Make this default behaviour with a new error to be raised: `tractor._exceptions.StreamOverrun` when a sender overruns a stream by the default size (2**6 for now). The old behavior can be enabled with `Context.open_stream(backpressure=True)` but now with warning log messages when there are overruns. Add task-linked-context error propagation using a "nursery raising" technique such that if either end of context linked pair of tasks errors, that error can be relayed to other side and raised as a form of interrupt at the receiving task's next `trio` checkpoint. This enables reliable error relay without expecting the (error) receiving task to call an API which would raise the remote exception (which it might never currently if using `tractor.MsgStream` APIs). Further internal implementation details: - define the default msg buffer size as `Actor.msg_buffer_size` - expose a `msg_buffer_size: int` kwarg from `Actor.get_context()` - maybe raise aforementioned context errors using `Context._maybe_error_from_remote_msg()` inside `Actor._push_result()` - support optional backpressure on a stream when pushing messages in `Actor._push_result()` - in `_invote()` handle multierrors raised from a `@tractor.context` entrypoint as being potentially caused by a relayed error from the remote caller task, if `Context._error` has been set then raise that error inside the `RemoteActorError` that will be relayed back to that caller more or less proxying through the source side error back to its origin.
2021-12-06 00:31:41 +00:00
# adjust buffer size if specified
2021-12-06 01:00:40 +00:00
state = ctx._send_chan._state # type: ignore
Disable msg stream backpressure by default Half of portal API usage requires a 1 message response (`.run()`, `.run_in_actor()`) and the streaming APIs should probably be explicitly enabled for backpressure if desired by the user. This makes more sense in (psuedo) realtime systems where it's better to notify on a block then freeze without notice. Make this default behaviour with a new error to be raised: `tractor._exceptions.StreamOverrun` when a sender overruns a stream by the default size (2**6 for now). The old behavior can be enabled with `Context.open_stream(backpressure=True)` but now with warning log messages when there are overruns. Add task-linked-context error propagation using a "nursery raising" technique such that if either end of context linked pair of tasks errors, that error can be relayed to other side and raised as a form of interrupt at the receiving task's next `trio` checkpoint. This enables reliable error relay without expecting the (error) receiving task to call an API which would raise the remote exception (which it might never currently if using `tractor.MsgStream` APIs). Further internal implementation details: - define the default msg buffer size as `Actor.msg_buffer_size` - expose a `msg_buffer_size: int` kwarg from `Actor.get_context()` - maybe raise aforementioned context errors using `Context._maybe_error_from_remote_msg()` inside `Actor._push_result()` - support optional backpressure on a stream when pushing messages in `Actor._push_result()` - in `_invote()` handle multierrors raised from a `@tractor.context` entrypoint as being potentially caused by a relayed error from the remote caller task, if `Context._error` has been set then raise that error inside the `RemoteActorError` that will be relayed back to that caller more or less proxying through the source side error back to its origin.
2021-12-06 00:31:41 +00:00
if msg_buffer_size and state.max_buffer_size != msg_buffer_size:
state.max_buffer_size = msg_buffer_size
2019-02-15 21:23:58 +00:00
except KeyError:
log.runtime(
f'Creating NEW IPC ctx for\n'
f'peer: {chan.uid}\n'
f'cid: {cid}\n'
)
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
ctx = mk_context(
chan,
cid,
nsf=nsf,
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
msg_buffer_size=msg_buffer_size or self.msg_buffer_size,
_allow_overruns=allow_overruns,
)
self._contexts[(actor_uid, cid)] = ctx
2019-02-15 21:23:58 +00:00
return ctx
2018-07-14 20:09:05 +00:00
async def start_remote_task(
2019-02-15 21:23:58 +00:00
self,
chan: Channel,
nsf: NamespacePath,
Disable msg stream backpressure by default Half of portal API usage requires a 1 message response (`.run()`, `.run_in_actor()`) and the streaming APIs should probably be explicitly enabled for backpressure if desired by the user. This makes more sense in (psuedo) realtime systems where it's better to notify on a block then freeze without notice. Make this default behaviour with a new error to be raised: `tractor._exceptions.StreamOverrun` when a sender overruns a stream by the default size (2**6 for now). The old behavior can be enabled with `Context.open_stream(backpressure=True)` but now with warning log messages when there are overruns. Add task-linked-context error propagation using a "nursery raising" technique such that if either end of context linked pair of tasks errors, that error can be relayed to other side and raised as a form of interrupt at the receiving task's next `trio` checkpoint. This enables reliable error relay without expecting the (error) receiving task to call an API which would raise the remote exception (which it might never currently if using `tractor.MsgStream` APIs). Further internal implementation details: - define the default msg buffer size as `Actor.msg_buffer_size` - expose a `msg_buffer_size: int` kwarg from `Actor.get_context()` - maybe raise aforementioned context errors using `Context._maybe_error_from_remote_msg()` inside `Actor._push_result()` - support optional backpressure on a stream when pushing messages in `Actor._push_result()` - in `_invote()` handle multierrors raised from a `@tractor.context` entrypoint as being potentially caused by a relayed error from the remote caller task, if `Context._error` has been set then raise that error inside the `RemoteActorError` that will be relayed back to that caller more or less proxying through the source side error back to its origin.
2021-12-06 00:31:41 +00:00
kwargs: dict,
# IPC channel config
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
msg_buffer_size: int | None = None,
allow_overruns: bool = False,
load_nsf: bool = False,
) -> Context:
'''
Send a ``'cmd'`` message to a remote actor, which starts
a remote task-as-function entrypoint.
Synchronously validates the endpoint type and return a caller
side task ``Context`` that can be used to wait for responses
delivered by the local runtime's message processing loop.
'''
cid = str(uuid.uuid4())
2018-08-31 21:16:24 +00:00
assert chan.uid
ctx = self.get_context(
chan=chan,
cid=cid,
nsf=nsf,
msg_buffer_size=msg_buffer_size,
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
allow_overruns=allow_overruns,
)
if (
'self' in nsf
or not load_nsf
):
ns, _, func = nsf.partition(':')
else:
# TODO: pass nsf directly over wire!
# -[ ] but, how to do `self:<Actor.meth>`??
ns, func = nsf.to_tuple()
log.runtime(
'Sending cmd to\n'
f'peer: {chan.uid} => \n'
'\n'
f'=> {ns}.{func}({kwargs})\n'
)
await chan.send(
{'cmd': (
ns,
func,
kwargs,
self.uid,
cid,
)}
)
# Wait on first response msg and validate; this should be
# immediate.
first_msg: dict = await ctx._recv_chan.receive()
functype: str = first_msg.get('functype')
if 'error' in first_msg:
raise unpack_error(first_msg, chan)
elif functype not in (
'asyncfunc',
'asyncgen',
'context',
):
raise ValueError(f"{first_msg} is an invalid response packet?")
ctx._remote_func_type = functype
return ctx
2018-07-14 20:09:05 +00:00
async def _from_parent(
self,
2023-05-25 20:00:05 +00:00
parent_addr: tuple[str, int] | None,
) -> tuple[Channel, tuple[str, int] | None]:
try:
# Connect back to the parent actor and conduct initial
# handshake. From this point on if we error, we
# attempt to ship the exception back to the parent.
chan = Channel(
destaddr=parent_addr,
)
await chan.connect()
# Initial handshake: swap names.
await self._do_handshake(chan)
2023-05-25 20:00:05 +00:00
accept_addr: tuple[str, int] | None = None
2020-08-09 00:57:18 +00:00
if self._spawn_method == "trio":
# Receive runtime state from our parent
2021-09-08 00:24:02 +00:00
parent_data: dict[str, Any]
parent_data = await chan.recv()
log.runtime(
'Received state from parent:\n\n'
# TODO: eventually all these msgs as
# `msgspec.Struct` with a special mode that
# pformats them in multi-line mode, BUT only
# if "trace"/"util" mode is enabled?
f'{pformat(parent_data)}\n'
)
accept_addr = (
parent_data.pop('bind_host'),
parent_data.pop('bind_port'),
)
rvs = parent_data.pop('_runtime_vars')
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
if rvs['_debug_mode']:
try:
log.info('Enabling `stackscope` traces on SIGUSR1')
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
from .devx import enable_stack_on_sig
enable_stack_on_sig()
except ImportError:
log.warning(
'`stackscope` not installed for use in debug mode!'
)
log.runtime(f"Runtime vars are: {rvs}")
rvs['_is_root'] = False
_state._runtime_vars.update(rvs)
for attr, value in parent_data.items():
2021-06-11 20:21:26 +00:00
if attr == '_arb_addr':
2021-09-08 00:24:02 +00:00
# XXX: ``msgspec`` doesn't support serializing tuples
2021-06-11 20:21:26 +00:00
# so just cash manually here since it's what our
# internals expect.
2021-09-08 00:24:02 +00:00
value = tuple(value) if value else None
self._arb_addr = value
2021-06-11 20:21:26 +00:00
else:
setattr(self, attr, value)
return chan, accept_addr
except OSError: # failed to connect
log.warning(
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
f'Failed to connect to parent!?\n\n'
'Closing IPC [TCP] transport server to\n'
f'{parent_addr}\n'
f'|_{self}\n\n'
)
await self.cancel(chan=None) # self cancel
raise
2018-07-14 20:09:05 +00:00
async def _serve_forever(
self,
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
handler_nursery: Nursery,
2018-07-14 20:09:05 +00:00
*,
# (host, port) to bind for channel server
accept_host: tuple[str, int] | None = None,
accept_port: int = 0,
2020-08-09 00:57:18 +00:00
task_status: TaskStatus[trio.Nursery] = trio.TASK_STATUS_IGNORED,
) -> None:
2022-08-03 20:09:16 +00:00
'''
Start the channel server, begin listening for new connections.
2018-07-14 20:09:05 +00:00
This will cause an actor to continue living (blocking) until
``cancel_server()`` is called.
2022-08-03 20:09:16 +00:00
'''
self._server_down = trio.Event()
2018-07-14 20:09:05 +00:00
try:
async with trio.open_nursery() as server_n:
2023-05-25 20:00:05 +00:00
listeners: list[trio.abc.Listener] = await server_n.start(
partial(
trio.serve_tcp,
self._stream_handler,
# new connections will stay alive even if this server
# is cancelled
handler_nursery=handler_nursery,
port=accept_port,
host=accept_host,
)
)
2023-05-25 20:00:05 +00:00
sockets: list[trio.socket] = [
getattr(listener, 'socket', 'unknown socket')
for listener in listeners
]
log.runtime(
'Started TCP server(s)\n'
f'|_{sockets}\n'
)
2023-05-25 20:00:05 +00:00
self._listeners.extend(listeners)
task_status.started(server_n)
finally:
# signal the server is down since nursery above terminated
self._server_down.set()
2018-07-14 20:09:05 +00:00
def cancel_soon(self) -> None:
2022-08-03 20:09:16 +00:00
'''
Cancel this actor asap; can be called from a sync context.
Schedules `.cancel()` to be run immediately just like when
cancelled by the parent.
2022-08-03 20:09:16 +00:00
'''
2021-04-27 16:07:16 +00:00
assert self._service_n
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
self._service_n.start_soon(
self.cancel,
None, # self cancel all rpc tasks
)
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
async def cancel(
self,
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
# chan whose lifetime limits the lifetime of its remotely
# requested and locally spawned RPC tasks - similar to the
# supervision semantics of a nursery wherein the actual
# implementation does start all such tasks in
# a sub-nursery.
req_chan: Channel|None,
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
) -> bool:
2022-08-03 20:09:16 +00:00
'''
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
Cancel this actor's runtime, eventually resulting in
the exit its containing process.
2018-08-01 19:15:18 +00:00
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
The ideal "deterministic" teardown sequence in order is:
- cancel all ongoing rpc tasks by cancel scope
- cancel the channel server to prevent new inbound
connections
- cancel the "service" nursery reponsible for
spawning new rpc tasks
- return control the parent channel message loop
2022-08-03 20:09:16 +00:00
'''
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
(
requesting_uid,
requester_type,
req_chan,
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
log_meth,
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
) = (
req_chan.uid,
'peer',
req_chan,
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
log.cancel,
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
) if req_chan else (
# a self cancel of ALL rpc tasks
self.uid,
'self',
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
self,
log.runtime,
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
)
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# TODO: just use the new `Context.repr_rpc: str` (and
# other) repr fields instead of doing this all manual..
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
msg: str = (
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
f'Runtime cancel request from {requester_type}:\n\n'
f'<= .cancel(): {requesting_uid}\n'
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
)
# TODO: what happens here when we self-cancel tho?
self._cancel_called_by_remote: tuple = requesting_uid
self._cancel_called = True
2018-08-01 19:15:18 +00:00
# cancel all ongoing rpc tasks
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
with CancelScope(shield=True):
# kill any debugger request task to avoid deadlock
# with the root actor in this tree
dbcs = _debug.Lock._debugger_request_cs
if dbcs is not None:
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
msg += (
'>> Cancelling active debugger request..\n'
f'|_{_debug.Lock}\n'
)
dbcs.cancel()
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
# self-cancel **all** ongoing RPC tasks
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
await self.cancel_rpc_tasks(
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
req_uid=requesting_uid,
parent_chan=None,
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
)
2020-08-09 00:57:18 +00:00
# stop channel server
self.cancel_server()
if self._server_down is not None:
await self._server_down.wait()
else:
log.warning(
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
'Transport[TCP] server was cancelled start?'
)
2020-08-09 00:57:18 +00:00
# cancel all rpc tasks permanently
if self._service_n:
self._service_n.cancel_scope.cancel()
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
log_meth(msg)
self._cancel_complete.set()
return True
# XXX: hard kill logic if needed?
# def _hard_mofo_kill(self):
# # If we're the root actor or zombied kill everything
# if self._parent_chan is None: # TODO: more robust check
# root = trio.lowlevel.current_root_task()
# for n in root.child_nurseries:
# n.cancel_scope.cancel()
2018-07-14 20:09:05 +00:00
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
async def _cancel_task(
self,
cid: str,
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
parent_chan: Channel,
requesting_uid: tuple[str, str]|None,
ipc_msg: dict|None|bool = False,
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
) -> bool:
'''
Cancel a local task by call-id / channel.
Note this method will be treated as a streaming function
by remote actor-callers due to the declaration of ``ctx``
in the signature (for now).
'''
# this ctx based lookup ensures the requested task to be
# cancelled was indeed spawned by a request from its
# parent (or some grandparent's) channel
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
ctx: Context
func: Callable
is_complete: trio.Event
try:
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
(
ctx,
func,
is_complete,
) = self._rpc_tasks[(
parent_chan,
cid,
)]
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
scope: CancelScope = ctx._scope
except KeyError:
# NOTE: during msging race conditions this will often
# emit, some examples:
# - callee returns a result before cancel-msg/ctxc-raised
# - callee self raises ctxc before caller send request,
# - callee errors prior to cancel req.
log.cancel(
'Cancel request invalid, RPC task already completed?\n'
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
f'<= canceller: {requesting_uid}\n\n'
f'=>{parent_chan}\n'
f' |_ctx-id: {cid}\n'
)
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
return True
log.cancel(
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
'Cancel request for RPC task\n\n'
f'<= Actor.cancel_task(): {requesting_uid}\n\n'
f'=> {ctx._task}\n'
f' |_ >> {ctx.repr_rpc}\n'
# f' >> Actor._cancel_task() => {ctx._task}\n'
# f' |_ {ctx._task}\n\n'
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
# TODO: better ascii repr for "supervisor" like
# a nursery or context scope?
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# f'=> {parent_chan}\n'
# f' |_{ctx._task}\n'
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
# TODO: simplified `Context.__repr__()` fields output
# shows only application state-related stuff like,
# - ._stream
# - .closed
# - .started_called
# - .. etc.
# f' >> {ctx.repr_rpc}\n'
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# f' |_ctx: {cid}\n'
# f' >> {ctx._nsf}()\n'
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
)
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
if (
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
ctx._canceller is None
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
and requesting_uid
):
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
ctx._canceller: tuple = requesting_uid
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
# TODO: pack the RPC `{'cmd': <blah>}` msg into a ctxc and
# then raise and pack it here?
if (
ipc_msg
and ctx._cancel_msg is None
):
# assign RPC msg directly from the loop which usually
# the case with `ctx.cancel()` on the other side.
ctx._cancel_msg = ipc_msg
# don't allow cancelling this function mid-execution
# (is this necessary?)
if func is self._cancel_task:
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
log.error('Do not cancel a cancel!?')
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
return True
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
# TODO: shouldn't we eventually be calling ``Context.cancel()``
# directly here instead (since that method can handle both
# side's calls into it?
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
# await ctx.cancel()
scope.cancel()
# wait for _invoke to mark the task complete
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
flow_info: str = (
f'<= canceller: {requesting_uid}\n'
f'=> ipc-parent: {parent_chan}\n'
f' |_{ctx}\n'
)
log.runtime(
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
'Waiting on RPC task to cancel\n'
f'{flow_info}'
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
)
await is_complete.wait()
log.runtime(
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
f'Sucessfully cancelled RPC task\n'
f'{flow_info}'
)
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
return True
async def cancel_rpc_tasks(
self,
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
req_uid: tuple[str, str],
# NOTE: when None is passed we cancel **all** rpc
# tasks running in this actor!
parent_chan: Channel|None,
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
) -> None:
2022-08-03 20:09:16 +00:00
'''
Cancel all existing RPC responder tasks using the cancel scope
2018-08-01 19:15:18 +00:00
registered for each.
2022-08-03 20:09:16 +00:00
'''
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
tasks: dict = self._rpc_tasks
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
if not tasks:
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
log.runtime(
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
'Actor has no cancellable RPC tasks?\n'
2024-02-22 20:06:39 +00:00
f'<= canceller: {req_uid}\n'
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
)
return
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
# TODO: seriously factor this into some helper funcs XD
tasks_str: str = ''
for (ctx, func, _) in tasks.values():
# TODO: std repr of all primitives in
# a hierarchical tree format, since we can!!
# like => repr for funcs/addrs/msg-typing:
#
# -[ ] use a proper utf8 "arm" like
# `stackscope` has!
# -[ ] for typed msging, show the
# py-type-annot style?
# - maybe auto-gen via `inspect` / `typing` type-sig:
# https://stackoverflow.com/a/57110117
# => see ex. code pasted into `.msg.types`
#
# -[ ] proper .maddr() for IPC primitives?
# - `Channel.maddr() -> str:` obvi!
# - `Context.maddr() -> str:`
tasks_str += (
f' |_@ /ipv4/tcp/cid="{ctx.cid[-16:]} .."\n'
f' |>> {ctx._nsf}() -> dict:\n'
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
)
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
descr: str = (
'all' if not parent_chan
else
"IPC channel's "
)
rent_chan_repr: str = (
f'|_{parent_chan}'
if parent_chan
else ''
)
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
log.cancel(
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
f'Cancelling {descr} {len(tasks)} rpc tasks\n\n'
f'<= `Actor.cancel_rpc_tasks()`: {req_uid}\n'
f' {rent_chan_repr}\n'
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# f'{self}\n'
# f'{tasks_str}'
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
)
for (
(task_caller_chan, cid),
(ctx, func, is_complete),
) in tasks.copy().items():
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
if (
# maybe filter to specific IPC channel?
(parent_chan
and
task_caller_chan != parent_chan)
# never "cancel-a-cancel" XD
or (func == self._cancel_task)
):
continue
# TODO: this maybe block on the task cancellation
# and so should really done in a nursery batch?
await self._cancel_task(
cid,
task_caller_chan,
requesting_uid=req_uid,
Be mega-pedantic with `ContextCancelled` semantics As part of extremely detailed inter-peer-actor testing, add much more granular `Context` cancellation state tracking via the following (new) fields: - `.canceller: tuple[str, str]` the uuid of the actor responsible for the cancellation condition - always set by `Context._maybe_cancel_and_set_remote_error()` and replaces `._cancelled_remote` and `.cancel_called_remote`. If set, this value should normally always match a value from some `ContextCancelled` raised or caught by one side of the context. - `._local_error` which is always set to the locally raised (and caller or callee task's scope-internal) error which caused any eventual cancellation/error condition and thus any closure of the context's per-task-side-`trio.Nursery`. - `.cancelled_caught: bool` is now always `True` whenever the local task catches (or "silently absorbs") a `ContextCancelled` (a `ctxc`) that indeed originated from one of the context's linked tasks or any other context which raised its own `ctxc` in the current `.open_context()` scope. => whenever there is a case that no `ContextCancelled` was raised **in** the `.open_context().__aexit__()` (eg. `ctx.result()` called after a call `ctx.cancel()`), we still consider the context's as having "caught a cancellation" since the `ctxc` was indeed silently handled by the cancel requester; all other error cases are already represented by mirroring the state of the `._scope: trio.CancelScope` => IOW there should be **no case** where an error is **not raised** in the context's scope and `.cancelled_caught: bool == False`, i.e. no case where `._scope.cancelled_caught == False and ._local_error is not None`! - always raise any `ctxc` from `.open_stream()` if `._cancel_called == True` - if the cancellation request has not already resulted in a `._remote_error: ContextCancelled` we raise a `RuntimeError` to indicate improper usage to the guilty side's task code. - make `._maybe_raise_remote_err()` a sync func and don't raise any `ctxc` which is matched against a `.canceller` determined to be the current actor, aka a "self cancel", and always set the `._local_error` to any such `ctxc`. - `.side: str` taken from inside `.cancel()` and unused as of now since it might be better re-written as a similar `.is_opener() -> bool`? - drop unused `._started_received: bool`.. - TONS and TONS of detailed comments/docs to attempt to explain all the possible cancellation/exit cases and how they should exhibit as either silent closes or raises from the `Context` API! Adjust the `._runtime._invoke()` code to match: - use `ctx._maybe_raise_remote_err()` in `._invoke()`. - adjust to new `.canceller` property. - more type hints. - better `log.cancel()` msging around self-cancels vs. peer-cancels. - always set the `._local_error: BaseException` for the "callee" task just like `Portal.open_context()` now will do B) Prior we were raising any `Context._remote_error` directly and doing (more or less) the same `ContextCancelled` "absorbing" logic (well kinda) in block; instead delegate to the method
2023-10-23 18:35:36 +00:00
)
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
if tasks:
log.cancel(
'Waiting for remaining rpc tasks to complete\n'
f'|_{tasks}'
)
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
await self._ongoing_rpc_tasks.wait()
2018-08-01 19:15:18 +00:00
def cancel_server(self) -> None:
2022-08-03 20:09:16 +00:00
'''
Cancel the internal channel server nursery thereby
2018-07-14 20:09:05 +00:00
preventing any new inbound connections from being established.
2022-08-03 20:09:16 +00:00
'''
if self._server_n:
log.runtime("Shutting down channel server")
self._server_n.cancel_scope.cancel()
2018-07-14 20:09:05 +00:00
@property
2023-05-25 20:00:05 +00:00
def accept_addr(self) -> tuple[str, int] | None:
2022-08-03 20:09:16 +00:00
'''
Primary address to which the channel server is bound.
'''
2019-12-10 05:55:03 +00:00
# throws OSError on failure
return self._listeners[0].socket.getsockname() # type: ignore
2018-07-14 20:09:05 +00:00
def get_parent(self) -> Portal:
2022-08-03 20:09:16 +00:00
'''
Return a portal to our parent actor.
'''
2018-08-31 21:16:24 +00:00
assert self._parent_chan, "No parent channel for this actor?"
2018-07-14 20:09:05 +00:00
return Portal(self._parent_chan)
def get_chans(self, uid: tuple[str, str]) -> list[Channel]:
2022-08-03 20:09:16 +00:00
'''
Return all channels to the actor with provided uid.
'''
return self._peers[uid]
2018-07-14 20:09:05 +00:00
async def _do_handshake(
self,
chan: Channel
2021-07-01 18:52:52 +00:00
) -> tuple[str, str]:
2022-08-03 20:09:16 +00:00
'''
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
Exchange `(name, UUIDs)` identifiers as the first
communication step.
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
These are essentially the "mailbox addresses" found in
actor model parlance.
2022-08-03 20:09:16 +00:00
'''
await chan.send(self.uid)
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
value: tuple = await chan.recv()
uid: tuple[str, str] = (str(value[0]), str(value[1]))
2021-07-01 18:52:52 +00:00
if not isinstance(uid, tuple):
raise ValueError(f"{uid} is not a valid uid?!")
2021-09-08 00:24:02 +00:00
chan.uid = str(uid[0]), str(uid[1])
return uid
def is_infected_aio(self) -> bool:
return self._infected_aio
2022-08-03 19:29:34 +00:00
async def async_main(
actor: Actor,
2023-05-25 20:00:05 +00:00
accept_addr: tuple[str, int] | None = None,
2022-08-03 19:29:34 +00:00
# XXX: currently ``parent_addr`` is only needed for the
# ``multiprocessing`` backend (which pickles state sent to
# the child instead of relaying it over the connect-back
# channel). Once that backend is removed we can likely just
# change this to a simple ``is_subactor: bool`` which will
# be False when running as root actor and True when as
# a subactor.
2023-05-25 20:00:05 +00:00
parent_addr: tuple[str, int] | None = None,
2022-08-03 19:29:34 +00:00
task_status: TaskStatus[None] = trio.TASK_STATUS_IGNORED,
) -> None:
'''
Actor runtime entrypoint; start the IPC channel server, maybe connect
back to the parent, and startup all core machinery tasks.
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
A "root" (or "top-level") nursery for this actor is opened here and
when cancelled/terminated effectively closes the actor's "runtime".
2022-08-03 19:29:34 +00:00
'''
# attempt to retreive ``trio``'s sigint handler and stash it
# on our debugger lock state.
_debug.Lock._trio_handler = signal.getsignal(signal.SIGINT)
2022-08-03 19:29:34 +00:00
registered_with_arbiter = False
try:
# establish primary connection with immediate parent
actor._parent_chan = None
if parent_addr is not None:
actor._parent_chan, accept_addr_rent = await actor._from_parent(
parent_addr)
# either it's passed in because we're not a child
# or because we're running in mp mode
if accept_addr_rent is not None:
accept_addr = accept_addr_rent
# load exposed/allowed RPC modules
# XXX: do this **after** establishing a channel to the parent
# but **before** starting the message loop for that channel
# such that import errors are properly propagated upwards
actor.load_modules()
# The "root" nursery ensures the channel with the immediate
# parent is kept alive as a resilient service until
# cancellation steps have (mostly) occurred in
# a deterministic way.
async with trio.open_nursery() as root_nursery:
actor._root_n = root_nursery
assert actor._root_n
async with trio.open_nursery() as service_nursery:
# This nursery is used to handle all inbound
# connections to us such that if the TCP server
# is killed, connections can continue to process
# in the background until this nursery is cancelled.
actor._service_n = service_nursery
assert actor._service_n
# Startup up the channel server with,
# - subactor: the bind address is sent by our parent
# over our established channel
# - root actor: the ``accept_addr`` passed to this method
assert accept_addr
host, port = accept_addr
actor._server_n = await service_nursery.start(
partial(
actor._serve_forever,
service_nursery,
accept_host=host,
accept_port=port
)
)
accept_addr = actor.accept_addr
if _state._runtime_vars['_is_root']:
_state._runtime_vars['_root_mailbox'] = accept_addr
# Register with the arbiter if we're told its addr
log.runtime(f"Registering {actor} for role `{actor.name}`")
assert isinstance(actor._arb_addr, tuple)
async with get_arbiter(*actor._arb_addr) as arb_portal:
await arb_portal.run_from_ns(
'self',
'register_actor',
uid=actor.uid,
sockaddr=accept_addr,
)
registered_with_arbiter = True
# init steps complete
task_status.started()
# Begin handling our new connection back to our
# parent. This is done last since we don't want to
# start processing parent requests until our channel
# server is 100% up and running.
if actor._parent_chan:
await root_nursery.start(
partial(
process_messages,
actor,
actor._parent_chan,
shield=True,
)
)
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
log.runtime(
'Actor runtime is up!'
# 'Blocking on service nursery to exit..\n'
)
log.runtime(
"Service nursery complete\n"
"Waiting on root nursery to complete"
)
2022-08-03 19:29:34 +00:00
# Blocks here as expected until the root nursery is
# killed (i.e. this actor is cancelled or signalled by the parent)
except Exception as err:
log.runtime("Closing all actor lifetime contexts")
actor.lifetime_stack.close()
2022-08-03 19:29:34 +00:00
if not registered_with_arbiter:
# TODO: I guess we could try to connect back
# to the parent through a channel and engage a debugger
# once we have that all working with std streams locking?
log.exception(
f"Actor errored and failed to register with arbiter "
f"@ {actor._arb_addr}?")
log.error(
"\n\n\t^^^ THIS IS PROBABLY A TRACTOR BUGGGGG!!! ^^^\n"
"\tCALMLY CALL THE AUTHORITIES AND HIDE YOUR CHILDREN.\n\n"
"\tYOUR PARENT CODE IS GOING TO KEEP WORKING FINE!!!\n"
"\tTHIS IS HOW RELIABlE SYSTEMS ARE SUPPOSED TO WORK!?!?\n"
)
if actor._parent_chan:
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
await try_ship_error_to_parent(
actor._parent_chan,
err,
)
2022-08-03 19:29:34 +00:00
# always!
match err:
case ContextCancelled():
log.cancel(
f'Actor: {actor.uid} was task-context-cancelled with,\n'
f'str(err)'
)
case _:
log.exception("Actor errored:")
2022-08-03 19:29:34 +00:00
raise
finally:
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
log.runtime(
'Runtime nursery complete'
'-> Closing all actor lifetime contexts..'
)
2022-08-03 19:29:34 +00:00
# tear down all lifetime contexts if not in guest mode
# XXX: should this just be in the entrypoint?
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
actor.lifetime_stack.close()
2022-08-03 19:29:34 +00:00
# TODO: we can't actually do this bc the debugger
# uses the _service_n to spawn the lock task, BUT,
# in theory if we had the root nursery surround this finally
# block it might be actually possible to debug THIS
# machinery in the same way as user task code?
# if actor.name == 'brokerd.ib':
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
# with CancelScope(shield=True):
2022-08-03 19:29:34 +00:00
# await _debug.breakpoint()
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# Unregister actor from the registry-sys / registrar.
if (
registered_with_arbiter
and not actor.is_arbiter
2022-08-03 19:29:34 +00:00
):
failed = False
assert isinstance(actor._arb_addr, tuple)
2022-08-03 19:29:34 +00:00
with trio.move_on_after(0.5) as cs:
cs.shield = True
try:
async with get_arbiter(*actor._arb_addr) as arb_portal:
await arb_portal.run_from_ns(
'self',
'unregister_actor',
uid=actor.uid
)
except OSError:
failed = True
if cs.cancelled_caught:
failed = True
if failed:
log.warning(
f"Failed to unregister {actor.name} from arbiter")
# Ensure all peers (actors connected to us as clients) are finished
if not actor._no_more_peers.is_set():
if any(
chan.connected() for chan in chain(*actor._peers.values())
):
log.runtime(
f"Waiting for remaining peers {actor._peers} to clear")
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
with CancelScope(shield=True):
2022-08-03 19:29:34 +00:00
await actor._no_more_peers.wait()
log.runtime("All peer channels are complete")
log.runtime("Runtime completed")
2022-08-03 19:14:36 +00:00
async def process_messages(
actor: Actor,
chan: Channel,
shield: bool = False,
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
task_status: TaskStatus[CancelScope] = trio.TASK_STATUS_IGNORED,
2022-08-03 19:14:36 +00:00
) -> bool:
'''
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
This is the per-channel, low level RPC task scheduler loop.
2022-08-03 19:14:36 +00:00
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
Receive multiplexed RPC request messages from some remote process,
spawn handler tasks depending on request type and deliver responses
or boxed errors back to the remote caller (task).
2022-08-03 19:14:36 +00:00
'''
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
# TODO: once `trio` get's an "obvious way" for req/resp we
# should use it?
# https://github.com/python-trio/trio/issues/467
log.runtime(
'Entering IPC msg loop:\n'
f'peer: {chan.uid}\n'
f'|_{chan}\n'
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
)
2022-08-03 19:14:36 +00:00
nursery_cancelled_before_task: bool = False
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
msg: dict | None = None
2022-08-03 19:14:36 +00:00
try:
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
# NOTE: this internal scope allows for keeping this
# message loop running despite the current task having
# been cancelled (eg. `open_portal()` may call this method
# from a locally spawned task) and recieve this scope
# using ``scope = Nursery.start()``
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
with CancelScope(shield=shield) as loop_cs:
2022-08-03 19:14:36 +00:00
task_status.started(loop_cs)
async for msg in chan:
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
# dedicated loop terminate sentinel
if msg is None:
2022-08-03 19:14:36 +00:00
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
tasks: dict[
tuple[Channel, str],
tuple[Context, Callable, trio.Event]
] = actor._rpc_tasks.copy()
2022-08-03 19:14:36 +00:00
log.cancel(
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
f'Peer IPC channel terminated via `None` setinel msg?\n'
f'=> Cancelling all {len(tasks)} local RPC tasks..\n'
f'peer: {chan.uid}\n'
f'|_{chan}\n'
)
for (channel, cid) in tasks:
2022-08-03 19:14:36 +00:00
if channel is chan:
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
await actor._cancel_task(
cid,
channel,
requesting_uid=channel.uid,
ipc_msg=msg,
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
)
2022-08-03 19:14:36 +00:00
break
log.transport( # type: ignore
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
f'<= IPC msg from peer: {chan.uid}\n\n'
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
# TODO: conditionally avoid fmting depending
# on log level (for perf)?
# => specifically `pformat()` sub-call..?
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
f'{pformat(msg)}\n'
)
2022-08-03 19:14:36 +00:00
cid = msg.get('cid')
if cid:
# deliver response to local caller/waiter
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
# via its per-remote-context memory channel.
await actor._push_result(
chan,
cid,
msg,
)
2022-08-03 19:14:36 +00:00
log.runtime(
'Waiting on next IPC msg from\n'
f'peer: {chan.uid}:\n'
f'|_{chan}\n'
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
# f'last msg: {msg}\n'
)
2022-08-03 19:14:36 +00:00
continue
# process a 'cmd' request-msg upack
# TODO: impl with native `msgspec.Struct` support !!
# -[ ] implement with ``match:`` syntax?
# -[ ] discard un-authed msgs as per,
# <TODO put issue for typed msging structs>
2022-08-03 19:14:36 +00:00
try:
(
ns,
funcname,
kwargs,
actorid,
cid,
) = msg['cmd']
2022-08-03 19:14:36 +00:00
except KeyError:
# This is the non-rpc error case, that is, an
# error **not** raised inside a call to ``_invoke()``
# (i.e. no cid was provided in the msg - see above).
# Push this error to all local channel consumers
# (normally portals) by marking the channel as errored
assert chan.uid
exc = unpack_error(msg, chan=chan)
chan._exc = exc
raise exc
log.runtime(
'Handling RPC cmd from\n'
f'peer: {actorid}\n'
'\n'
f'=> {ns}.{funcname}({kwargs})\n'
)
2022-08-03 19:14:36 +00:00
if ns == 'self':
if funcname == 'cancel':
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
func: Callable = actor.cancel
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
kwargs |= {
'req_chan': chan,
}
2022-08-03 19:14:36 +00:00
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
# don't start entire actor runtime cancellation
# if this actor is currently in debug mode!
pdb_complete: trio.Event|None = _debug.Lock.local_pdb_complete
2022-08-03 19:14:36 +00:00
if pdb_complete:
await pdb_complete.wait()
# Either of `Actor.cancel()`/`.cancel_soon()`
# was called, so terminate this IPC msg
# loop, exit back out into `async_main()`,
# and immediately start the core runtime
# machinery shutdown!
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
with CancelScope(shield=True):
2022-08-03 19:14:36 +00:00
await _invoke(
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
actor,
cid,
chan,
func,
kwargs,
is_rpc=False,
2022-08-03 19:14:36 +00:00
)
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
log.runtime(
'Cancelling IPC transport msg-loop with peer:\n'
f'|_{chan}\n'
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
)
2022-08-03 19:14:36 +00:00
loop_cs.cancel()
break
if funcname == '_cancel_task':
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
func: Callable = actor._cancel_task
2022-08-03 19:14:36 +00:00
# we immediately start the runtime machinery
# shutdown
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
# with CancelScope(shield=True):
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
target_cid: str = kwargs['cid']
kwargs |= {
# NOTE: ONLY the rpc-task-owning
# parent IPC channel should be able to
# cancel it!
'parent_chan': chan,
'requesting_uid': chan.uid,
'ipc_msg': msg,
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
}
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
# TODO: remove? already have emit in meth.
# log.runtime(
# f'Rx RPC task cancel request\n'
# f'<= canceller: {chan.uid}\n'
# f' |_{chan}\n\n'
# f'=> {actor}\n'
# f' |_cid: {target_cid}\n'
# )
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
try:
await _invoke(
actor,
cid,
chan,
func,
kwargs,
is_rpc=False,
2022-08-03 19:14:36 +00:00
)
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
except BaseException:
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
log.exception(
'Failed to cancel task?\n'
f'<= canceller: {chan.uid}\n'
f' |_{chan}\n\n'
f'=> {actor}\n'
f' |_cid: {target_cid}\n'
)
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
continue
else:
# normally registry methods, eg.
# ``.register_actor()`` etc.
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
func: Callable = getattr(actor, funcname)
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
2022-08-03 19:14:36 +00:00
else:
# complain to client about restricted modules
try:
func = actor._get_rpc_func(ns, funcname)
except (ModuleNotExposed, AttributeError) as err:
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
err_msg: dict[str, dict] = pack_error(
err,
cid=cid,
)
2022-08-03 19:14:36 +00:00
await chan.send(err_msg)
continue
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
# schedule a task for the requested RPC function
# in the actor's main "service nursery".
# TODO: possibly a service-tn per IPC channel for
# supervision isolation? would avoid having to
# manage RPC tasks individually in `._rpc_tasks`
# table?
log.runtime(
f'Spawning task for RPC request\n'
f'<= caller: {chan.uid}\n'
f' |_{chan}\n\n'
# TODO: maddr style repr?
# f' |_@ /ipv4/{chan.raddr}/tcp/{chan.rport}/'
# f'cid="{cid[-16:]} .."\n\n'
f'=> {actor}\n'
f' |_cid: {cid}\n'
f' |>> {func}()\n'
)
assert actor._service_n # wait why? do it at top?
2022-08-03 19:14:36 +00:00
try:
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
ctx: Context = await actor._service_n.start(
partial(
_invoke,
actor,
cid,
chan,
func,
kwargs,
),
2022-08-03 19:14:36 +00:00
name=funcname,
)
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
except (
RuntimeError,
BaseExceptionGroup,
):
2022-08-03 19:14:36 +00:00
# avoid reporting a benign race condition
# during actor runtime teardown.
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
nursery_cancelled_before_task: bool = True
2022-08-03 19:14:36 +00:00
break
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
# in the lone case where a ``Context`` is not
# delivered, it's likely going to be a locally
# scoped exception from ``_invoke()`` itself.
if isinstance(err := ctx, Exception):
2022-08-03 19:14:36 +00:00
log.warning(
'Task for RPC failed?'
f'|_ {func}()\n\n'
f'{err}'
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
)
continue
2022-08-03 19:14:36 +00:00
else:
# mark that we have ongoing rpc tasks
actor._ongoing_rpc_tasks = trio.Event()
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
2022-08-03 19:14:36 +00:00
# store cancel scope such that the rpc task can be
# cancelled gracefully if requested
actor._rpc_tasks[(chan, cid)] = (
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
ctx,
func,
trio.Event(),
)
2022-08-03 19:14:36 +00:00
log.runtime(
'Waiting on next IPC msg from\n'
f'peer: {chan.uid}\n'
f'|_{chan}\n'
)
2022-08-03 19:14:36 +00:00
# end of async for, channel disconnect vis
# ``trio.EndOfChannel``
log.runtime(
f"{chan} for {chan.uid} disconnected, cancelling tasks"
)
Tweak `Actor` cancel method signatures Besides improving a bunch more log msg contents similarly as before this changes the cancel method signatures slightly with different arg names: for `.cancel()`: - instead of `requesting_uid: str` take in a `req_chan: Channel` since we can always just read its `.uid: tuple` for logging and further we can then offer the `chan=None` case indicating a "self cancel" (since there's no "requesting channel"). - the semantics of "requesting" here better indicate that the IPC connection is an IPC peer and further (eventually) will allow permission checking against given peers for cancellation requests. - when `chan==None` we also define a meth-internal `requester_type: str` differently for logging content :) - add much more detailed `.cancel()` content around the requester, its type, and any debugger related locking steps. for `._cancel_task()`: - change the `chan` arg to `parent_chan: Channel` since "parent" correctly indicates that the channel is the parent of the locally spawned rpc task to cancel; in fact no other chan should be able to cancel tasks parented/spawned by other channels obvi! - also add more extensive meth-internal `.cancel()` logging with a #TODO around showing only the "relevant/lasest" `Context` state vars in such logging content. for `.cancel_rpc_tasks()`: - shorten `requesting_uid` -> `req_uid`. - add `parent_chan: Channel` to be similar as above in `._cancel_task()` (since it's internally delegated to anyway) which replaces the prior `only_chan` and use it to filter to only tasks spawned by this channel (thus as their "parent") as before. - instead of `if tasks:` to enter, invert and `return` early on `if not tasks`, for less indentation B) - add WIP str-repr format (for `.cancel()` emissions) to show a multi-address (maddr) + task func (via the new `Context._nsf`) and report all cancel task targets with it a "tree"; include #TODO to finalize and implement some utils for all this! To match ensure we adjust `process_messages()` self/`Actor` cancel handling blocks to provide the new `kwargs` (now with `dict`-merge syntax) to `._invoke()`.
2024-02-22 18:42:48 +00:00
await actor.cancel_rpc_tasks(
req_uid=actor.uid,
# a "self cancel" in terms of the lifetime of the
# IPC connection which is presumed to be the
# source of any requests for spawned tasks.
parent_chan=chan,
)
2022-08-03 19:14:36 +00:00
except (
TransportClosed,
):
# channels "breaking" (for TCP streams by EOF or 104
# connection-reset) is ok since we don't have a teardown
# handshake for them (yet) and instead we simply bail out of
# the message loop and expect the teardown sequence to clean
# up.
# TODO: don't show this msg if it's an emphemeral
# discovery ep call?
2023-01-26 00:13:34 +00:00
log.runtime(
f'channel closed abruptly with\n'
f'peer: {chan.uid}\n'
f'|_{chan.raddr}\n'
2023-01-26 00:13:34 +00:00
)
2022-08-03 19:14:36 +00:00
# transport **was** disconnected
return True
except (
Exception,
BaseExceptionGroup,
) as err:
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
2022-08-03 19:14:36 +00:00
if nursery_cancelled_before_task:
Guarding for IPC failures in `._runtime._invoke()` Took me longer then i wanted to figure out the source of a failed-response to a remote-cancellation (in this case in `modden` where a client was cancelling a workspace layer.. but disconnects before receiving the ack msg) that was triggering an IPC error when sending the error msg for the cancellation of a `Actor._cancel_task()`, but since this (non-rpc) `._invoke()` task was trying to send to a now disconnected canceller it was resulting in a `BrokenPipeError` (or similar) error. Now, we except for such IPC errors and only raise them when, 1. the transport `Channel` is for sure up (bc ow what's the point of trying to send an error on the thing that caused it..) 2. it's definitely for handling an RPC task Similarly if the entire main invoke `try:` excepts, - we only hide the call-stack frame from the debugger (with `__tracebackhide__: bool`) if it's an RPC task that has a connected channel since we always want to see the frame when debugging internal task or IPC failures. - we don't bother trying to send errors to the context caller (actor) when it's a non-RPC request since failures on actor-runtime-internal tasks shouldn't really ever be reported remotely, only maybe raised locally. Also some other tidying, - this properly corrects for the self-cancel case where an RPC context is cancelled due to a local (runtime) task calling a method like `Actor.cancel_soon()`. We now set our own `.uid` as the `ContextCancelled.canceller` value so that other-end tasks know that the cancellation was due to a self-cancellation by the actor itself. We still need to properly test for this though! - add a more detailed module doc-str. - more explicit imports for `trio` core types throughout.
2024-01-02 14:08:39 +00:00
sn: Nursery = actor._service_n
assert sn and sn.cancel_scope.cancel_called # sanity
2022-08-03 19:14:36 +00:00
log.cancel(
f'Service nursery cancelled before it handled {funcname}'
)
else:
# ship any "internal" exception (i.e. one from internal
# machinery not from an rpc task) to parent
match err:
case ContextCancelled():
log.cancel(
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
f'Actor: {actor.uid} was context-cancelled with,\n'
f'str(err)'
)
case _:
log.exception("Actor errored:")
2022-08-03 19:14:36 +00:00
if actor._parent_chan:
Improved log msg formatting in core As part of solving some final edge cases todo with inter-peer remote cancellation (particularly a remote cancel from a separate actor tree-client hanging on the request side in `modden`..) I needed less dense, more line-delimited log msg formats when understanding ipc channel and context cancels from console logging; this adds a ton of that to: - `._invoke()` which now does, - better formatting of `Context`-task info as multi-line `'<field>: <value>\n'` messages, - use of `trio.Task` (from `.lowlevel.current_task()` for full rpc-func namespace-path info, - better "msg flow annotations" with `<=` for understanding `ContextCancelled` flow. - `Actor._stream_handler()` where in we break down IPC peers reporting better as multi-line `|_<Channel>` log msgs instead of all jammed on one line.. - `._ipc.Channel.send()` use `pformat()` for repr of packet. Also tweak some optional deps imports for debug mode: - add `maybe_import_gb()` for attempting to import `greenback`. - maybe enable `stackscope` tree pprinter on `SIGUSR1` if installed. Add a further stale-debugger-lock guard before removal: - read the `._debug.Lock.global_actor_in_debug: tuple` uid and possibly `maybe_wait_for_debugger()` when the child-user is known to have a live process in our tree. - only cancel `Lock._root_local_task_cs_in_debug: CancelScope` when the disconnected channel maps to the `Lock.global_actor_in_debug`, though not sure this is correct yet? Started adding missing type annots in sections that were modified.
2024-02-19 17:25:08 +00:00
await try_ship_error_to_parent(
actor._parent_chan,
err,
)
2022-08-03 19:14:36 +00:00
# if this is the `MainProcess` we expect the error broadcasting
# above to trigger an error at consuming portal "checkpoints"
raise
finally:
# msg debugging for when he machinery is brokey
log.runtime(
'Exiting IPC msg loop with\n'
f'peer: {chan.uid}\n'
f'|_{chan}\n\n'
'final msg:\n'
f'{pformat(msg)}\n'
Remote `Context` cancellation semantics rework B) This adds remote cancellation semantics to our `tractor.Context` machinery to more closely match that of `trio.CancelScope` but with operational differences to handle the nature of parallel tasks interoperating across multiple memory boundaries: - if an actor task cancels some context it has opened via `Context.cancel()`, the remote (scope linked) task will be cancelled using the normal `CancelScope` semantics of `trio` meaning the remote cancel scope surrounding the far side task is cancelled and `trio.Cancelled`s are expected to be raised in that scope as per normal `trio` operation, and in the case where no error is raised in that remote scope, a `ContextCancelled` error is raised inside the runtime machinery and relayed back to the opener/caller side of the context. - if any actor task cancels a full remote actor runtime using `Portal.cancel_actor()` the same semantics as above apply except every other remote actor task which also has an open context with the actor which was cancelled will also be sent a `ContextCancelled` **but** with the `.canceller` field set to the uid of the original cancel requesting actor. This changeset also includes a more "proper" solution to the issue of "allowing overruns" during streaming without attempting to implement any form of IPC streaming backpressure. Implementing task-granularity backpressure cross-process turns out to be more or less impossible without augmenting out streaming protocol (likely at the cost of performance). Further allowing overruns requires special care since any blocking of the runtime RPC msg loop task effectively can block control msgs such as cancels and stream terminations. The implementation details per abstraction layer are as follows. ._streaming.Context: - add a new contructor factor func `mk_context()` which provides a strictly private init-er whilst allowing us to not have to define an `.__init__()` on the type def. - add public `.cancel_called` and `.cancel_called_remote` properties. - general rename of what was the internal `._backpressure` var to `._allow_overruns: bool`. - move the old contents of `Actor._push_result()` into a new `._deliver_msg()` allowing for better encapsulation of per-ctx msg handling. - always check for received 'error' msgs and process them with the new `_maybe_cancel_and_set_remote_error()` **before** any msg delivery to the local task, thus guaranteeing error and cancellation handling despite any overflow handling. - add a new `._drain_overflows()` task-method for use with new `._allow_overruns: bool = True` mode. - add back a `._scope_nursery: trio.Nursery` (allocated in `Portal.open_context()`) who's sole purpose is to spawn a single task which runs the above method; anything else is an error. - augment `._deliver_msg()` to start a task and run the above method when operating in no overrun mode; the task queues overflow msgs and attempts to send them to the underlying mem chan using a blocking `.send()` call. - on context exit, any existing "drainer task" will be cancelled and remaining overflow queued msgs are discarded with a warning. - rename `._error` -> `_remote_error` and set it in a new method `_maybe_cancel_and_set_remote_error()` which is called before processing - adjust `.result()` to always call `._maybe_raise_remote_err()` at its start such that whenever a `ContextCancelled` arrives we do logic for whether or not to immediately raise that error or ignore it due to the current actor being the one who requested the cancel, by checking the error's `.canceller` field. - set the default value of `._result` to be `id(Context()` thus avoiding conflict with any `.result()` actually being `False`.. ._runtime.Actor: - augment `.cancel()` and `._cancel_task()` and `.cancel_rpc_tasks()` to take a `requesting_uid: tuple` indicating the source actor of every cancellation request. - pass through the new `Context._allow_overruns` through `.get_context()` - call the new `Context._deliver_msg()` from `._push_result()` (since the factoring out that method's contents). ._runtime._invoke: - `TastStatus.started()` back a `Context` (unless an error is raised) instead of the cancel scope to make it easy to set/get state on that context for the purposes of cancellation and remote error relay. - always raise any remote error via `Context._maybe_raise_remote_err()` before doing any `ContextCancelled` logic. - assign any `Context._cancel_called_remote` set by the `requesting_uid` cancel methods (mentioned above) to the `ContextCancelled.canceller`. ._runtime.process_messages: - always pass a `requesting_uid: tuple` to `Actor.cancel()` and `._cancel_task` to that any corresponding `ContextCancelled.canceller` can be set inside `._invoke()`.
2023-04-13 20:03:35 +00:00
)
2022-08-03 19:14:36 +00:00
# transport **was not** disconnected
return False
2018-07-14 20:09:05 +00:00
class Arbiter(Actor):
'''
A special actor who knows all the other actors and always has
access to a top level nursery.
2018-07-14 20:09:05 +00:00
The arbiter is by default the first actor spawned on each host
and is responsible for keeping track of all other actors for
coordination purposes. If a new main process is launched and an
arbiter is already running that arbiter will be used.
'''
2018-07-14 20:09:05 +00:00
is_arbiter = True
def __init__(self, *args, **kwargs) -> None:
2021-12-02 17:34:27 +00:00
self._registry: dict[
tuple[str, str],
tuple[str, int],
2021-09-08 00:24:02 +00:00
] = {}
self._waiters: dict[
str,
# either an event to sync to receiving an actor uid (which
# is filled in once the actor has sucessfully registered),
# or that uid after registry is complete.
list[trio.Event | tuple[str, str]]
] = {}
super().__init__(*args, **kwargs)
async def find_actor(
self,
name: str,
) -> tuple[str, int] | None:
for uid, sockaddr in self._registry.items():
if name in uid:
return sockaddr
2018-08-31 21:16:24 +00:00
return None
async def get_registry(
self
2022-02-15 13:52:04 +00:00
) -> dict[str, tuple[str, int]]:
'''
Return current name registry.
This method is async to allow for cross-actor invocation.
'''
# NOTE: requires ``strict_map_key=False`` to the msgpack
# unpacker since we have tuples as keys (not this makes the
# arbiter suscetible to hashdos):
# https://github.com/msgpack/msgpack-python#major-breaking-changes-in-msgpack-10
return {'.'.join(key): val for key, val in self._registry.items()}
2018-08-26 17:12:29 +00:00
async def wait_for_actor(
self,
name: str,
) -> list[tuple[str, int]]:
'''
Wait for a particular actor to register.
This is a blocking call if no actor by the provided name is currently
registered.
'''
sockaddrs: list[tuple[str, int]] = []
sockaddr: tuple[str, int]
for (aname, _), sockaddr in self._registry.items():
log.runtime(
Mega-refactor on `._invoke()` targeting `@context`s Since eventually we want to implement all other RPC "func types" as contexts underneath this starts the rework to move all the other cases into a separate func not only to simplify the main `._invoke()` body but also as a reminder of the intention to do it XD Details of re-factor: - add a new `._invoke_non_context()` which just moves all the old blocks for non-context handling to a single def. - factor what was basically just the `finally:` block handler (doing all the task bookkeeping) into a new `@acm`: `_errors_relayed_via_ipc()` with that content packed into the post-`yield` (also with a `hide_tb: bool` flag added of course). * include a `debug_kbis: bool` for when needed. - since the `@context` block is the only type left in the main `_invoke()` body, de-dent it so it's more grok-able B) Obviously this patch also includes a few improvements regarding context-cancellation-semantics (for the `context` RPC case) on the callee side in order to match previous changes to the `Context` api: - always setting any ctxc as the `Context._local_error`. - using the new convenience `.maybe_raise()` topically (for now). - avoiding any previous reliance on `Context.cancelled_caught` for anything public of meaning. Further included is more logging content updates: - being pedantic in `.cancel()` msgs about whether termination is caused by error or ctxc. - optional `._invoke()` traceback hiding via a `hide_tb: bool`. - simpler log headers throughout instead leveraging new `.__repr__()` on primitives. - buncha `<= <actor-uid>` sent some message emissions. - simplified handshake statuses reporting. Other subsys api changes we need to match: - change to `Channel.transport`. - avoiding any `local_nursery: ActorNursery` waiting when the `._implicit_runtime_started` is set. And yes, lotsa more comments for #TODOs dawg.. since there's always somethin!
2024-03-03 00:26:40 +00:00
f'Actor mailbox info:\n'
f'aname: {aname}\n'
f'sockaddr: {sockaddr}\n'
)
if name == aname:
sockaddrs.append(sockaddr)
if not sockaddrs:
waiter = trio.Event()
self._waiters.setdefault(name, []).append(waiter)
await waiter.wait()
for uid in self._waiters[name]:
if not isinstance(uid, trio.Event):
sockaddrs.append(self._registry[uid])
return sockaddrs
2018-07-14 20:09:05 +00:00
async def register_actor(
2021-07-01 18:52:52 +00:00
self,
uid: tuple[str, str],
sockaddr: tuple[str, int]
2021-07-01 18:52:52 +00:00
2018-08-26 17:12:29 +00:00
) -> None:
uid = name, _ = (str(uid[0]), str(uid[1]))
2021-09-08 00:24:02 +00:00
self._registry[uid] = (str(sockaddr[0]), int(sockaddr[1]))
# pop and signal all waiter events
events = self._waiters.pop(name, [])
self._waiters.setdefault(name, []).append(uid)
for event in events:
if isinstance(event, trio.Event):
event.set()
2018-07-14 20:09:05 +00:00
async def unregister_actor(
self,
uid: tuple[str, str]
) -> None:
2021-09-08 00:24:02 +00:00
uid = (str(uid[0]), str(uid[1]))
self._registry.pop(uid)