Go back to only logging crashes if no pdb gets engaged

transport_hardening
Tyler Goodlet 2021-06-29 15:55:03 -04:00
parent f57cc66a7f
commit 6d72a4ef45
1 changed files with 99 additions and 47 deletions

View File

@ -12,6 +12,7 @@ import uuid
import typing import typing
from typing import Dict, List, Tuple, Any, Optional, Union from typing import Dict, List, Tuple, Any, Optional, Union
from types import ModuleType from types import ModuleType
import signal
import sys import sys
import os import os
from contextlib import ExitStack from contextlib import ExitStack
@ -47,6 +48,7 @@ class ActorFailure(Exception):
async def _invoke( async def _invoke(
actor: 'Actor', actor: 'Actor',
cid: str, cid: str,
chan: Channel, chan: Channel,
@ -60,13 +62,14 @@ async def _invoke(
""" """
treat_as_gen = False treat_as_gen = False
# possibly a traceback object # possible a traceback (not sure what typing is for this..)
# (not sure what typing is for this..)
tb = None tb = None
cancel_scope = trio.CancelScope() cancel_scope = trio.CancelScope()
ctx = Context(chan, cid, _cancel_scope=cancel_scope) cs: trio.CancelScope = None
context = False
ctx = Context(chan, cid)
context: bool = False
if getattr(func, '_tractor_stream_function', False): if getattr(func, '_tractor_stream_function', False):
# handle decorated ``@tractor.stream`` async functions # handle decorated ``@tractor.stream`` async functions
@ -154,7 +157,9 @@ async def _invoke(
# context func with support for bi-dir streaming # context func with support for bi-dir streaming
await chan.send({'functype': 'context', 'cid': cid}) await chan.send({'functype': 'context', 'cid': cid})
with cancel_scope as cs: async with trio.open_nursery() as scope_nursery:
ctx._scope_nursery = scope_nursery
cs = scope_nursery.cancel_scope
task_status.started(cs) task_status.started(cs)
try: try:
await chan.send({'return': await coro, 'cid': cid}) await chan.send({'return': await coro, 'cid': cid})
@ -192,19 +197,32 @@ async def _invoke(
except (Exception, trio.MultiError) as err: except (Exception, trio.MultiError) as err:
# TODO: maybe we'll want differnet "levels" of debugging if not is_multi_cancelled(err):
# eventualy such as ('app', 'supervisory', 'runtime') ?
if not isinstance(err, trio.ClosedResourceError) and ( log.exception("Actor crashed:")
not is_multi_cancelled(err)) and (
not isinstance(err, ContextCancelled) # TODO: maybe we'll want different "levels" of debugging
): # eventualy such as ('app', 'supervisory', 'runtime') ?
# XXX: is there any case where we'll want to debug IPC
# disconnects? I can't think of a reason that inspecting # if not isinstance(err, trio.ClosedResourceError) and (
# this type of failure will be useful for respawns or # if not is_multi_cancelled(err) and (
# recovery logic - the only case is some kind of strange bug
# in `trio` itself? entered_debug: bool = False
entered = await _debug._maybe_enter_pm(err) if not isinstance(err, ContextCancelled) or (
if not entered: isinstance(err, ContextCancelled) and ctx._cancel_called
):
# 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:") log.exception("Actor crashed:")
# always ship errors back to caller # always ship errors back to caller
@ -214,8 +232,10 @@ async def _invoke(
await chan.send(err_msg) await chan.send(err_msg)
except trio.ClosedResourceError: except trio.ClosedResourceError:
log.warning( # if we can't propagate the error that's a big boo boo
f"Failed to ship error to caller @ {chan.uid}") log.error(
f"Failed to ship error to caller @ {chan.uid} !?"
)
if cs is None: if cs is None:
# error is from above code not from rpc invocation # error is from above code not from rpc invocation
@ -271,7 +291,7 @@ class Actor:
enable_modules: List[str] = [], enable_modules: List[str] = [],
uid: str = None, uid: str = None,
loglevel: str = None, loglevel: str = None,
arbiter_addr: Optional[Tuple[str, int]] = None, arbiter_addr: Optional[Tuple[str, int]] = (None, None),
spawn_method: Optional[str] = None spawn_method: Optional[str] = None
) -> None: ) -> None:
"""This constructor is called in the parent actor **before** the spawning """This constructor is called in the parent actor **before** the spawning
@ -301,8 +321,7 @@ class Actor:
# TODO: consider making this a dynamically defined # TODO: consider making this a dynamically defined
# @dataclass once we get py3.7 # @dataclass once we get py3.7
self.loglevel = loglevel self.loglevel = loglevel
self._arb_addr = tuple(arbiter_addr)
self._arb_addr = arbiter_addr or (None, None)
# marked by the process spawning backend at startup # marked by the process spawning backend at startup
# will be None for the parent most process started manually # will be None for the parent most process started manually
@ -396,12 +415,16 @@ class Actor:
raise mne raise mne
async def _stream_handler( async def _stream_handler(
self, self,
stream: trio.SocketStream, stream: trio.SocketStream,
) -> None: ) -> None:
"""Entry point for new inbound connections to the channel server. """Entry point for new inbound connections to the channel server.
""" """
self._no_more_peers = trio.Event() # unset self._no_more_peers = trio.Event() # unset
chan = Channel(stream=stream) chan = Channel(stream=stream)
log.runtime(f"New connection to us {chan}") log.runtime(f"New connection to us {chan}")
@ -435,8 +458,9 @@ class Actor:
chans = self._peers[uid] chans = self._peers[uid]
# TODO: re-use channels for new connections instead of always # TODO: re-use channels for new connections instead
# new ones; will require changing all the discovery funcs # of always new ones; will require changing all the
# discovery funcs
if chans: if chans:
log.runtime( log.runtime(
f"already have channel(s) for {uid}:{chans}?" f"already have channel(s) for {uid}:{chans}?"
@ -451,10 +475,24 @@ class Actor:
try: try:
await self._process_messages(chan) await self._process_messages(chan)
finally: finally:
# channel cleanup sequence
# for (channel, cid) in self._rpc_tasks.copy():
# if channel is chan:
# with trio.CancelScope(shield=True):
# await self._cancel_task(cid, channel)
# # close all consumer side task mem chans
# send_chan, _ = self._cids2qs[(chan.uid, cid)]
# assert send_chan.cid == cid # type: ignore
# await send_chan.aclose()
# Drop ref to channel so it can be gc-ed and disconnected # Drop ref to channel so it can be gc-ed and disconnected
log.debug(f"Releasing channel {chan} from {chan.uid}") log.debug(f"Releasing channel {chan} from {chan.uid}")
chans = self._peers.get(chan.uid) chans = self._peers.get(chan.uid)
chans.remove(chan) chans.remove(chan)
if not chans: if not chans:
log.debug(f"No more channels for {chan.uid}") log.debug(f"No more channels for {chan.uid}")
self._peers.pop(chan.uid, None) self._peers.pop(chan.uid, None)
@ -467,14 +505,22 @@ class Actor:
# # XXX: is this necessary (GC should do it?) # # XXX: is this necessary (GC should do it?)
if chan.connected(): if 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.debug(f"Disconnecting channel {chan}") log.debug(f"Disconnecting channel {chan}")
try: try:
# send our msg loop terminate sentinel # send a msg loop terminate sentinel
await chan.send(None) await chan.send(None)
# XXX: do we want this?
# causes "[104] connection reset by peer" on other end
# await chan.aclose() # await chan.aclose()
except trio.BrokenResourceError: except trio.BrokenResourceError:
log.exception( log.warning(f"Channel for {chan.uid} was already closed")
f"Channel for {chan.uid} was already zonked..")
async def _push_result( async def _push_result(
self, self,
@ -484,18 +530,22 @@ class Actor:
) -> None: ) -> None:
"""Push an RPC result to the local consumer's queue. """Push an RPC result to the local consumer's queue.
""" """
actorid = chan.uid # actorid = chan.uid
assert actorid, f"`actorid` can't be {actorid}" assert chan.uid, f"`chan.uid` can't be {chan.uid}"
send_chan, recv_chan = self._cids2qs[(actorid, cid)] send_chan, recv_chan = self._cids2qs[(chan.uid, cid)]
assert send_chan.cid == cid # type: ignore assert send_chan.cid == cid # type: ignore
# if 'stop' in msg: if 'error' in msg:
ctx = getattr(recv_chan, '_ctx', None)
# if ctx:
# ctx._error_from_remote_msg(msg)
# log.debug(f"{send_chan} was terminated at remote end") # log.debug(f"{send_chan} was terminated at remote end")
# # indicate to consumer that far end has stopped # # indicate to consumer that far end has stopped
# return await send_chan.aclose() # return await send_chan.aclose()
try: try:
log.debug(f"Delivering {msg} from {actorid} to caller {cid}") log.debug(f"Delivering {msg} from {chan.uid} to caller {cid}")
# maintain backpressure # maintain backpressure
await send_chan.send(msg) await send_chan.send(msg)
@ -514,7 +564,9 @@ class Actor:
self, self,
actorid: Tuple[str, str], actorid: Tuple[str, str],
cid: str cid: str
) -> Tuple[trio.abc.SendChannel, trio.abc.ReceiveChannel]: ) -> Tuple[trio.abc.SendChannel, trio.abc.ReceiveChannel]:
log.debug(f"Getting result queue for {actorid} cid {cid}") log.debug(f"Getting result queue for {actorid} cid {cid}")
try: try:
send_chan, recv_chan = self._cids2qs[(actorid, cid)] send_chan, recv_chan = self._cids2qs[(actorid, cid)]
@ -577,9 +629,15 @@ class Actor:
if channel is chan: if channel is chan:
await self._cancel_task(cid, channel) await self._cancel_task(cid, channel)
# close all consumer side task mem chans
# send_chan, _ = self._cids2qs[(chan.uid, cid)]
# assert send_chan.cid == cid # type: ignore
# await send_chan.aclose()
log.debug( log.debug(
f"Msg loop signalled to terminate for" f"Msg loop signalled to terminate for"
f" {chan} from {chan.uid}") f" {chan} from {chan.uid}")
break break
log.trace( # type: ignore log.trace( # type: ignore
@ -685,9 +743,6 @@ class Actor:
# caller's teardown sequence to clean up. # caller's teardown sequence to clean up.
log.warning(f"Channel from {chan.uid} closed abruptly") log.warning(f"Channel from {chan.uid} closed abruptly")
except trio.ClosedResourceError:
log.error(f"{chan} form {chan.uid} broke")
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
@ -750,8 +805,7 @@ class Actor:
# XXX: msgspec doesn't support serializing tuples # XXX: msgspec doesn't support serializing tuples
# so just cash manually here since it's what our # so just cash manually here since it's what our
# internals expect. # internals expect.
address: Tuple[str, int] = value self._arb_addr = tuple(value)
self._arb_addr = address
else: else:
setattr(self, attr, value) setattr(self, attr, value)
@ -1132,7 +1186,6 @@ class Actor:
async def _do_handshake( async def _do_handshake(
self, self,
chan: Channel chan: Channel
) -> Tuple[str, str]: ) -> Tuple[str, str]:
"""Exchange (name, UUIDs) identifiers as the first communication step. """Exchange (name, UUIDs) identifiers as the first communication step.
@ -1140,10 +1193,11 @@ class Actor:
parlance. parlance.
""" """
await chan.send(self.uid) await chan.send(self.uid)
uid: Tuple[str, str] = await chan.recv() # breakpoint()
uid: Tuple[str, str] = tuple(await chan.recv())
if not isinstance(uid, tuple): # if not isinstance(uid, tuple):
raise ValueError(f"{uid} is not a valid uid?!") # raise ValueError(f"{uid} is not a valid uid?!")
chan.uid = uid chan.uid = uid
log.runtime(f"Handshake with actor {uid}@{chan.raddr} complete") log.runtime(f"Handshake with actor {uid}@{chan.raddr} complete")
@ -1208,12 +1262,10 @@ class Arbiter(Actor):
return sockaddrs return sockaddrs
async def register_actor( async def register_actor(
self, self, uid: Tuple[str, str], sockaddr: Tuple[str, int]
uid: Tuple[str, str],
sockaddr: Tuple[str, str]
) -> None: ) -> None:
name, uuid = tuple(uid) uid = tuple(uid)
name, uuid = uid
self._registry[uid] = tuple(sockaddr) self._registry[uid] = tuple(sockaddr)
# pop and signal all waiter events # pop and signal all waiter events