Merge pull request #143 from goodboy/ensure_deregister
Ensure actors de-register with arbiter when cancelled during infitinite streaming.matrix
commit
8da45eedf4
|
@ -1,9 +1,13 @@
|
|||
"""
|
||||
``tractor`` testing!!
|
||||
"""
|
||||
import sys
|
||||
import subprocess
|
||||
import os
|
||||
import random
|
||||
import signal
|
||||
import platform
|
||||
import time
|
||||
|
||||
import pytest
|
||||
import tractor
|
||||
|
@ -16,6 +20,19 @@ pytest_plugins = ['pytester']
|
|||
_arb_addr = '127.0.0.1', random.randint(1000, 9999)
|
||||
|
||||
|
||||
# Sending signal.SIGINT on subprocess fails on windows. Use CTRL_* alternatives
|
||||
if platform.system() == 'Windows':
|
||||
_KILL_SIGNAL = signal.CTRL_BREAK_EVENT
|
||||
_INT_SIGNAL = signal.CTRL_C_EVENT
|
||||
_INT_RETURN_CODE = 3221225786
|
||||
_PROC_SPAWN_WAIT = 2
|
||||
else:
|
||||
_KILL_SIGNAL = signal.SIGKILL
|
||||
_INT_SIGNAL = signal.SIGINT
|
||||
_INT_RETURN_CODE = 1 if sys.version_info < (3, 8) else -signal.SIGINT.value
|
||||
_PROC_SPAWN_WAIT = 0.6 if sys.version_info < (3, 7) else 0.4
|
||||
|
||||
|
||||
no_windows = pytest.mark.skipif(
|
||||
platform.system() == "Windows",
|
||||
reason="Test is unsupported on windows",
|
||||
|
@ -89,3 +106,43 @@ def pytest_generate_tests(metafunc):
|
|||
methods = ['trio']
|
||||
|
||||
metafunc.parametrize("start_method", methods, scope='module')
|
||||
|
||||
|
||||
def sig_prog(proc, sig):
|
||||
"Kill the actor-process with ``sig``."
|
||||
proc.send_signal(sig)
|
||||
time.sleep(0.1)
|
||||
if not proc.poll():
|
||||
# TODO: why sometimes does SIGINT not work on teardown?
|
||||
# seems to happen only when trace logging enabled?
|
||||
proc.send_signal(_KILL_SIGNAL)
|
||||
ret = proc.wait()
|
||||
assert ret
|
||||
|
||||
|
||||
@pytest.fixture
|
||||
def daemon(loglevel, testdir, arb_addr):
|
||||
"""Run a daemon actor as a "remote arbiter".
|
||||
"""
|
||||
cmdargs = [
|
||||
sys.executable, '-c',
|
||||
"import tractor; tractor.run_daemon((), arbiter_addr={}, loglevel={})"
|
||||
.format(
|
||||
arb_addr,
|
||||
"'{}'".format(loglevel) if loglevel else None)
|
||||
]
|
||||
kwargs = dict()
|
||||
if platform.system() == 'Windows':
|
||||
# without this, tests hang on windows forever
|
||||
kwargs['creationflags'] = subprocess.CREATE_NEW_PROCESS_GROUP
|
||||
|
||||
proc = testdir.popen(
|
||||
cmdargs,
|
||||
stdout=subprocess.PIPE,
|
||||
stderr=subprocess.PIPE,
|
||||
**kwargs,
|
||||
)
|
||||
assert not proc.returncode
|
||||
time.sleep(_PROC_SPAWN_WAIT)
|
||||
yield proc
|
||||
sig_prog(proc, _INT_SIGNAL)
|
||||
|
|
|
@ -1,6 +1,12 @@
|
|||
"""
|
||||
Actor "discovery" testing
|
||||
"""
|
||||
import os
|
||||
import signal
|
||||
import platform
|
||||
from functools import partial
|
||||
import itertools
|
||||
|
||||
import pytest
|
||||
import tractor
|
||||
import trio
|
||||
|
@ -80,3 +86,145 @@ async def test_trynamic_trio(func, start_method):
|
|||
print(await gretchen.result())
|
||||
print(await donny.result())
|
||||
print("CUTTTT CUUTT CUT!!?! Donny!! You're supposed to say...")
|
||||
|
||||
|
||||
async def stream_forever():
|
||||
for i in itertools.count():
|
||||
yield i
|
||||
await trio.sleep(0.01)
|
||||
|
||||
|
||||
async def cancel(use_signal, delay=0):
|
||||
# hold on there sally
|
||||
await trio.sleep(delay)
|
||||
|
||||
# trigger cancel
|
||||
if use_signal:
|
||||
if platform.system() == 'Windows':
|
||||
pytest.skip("SIGINT not supported on windows")
|
||||
os.kill(os.getpid(), signal.SIGINT)
|
||||
else:
|
||||
raise KeyboardInterrupt
|
||||
|
||||
|
||||
async def stream_from(portal):
|
||||
async for value in await portal.result():
|
||||
print(value)
|
||||
|
||||
|
||||
async def spawn_and_check_registry(
|
||||
arb_addr: tuple,
|
||||
use_signal: bool,
|
||||
remote_arbiter: bool = False,
|
||||
with_streaming: bool = False,
|
||||
) -> None:
|
||||
actor = tractor.current_actor()
|
||||
|
||||
if remote_arbiter:
|
||||
assert not actor.is_arbiter
|
||||
|
||||
async with tractor.get_arbiter(*arb_addr) as portal:
|
||||
if actor.is_arbiter:
|
||||
async def get_reg():
|
||||
return actor._registry
|
||||
extra = 1 # arbiter is local root actor
|
||||
else:
|
||||
get_reg = partial(portal.run, 'self', 'get_registry')
|
||||
extra = 2 # local root actor + remote arbiter
|
||||
|
||||
# ensure current actor is registered
|
||||
registry = await get_reg()
|
||||
assert actor.uid in registry
|
||||
|
||||
if with_streaming:
|
||||
to_run = stream_forever
|
||||
else:
|
||||
to_run = trio.sleep_forever
|
||||
|
||||
async with trio.open_nursery() as trion:
|
||||
try:
|
||||
async with tractor.open_nursery() as n:
|
||||
portals = {}
|
||||
for i in range(3):
|
||||
name = f'a{i}'
|
||||
portals[name] = await n.run_in_actor(name, to_run)
|
||||
|
||||
# wait on last actor to come up
|
||||
async with tractor.wait_for_actor(name):
|
||||
registry = await get_reg()
|
||||
for uid in n._children:
|
||||
assert uid in registry
|
||||
|
||||
assert len(portals) + extra == len(registry)
|
||||
|
||||
if with_streaming:
|
||||
await trio.sleep(0.1)
|
||||
|
||||
pts = list(portals.values())
|
||||
for p in pts[:-1]:
|
||||
trion.start_soon(stream_from, p)
|
||||
|
||||
# stream for 1 sec
|
||||
trion.start_soon(cancel, use_signal, 1)
|
||||
|
||||
last_p = pts[-1]
|
||||
async for value in await last_p.result():
|
||||
print(value)
|
||||
else:
|
||||
await cancel(use_signal)
|
||||
|
||||
finally:
|
||||
with trio.CancelScope(shield=True):
|
||||
await trio.sleep(0.5)
|
||||
|
||||
# all subactors should have de-registered
|
||||
registry = await get_reg()
|
||||
assert len(registry) == extra
|
||||
assert actor.uid in registry
|
||||
|
||||
|
||||
@pytest.mark.parametrize('use_signal', [False, True])
|
||||
@pytest.mark.parametrize('with_streaming', [False, True])
|
||||
def test_subactors_unregister_on_cancel(
|
||||
start_method,
|
||||
use_signal,
|
||||
arb_addr,
|
||||
with_streaming,
|
||||
):
|
||||
"""Verify that cancelling a nursery results in all subactors
|
||||
deregistering themselves with the arbiter.
|
||||
"""
|
||||
with pytest.raises(KeyboardInterrupt):
|
||||
tractor.run(
|
||||
spawn_and_check_registry,
|
||||
arb_addr,
|
||||
use_signal,
|
||||
False,
|
||||
with_streaming,
|
||||
arbiter_addr=arb_addr
|
||||
)
|
||||
|
||||
|
||||
@pytest.mark.parametrize('use_signal', [False, True])
|
||||
@pytest.mark.parametrize('with_streaming', [False, True])
|
||||
def test_subactors_unregister_on_cancel_remote_daemon(
|
||||
daemon,
|
||||
start_method,
|
||||
use_signal,
|
||||
arb_addr,
|
||||
with_streaming,
|
||||
):
|
||||
"""Verify that cancelling a nursery results in all subactors
|
||||
deregistering themselves with a **remote** (not in the local process
|
||||
tree) arbiter.
|
||||
"""
|
||||
with pytest.raises(KeyboardInterrupt):
|
||||
tractor.run(
|
||||
spawn_and_check_registry,
|
||||
arb_addr,
|
||||
use_signal,
|
||||
True,
|
||||
with_streaming,
|
||||
# XXX: required to use remote daemon!
|
||||
arbiter_addr=arb_addr
|
||||
)
|
||||
|
|
|
@ -2,64 +2,16 @@
|
|||
Multiple python programs invoking ``tractor.run()``
|
||||
"""
|
||||
import platform
|
||||
import sys
|
||||
import time
|
||||
import signal
|
||||
import subprocess
|
||||
|
||||
import pytest
|
||||
import tractor
|
||||
from conftest import tractor_test
|
||||
|
||||
# Sending signal.SIGINT on subprocess fails on windows. Use CTRL_* alternatives
|
||||
if platform.system() == 'Windows':
|
||||
_KILL_SIGNAL = signal.CTRL_BREAK_EVENT
|
||||
_INT_SIGNAL = signal.CTRL_C_EVENT
|
||||
_INT_RETURN_CODE = 3221225786
|
||||
_PROC_SPAWN_WAIT = 2
|
||||
else:
|
||||
_KILL_SIGNAL = signal.SIGKILL
|
||||
_INT_SIGNAL = signal.SIGINT
|
||||
_INT_RETURN_CODE = 1 if sys.version_info < (3, 8) else -signal.SIGINT.value
|
||||
_PROC_SPAWN_WAIT = 0.6 if sys.version_info < (3, 7) else 0.4
|
||||
|
||||
|
||||
def sig_prog(proc, sig):
|
||||
"Kill the actor-process with ``sig``."
|
||||
proc.send_signal(sig)
|
||||
time.sleep(0.1)
|
||||
if not proc.poll():
|
||||
# TODO: why sometimes does SIGINT not work on teardown?
|
||||
# seems to happen only when trace logging enabled?
|
||||
proc.send_signal(_KILL_SIGNAL)
|
||||
ret = proc.wait()
|
||||
assert ret
|
||||
|
||||
|
||||
@pytest.fixture
|
||||
def daemon(loglevel, testdir, arb_addr):
|
||||
cmdargs = [
|
||||
sys.executable, '-c',
|
||||
"import tractor; tractor.run_daemon((), arbiter_addr={}, loglevel={})"
|
||||
.format(
|
||||
arb_addr,
|
||||
"'{}'".format(loglevel) if loglevel else None)
|
||||
]
|
||||
kwargs = dict()
|
||||
if platform.system() == 'Windows':
|
||||
# without this, tests hang on windows forever
|
||||
kwargs['creationflags'] = subprocess.CREATE_NEW_PROCESS_GROUP
|
||||
|
||||
proc = testdir.popen(
|
||||
cmdargs,
|
||||
stdout=subprocess.PIPE,
|
||||
stderr=subprocess.PIPE,
|
||||
**kwargs,
|
||||
)
|
||||
assert not proc.returncode
|
||||
time.sleep(_PROC_SPAWN_WAIT)
|
||||
yield proc
|
||||
sig_prog(proc, _INT_SIGNAL)
|
||||
from conftest import (
|
||||
tractor_test,
|
||||
sig_prog,
|
||||
_INT_SIGNAL,
|
||||
_INT_RETURN_CODE,
|
||||
)
|
||||
|
||||
|
||||
def test_abort_on_sigint(daemon):
|
||||
|
@ -67,6 +19,7 @@ def test_abort_on_sigint(daemon):
|
|||
time.sleep(0.1)
|
||||
sig_prog(daemon, _INT_SIGNAL)
|
||||
assert daemon.returncode == _INT_RETURN_CODE
|
||||
|
||||
# XXX: oddly, couldn't get capfd.readouterr() to work here?
|
||||
if platform.system() != 'Windows':
|
||||
# don't check stderr on windows as its empty when sending CTRL_C_EVENT
|
||||
|
|
|
@ -440,7 +440,7 @@ class Actor:
|
|||
f"Cancelling all tasks for {chan} from {chan.uid}")
|
||||
for (channel, cid) in self._rpc_tasks:
|
||||
if channel is chan:
|
||||
self._cancel_task(cid, channel)
|
||||
await self._cancel_task(cid, channel)
|
||||
log.debug(
|
||||
f"Msg loop signalled to terminate for"
|
||||
f" {chan} from {chan.uid}")
|
||||
|
@ -678,7 +678,10 @@ class Actor:
|
|||
|
||||
finally:
|
||||
if registered_with_arbiter:
|
||||
# with trio.move_on_after(3) as cs:
|
||||
# cs.shield = True
|
||||
await self._do_unreg(self._arb_addr)
|
||||
|
||||
# terminate actor once all it's peers (actors that connected
|
||||
# to it as clients) have disappeared
|
||||
if not self._no_more_peers.is_set():
|
||||
|
@ -863,6 +866,17 @@ class Arbiter(Actor):
|
|||
|
||||
return None
|
||||
|
||||
async def get_registry(
|
||||
self
|
||||
) -> Dict[str, Tuple[str, str]]:
|
||||
"""Return current name registry.
|
||||
"""
|
||||
# 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 self._registry
|
||||
|
||||
async def wait_for_actor(
|
||||
self, name: str
|
||||
) -> List[Tuple[str, int]]:
|
||||
|
|
|
@ -23,6 +23,7 @@ async def get_arbiter(
|
|||
arbiter.
|
||||
"""
|
||||
actor = current_actor()
|
||||
|
||||
if not actor:
|
||||
raise RuntimeError("No actor instance has been defined yet?")
|
||||
|
||||
|
@ -38,7 +39,8 @@ async def get_arbiter(
|
|||
|
||||
@asynccontextmanager
|
||||
async def find_actor(
|
||||
name: str, arbiter_sockaddr: Tuple[str, int] = None
|
||||
name: str,
|
||||
arbiter_sockaddr: Tuple[str, int] = None
|
||||
) -> typing.AsyncGenerator[Optional[Portal], None]:
|
||||
"""Ask the arbiter to find actor(s) by name.
|
||||
|
||||
|
|
|
@ -72,8 +72,10 @@ def _trio_main(
|
|||
actor._async_main,
|
||||
parent_addr=parent_addr
|
||||
)
|
||||
|
||||
try:
|
||||
trio.run(trio_main)
|
||||
except KeyboardInterrupt:
|
||||
pass # handle it the same way trio does?
|
||||
log.info(f"Actor {actor.uid} terminated")
|
||||
log.warning(f"Actor {actor.uid} received KBI")
|
||||
|
||||
log.info(f"Actor {actor.uid} terminated")
|
||||
|
|
|
@ -3,6 +3,8 @@ Inter-process comms abstractions
|
|||
"""
|
||||
import typing
|
||||
from typing import Any, Tuple, Optional
|
||||
from functools import partial
|
||||
import inspect
|
||||
|
||||
import msgpack
|
||||
import trio
|
||||
|
@ -11,6 +13,14 @@ from async_generator import asynccontextmanager
|
|||
from .log import get_logger
|
||||
log = get_logger('ipc')
|
||||
|
||||
# :eyeroll:
|
||||
try:
|
||||
import msgpack_numpy
|
||||
Unpacker = msgpack_numpy.Unpacker
|
||||
except ImportError:
|
||||
# just plain ``msgpack`` requires tweaking key settings
|
||||
Unpacker = partial(msgpack.Unpacker, strict_map_key=False)
|
||||
|
||||
|
||||
class MsgpackStream:
|
||||
"""A ``trio.SocketStream`` delivering ``msgpack`` formatted data.
|
||||
|
@ -32,7 +42,10 @@ class MsgpackStream:
|
|||
async def _iter_packets(self) -> typing.AsyncGenerator[dict, None]:
|
||||
"""Yield packets from the underlying stream.
|
||||
"""
|
||||
unpacker = msgpack.Unpacker(raw=False, use_list=False)
|
||||
unpacker = Unpacker(
|
||||
raw=False,
|
||||
use_list=False,
|
||||
)
|
||||
while True:
|
||||
try:
|
||||
data = await self.stream.receive_some(2**10)
|
||||
|
|
|
@ -157,7 +157,6 @@ async def cancel_on_completion(
|
|||
@asynccontextmanager
|
||||
async def spawn_subactor(
|
||||
subactor: 'Actor',
|
||||
accept_addr: Tuple[str, int],
|
||||
parent_addr: Tuple[str, int],
|
||||
):
|
||||
|
||||
|
@ -167,8 +166,13 @@ async def spawn_subactor(
|
|||
# Hardcode this (instead of using ``_child.__name__`` to avoid a
|
||||
# double import warning: https://stackoverflow.com/a/45070583
|
||||
"tractor._child",
|
||||
# We provide the child's unique identifier on this exec/spawn
|
||||
# line for debugging purposes when viewing the process tree from
|
||||
# the OS; it otherwise can be passed via the parent channel if
|
||||
# we prefer in the future (for privacy).
|
||||
"--uid",
|
||||
str(subactor.uid),
|
||||
# Address the child must connect to on startup
|
||||
"--parent_addr",
|
||||
str(parent_addr)
|
||||
]
|
||||
|
@ -179,8 +183,14 @@ async def spawn_subactor(
|
|||
subactor.loglevel
|
||||
]
|
||||
|
||||
async with await trio.open_process(spawn_cmd) as proc:
|
||||
yield proc
|
||||
proc = await trio.open_process(spawn_cmd)
|
||||
yield proc
|
||||
|
||||
# XXX: do this **after** cancellation/tearfown
|
||||
# to avoid killing the process too early
|
||||
# since trio does this internally on ``__aexit__()``
|
||||
async with proc:
|
||||
log.debug(f"Terminating {proc}")
|
||||
|
||||
|
||||
async def new_proc(
|
||||
|
@ -206,7 +216,6 @@ async def new_proc(
|
|||
if use_trio_run_in_process or _spawn_method == 'trio':
|
||||
async with spawn_subactor(
|
||||
subactor,
|
||||
bind_addr,
|
||||
parent_addr,
|
||||
) as proc:
|
||||
log.info(f"Started {proc}")
|
||||
|
|
|
@ -230,7 +230,7 @@ async def open_nursery() -> typing.AsyncGenerator[ActorNursery, None]:
|
|||
f"Waiting on subactors {anursery._children}"
|
||||
"to complete"
|
||||
)
|
||||
except (BaseException, Exception) as err:
|
||||
except BaseException as err:
|
||||
# if the caller's scope errored then we activate our
|
||||
# one-cancels-all supervisor strategy (don't
|
||||
# worry more are coming).
|
||||
|
@ -241,10 +241,11 @@ async def open_nursery() -> typing.AsyncGenerator[ActorNursery, None]:
|
|||
# the `else:` block here might not complete?
|
||||
# For now, shield both.
|
||||
with trio.CancelScope(shield=True):
|
||||
if err in (trio.Cancelled, KeyboardInterrupt):
|
||||
etype = type(err)
|
||||
if etype in (trio.Cancelled, KeyboardInterrupt):
|
||||
log.warning(
|
||||
f"Nursery for {current_actor().uid} was "
|
||||
f"cancelled with {err}")
|
||||
f"cancelled with {etype}")
|
||||
else:
|
||||
log.exception(
|
||||
f"Nursery for {current_actor().uid} "
|
||||
|
|
Loading…
Reference in New Issue