forked from goodboy/tractor
1
0
Fork 0

Merge pull request #215 from goodboy/transport_cleaning

Transport cleaning: attempt to define our graceful channel close signal.
wats_da_nooz
goodboy 2021-07-06 08:20:19 -04:00 committed by GitHub
commit 4d530deac3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 151 additions and 28 deletions

View File

@ -38,13 +38,20 @@ setup(
'tractor.testing', 'tractor.testing',
], ],
install_requires=[ install_requires=[
# trio related
'trio>0.8', 'trio>0.8',
'msgpack',
'async_generator', 'async_generator',
'trio_typing',
# tooling
'colorlog', 'colorlog',
'wrapt', 'wrapt',
'trio_typing',
'pdbpp', 'pdbpp',
# serialization
'msgpack',
], ],
tests_require=['pytest'], tests_require=['pytest'],
python_requires=">=3.7", python_requires=">=3.7",

View File

@ -123,8 +123,15 @@ def test_multierror_fast_nursery(arb_addr, start_method, num_subactors, delay):
assert exc_info.type == tractor.MultiError assert exc_info.type == tractor.MultiError
err = exc_info.value err = exc_info.value
assert len(err.exceptions) == num_subactors exceptions = err.exceptions
for exc in 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 isinstance(exc, tractor.RemoteActorError)
assert exc.type == AssertionError assert exc.type == AssertionError

View File

@ -297,15 +297,31 @@ def test_multi_daemon_subactors(spawn, loglevel):
child.expect(r"\(Pdb\+\+\)") child.expect(r"\(Pdb\+\+\)")
# there is a race for which subactor will acquire
# the root's tty lock first
before = str(child.before.decode()) 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') child.sendline('c')
# first name_error failure # first name_error failure
child.expect(r"\(Pdb\+\+\)") child.expect(r"\(Pdb\+\+\)")
before = str(child.before.decode()) before = str(child.before.decode())
assert "NameError" in before
if next_msg:
assert next_msg in before
child.sendline('c') child.sendline('c')
@ -316,9 +332,10 @@ def test_multi_daemon_subactors(spawn, loglevel):
try: try:
child.sendline('c') child.sendline('c')
child.expect(pexpect.EOF) 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.sendline('q')
child.expect(pexpect.EOF) child.expect(pexpect.EOF)

View File

@ -27,6 +27,7 @@ from ._exceptions import (
unpack_error, unpack_error,
ModuleNotExposed, ModuleNotExposed,
is_multi_cancelled, is_multi_cancelled,
TransportClosed,
) )
from . import _debug from . import _debug
from ._discovery import get_arbiter from ._discovery import get_arbiter
@ -262,7 +263,7 @@ class Actor:
self._parent_chan: Optional[Channel] = None self._parent_chan: Optional[Channel] = None
self._forkserver_info: Optional[ self._forkserver_info: Optional[
Tuple[Any, Any, Any, Any, Any]] = None 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( async def wait_for_peer(
self, uid: Tuple[str, str] self, uid: Tuple[str, str]
@ -338,7 +339,18 @@ class Actor:
# send/receive initial handshake response # send/receive initial handshake response
try: try:
uid = await self._do_handshake(chan) 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") log.warning(f"Channel {chan} failed to handshake")
return return
@ -578,22 +590,35 @@ class Actor:
) )
await self.cancel_rpc_tasks(chan) await self.cancel_rpc_tasks(chan)
except trio.ClosedResourceError: except (
log.error(f"{chan} form {chan.uid} broke") 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: except (Exception, trio.MultiError) as err:
# ship any "internal" exception (i.e. one from internal machinery # ship any "internal" exception (i.e. one from internal machinery
# not from an rpc task) to parent # not from an rpc task) to parent
log.exception("Actor errored:") log.exception("Actor errored:")
if self._parent_chan: if self._parent_chan:
await self._parent_chan.send(pack_error(err)) await self._parent_chan.send(pack_error(err))
raise
# if this is the `MainProcess` we expect the error broadcasting # if this is the `MainProcess` we expect the error broadcasting
# above to trigger an error at consuming portal "checkpoints" # above to trigger an error at consuming portal "checkpoints"
raise
except trio.Cancelled: except trio.Cancelled:
# debugging only # debugging only
log.debug(f"Msg loop was cancelled for {chan}") log.debug(f"Msg loop was cancelled for {chan}")
raise raise
finally: finally:
# msg debugging for when he machinery is brokey
log.debug( log.debug(
f"Exiting msg loop for {chan} from {chan.uid} " f"Exiting msg loop for {chan} from {chan.uid} "
f"with last msg:\n{msg}") f"with last msg:\n{msg}")

View File

@ -38,6 +38,10 @@ class InternalActorError(RemoteActorError):
""" """
class TransportClosed(trio.ClosedResourceError):
"Underlying channel transport was closed prior to use"
class NoResult(RuntimeError): class NoResult(RuntimeError):
"No final result is expected for this actor" "No final result is expected for this actor"
@ -63,12 +67,15 @@ def pack_error(exc: BaseException) -> Dict[str, Any]:
def unpack_error( def unpack_error(
msg: Dict[str, Any], msg: Dict[str, Any],
chan=None, chan=None,
err_type=RemoteActorError err_type=RemoteActorError
) -> Exception: ) -> Exception:
"""Unpack an 'error' message from the wire """Unpack an 'error' message from the wire
into a local ``RemoteActorError``. into a local ``RemoteActorError``.
""" """
tb_str = msg['error'].get('tb_str', '') tb_str = msg['error'].get('tb_str', '')
return err_type( return err_type(

View File

@ -1,6 +1,7 @@
""" """
Inter-process comms abstractions Inter-process comms abstractions
""" """
import platform
import typing import typing
from typing import Any, Tuple, Optional from typing import Any, Tuple, Optional
from functools import partial from functools import partial
@ -10,7 +11,11 @@ import trio
from async_generator import asynccontextmanager from async_generator import asynccontextmanager
from .log import get_logger from .log import get_logger
log = get_logger('ipc') from ._exceptions import TransportClosed
log = get_logger(__name__)
_is_windows = platform.system() == 'Windows'
# :eyeroll: # :eyeroll:
try: try:
@ -21,10 +26,17 @@ except ImportError:
Unpacker = partial(msgpack.Unpacker, strict_map_key=False) Unpacker = partial(msgpack.Unpacker, strict_map_key=False)
class MsgpackStream: class MsgpackTCPStream:
"""A ``trio.SocketStream`` delivering ``msgpack`` formatted data. '''A ``trio.SocketStream`` delivering ``msgpack`` formatted data
""" using ``msgpack-python``.
def __init__(self, stream: trio.SocketStream) -> None:
'''
def __init__(
self,
stream: trio.SocketStream,
) -> None:
self.stream = stream self.stream = stream
assert self.stream.socket assert self.stream.socket
# should both be IP sockets # should both be IP sockets
@ -35,7 +47,10 @@ class MsgpackStream:
assert isinstance(rsockname, tuple) assert isinstance(rsockname, tuple)
self._raddr = rsockname[:2] self._raddr = rsockname[:2]
# start and seed first entry to read loop
self._agen = self._iter_packets() self._agen = self._iter_packets()
# self._agen.asend(None) is None
self._send_lock = trio.StrictFIFOLock() self._send_lock = trio.StrictFIFOLock()
async def _iter_packets(self) -> typing.AsyncGenerator[dict, None]: async def _iter_packets(self) -> typing.AsyncGenerator[dict, None]:
@ -46,16 +61,39 @@ class MsgpackStream:
use_list=False, use_list=False,
) )
while True: while True:
try: try:
data = await self.stream.receive_some(2**10) data = await self.stream.receive_some(2**10)
log.trace(f"received {data}") # type: ignore
except trio.BrokenResourceError: except trio.BrokenResourceError as err:
log.warning(f"Stream connection {self.raddr} broke") msg = err.args[0]
return
# 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
if data == b'': if data == b'':
log.debug(f"Stream connection {self.raddr} was closed") raise TransportClosed(
return f'transport {self} was already closed prior ro read'
)
unpacker.feed(data) unpacker.feed(data)
for packet in unpacker: for packet in unpacker:
@ -96,10 +134,11 @@ class Channel:
on_reconnect: typing.Callable[..., typing.Awaitable] = None, on_reconnect: typing.Callable[..., typing.Awaitable] = None,
auto_reconnect: bool = False, auto_reconnect: bool = False,
stream: trio.SocketStream = None, # expected to be active stream: trio.SocketStream = None, # expected to be active
) -> None: ) -> None:
self._recon_seq = on_reconnect self._recon_seq = on_reconnect
self._autorecon = auto_reconnect self._autorecon = auto_reconnect
self.msgstream: Optional[MsgpackStream] = MsgpackStream( self.msgstream: Optional[MsgpackTCPStream] = MsgpackTCPStream(
stream) if stream else None stream) if stream else None
if self.msgstream and destaddr: if self.msgstream and destaddr:
raise ValueError( raise ValueError(
@ -112,6 +151,8 @@ class Channel:
self._exc: Optional[Exception] = None self._exc: Optional[Exception] = None
self._agen = self._aiter_recv() self._agen = self._aiter_recv()
self._closed: bool = False
def __repr__(self) -> str: def __repr__(self) -> str:
if self.msgstream: if self.msgstream:
return repr( return repr(
@ -128,35 +169,49 @@ class Channel:
return self.msgstream.raddr if self.msgstream else None return self.msgstream.raddr if self.msgstream else None
async def connect( async def connect(
self, destaddr: Tuple[Any, ...] = None, self,
destaddr: Tuple[Any, ...] = None,
**kwargs **kwargs
) -> trio.SocketStream: ) -> trio.SocketStream:
if self.connected(): if self.connected():
raise RuntimeError("channel is already connected?") raise RuntimeError("channel is already connected?")
destaddr = destaddr or self._destaddr destaddr = destaddr or self._destaddr
assert isinstance(destaddr, tuple) assert isinstance(destaddr, tuple)
stream = await trio.open_tcp_stream(*destaddr, **kwargs) stream = await trio.open_tcp_stream(*destaddr, **kwargs)
self.msgstream = MsgpackStream(stream) self.msgstream = MsgpackTCPStream(stream)
return stream return stream
async def send(self, item: Any) -> None: async def send(self, item: Any) -> None:
log.trace(f"send `{item}`") # type: ignore log.trace(f"send `{item}`") # type: ignore
assert self.msgstream assert self.msgstream
await self.msgstream.send(item) await self.msgstream.send(item)
async def recv(self) -> Any: async def recv(self) -> Any:
assert self.msgstream assert self.msgstream
try: try:
return await self.msgstream.recv() return await self.msgstream.recv()
except trio.BrokenResourceError: except trio.BrokenResourceError:
if self._autorecon: if self._autorecon:
await self._reconnect() await self._reconnect()
return await self.recv() return await self.recv()
raise
async def aclose(self) -> None: 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 assert self.msgstream
await self.msgstream.stream.aclose() await self.msgstream.stream.aclose()
self._closed = True
async def __aenter__(self): async def __aenter__(self):
await self.connect() await self.connect()

View File

@ -105,6 +105,11 @@ async def open_root_actor(
arbiter_found = False arbiter_found = False
try: 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): async with _connect_chan(host, port):
arbiter_found = True arbiter_found = True