forked from goodboy/tractor
				
			
						commit
						ae6d751d71
					
				| 
						 | 
				
			
			@ -0,0 +1,16 @@
 | 
			
		|||
This (finally) adds fully acknowledged remote cancellation messaging
 | 
			
		||||
support for both explicit ``Portal.cancel_actor()`` calls as well as
 | 
			
		||||
when there is a "runtime-wide" cancellations (eg. during KBI or general
 | 
			
		||||
actor nursery exception handling which causes a full actor
 | 
			
		||||
"crash"/termination).
 | 
			
		||||
 | 
			
		||||
You can think of this as the most ideal case in 2-generals where the
 | 
			
		||||
actor requesting the cancel of its child is able to always receive back
 | 
			
		||||
the ACK to that request. This leads to a more deterministic shutdown of
 | 
			
		||||
the child where the parent is able to wait for the child to fully
 | 
			
		||||
respond to the request. On a localhost setup, where the parent can
 | 
			
		||||
monitor the state of the child through process or other OS APIs instead
 | 
			
		||||
of solely through IPC messaging, the parent can know whether or not the
 | 
			
		||||
child decided to cancel with more certainty. In the case of separate
 | 
			
		||||
hosts, we still rely on a simple timeout approach until such a time
 | 
			
		||||
where we prefer to get "fancier".
 | 
			
		||||
| 
						 | 
				
			
			@ -128,7 +128,11 @@ def test_multierror_fast_nursery(arb_addr, start_method, num_subactors, delay):
 | 
			
		|||
 | 
			
		||||
    if len(exceptions) == 2:
 | 
			
		||||
        # sometimes oddly now there's an embedded BrokenResourceError ?
 | 
			
		||||
        exceptions = exceptions[1].exceptions
 | 
			
		||||
        for exc in exceptions:
 | 
			
		||||
            excs = getattr(exc, 'exceptions', None)
 | 
			
		||||
            if excs:
 | 
			
		||||
                exceptions = excs
 | 
			
		||||
                break
 | 
			
		||||
 | 
			
		||||
    assert len(exceptions) == num_subactors
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -2,6 +2,7 @@
 | 
			
		|||
Actor primitives and helpers
 | 
			
		||||
 | 
			
		||||
"""
 | 
			
		||||
from __future__ import annotations
 | 
			
		||||
from collections import defaultdict
 | 
			
		||||
from functools import partial
 | 
			
		||||
from itertools import chain
 | 
			
		||||
| 
						 | 
				
			
			@ -10,7 +11,7 @@ import importlib.util
 | 
			
		|||
import inspect
 | 
			
		||||
import uuid
 | 
			
		||||
import typing
 | 
			
		||||
from typing import Dict, List, Tuple, Any, Optional, Union
 | 
			
		||||
from typing import List, Tuple, Any, Optional, Union
 | 
			
		||||
from types import ModuleType
 | 
			
		||||
import sys
 | 
			
		||||
import os
 | 
			
		||||
| 
						 | 
				
			
			@ -48,7 +49,7 @@ async def _invoke(
 | 
			
		|||
    cid: str,
 | 
			
		||||
    chan: Channel,
 | 
			
		||||
    func: typing.Callable,
 | 
			
		||||
    kwargs: Dict[str, Any],
 | 
			
		||||
    kwargs: dict[str, Any],
 | 
			
		||||
    is_rpc: bool = True,
 | 
			
		||||
    task_status: TaskStatus[
 | 
			
		||||
        Union[trio.CancelScope, BaseException]
 | 
			
		||||
| 
						 | 
				
			
			@ -57,6 +58,8 @@ async def _invoke(
 | 
			
		|||
    '''
 | 
			
		||||
    Invoke local func and deliver result(s) over provided channel.
 | 
			
		||||
 | 
			
		||||
    This is the core "RPC task" starting machinery.
 | 
			
		||||
 | 
			
		||||
    '''
 | 
			
		||||
    __tracebackhide__ = True
 | 
			
		||||
    treat_as_gen = False
 | 
			
		||||
| 
						 | 
				
			
			@ -263,14 +266,51 @@ def _get_mod_abspath(module):
 | 
			
		|||
_lifetime_stack: ExitStack = ExitStack()
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
class Actor:
 | 
			
		||||
    """The fundamental concurrency primitive.
 | 
			
		||||
async def try_ship_error_to_parent(
 | 
			
		||||
    channel: Channel,
 | 
			
		||||
    err: Union[Exception, trio.MultiError],
 | 
			
		||||
 | 
			
		||||
    An *actor* is the combination of a regular Python process
 | 
			
		||||
    executing a ``trio`` task tree, communicating
 | 
			
		||||
    with other actors through "portals" which provide a native async API
 | 
			
		||||
    around various IPC transport "channels".
 | 
			
		||||
    """
 | 
			
		||||
) -> None:
 | 
			
		||||
    with trio.CancelScope(shield=True):
 | 
			
		||||
        try:
 | 
			
		||||
            # internal error so ship to parent without cid
 | 
			
		||||
            await channel.send(pack_error(err))
 | 
			
		||||
        except (
 | 
			
		||||
            trio.ClosedResourceError,
 | 
			
		||||
            trio.BrokenResourceError,
 | 
			
		||||
        ):
 | 
			
		||||
            log.error(
 | 
			
		||||
                f"Failed to ship error to parent "
 | 
			
		||||
                f"{channel.uid}, channel was closed"
 | 
			
		||||
            )
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
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.
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
    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 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.
 | 
			
		||||
 | 
			
		||||
    '''
 | 
			
		||||
    # ugh, we need to get rid of this and replace with a "registry" sys
 | 
			
		||||
    # https://github.com/goodboy/tractor/issues/216
 | 
			
		||||
    is_arbiter: bool = False
 | 
			
		||||
 | 
			
		||||
    # nursery placeholders filled in by `_async_main()` after fork
 | 
			
		||||
| 
						 | 
				
			
			@ -279,7 +319,7 @@ class Actor:
 | 
			
		|||
    _server_n: Optional[trio.Nursery] = None
 | 
			
		||||
 | 
			
		||||
    # Information about `__main__` from parent
 | 
			
		||||
    _parent_main_data: Dict[str, str]
 | 
			
		||||
    _parent_main_data: dict[str, str]
 | 
			
		||||
    _parent_chan_cs: Optional[trio.CancelScope] = None
 | 
			
		||||
 | 
			
		||||
    # syncs for setup/teardown sequences
 | 
			
		||||
| 
						 | 
				
			
			@ -317,7 +357,7 @@ class Actor:
 | 
			
		|||
            mods[name] = _get_mod_abspath(mod)
 | 
			
		||||
 | 
			
		||||
        self.enable_modules = mods
 | 
			
		||||
        self._mods: Dict[str, ModuleType] = {}
 | 
			
		||||
        self._mods: dict[str, ModuleType] = {}
 | 
			
		||||
 | 
			
		||||
        # TODO: consider making this a dynamically defined
 | 
			
		||||
        # @dataclass once we get py3.7
 | 
			
		||||
| 
						 | 
				
			
			@ -340,12 +380,12 @@ class Actor:
 | 
			
		|||
        self._ongoing_rpc_tasks = trio.Event()
 | 
			
		||||
        self._ongoing_rpc_tasks.set()
 | 
			
		||||
        # (chan, cid) -> (cancel_scope, func)
 | 
			
		||||
        self._rpc_tasks: Dict[
 | 
			
		||||
        self._rpc_tasks: dict[
 | 
			
		||||
            Tuple[Channel, str],
 | 
			
		||||
            Tuple[trio.CancelScope, typing.Callable, trio.Event]
 | 
			
		||||
        ] = {}
 | 
			
		||||
        # map {uids -> {callids -> waiter queues}}
 | 
			
		||||
        self._cids2qs: Dict[
 | 
			
		||||
        self._cids2qs: dict[
 | 
			
		||||
            Tuple[Tuple[str, str], str],
 | 
			
		||||
            Tuple[
 | 
			
		||||
                trio.abc.SendChannel[Any],
 | 
			
		||||
| 
						 | 
				
			
			@ -356,7 +396,7 @@ class Actor:
 | 
			
		|||
        self._parent_chan: Optional[Channel] = None
 | 
			
		||||
        self._forkserver_info: Optional[
 | 
			
		||||
            Tuple[Any, Any, Any, Any, Any]] = None
 | 
			
		||||
        self._actoruid2nursery: Dict[str, 'ActorNursery'] = {}  # type: ignore  # noqa
 | 
			
		||||
        self._actoruid2nursery: dict[Optional[tuple[str, str]], 'ActorNursery'] = {}  # type: ignore  # noqa
 | 
			
		||||
 | 
			
		||||
    async def wait_for_peer(
 | 
			
		||||
        self, uid: Tuple[str, str]
 | 
			
		||||
| 
						 | 
				
			
			@ -441,8 +481,8 @@ class Actor:
 | 
			
		|||
            # 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()``
 | 
			
		||||
| 
						 | 
				
			
			@ -482,7 +522,50 @@ class Actor:
 | 
			
		|||
        # process received reponses.
 | 
			
		||||
        try:
 | 
			
		||||
            await self._process_messages(chan)
 | 
			
		||||
 | 
			
		||||
        except trio.Cancelled:
 | 
			
		||||
            log.cancel(f"Msg loop was cancelled for {chan}")
 | 
			
		||||
            raise
 | 
			
		||||
 | 
			
		||||
        finally:
 | 
			
		||||
            # 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.
 | 
			
		||||
            local_nursery = self._actoruid2nursery.get(chan.uid)
 | 
			
		||||
            if (
 | 
			
		||||
                local_nursery
 | 
			
		||||
            ):
 | 
			
		||||
                log.cancel(f"Waiting on cancel request to peer {chan.uid}")
 | 
			
		||||
                # 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)
 | 
			
		||||
                # 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.
 | 
			
		||||
                with trio.move_on_after(0.1) as cs:
 | 
			
		||||
                    cs.shield = True
 | 
			
		||||
                    # Attempt to wait for the far end to close the channel
 | 
			
		||||
                    # and bail after timeout (2-generals on closure).
 | 
			
		||||
                    assert chan.msgstream
 | 
			
		||||
                    async for msg in chan.msgstream.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.runtime(f'drained {msg} for {chan.uid}')
 | 
			
		||||
                        cid = msg.get('cid')
 | 
			
		||||
                        if cid:
 | 
			
		||||
                            # deliver response to local caller/waiter
 | 
			
		||||
                            await self._push_result(chan, cid, msg)
 | 
			
		||||
 | 
			
		||||
                    await local_nursery.exited.wait()
 | 
			
		||||
 | 
			
		||||
            # channel cleanup sequence
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -534,7 +617,7 @@ class Actor:
 | 
			
		|||
        self,
 | 
			
		||||
        chan: Channel,
 | 
			
		||||
        cid: str,
 | 
			
		||||
        msg: Dict[str, Any],
 | 
			
		||||
        msg: dict[str, Any],
 | 
			
		||||
    ) -> None:
 | 
			
		||||
        """Push an RPC result to the local consumer's queue.
 | 
			
		||||
        """
 | 
			
		||||
| 
						 | 
				
			
			@ -593,10 +676,12 @@ class Actor:
 | 
			
		|||
        func: str,
 | 
			
		||||
        kwargs: dict
 | 
			
		||||
    ) -> Tuple[str, trio.abc.ReceiveChannel]:
 | 
			
		||||
        """Send a ``'cmd'`` message to a remote actor and return a
 | 
			
		||||
        '''
 | 
			
		||||
        Send a ``'cmd'`` message to a remote actor and return a
 | 
			
		||||
        caller id and a ``trio.Queue`` that can be used to wait for
 | 
			
		||||
        responses delivered by the local message processing loop.
 | 
			
		||||
        """
 | 
			
		||||
 | 
			
		||||
        '''
 | 
			
		||||
        cid = str(uuid.uuid4())
 | 
			
		||||
        assert chan.uid
 | 
			
		||||
        send_chan, recv_chan = self.get_memchans(chan.uid, cid)
 | 
			
		||||
| 
						 | 
				
			
			@ -609,11 +694,14 @@ class Actor:
 | 
			
		|||
        chan: Channel,
 | 
			
		||||
        shield: bool = False,
 | 
			
		||||
        task_status: TaskStatus[trio.CancelScope] = trio.TASK_STATUS_IGNORED,
 | 
			
		||||
 | 
			
		||||
    ) -> None:
 | 
			
		||||
        """Process messages for the channel async-RPC style.
 | 
			
		||||
        '''
 | 
			
		||||
        Process messages for the channel async-RPC style.
 | 
			
		||||
 | 
			
		||||
        Receive multiplexed RPC requests and deliver responses over ``chan``.
 | 
			
		||||
        """
 | 
			
		||||
 | 
			
		||||
        '''
 | 
			
		||||
        # TODO: once https://github.com/python-trio/trio/issues/467 gets
 | 
			
		||||
        # worked out we'll likely want to use that!
 | 
			
		||||
        msg = None
 | 
			
		||||
| 
						 | 
				
			
			@ -692,8 +780,9 @@ class Actor:
 | 
			
		|||
                                # msg loop and break out into
 | 
			
		||||
                                # ``_async_main()``
 | 
			
		||||
                                log.cancel(
 | 
			
		||||
                                    f"Actor {self.uid} was remotely cancelled;"
 | 
			
		||||
                                    " waiting on cancellation completion..")
 | 
			
		||||
                                    f"Actor {self.uid} was remotely cancelled "
 | 
			
		||||
                                    f"by {chan.uid}"
 | 
			
		||||
                                )
 | 
			
		||||
                                await _invoke(
 | 
			
		||||
                                    self, cid, chan, func, kwargs, is_rpc=False
 | 
			
		||||
                                )
 | 
			
		||||
| 
						 | 
				
			
			@ -789,17 +878,12 @@ class Actor:
 | 
			
		|||
                # machinery not from an rpc task) to parent
 | 
			
		||||
                log.exception("Actor errored:")
 | 
			
		||||
                if self._parent_chan:
 | 
			
		||||
                    await self._parent_chan.send(pack_error(err))
 | 
			
		||||
                    await try_ship_error_to_parent(self._parent_chan, err)
 | 
			
		||||
 | 
			
		||||
            # if this is the `MainProcess` we expect the error broadcasting
 | 
			
		||||
            # above to trigger an error at consuming portal "checkpoints"
 | 
			
		||||
            raise
 | 
			
		||||
 | 
			
		||||
        except trio.Cancelled:
 | 
			
		||||
            # debugging only
 | 
			
		||||
            log.runtime(f"Msg loop was cancelled for {chan}")
 | 
			
		||||
            raise
 | 
			
		||||
 | 
			
		||||
        finally:
 | 
			
		||||
            # msg debugging for when he machinery is brokey
 | 
			
		||||
            log.runtime(
 | 
			
		||||
| 
						 | 
				
			
			@ -891,6 +975,7 @@ class Actor:
 | 
			
		|||
            # establish primary connection with immediate parent
 | 
			
		||||
            self._parent_chan = None
 | 
			
		||||
            if parent_addr is not None:
 | 
			
		||||
 | 
			
		||||
                self._parent_chan, accept_addr_rent = await self._from_parent(
 | 
			
		||||
                    parent_addr)
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -994,14 +1079,7 @@ class Actor:
 | 
			
		|||
                )
 | 
			
		||||
 | 
			
		||||
            if self._parent_chan:
 | 
			
		||||
                with trio.CancelScope(shield=True):
 | 
			
		||||
                    try:
 | 
			
		||||
                        # internal error so ship to parent without cid
 | 
			
		||||
                        await self._parent_chan.send(pack_error(err))
 | 
			
		||||
                    except trio.ClosedResourceError:
 | 
			
		||||
                        log.error(
 | 
			
		||||
                            f"Failed to ship error to parent "
 | 
			
		||||
                            f"{self._parent_chan.uid}, channel was closed")
 | 
			
		||||
                await try_ship_error_to_parent(self._parent_chan, err)
 | 
			
		||||
 | 
			
		||||
            # always!
 | 
			
		||||
            log.exception("Actor errored:")
 | 
			
		||||
| 
						 | 
				
			
			@ -1283,7 +1361,7 @@ class Arbiter(Actor):
 | 
			
		|||
 | 
			
		||||
    def __init__(self, *args, **kwargs):
 | 
			
		||||
 | 
			
		||||
        self._registry: Dict[
 | 
			
		||||
        self._registry: dict[
 | 
			
		||||
            Tuple[str, str],
 | 
			
		||||
            Tuple[str, int],
 | 
			
		||||
        ] = {}
 | 
			
		||||
| 
						 | 
				
			
			@ -1300,7 +1378,7 @@ class Arbiter(Actor):
 | 
			
		|||
 | 
			
		||||
    async def get_registry(
 | 
			
		||||
        self
 | 
			
		||||
    ) -> Dict[Tuple[str, str], Tuple[str, int]]:
 | 
			
		||||
    ) -> dict[Tuple[str, str], Tuple[str, int]]:
 | 
			
		||||
        '''Return current name registry.
 | 
			
		||||
 | 
			
		||||
        This method is async to allow for cross-actor invocation.
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -6,9 +6,10 @@ from __future__ import annotations
 | 
			
		|||
import platform
 | 
			
		||||
import struct
 | 
			
		||||
import typing
 | 
			
		||||
from collections.abc import AsyncGenerator, AsyncIterator
 | 
			
		||||
from typing import (
 | 
			
		||||
    Any, Tuple, Optional,
 | 
			
		||||
    Type, Protocol, TypeVar
 | 
			
		||||
    Type, Protocol, TypeVar,
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
from tricycle import BufferedReceiveStream
 | 
			
		||||
| 
						 | 
				
			
			@ -46,6 +47,7 @@ MsgType = TypeVar("MsgType")
 | 
			
		|||
class MsgTransport(Protocol[MsgType]):
 | 
			
		||||
 | 
			
		||||
    stream: trio.SocketStream
 | 
			
		||||
    drained: list[MsgType]
 | 
			
		||||
 | 
			
		||||
    def __init__(self, stream: trio.SocketStream) -> None:
 | 
			
		||||
        ...
 | 
			
		||||
| 
						 | 
				
			
			@ -63,6 +65,11 @@ class MsgTransport(Protocol[MsgType]):
 | 
			
		|||
    def connected(self) -> bool:
 | 
			
		||||
        ...
 | 
			
		||||
 | 
			
		||||
    # defining this sync otherwise it causes a mypy error because it
 | 
			
		||||
    # can't figure out it's a generator i guess?..?
 | 
			
		||||
    def drain(self) -> AsyncIterator[dict]:
 | 
			
		||||
        ...
 | 
			
		||||
 | 
			
		||||
    @property
 | 
			
		||||
    def laddr(self) -> Tuple[str, int]:
 | 
			
		||||
        ...
 | 
			
		||||
| 
						 | 
				
			
			@ -93,7 +100,10 @@ class MsgpackTCPStream:
 | 
			
		|||
        self._agen = self._iter_packets()
 | 
			
		||||
        self._send_lock = trio.StrictFIFOLock()
 | 
			
		||||
 | 
			
		||||
    async def _iter_packets(self) -> typing.AsyncGenerator[dict, None]:
 | 
			
		||||
        # public i guess?
 | 
			
		||||
        self.drained: list[dict] = []
 | 
			
		||||
 | 
			
		||||
    async def _iter_packets(self) -> AsyncGenerator[dict, None]:
 | 
			
		||||
        """Yield packets from the underlying stream.
 | 
			
		||||
        """
 | 
			
		||||
        unpacker = msgpack.Unpacker(
 | 
			
		||||
| 
						 | 
				
			
			@ -132,7 +142,7 @@ class MsgpackTCPStream:
 | 
			
		|||
 | 
			
		||||
            if data == b'':
 | 
			
		||||
                raise TransportClosed(
 | 
			
		||||
                    f'transport {self} was already closed prior ro read'
 | 
			
		||||
                    f'transport {self} was already closed prior to read'
 | 
			
		||||
                )
 | 
			
		||||
 | 
			
		||||
            unpacker.feed(data)
 | 
			
		||||
| 
						 | 
				
			
			@ -156,6 +166,20 @@ class MsgpackTCPStream:
 | 
			
		|||
    async def recv(self) -> Any:
 | 
			
		||||
        return await self._agen.asend(None)
 | 
			
		||||
 | 
			
		||||
    async def drain(self) -> AsyncIterator[dict]:
 | 
			
		||||
        '''
 | 
			
		||||
        Drain the stream's remaining messages sent from
 | 
			
		||||
        the far end until the connection is closed by
 | 
			
		||||
        the peer.
 | 
			
		||||
 | 
			
		||||
        '''
 | 
			
		||||
        try:
 | 
			
		||||
            async for msg in self._iter_packets():
 | 
			
		||||
                self.drained.append(msg)
 | 
			
		||||
        except TransportClosed:
 | 
			
		||||
            for msg in self.drained:
 | 
			
		||||
                yield msg
 | 
			
		||||
 | 
			
		||||
    def __aiter__(self):
 | 
			
		||||
        return self._agen
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -164,7 +188,8 @@ class MsgpackTCPStream:
 | 
			
		|||
 | 
			
		||||
 | 
			
		||||
class MsgspecTCPStream(MsgpackTCPStream):
 | 
			
		||||
    '''A ``trio.SocketStream`` delivering ``msgpack`` formatted data
 | 
			
		||||
    '''
 | 
			
		||||
    A ``trio.SocketStream`` delivering ``msgpack`` formatted data
 | 
			
		||||
    using ``msgspec``.
 | 
			
		||||
 | 
			
		||||
    '''
 | 
			
		||||
| 
						 | 
				
			
			@ -184,7 +209,7 @@ class MsgspecTCPStream(MsgpackTCPStream):
 | 
			
		|||
        self.encode = msgspec.Encoder().encode
 | 
			
		||||
        self.decode = msgspec.Decoder().decode  # dict[str, Any])
 | 
			
		||||
 | 
			
		||||
    async def _iter_packets(self) -> typing.AsyncGenerator[dict, None]:
 | 
			
		||||
    async def _iter_packets(self) -> AsyncGenerator[dict, None]:
 | 
			
		||||
        '''Yield packets from the underlying stream.
 | 
			
		||||
 | 
			
		||||
        '''
 | 
			
		||||
| 
						 | 
				
			
			@ -259,9 +284,12 @@ def get_msg_transport(
 | 
			
		|||
 | 
			
		||||
 | 
			
		||||
class Channel:
 | 
			
		||||
    '''An inter-process channel for communication between (remote) actors.
 | 
			
		||||
    '''
 | 
			
		||||
    An inter-process channel for communication between (remote) actors.
 | 
			
		||||
 | 
			
		||||
    Currently the only supported transport is a ``trio.SocketStream``.
 | 
			
		||||
    Wraps a ``MsgStream``: transport + encoding IPC connection.
 | 
			
		||||
    Currently we only support ``trio.SocketStream`` for transport
 | 
			
		||||
    (aka TCP).
 | 
			
		||||
 | 
			
		||||
    '''
 | 
			
		||||
    def __init__(
 | 
			
		||||
| 
						 | 
				
			
			@ -299,10 +327,12 @@ class Channel:
 | 
			
		|||
        # set after handshake - always uid of far end
 | 
			
		||||
        self.uid: Optional[Tuple[str, str]] = None
 | 
			
		||||
 | 
			
		||||
        # set if far end actor errors internally
 | 
			
		||||
        self._exc: Optional[Exception] = None
 | 
			
		||||
        self._agen = self._aiter_recv()
 | 
			
		||||
        self._exc: Optional[Exception] = None  # set if far end actor errors
 | 
			
		||||
        self._closed: bool = False
 | 
			
		||||
        # flag set on ``Portal.cancel_actor()`` indicating
 | 
			
		||||
        # remote (peer) cancellation of the far end actor runtime.
 | 
			
		||||
        self._cancel_called: bool = False  # set on ``Portal.cancel_actor()``
 | 
			
		||||
 | 
			
		||||
    @classmethod
 | 
			
		||||
    def from_stream(
 | 
			
		||||
| 
						 | 
				
			
			@ -441,9 +471,11 @@ class Channel:
 | 
			
		|||
 | 
			
		||||
    async def _aiter_recv(
 | 
			
		||||
        self
 | 
			
		||||
    ) -> typing.AsyncGenerator[Any, None]:
 | 
			
		||||
        """Async iterate items from underlying stream.
 | 
			
		||||
        """
 | 
			
		||||
    ) -> AsyncGenerator[Any, None]:
 | 
			
		||||
        '''
 | 
			
		||||
        Async iterate items from underlying stream.
 | 
			
		||||
 | 
			
		||||
        '''
 | 
			
		||||
        assert self.msgstream
 | 
			
		||||
        while True:
 | 
			
		||||
            try:
 | 
			
		||||
| 
						 | 
				
			
			@ -473,9 +505,11 @@ class Channel:
 | 
			
		|||
async def _connect_chan(
 | 
			
		||||
    host: str, port: int
 | 
			
		||||
) -> typing.AsyncGenerator[Channel, None]:
 | 
			
		||||
    """Create and connect a channel with disconnect on context manager
 | 
			
		||||
    '''
 | 
			
		||||
    Create and connect a channel with disconnect on context manager
 | 
			
		||||
    teardown.
 | 
			
		||||
    """
 | 
			
		||||
 | 
			
		||||
    '''
 | 
			
		||||
    chan = Channel((host, port))
 | 
			
		||||
    await chan.connect()
 | 
			
		||||
    yield chan
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -1,5 +1,6 @@
 | 
			
		|||
"""
 | 
			
		||||
Portal api
 | 
			
		||||
Memory boundary "Portals": an API for structured
 | 
			
		||||
concurrency linked tasks running in disparate memory domains.
 | 
			
		||||
 | 
			
		||||
"""
 | 
			
		||||
import importlib
 | 
			
		||||
| 
						 | 
				
			
			@ -21,7 +22,6 @@ from .log import get_logger
 | 
			
		|||
from ._exceptions import (
 | 
			
		||||
    unpack_error,
 | 
			
		||||
    NoResult,
 | 
			
		||||
    # RemoteActorError,
 | 
			
		||||
    ContextCancelled,
 | 
			
		||||
)
 | 
			
		||||
from ._streaming import Context, ReceiveMsgStream
 | 
			
		||||
| 
						 | 
				
			
			@ -35,10 +35,12 @@ async def maybe_open_nursery(
 | 
			
		|||
    nursery: trio.Nursery = None,
 | 
			
		||||
    shield: bool = False,
 | 
			
		||||
) -> AsyncGenerator[trio.Nursery, Any]:
 | 
			
		||||
    """Create a new nursery if None provided.
 | 
			
		||||
    '''
 | 
			
		||||
    Create a new nursery if None provided.
 | 
			
		||||
 | 
			
		||||
    Blocks on exit as expected if no input nursery is provided.
 | 
			
		||||
    """
 | 
			
		||||
 | 
			
		||||
    '''
 | 
			
		||||
    if nursery is not None:
 | 
			
		||||
        yield nursery
 | 
			
		||||
    else:
 | 
			
		||||
| 
						 | 
				
			
			@ -87,14 +89,18 @@ class Portal:
 | 
			
		|||
    like having a "portal" between the seperate actor memory spaces.
 | 
			
		||||
 | 
			
		||||
    '''
 | 
			
		||||
    # the timeout for a remote cancel request sent to
 | 
			
		||||
    # a(n) (peer) actor.
 | 
			
		||||
    cancel_timeout = 0.5
 | 
			
		||||
 | 
			
		||||
    def __init__(self, channel: Channel) -> None:
 | 
			
		||||
        self.channel = channel
 | 
			
		||||
        # when this is set to a tuple returned from ``_submit()`` then
 | 
			
		||||
        # it is expected that ``result()`` will be awaited at some point
 | 
			
		||||
        # during the portal's lifetime
 | 
			
		||||
        self._result_msg: Optional[dict] = None
 | 
			
		||||
 | 
			
		||||
        # set when _submit_for_result is called
 | 
			
		||||
        # When this is set to a tuple returned from ``_submit()`` then
 | 
			
		||||
        # it is expected that ``result()`` will be awaited at some
 | 
			
		||||
        # point. Set when _submit_for_result is called
 | 
			
		||||
        self._expect_result: Optional[
 | 
			
		||||
            Tuple[str, Any, str, Dict[str, Any]]
 | 
			
		||||
        ] = None
 | 
			
		||||
| 
						 | 
				
			
			@ -199,36 +205,46 @@ class Portal:
 | 
			
		|||
        # we'll need to .aclose all those channels here
 | 
			
		||||
        await self._cancel_streams()
 | 
			
		||||
 | 
			
		||||
    async def cancel_actor(self):
 | 
			
		||||
        """Cancel the actor on the other end of this portal.
 | 
			
		||||
        """
 | 
			
		||||
        if not self.channel.connected():
 | 
			
		||||
            log.cancel("This portal is already closed can't cancel")
 | 
			
		||||
            return False
 | 
			
		||||
    async def cancel_actor(
 | 
			
		||||
        self,
 | 
			
		||||
        timeout: float = None,
 | 
			
		||||
 | 
			
		||||
        await self._cancel_streams()
 | 
			
		||||
    ) -> bool:
 | 
			
		||||
        '''
 | 
			
		||||
        Cancel the actor on the other end of this portal.
 | 
			
		||||
 | 
			
		||||
        '''
 | 
			
		||||
        if not self.channel.connected():
 | 
			
		||||
            log.cancel("This channel is already closed can't cancel")
 | 
			
		||||
            return False
 | 
			
		||||
 | 
			
		||||
        log.cancel(
 | 
			
		||||
            f"Sending actor cancel request to {self.channel.uid} on "
 | 
			
		||||
            f"{self.channel}")
 | 
			
		||||
 | 
			
		||||
        self.channel._cancel_called = True
 | 
			
		||||
 | 
			
		||||
        try:
 | 
			
		||||
            # send cancel cmd - might not get response
 | 
			
		||||
            # XXX: sure would be nice to make this work with a proper shield
 | 
			
		||||
            with trio.move_on_after(0.5) as cancel_scope:
 | 
			
		||||
                cancel_scope.shield = True
 | 
			
		||||
            with trio.move_on_after(timeout or self.cancel_timeout) as cs:
 | 
			
		||||
                cs.shield = True
 | 
			
		||||
 | 
			
		||||
                await self.run_from_ns('self', 'cancel')
 | 
			
		||||
                return True
 | 
			
		||||
 | 
			
		||||
            if cancel_scope.cancelled_caught:
 | 
			
		||||
            if cs.cancelled_caught:
 | 
			
		||||
                log.cancel(f"May have failed to cancel {self.channel.uid}")
 | 
			
		||||
 | 
			
		||||
            # if we get here some weird cancellation case happened
 | 
			
		||||
            return False
 | 
			
		||||
 | 
			
		||||
        except trio.ClosedResourceError:
 | 
			
		||||
        except (
 | 
			
		||||
            trio.ClosedResourceError,
 | 
			
		||||
            trio.BrokenResourceError,
 | 
			
		||||
        ):
 | 
			
		||||
            log.cancel(
 | 
			
		||||
                f"{self.channel} for {self.channel.uid} was already closed?")
 | 
			
		||||
                f"{self.channel} for {self.channel.uid} was already closed or broken?")
 | 
			
		||||
            return False
 | 
			
		||||
 | 
			
		||||
    async def run_from_ns(
 | 
			
		||||
| 
						 | 
				
			
			@ -237,7 +253,9 @@ class Portal:
 | 
			
		|||
        function_name: str,
 | 
			
		||||
        **kwargs,
 | 
			
		||||
    ) -> Any:
 | 
			
		||||
        """Run a function from a (remote) namespace in a new task on the far-end actor.
 | 
			
		||||
        '''
 | 
			
		||||
        Run a function from a (remote) namespace in a new task on the
 | 
			
		||||
        far-end actor.
 | 
			
		||||
 | 
			
		||||
        This is a more explitcit way to run tasks in a remote-process
 | 
			
		||||
        actor using explicit object-path syntax. Hint: this is how
 | 
			
		||||
| 
						 | 
				
			
			@ -246,9 +264,11 @@ class Portal:
 | 
			
		|||
        Note::
 | 
			
		||||
 | 
			
		||||
            A special namespace `self` can be used to invoke `Actor`
 | 
			
		||||
            instance methods in the remote runtime. Currently this should only
 | 
			
		||||
            be used for `tractor` internals.
 | 
			
		||||
        """
 | 
			
		||||
            instance methods in the remote runtime. Currently this
 | 
			
		||||
            should only be used solely for ``tractor`` runtime
 | 
			
		||||
            internals.
 | 
			
		||||
 | 
			
		||||
        '''
 | 
			
		||||
        msg = await self._return_once(
 | 
			
		||||
            *(await self._submit(namespace_path, function_name, kwargs))
 | 
			
		||||
        )
 | 
			
		||||
| 
						 | 
				
			
			@ -447,7 +467,8 @@ class Portal:
 | 
			
		|||
        except (
 | 
			
		||||
            BaseException,
 | 
			
		||||
 | 
			
		||||
            # more specifically, we need to handle:
 | 
			
		||||
            # more specifically, we need to handle these but not
 | 
			
		||||
            # sure it's worth being pedantic:
 | 
			
		||||
            # Exception,
 | 
			
		||||
            # trio.Cancelled,
 | 
			
		||||
            # trio.MultiError,
 | 
			
		||||
| 
						 | 
				
			
			@ -495,19 +516,22 @@ class Portal:
 | 
			
		|||
 | 
			
		||||
@dataclass
 | 
			
		||||
class LocalPortal:
 | 
			
		||||
    """A 'portal' to a local ``Actor``.
 | 
			
		||||
    '''
 | 
			
		||||
    A 'portal' to a local ``Actor``.
 | 
			
		||||
 | 
			
		||||
    A compatibility shim for normal portals but for invoking functions
 | 
			
		||||
    using an in process actor instance.
 | 
			
		||||
    """
 | 
			
		||||
 | 
			
		||||
    '''
 | 
			
		||||
    actor: 'Actor'  # type: ignore # noqa
 | 
			
		||||
    channel: Channel
 | 
			
		||||
 | 
			
		||||
    async def run_from_ns(self, ns: str, func_name: str, **kwargs) -> Any:
 | 
			
		||||
        """Run a requested local function from a namespace path and
 | 
			
		||||
        '''
 | 
			
		||||
        Run a requested local function from a namespace path and
 | 
			
		||||
        return it's result.
 | 
			
		||||
 | 
			
		||||
        """
 | 
			
		||||
        '''
 | 
			
		||||
        obj = self.actor if ns == 'self' else importlib.import_module(ns)
 | 
			
		||||
        func = getattr(obj, func_name)
 | 
			
		||||
        return await func(**kwargs)
 | 
			
		||||
| 
						 | 
				
			
			@ -522,10 +546,13 @@ async def open_portal(
 | 
			
		|||
    shield: bool = False,
 | 
			
		||||
 | 
			
		||||
) -> AsyncGenerator[Portal, None]:
 | 
			
		||||
    """Open a ``Portal`` through the provided ``channel``.
 | 
			
		||||
    '''
 | 
			
		||||
    Open a ``Portal`` through the provided ``channel``.
 | 
			
		||||
 | 
			
		||||
    Spawns a background task to handle message processing.
 | 
			
		||||
    """
 | 
			
		||||
    Spawns a background task to handle message processing (normally
 | 
			
		||||
    done by the actor-runtime implicitly).
 | 
			
		||||
 | 
			
		||||
    '''
 | 
			
		||||
    actor = current_actor()
 | 
			
		||||
    assert actor
 | 
			
		||||
    was_connected = False
 | 
			
		||||
| 
						 | 
				
			
			@ -553,7 +580,6 @@ async def open_portal(
 | 
			
		|||
        portal = Portal(channel)
 | 
			
		||||
        try:
 | 
			
		||||
            yield portal
 | 
			
		||||
 | 
			
		||||
        finally:
 | 
			
		||||
            await portal.aclose()
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -5,7 +5,11 @@ Machinery for actor process spawning using multiple backends.
 | 
			
		|||
import sys
 | 
			
		||||
import multiprocessing as mp
 | 
			
		||||
import platform
 | 
			
		||||
from typing import Any, Dict, Optional
 | 
			
		||||
from typing import (
 | 
			
		||||
    Any, Dict, Optional, Union, Callable,
 | 
			
		||||
    TypeVar,
 | 
			
		||||
)
 | 
			
		||||
from collections.abc import Awaitable, Coroutine
 | 
			
		||||
 | 
			
		||||
import trio
 | 
			
		||||
from trio_typing import TaskStatus
 | 
			
		||||
| 
						 | 
				
			
			@ -41,6 +45,7 @@ from ._exceptions import ActorFailure
 | 
			
		|||
 | 
			
		||||
 | 
			
		||||
log = get_logger('tractor')
 | 
			
		||||
ProcessType = TypeVar('ProcessType', mp.Process, trio.Process)
 | 
			
		||||
 | 
			
		||||
# placeholder for an mp start context if so using that backend
 | 
			
		||||
_ctx: Optional[mp.context.BaseContext] = None
 | 
			
		||||
| 
						 | 
				
			
			@ -97,14 +102,17 @@ def try_set_start_method(name: str) -> Optional[mp.context.BaseContext]:
 | 
			
		|||
 | 
			
		||||
 | 
			
		||||
async def exhaust_portal(
 | 
			
		||||
 | 
			
		||||
    portal: Portal,
 | 
			
		||||
    actor: Actor
 | 
			
		||||
 | 
			
		||||
) -> Any:
 | 
			
		||||
    """Pull final result from portal (assuming it has one).
 | 
			
		||||
    '''
 | 
			
		||||
    Pull final result from portal (assuming it has one).
 | 
			
		||||
 | 
			
		||||
    If the main task is an async generator do our best to consume
 | 
			
		||||
    what's left of it.
 | 
			
		||||
    """
 | 
			
		||||
    '''
 | 
			
		||||
    try:
 | 
			
		||||
        log.debug(f"Waiting on final result from {actor.uid}")
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -126,18 +134,19 @@ async def exhaust_portal(
 | 
			
		|||
 | 
			
		||||
 | 
			
		||||
async def cancel_on_completion(
 | 
			
		||||
 | 
			
		||||
    portal: Portal,
 | 
			
		||||
    actor: Actor,
 | 
			
		||||
    errors: Dict[Tuple[str, str], Exception],
 | 
			
		||||
 | 
			
		||||
) -> None:
 | 
			
		||||
    """
 | 
			
		||||
    '''
 | 
			
		||||
    Cancel actor gracefully once it's "main" portal's
 | 
			
		||||
    result arrives.
 | 
			
		||||
 | 
			
		||||
    Should only be called for actors spawned with `run_in_actor()`.
 | 
			
		||||
 | 
			
		||||
    """
 | 
			
		||||
    '''
 | 
			
		||||
    # if this call errors we store the exception for later
 | 
			
		||||
    # in ``errors`` which will be reraised inside
 | 
			
		||||
    # a MultiError and we still send out a cancel request
 | 
			
		||||
| 
						 | 
				
			
			@ -175,10 +184,37 @@ async def do_hard_kill(
 | 
			
		|||
        # XXX: should pretty much never get here unless we have
 | 
			
		||||
        # to move the bits from ``proc.__aexit__()`` out and
 | 
			
		||||
        # into here.
 | 
			
		||||
        log.critical(f"HARD KILLING {proc}")
 | 
			
		||||
        log.critical(f"#ZOMBIE_LORD_IS_HERE: {proc}")
 | 
			
		||||
        proc.kill()
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
async def soft_wait(
 | 
			
		||||
 | 
			
		||||
    proc: ProcessType,
 | 
			
		||||
    wait_func: Callable[
 | 
			
		||||
        [ProcessType],
 | 
			
		||||
        Awaitable,
 | 
			
		||||
    ],
 | 
			
		||||
    portal: Portal,
 | 
			
		||||
 | 
			
		||||
) -> None:
 | 
			
		||||
    # Wait for proc termination but **dont' yet** call
 | 
			
		||||
    # ``trio.Process.__aexit__()`` (it tears down stdio
 | 
			
		||||
    # which will kill any waiting remote pdb trace).
 | 
			
		||||
    # This is a "soft" (cancellable) join/reap.
 | 
			
		||||
    try:
 | 
			
		||||
        await wait_func(proc)
 | 
			
		||||
    except trio.Cancelled:
 | 
			
		||||
        # if cancelled during a soft wait, cancel the child
 | 
			
		||||
        # actor before entering the hard reap sequence
 | 
			
		||||
        # below. This means we try to do a graceful teardown
 | 
			
		||||
        # via sending a cancel message before getting out
 | 
			
		||||
        # zombie killing tools.
 | 
			
		||||
        with trio.CancelScope(shield=True):
 | 
			
		||||
            await portal.cancel_actor()
 | 
			
		||||
        raise
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
async def new_proc(
 | 
			
		||||
 | 
			
		||||
    name: str,
 | 
			
		||||
| 
						 | 
				
			
			@ -195,11 +231,14 @@ async def new_proc(
 | 
			
		|||
    task_status: TaskStatus[Portal] = trio.TASK_STATUS_IGNORED
 | 
			
		||||
 | 
			
		||||
) -> None:
 | 
			
		||||
    """
 | 
			
		||||
    Create a new ``multiprocessing.Process`` using the
 | 
			
		||||
    spawn method as configured using ``try_set_start_method()``.
 | 
			
		||||
    '''
 | 
			
		||||
    Create a new ``Process`` using a "spawn method" as (configured using
 | 
			
		||||
    ``try_set_start_method()``).
 | 
			
		||||
 | 
			
		||||
    """
 | 
			
		||||
    This routine should be started in a actor runtime task and the logic
 | 
			
		||||
    here is to be considered the core supervision strategy.
 | 
			
		||||
 | 
			
		||||
    '''
 | 
			
		||||
    # mark the new actor with the global spawn method
 | 
			
		||||
    subactor._spawn_method = _spawn_method
 | 
			
		||||
    uid = subactor.uid
 | 
			
		||||
| 
						 | 
				
			
			@ -230,17 +269,19 @@ async def new_proc(
 | 
			
		|||
            ]
 | 
			
		||||
 | 
			
		||||
        cancelled_during_spawn: bool = False
 | 
			
		||||
        proc: Optional[trio.Process] = None
 | 
			
		||||
        try:
 | 
			
		||||
            proc = await trio.open_process(spawn_cmd)
 | 
			
		||||
 | 
			
		||||
            log.runtime(f"Started {proc}")
 | 
			
		||||
 | 
			
		||||
            # wait for actor to spawn and connect back to us
 | 
			
		||||
            # channel should have handshake completed by the
 | 
			
		||||
            # local actor by the time we get a ref to it
 | 
			
		||||
            try:
 | 
			
		||||
                proc = await trio.open_process(spawn_cmd)
 | 
			
		||||
 | 
			
		||||
                log.runtime(f"Started {proc}")
 | 
			
		||||
 | 
			
		||||
                # wait for actor to spawn and connect back to us
 | 
			
		||||
                # channel should have handshake completed by the
 | 
			
		||||
                # local actor by the time we get a ref to it
 | 
			
		||||
                event, chan = await actor_nursery._actor.wait_for_peer(
 | 
			
		||||
                    subactor.uid)
 | 
			
		||||
 | 
			
		||||
            except trio.Cancelled:
 | 
			
		||||
                cancelled_during_spawn = True
 | 
			
		||||
                # we may cancel before the child connects back in which
 | 
			
		||||
| 
						 | 
				
			
			@ -250,7 +291,8 @@ async def new_proc(
 | 
			
		|||
                        # don't clobber an ongoing pdb
 | 
			
		||||
                        if is_root_process():
 | 
			
		||||
                            await maybe_wait_for_debugger()
 | 
			
		||||
                        else:
 | 
			
		||||
 | 
			
		||||
                        elif proc is not None:
 | 
			
		||||
                            async with acquire_debug_lock(uid):
 | 
			
		||||
                                # soft wait on the proc to terminate
 | 
			
		||||
                                with trio.move_on_after(0.5):
 | 
			
		||||
| 
						 | 
				
			
			@ -291,21 +333,14 @@ async def new_proc(
 | 
			
		|||
                        errors
 | 
			
		||||
                    )
 | 
			
		||||
 | 
			
		||||
                # Wait for proc termination but **dont' yet** call
 | 
			
		||||
                # ``trio.Process.__aexit__()`` (it tears down stdio
 | 
			
		||||
                # which will kill any waiting remote pdb trace).
 | 
			
		||||
                # This is a "soft" (cancellable) join/reap.
 | 
			
		||||
                try:
 | 
			
		||||
                    await proc.wait()
 | 
			
		||||
                except trio.Cancelled:
 | 
			
		||||
                    # if cancelled during a soft wait, cancel the child
 | 
			
		||||
                    # actor before entering the hard reap sequence
 | 
			
		||||
                    # below. This means we try to do a graceful teardown
 | 
			
		||||
                    # via sending a cancel message before getting out
 | 
			
		||||
                    # zombie killing tools.
 | 
			
		||||
                    with trio.CancelScope(shield=True):
 | 
			
		||||
                        await portal.cancel_actor()
 | 
			
		||||
                    raise
 | 
			
		||||
                # This is a "soft" (cancellable) join/reap which
 | 
			
		||||
                # will remote cancel the actor on a ``trio.Cancelled``
 | 
			
		||||
                # condition.
 | 
			
		||||
                await soft_wait(
 | 
			
		||||
                    proc,
 | 
			
		||||
                    trio.Process.wait,
 | 
			
		||||
                    portal
 | 
			
		||||
                )
 | 
			
		||||
 | 
			
		||||
                # cancel result waiter that may have been spawned in
 | 
			
		||||
                # tandem if not done already
 | 
			
		||||
| 
						 | 
				
			
			@ -320,23 +355,26 @@ async def new_proc(
 | 
			
		|||
            # killing the process too early.
 | 
			
		||||
            log.cancel(f'Hard reap sequence starting for {uid}')
 | 
			
		||||
 | 
			
		||||
            with trio.CancelScope(shield=True):
 | 
			
		||||
            if proc:
 | 
			
		||||
                with trio.CancelScope(shield=True):
 | 
			
		||||
 | 
			
		||||
                # don't clobber an ongoing pdb
 | 
			
		||||
                if cancelled_during_spawn:
 | 
			
		||||
                    # Try again to avoid TTY clobbering.
 | 
			
		||||
                    async with acquire_debug_lock(uid):
 | 
			
		||||
                        with trio.move_on_after(0.5):
 | 
			
		||||
                            await proc.wait()
 | 
			
		||||
                    # don't clobber an ongoing pdb
 | 
			
		||||
                    if cancelled_during_spawn:
 | 
			
		||||
                        # Try again to avoid TTY clobbering.
 | 
			
		||||
                        async with acquire_debug_lock(uid):
 | 
			
		||||
                            with trio.move_on_after(0.5):
 | 
			
		||||
                                await proc.wait()
 | 
			
		||||
 | 
			
		||||
                if is_root_process():
 | 
			
		||||
                    await maybe_wait_for_debugger()
 | 
			
		||||
                    if is_root_process():
 | 
			
		||||
                        await maybe_wait_for_debugger()
 | 
			
		||||
 | 
			
		||||
                if proc.poll() is None:
 | 
			
		||||
                    log.cancel(f"Attempting to hard kill {proc}")
 | 
			
		||||
                    await do_hard_kill(proc)
 | 
			
		||||
                    if proc.poll() is None:
 | 
			
		||||
                        log.cancel(f"Attempting to hard kill {proc}")
 | 
			
		||||
                        await do_hard_kill(proc)
 | 
			
		||||
 | 
			
		||||
            log.debug(f"Joined {proc}")
 | 
			
		||||
                    log.debug(f"Joined {proc}")
 | 
			
		||||
            else:
 | 
			
		||||
                log.warning('Nursery cancelled before sub-proc started')
 | 
			
		||||
 | 
			
		||||
            if not cancelled_during_spawn:
 | 
			
		||||
                # pop child entry to indicate we no longer managing this
 | 
			
		||||
| 
						 | 
				
			
			@ -351,6 +389,7 @@ async def new_proc(
 | 
			
		|||
            actor_nursery=actor_nursery,
 | 
			
		||||
            subactor=subactor,
 | 
			
		||||
            errors=errors,
 | 
			
		||||
 | 
			
		||||
            # passed through to actor main
 | 
			
		||||
            bind_addr=bind_addr,
 | 
			
		||||
            parent_addr=parent_addr,
 | 
			
		||||
| 
						 | 
				
			
			@ -469,7 +508,14 @@ async def mp_new_proc(
 | 
			
		|||
                    errors
 | 
			
		||||
                )
 | 
			
		||||
 | 
			
		||||
            await proc_waiter(proc)
 | 
			
		||||
            # This is a "soft" (cancellable) join/reap which
 | 
			
		||||
            # will remote cancel the actor on a ``trio.Cancelled``
 | 
			
		||||
            # condition.
 | 
			
		||||
            await soft_wait(
 | 
			
		||||
                proc,
 | 
			
		||||
                proc_waiter,
 | 
			
		||||
                portal
 | 
			
		||||
            )
 | 
			
		||||
 | 
			
		||||
            # cancel result waiter that may have been spawned in
 | 
			
		||||
            # tandem if not done already
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -52,6 +52,7 @@ class ActorNursery:
 | 
			
		|||
        self.cancelled: bool = False
 | 
			
		||||
        self._join_procs = trio.Event()
 | 
			
		||||
        self.errors = errors
 | 
			
		||||
        self.exited = trio.Event()
 | 
			
		||||
 | 
			
		||||
    async def start_actor(
 | 
			
		||||
        self,
 | 
			
		||||
| 
						 | 
				
			
			@ -207,7 +208,8 @@ class ActorNursery:
 | 
			
		|||
 | 
			
		||||
                        # spawn cancel tasks for each sub-actor
 | 
			
		||||
                        assert portal
 | 
			
		||||
                        nursery.start_soon(portal.cancel_actor)
 | 
			
		||||
                        if portal.channel.connected():
 | 
			
		||||
                            nursery.start_soon(portal.cancel_actor)
 | 
			
		||||
 | 
			
		||||
        # if we cancelled the cancel (we hung cancelling remote actors)
 | 
			
		||||
        # then hard kill all sub-processes
 | 
			
		||||
| 
						 | 
				
			
			@ -401,18 +403,23 @@ async def open_nursery(
 | 
			
		|||
            async with open_root_actor(**kwargs) as actor:
 | 
			
		||||
                assert actor is current_actor()
 | 
			
		||||
 | 
			
		||||
                # try:
 | 
			
		||||
                try:
 | 
			
		||||
                    async with _open_and_supervise_one_cancels_all_nursery(
 | 
			
		||||
                        actor
 | 
			
		||||
                    ) as anursery:
 | 
			
		||||
                        yield anursery
 | 
			
		||||
                finally:
 | 
			
		||||
                    anursery.exited.set()
 | 
			
		||||
 | 
			
		||||
        else:  # sub-nursery case
 | 
			
		||||
 | 
			
		||||
            try:
 | 
			
		||||
                async with _open_and_supervise_one_cancels_all_nursery(
 | 
			
		||||
                    actor
 | 
			
		||||
                ) as anursery:
 | 
			
		||||
                    yield anursery
 | 
			
		||||
 | 
			
		||||
        else:  # sub-nursery case
 | 
			
		||||
 | 
			
		||||
            async with _open_and_supervise_one_cancels_all_nursery(
 | 
			
		||||
                actor
 | 
			
		||||
            ) as anursery:
 | 
			
		||||
                yield anursery
 | 
			
		||||
            finally:
 | 
			
		||||
                anursery.exited.set()
 | 
			
		||||
 | 
			
		||||
    finally:
 | 
			
		||||
        log.debug("Nursery teardown complete")
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
		Loading…
	
		Reference in New Issue