forked from goodboy/tractor
Merge pull request #215 from goodboy/transport_cleaning
Transport cleaning: attempt to define our graceful channel close signal.wats_da_nooz
commit
4d530deac3
11
setup.py
11
setup.py
|
@ -38,13 +38,20 @@ setup(
|
|||
'tractor.testing',
|
||||
],
|
||||
install_requires=[
|
||||
|
||||
# trio related
|
||||
'trio>0.8',
|
||||
'msgpack',
|
||||
'async_generator',
|
||||
'trio_typing',
|
||||
|
||||
# tooling
|
||||
'colorlog',
|
||||
'wrapt',
|
||||
'trio_typing',
|
||||
'pdbpp',
|
||||
|
||||
# serialization
|
||||
'msgpack',
|
||||
|
||||
],
|
||||
tests_require=['pytest'],
|
||||
python_requires=">=3.7",
|
||||
|
|
|
@ -123,8 +123,15 @@ def test_multierror_fast_nursery(arb_addr, start_method, num_subactors, delay):
|
|||
|
||||
assert exc_info.type == tractor.MultiError
|
||||
err = exc_info.value
|
||||
assert len(err.exceptions) == num_subactors
|
||||
for exc in err.exceptions:
|
||||
exceptions = err.exceptions
|
||||
|
||||
if len(exceptions) == 2:
|
||||
# sometimes oddly now there's an embedded BrokenResourceError ?
|
||||
exceptions = exceptions[1].exceptions
|
||||
|
||||
assert len(exceptions) == num_subactors
|
||||
|
||||
for exc in exceptions:
|
||||
assert isinstance(exc, tractor.RemoteActorError)
|
||||
assert exc.type == AssertionError
|
||||
|
||||
|
|
|
@ -297,15 +297,31 @@ def test_multi_daemon_subactors(spawn, loglevel):
|
|||
|
||||
child.expect(r"\(Pdb\+\+\)")
|
||||
|
||||
# there is a race for which subactor will acquire
|
||||
# the root's tty lock first
|
||||
|
||||
before = str(child.before.decode())
|
||||
assert "Attaching pdb to actor: ('bp_forever'" in before
|
||||
|
||||
bp_forever_msg = "Attaching pdb to actor: ('bp_forever'"
|
||||
name_error_msg = "NameError"
|
||||
|
||||
if bp_forever_msg in before:
|
||||
next_msg = name_error_msg
|
||||
|
||||
elif name_error_msg in before:
|
||||
next_msg = None
|
||||
|
||||
else:
|
||||
raise ValueError("Neither log msg was found !?")
|
||||
|
||||
child.sendline('c')
|
||||
|
||||
# first name_error failure
|
||||
child.expect(r"\(Pdb\+\+\)")
|
||||
before = str(child.before.decode())
|
||||
assert "NameError" in before
|
||||
|
||||
if next_msg:
|
||||
assert next_msg in before
|
||||
|
||||
child.sendline('c')
|
||||
|
||||
|
@ -316,9 +332,10 @@ def test_multi_daemon_subactors(spawn, loglevel):
|
|||
try:
|
||||
child.sendline('c')
|
||||
child.expect(pexpect.EOF)
|
||||
except pexpect.exceptions.TIMEOUT:
|
||||
# Failed to exit using continue..?
|
||||
|
||||
except pexpect.exceptions.TIMEOUT:
|
||||
|
||||
# Failed to exit using continue..?
|
||||
child.sendline('q')
|
||||
child.expect(pexpect.EOF)
|
||||
|
||||
|
|
|
@ -27,6 +27,7 @@ from ._exceptions import (
|
|||
unpack_error,
|
||||
ModuleNotExposed,
|
||||
is_multi_cancelled,
|
||||
TransportClosed,
|
||||
)
|
||||
from . import _debug
|
||||
from ._discovery import get_arbiter
|
||||
|
@ -262,7 +263,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
|
||||
self._actoruid2nursery: Dict[str, 'ActorNursery'] = {} # type: ignore # noqa
|
||||
|
||||
async def wait_for_peer(
|
||||
self, uid: Tuple[str, str]
|
||||
|
@ -338,7 +339,18 @@ class Actor:
|
|||
# send/receive initial handshake response
|
||||
try:
|
||||
uid = await self._do_handshake(chan)
|
||||
except StopAsyncIteration:
|
||||
|
||||
except (
|
||||
# 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.
|
||||
log.warning(f"Channel {chan} failed to handshake")
|
||||
return
|
||||
|
||||
|
@ -578,22 +590,35 @@ class Actor:
|
|||
)
|
||||
await self.cancel_rpc_tasks(chan)
|
||||
|
||||
except trio.ClosedResourceError:
|
||||
log.error(f"{chan} form {chan.uid} broke")
|
||||
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.
|
||||
log.debug(f'channel from {chan.uid} closed abruptly:\n{chan}')
|
||||
|
||||
except (Exception, trio.MultiError) as err:
|
||||
|
||||
# ship any "internal" exception (i.e. one from internal machinery
|
||||
# not from an rpc task) to parent
|
||||
log.exception("Actor errored:")
|
||||
if self._parent_chan:
|
||||
await self._parent_chan.send(pack_error(err))
|
||||
raise
|
||||
|
||||
# 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.debug(f"Msg loop was cancelled for {chan}")
|
||||
raise
|
||||
|
||||
finally:
|
||||
# msg debugging for when he machinery is brokey
|
||||
log.debug(
|
||||
f"Exiting msg loop for {chan} from {chan.uid} "
|
||||
f"with last msg:\n{msg}")
|
||||
|
|
|
@ -38,6 +38,10 @@ class InternalActorError(RemoteActorError):
|
|||
"""
|
||||
|
||||
|
||||
class TransportClosed(trio.ClosedResourceError):
|
||||
"Underlying channel transport was closed prior to use"
|
||||
|
||||
|
||||
class NoResult(RuntimeError):
|
||||
"No final result is expected for this actor"
|
||||
|
||||
|
@ -63,12 +67,15 @@ def pack_error(exc: BaseException) -> Dict[str, Any]:
|
|||
|
||||
|
||||
def unpack_error(
|
||||
|
||||
msg: Dict[str, Any],
|
||||
chan=None,
|
||||
err_type=RemoteActorError
|
||||
|
||||
) -> Exception:
|
||||
"""Unpack an 'error' message from the wire
|
||||
into a local ``RemoteActorError``.
|
||||
|
||||
"""
|
||||
tb_str = msg['error'].get('tb_str', '')
|
||||
return err_type(
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
"""
|
||||
Inter-process comms abstractions
|
||||
"""
|
||||
import platform
|
||||
import typing
|
||||
from typing import Any, Tuple, Optional
|
||||
from functools import partial
|
||||
|
@ -10,7 +11,11 @@ import trio
|
|||
from async_generator import asynccontextmanager
|
||||
|
||||
from .log import get_logger
|
||||
log = get_logger('ipc')
|
||||
from ._exceptions import TransportClosed
|
||||
log = get_logger(__name__)
|
||||
|
||||
|
||||
_is_windows = platform.system() == 'Windows'
|
||||
|
||||
# :eyeroll:
|
||||
try:
|
||||
|
@ -21,10 +26,17 @@ except ImportError:
|
|||
Unpacker = partial(msgpack.Unpacker, strict_map_key=False)
|
||||
|
||||
|
||||
class MsgpackStream:
|
||||
"""A ``trio.SocketStream`` delivering ``msgpack`` formatted data.
|
||||
"""
|
||||
def __init__(self, stream: trio.SocketStream) -> None:
|
||||
class MsgpackTCPStream:
|
||||
'''A ``trio.SocketStream`` delivering ``msgpack`` formatted data
|
||||
using ``msgpack-python``.
|
||||
|
||||
'''
|
||||
def __init__(
|
||||
self,
|
||||
stream: trio.SocketStream,
|
||||
|
||||
) -> None:
|
||||
|
||||
self.stream = stream
|
||||
assert self.stream.socket
|
||||
# should both be IP sockets
|
||||
|
@ -35,7 +47,10 @@ class MsgpackStream:
|
|||
assert isinstance(rsockname, tuple)
|
||||
self._raddr = rsockname[:2]
|
||||
|
||||
# start and seed first entry to read loop
|
||||
self._agen = self._iter_packets()
|
||||
# self._agen.asend(None) is None
|
||||
|
||||
self._send_lock = trio.StrictFIFOLock()
|
||||
|
||||
async def _iter_packets(self) -> typing.AsyncGenerator[dict, None]:
|
||||
|
@ -46,16 +61,39 @@ class MsgpackStream:
|
|||
use_list=False,
|
||||
)
|
||||
while True:
|
||||
|
||||
try:
|
||||
data = await self.stream.receive_some(2**10)
|
||||
|
||||
except trio.BrokenResourceError as err:
|
||||
msg = err.args[0]
|
||||
|
||||
# XXX: handle connection-reset-by-peer the same as a EOF.
|
||||
# we're currently remapping this since we allow
|
||||
# a quick connect then drop for root actors when
|
||||
# checking to see if there exists an "arbiter"
|
||||
# on the chosen sockaddr (``_root.py:108`` or thereabouts)
|
||||
if (
|
||||
# nix
|
||||
'[Errno 104]' in msg or
|
||||
|
||||
# on windows it seems there are a variety of errors
|
||||
# to handle..
|
||||
_is_windows
|
||||
):
|
||||
raise TransportClosed(
|
||||
f'{self} was broken with {msg}'
|
||||
)
|
||||
|
||||
else:
|
||||
raise
|
||||
|
||||
log.trace(f"received {data}") # type: ignore
|
||||
except trio.BrokenResourceError:
|
||||
log.warning(f"Stream connection {self.raddr} broke")
|
||||
return
|
||||
|
||||
if data == b'':
|
||||
log.debug(f"Stream connection {self.raddr} was closed")
|
||||
return
|
||||
raise TransportClosed(
|
||||
f'transport {self} was already closed prior ro read'
|
||||
)
|
||||
|
||||
unpacker.feed(data)
|
||||
for packet in unpacker:
|
||||
|
@ -96,10 +134,11 @@ class Channel:
|
|||
on_reconnect: typing.Callable[..., typing.Awaitable] = None,
|
||||
auto_reconnect: bool = False,
|
||||
stream: trio.SocketStream = None, # expected to be active
|
||||
|
||||
) -> None:
|
||||
self._recon_seq = on_reconnect
|
||||
self._autorecon = auto_reconnect
|
||||
self.msgstream: Optional[MsgpackStream] = MsgpackStream(
|
||||
self.msgstream: Optional[MsgpackTCPStream] = MsgpackTCPStream(
|
||||
stream) if stream else None
|
||||
if self.msgstream and destaddr:
|
||||
raise ValueError(
|
||||
|
@ -112,6 +151,8 @@ class Channel:
|
|||
self._exc: Optional[Exception] = None
|
||||
self._agen = self._aiter_recv()
|
||||
|
||||
self._closed: bool = False
|
||||
|
||||
def __repr__(self) -> str:
|
||||
if self.msgstream:
|
||||
return repr(
|
||||
|
@ -128,35 +169,49 @@ class Channel:
|
|||
return self.msgstream.raddr if self.msgstream else None
|
||||
|
||||
async def connect(
|
||||
self, destaddr: Tuple[Any, ...] = None,
|
||||
self,
|
||||
destaddr: Tuple[Any, ...] = None,
|
||||
**kwargs
|
||||
|
||||
) -> trio.SocketStream:
|
||||
|
||||
if self.connected():
|
||||
raise RuntimeError("channel is already connected?")
|
||||
|
||||
destaddr = destaddr or self._destaddr
|
||||
assert isinstance(destaddr, tuple)
|
||||
stream = await trio.open_tcp_stream(*destaddr, **kwargs)
|
||||
self.msgstream = MsgpackStream(stream)
|
||||
self.msgstream = MsgpackTCPStream(stream)
|
||||
return stream
|
||||
|
||||
async def send(self, item: Any) -> None:
|
||||
|
||||
log.trace(f"send `{item}`") # type: ignore
|
||||
assert self.msgstream
|
||||
|
||||
await self.msgstream.send(item)
|
||||
|
||||
async def recv(self) -> Any:
|
||||
assert self.msgstream
|
||||
|
||||
try:
|
||||
return await self.msgstream.recv()
|
||||
|
||||
except trio.BrokenResourceError:
|
||||
if self._autorecon:
|
||||
await self._reconnect()
|
||||
return await self.recv()
|
||||
|
||||
raise
|
||||
|
||||
async def aclose(self) -> None:
|
||||
log.debug(f"Closing {self}")
|
||||
log.debug(
|
||||
f'Closing channel to {self.uid} '
|
||||
f'{self.laddr} -> {self.raddr}'
|
||||
)
|
||||
assert self.msgstream
|
||||
await self.msgstream.stream.aclose()
|
||||
self._closed = True
|
||||
|
||||
async def __aenter__(self):
|
||||
await self.connect()
|
||||
|
|
|
@ -105,6 +105,11 @@ async def open_root_actor(
|
|||
arbiter_found = False
|
||||
|
||||
try:
|
||||
# TODO: this connect-and-bail forces us to have to carefully
|
||||
# rewrap TCP 104-connection-reset errors as EOF so as to avoid
|
||||
# propagating cancel-causing errors to the channel-msg loop
|
||||
# machinery. Likely it would be better to eventually have
|
||||
# a "discovery" protocol with basic handshake instead.
|
||||
async with _connect_chan(host, port):
|
||||
arbiter_found = True
|
||||
|
||||
|
|
Loading…
Reference in New Issue