forked from goodboy/tractor
commit
e6763d4daf
|
@ -0,0 +1,6 @@
|
||||||
|
Fix keyboard interrupt handling in ``Portal.open_context()`` blocks.
|
||||||
|
|
||||||
|
Previously this not triggering cancellation of the remote task context
|
||||||
|
and could result in hangs if a stream was also opened. This fix is to
|
||||||
|
accept `BaseException` since it is likely any other top level exception
|
||||||
|
other then kbi (even though not expected) should also get this result.
|
|
@ -2,6 +2,8 @@
|
||||||
Bidirectional streaming and context API.
|
Bidirectional streaming and context API.
|
||||||
|
|
||||||
"""
|
"""
|
||||||
|
import platform
|
||||||
|
|
||||||
import pytest
|
import pytest
|
||||||
import trio
|
import trio
|
||||||
import tractor
|
import tractor
|
||||||
|
@ -51,26 +53,37 @@ async def assert_state(value: bool):
|
||||||
|
|
||||||
@pytest.mark.parametrize(
|
@pytest.mark.parametrize(
|
||||||
'error_parent',
|
'error_parent',
|
||||||
[False, True],
|
[False, ValueError, KeyboardInterrupt],
|
||||||
)
|
)
|
||||||
@pytest.mark.parametrize(
|
@pytest.mark.parametrize(
|
||||||
'callee_blocks_forever',
|
'callee_blocks_forever',
|
||||||
[False, True],
|
[False, True],
|
||||||
|
ids=lambda item: f'callee_blocks_forever={item}'
|
||||||
|
)
|
||||||
|
@pytest.mark.parametrize(
|
||||||
|
'pointlessly_open_stream',
|
||||||
|
[False, True],
|
||||||
|
ids=lambda item: f'open_stream={item}'
|
||||||
)
|
)
|
||||||
def test_simple_context(
|
def test_simple_context(
|
||||||
error_parent,
|
error_parent,
|
||||||
callee_blocks_forever,
|
callee_blocks_forever,
|
||||||
|
pointlessly_open_stream,
|
||||||
):
|
):
|
||||||
|
|
||||||
|
timeout = 1.5 if not platform.system() == 'Windows' else 3
|
||||||
|
|
||||||
async def main():
|
async def main():
|
||||||
|
|
||||||
async with tractor.open_nursery() as n:
|
with trio.fail_after(timeout):
|
||||||
|
async with tractor.open_nursery() as nursery:
|
||||||
|
|
||||||
portal = await n.start_actor(
|
portal = await nursery.start_actor(
|
||||||
'simple_context',
|
'simple_context',
|
||||||
enable_modules=[__name__],
|
enable_modules=[__name__],
|
||||||
)
|
)
|
||||||
|
|
||||||
|
try:
|
||||||
async with portal.open_context(
|
async with portal.open_context(
|
||||||
simple_setup_teardown,
|
simple_setup_teardown,
|
||||||
data=10,
|
data=10,
|
||||||
|
@ -81,15 +94,33 @@ def test_simple_context(
|
||||||
|
|
||||||
if callee_blocks_forever:
|
if callee_blocks_forever:
|
||||||
await portal.run(assert_state, value=True)
|
await portal.run(assert_state, value=True)
|
||||||
await ctx.cancel()
|
|
||||||
else:
|
else:
|
||||||
assert await ctx.result() == 'yo'
|
assert await ctx.result() == 'yo'
|
||||||
|
|
||||||
# after cancellation
|
if not error_parent:
|
||||||
await portal.run(assert_state, value=False)
|
await ctx.cancel()
|
||||||
|
|
||||||
|
if pointlessly_open_stream:
|
||||||
|
async with ctx.open_stream():
|
||||||
if error_parent:
|
if error_parent:
|
||||||
raise ValueError
|
raise error_parent
|
||||||
|
|
||||||
|
if callee_blocks_forever:
|
||||||
|
await ctx.cancel()
|
||||||
|
else:
|
||||||
|
# in this case the stream will send a
|
||||||
|
# 'stop' msg to the far end which needs
|
||||||
|
# to be ignored
|
||||||
|
pass
|
||||||
|
else:
|
||||||
|
if error_parent:
|
||||||
|
raise error_parent
|
||||||
|
|
||||||
|
finally:
|
||||||
|
|
||||||
|
# after cancellation
|
||||||
|
if not error_parent:
|
||||||
|
await portal.run(assert_state, value=False)
|
||||||
|
|
||||||
# shut down daemon
|
# shut down daemon
|
||||||
await portal.cancel_actor()
|
await portal.cancel_actor()
|
||||||
|
@ -97,8 +128,12 @@ def test_simple_context(
|
||||||
if error_parent:
|
if error_parent:
|
||||||
try:
|
try:
|
||||||
trio.run(main)
|
trio.run(main)
|
||||||
except ValueError:
|
except error_parent:
|
||||||
pass
|
pass
|
||||||
|
except trio.MultiError as me:
|
||||||
|
# XXX: on windows it seems we may have to expect the group error
|
||||||
|
from tractor._exceptions import is_multi_cancelled
|
||||||
|
assert is_multi_cancelled(me)
|
||||||
else:
|
else:
|
||||||
trio.run(main)
|
trio.run(main)
|
||||||
|
|
||||||
|
|
|
@ -218,3 +218,54 @@ def test_reqresp_ontopof_streaming():
|
||||||
trio.run(main)
|
trio.run(main)
|
||||||
except trio.TooSlowError:
|
except trio.TooSlowError:
|
||||||
pass
|
pass
|
||||||
|
|
||||||
|
|
||||||
|
async def async_gen_stream(sequence):
|
||||||
|
for i in sequence:
|
||||||
|
yield i
|
||||||
|
await trio.sleep(0.1)
|
||||||
|
|
||||||
|
|
||||||
|
@tractor.context
|
||||||
|
async def echo_ctx_stream(
|
||||||
|
ctx: tractor.Context,
|
||||||
|
) -> None:
|
||||||
|
await ctx.started()
|
||||||
|
|
||||||
|
async with ctx.open_stream() as stream:
|
||||||
|
async for msg in stream:
|
||||||
|
await stream.send(msg)
|
||||||
|
|
||||||
|
|
||||||
|
def test_sigint_both_stream_types():
|
||||||
|
'''Verify that running a bi-directional and recv only stream
|
||||||
|
side-by-side will cancel correctly from SIGINT.
|
||||||
|
|
||||||
|
'''
|
||||||
|
async def main():
|
||||||
|
with trio.fail_after(2):
|
||||||
|
async with tractor.open_nursery() as n:
|
||||||
|
# name of this actor will be same as target func
|
||||||
|
portal = await n.start_actor(
|
||||||
|
'2_way',
|
||||||
|
enable_modules=[__name__]
|
||||||
|
)
|
||||||
|
|
||||||
|
async with portal.open_context(echo_ctx_stream) as (ctx, _):
|
||||||
|
async with ctx.open_stream() as stream:
|
||||||
|
async with portal.open_stream_from(
|
||||||
|
async_gen_stream,
|
||||||
|
sequence=list(range(1)),
|
||||||
|
) as gen_stream:
|
||||||
|
|
||||||
|
msg = await gen_stream.receive()
|
||||||
|
await stream.send(msg)
|
||||||
|
resp = await stream.receive()
|
||||||
|
assert resp == msg
|
||||||
|
raise KeyboardInterrupt
|
||||||
|
|
||||||
|
try:
|
||||||
|
trio.run(main)
|
||||||
|
assert 0, "Didn't receive KBI!?"
|
||||||
|
except KeyboardInterrupt:
|
||||||
|
pass
|
||||||
|
|
|
@ -1,5 +1,6 @@
|
||||||
"""
|
"""
|
||||||
Broadcast channels for fan-out to local tasks.
|
Broadcast channels for fan-out to local tasks.
|
||||||
|
|
||||||
"""
|
"""
|
||||||
from contextlib import asynccontextmanager
|
from contextlib import asynccontextmanager
|
||||||
from functools import partial
|
from functools import partial
|
||||||
|
@ -332,6 +333,9 @@ def test_ensure_slow_consumers_lag_out(
|
||||||
await trio.sleep(delay)
|
await trio.sleep(delay)
|
||||||
|
|
||||||
if task.name == 'sub_1':
|
if task.name == 'sub_1':
|
||||||
|
# trigger checkpoint to clean out other subs
|
||||||
|
await trio.sleep(0)
|
||||||
|
|
||||||
# the non-lagger got
|
# the non-lagger got
|
||||||
# a ``trio.EndOfChannel``
|
# a ``trio.EndOfChannel``
|
||||||
# because the ``tx`` below was closed
|
# because the ``tx`` below was closed
|
||||||
|
|
|
@ -177,7 +177,6 @@ class Portal:
|
||||||
f"Cancelling all streams with {self.channel.uid}")
|
f"Cancelling all streams with {self.channel.uid}")
|
||||||
for stream in self._streams.copy():
|
for stream in self._streams.copy():
|
||||||
try:
|
try:
|
||||||
# with trio.CancelScope(shield=True):
|
|
||||||
await stream.aclose()
|
await stream.aclose()
|
||||||
except trio.ClosedResourceError:
|
except trio.ClosedResourceError:
|
||||||
# don't error the stream having already been closed
|
# don't error the stream having already been closed
|
||||||
|
@ -294,7 +293,6 @@ class Portal:
|
||||||
async def open_stream_from(
|
async def open_stream_from(
|
||||||
self,
|
self,
|
||||||
async_gen_func: Callable, # typing: ignore
|
async_gen_func: Callable, # typing: ignore
|
||||||
shield: bool = False,
|
|
||||||
**kwargs,
|
**kwargs,
|
||||||
|
|
||||||
) -> AsyncGenerator[ReceiveMsgStream, None]:
|
) -> AsyncGenerator[ReceiveMsgStream, None]:
|
||||||
|
@ -318,11 +316,17 @@ class Portal:
|
||||||
# receive only stream
|
# receive only stream
|
||||||
assert functype == 'asyncgen'
|
assert functype == 'asyncgen'
|
||||||
|
|
||||||
ctx = Context(self.channel, cid, _portal=self)
|
ctx = Context(
|
||||||
|
self.channel,
|
||||||
|
cid,
|
||||||
|
# do we need this to be closed implicitly?
|
||||||
|
# _recv_chan=recv_chan,
|
||||||
|
_portal=self
|
||||||
|
)
|
||||||
try:
|
try:
|
||||||
# deliver receive only stream
|
# deliver receive only stream
|
||||||
async with ReceiveMsgStream(
|
async with ReceiveMsgStream(
|
||||||
ctx, recv_chan, shield=shield
|
ctx, recv_chan,
|
||||||
) as rchan:
|
) as rchan:
|
||||||
self._streams.add(rchan)
|
self._streams.add(rchan)
|
||||||
yield rchan
|
yield rchan
|
||||||
|
@ -337,13 +341,16 @@ class Portal:
|
||||||
# message right now since there shouldn't be a reason to
|
# message right now since there shouldn't be a reason to
|
||||||
# stop and restart the stream, right?
|
# stop and restart the stream, right?
|
||||||
try:
|
try:
|
||||||
|
with trio.CancelScope(shield=True):
|
||||||
await ctx.cancel()
|
await ctx.cancel()
|
||||||
|
|
||||||
except trio.ClosedResourceError:
|
except trio.ClosedResourceError:
|
||||||
# if the far end terminates before we send a cancel the
|
# if the far end terminates before we send a cancel the
|
||||||
# underlying transport-channel may already be closed.
|
# underlying transport-channel may already be closed.
|
||||||
log.debug(f'Context {ctx} was already closed?')
|
log.warning(f'Context {ctx} was already closed?')
|
||||||
|
|
||||||
|
# XXX: should this always be done?
|
||||||
|
# await recv_chan.aclose()
|
||||||
self._streams.remove(rchan)
|
self._streams.remove(rchan)
|
||||||
|
|
||||||
@asynccontextmanager
|
@asynccontextmanager
|
||||||
|
@ -408,8 +415,8 @@ class Portal:
|
||||||
|
|
||||||
# pairs with handling in ``Actor._push_result()``
|
# pairs with handling in ``Actor._push_result()``
|
||||||
# recv_chan._ctx = ctx
|
# recv_chan._ctx = ctx
|
||||||
|
|
||||||
# await trio.lowlevel.checkpoint()
|
# await trio.lowlevel.checkpoint()
|
||||||
|
|
||||||
yield ctx, first
|
yield ctx, first
|
||||||
|
|
||||||
except ContextCancelled as err:
|
except ContextCancelled as err:
|
||||||
|
@ -427,9 +434,14 @@ class Portal:
|
||||||
log.debug(f'Context {ctx} cancelled gracefully')
|
log.debug(f'Context {ctx} cancelled gracefully')
|
||||||
|
|
||||||
except (
|
except (
|
||||||
trio.Cancelled,
|
BaseException,
|
||||||
trio.MultiError,
|
|
||||||
Exception,
|
# more specifically, we need to handle:
|
||||||
|
# Exception,
|
||||||
|
# trio.Cancelled,
|
||||||
|
# trio.MultiError,
|
||||||
|
# KeyboardInterrupt,
|
||||||
|
|
||||||
) as err:
|
) as err:
|
||||||
_err = err
|
_err = err
|
||||||
# the context cancels itself on any cancel
|
# the context cancels itself on any cancel
|
||||||
|
@ -440,6 +452,11 @@ class Portal:
|
||||||
raise
|
raise
|
||||||
|
|
||||||
finally:
|
finally:
|
||||||
|
# in the case where a runtime nursery (due to internal bug)
|
||||||
|
# or a remote actor transmits an error we want to be
|
||||||
|
# sure we get the error the underlying feeder mem chan.
|
||||||
|
# if it's not raised here it *should* be raised from the
|
||||||
|
# msg loop nursery right?
|
||||||
result = await ctx.result()
|
result = await ctx.result()
|
||||||
|
|
||||||
# though it should be impossible for any tasks
|
# though it should be impossible for any tasks
|
||||||
|
|
|
@ -51,7 +51,6 @@ class ReceiveMsgStream(trio.abc.ReceiveChannel):
|
||||||
self,
|
self,
|
||||||
ctx: 'Context', # typing: ignore # noqa
|
ctx: 'Context', # typing: ignore # noqa
|
||||||
rx_chan: trio.MemoryReceiveChannel,
|
rx_chan: trio.MemoryReceiveChannel,
|
||||||
shield: bool = False,
|
|
||||||
_broadcaster: Optional[BroadcastReceiver] = None,
|
_broadcaster: Optional[BroadcastReceiver] = None,
|
||||||
|
|
||||||
) -> None:
|
) -> None:
|
||||||
|
@ -295,6 +294,9 @@ class MsgStream(ReceiveMsgStream, trio.abc.Channel):
|
||||||
'''Send a message over this stream to the far end.
|
'''Send a message over this stream to the far end.
|
||||||
|
|
||||||
'''
|
'''
|
||||||
|
# if self._eoc:
|
||||||
|
# raise trio.ClosedResourceError('This stream is already ded')
|
||||||
|
|
||||||
await self._ctx.chan.send({'yield': data, 'cid': self._ctx.cid})
|
await self._ctx.chan.send({'yield': data, 'cid': self._ctx.cid})
|
||||||
|
|
||||||
|
|
||||||
|
@ -365,7 +367,7 @@ class Context:
|
||||||
'''
|
'''
|
||||||
side = 'caller' if self._portal else 'callee'
|
side = 'caller' if self._portal else 'callee'
|
||||||
|
|
||||||
log.warning(f'Cancelling {side} side of context to {self.chan}')
|
log.warning(f'Cancelling {side} side of context to {self.chan.uid}')
|
||||||
|
|
||||||
self._cancel_called = True
|
self._cancel_called = True
|
||||||
|
|
||||||
|
@ -396,6 +398,10 @@ class Context:
|
||||||
log.warning(
|
log.warning(
|
||||||
"May have failed to cancel remote task "
|
"May have failed to cancel remote task "
|
||||||
f"{cid} for {self._portal.channel.uid}")
|
f"{cid} for {self._portal.channel.uid}")
|
||||||
|
else:
|
||||||
|
log.warning(
|
||||||
|
"Timed out on cancelling remote task "
|
||||||
|
f"{cid} for {self._portal.channel.uid}")
|
||||||
else:
|
else:
|
||||||
# callee side remote task
|
# callee side remote task
|
||||||
|
|
||||||
|
@ -439,16 +445,6 @@ class Context:
|
||||||
# here we create a mem chan that corresponds to the
|
# here we create a mem chan that corresponds to the
|
||||||
# far end caller / callee.
|
# far end caller / callee.
|
||||||
|
|
||||||
# NOTE: in one way streaming this only happens on the
|
|
||||||
# caller side inside `Actor.send_cmd()` so if you try
|
|
||||||
# to send a stop from the caller to the callee in the
|
|
||||||
# single-direction-stream case you'll get a lookup error
|
|
||||||
# currently.
|
|
||||||
_, recv_chan = actor.get_memchans(
|
|
||||||
self.chan.uid,
|
|
||||||
self.cid
|
|
||||||
)
|
|
||||||
|
|
||||||
# Likewise if the surrounding context has been cancelled we error here
|
# Likewise if the surrounding context has been cancelled we error here
|
||||||
# since it likely means the surrounding block was exited or
|
# since it likely means the surrounding block was exited or
|
||||||
# killed
|
# killed
|
||||||
|
@ -459,6 +455,16 @@ class Context:
|
||||||
f'Context around {actor.uid[0]}:{task} was already cancelled!'
|
f'Context around {actor.uid[0]}:{task} was already cancelled!'
|
||||||
)
|
)
|
||||||
|
|
||||||
|
# NOTE: in one way streaming this only happens on the
|
||||||
|
# caller side inside `Actor.send_cmd()` so if you try
|
||||||
|
# to send a stop from the caller to the callee in the
|
||||||
|
# single-direction-stream case you'll get a lookup error
|
||||||
|
# currently.
|
||||||
|
_, recv_chan = actor.get_memchans(
|
||||||
|
self.chan.uid,
|
||||||
|
self.cid
|
||||||
|
)
|
||||||
|
|
||||||
# XXX: If the underlying channel feeder receive mem chan has
|
# XXX: If the underlying channel feeder receive mem chan has
|
||||||
# been closed then likely client code has already exited
|
# been closed then likely client code has already exited
|
||||||
# a ``.open_stream()`` block prior or there was some other
|
# a ``.open_stream()`` block prior or there was some other
|
||||||
|
@ -482,12 +488,6 @@ class Context:
|
||||||
# await trio.lowlevel.checkpoint()
|
# await trio.lowlevel.checkpoint()
|
||||||
yield rchan
|
yield rchan
|
||||||
|
|
||||||
except trio.EndOfChannel:
|
|
||||||
# likely the far end sent us a 'stop' message to
|
|
||||||
# terminate the stream.
|
|
||||||
raise
|
|
||||||
|
|
||||||
else:
|
|
||||||
# XXX: Make the stream "one-shot use". On exit, signal
|
# XXX: Make the stream "one-shot use". On exit, signal
|
||||||
# ``trio.EndOfChannel``/``StopAsyncIteration`` to the
|
# ``trio.EndOfChannel``/``StopAsyncIteration`` to the
|
||||||
# far end.
|
# far end.
|
||||||
|
|
|
@ -52,6 +52,8 @@ BOLD_PALETTE = {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
# TODO: this isn't showing the correct '{filename}'
|
||||||
|
# as it did before..
|
||||||
class StackLevelAdapter(logging.LoggerAdapter):
|
class StackLevelAdapter(logging.LoggerAdapter):
|
||||||
|
|
||||||
def transport(
|
def transport(
|
||||||
|
|
Loading…
Reference in New Issue