diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 9d91f17d..571bd1db 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -20,7 +20,7 @@ jobs: - name: Setup python uses: actions/setup-python@v2 with: - python-version: '3.10' + python-version: '3.11' - name: Install dependencies run: pip install -U . --upgrade-strategy eager -r requirements-test.txt @@ -41,7 +41,7 @@ jobs: - name: Setup python uses: actions/setup-python@v2 with: - python-version: '3.10' + python-version: '3.11' - name: Build sdist run: python setup.py sdist --formats=zip @@ -59,7 +59,7 @@ jobs: fail-fast: false matrix: os: [ubuntu-latest] - python: ['3.10'] + python: ['3.11'] spawn_backend: [ 'trio', 'mp_spawn', diff --git a/examples/advanced_faults/ipc_failure_during_stream.py b/examples/advanced_faults/ipc_failure_during_stream.py index 6728b8d2..9dca92b1 100644 --- a/examples/advanced_faults/ipc_failure_during_stream.py +++ b/examples/advanced_faults/ipc_failure_during_stream.py @@ -6,47 +6,115 @@ been an outage) and we want to ensure that despite being in debug mode actor tree will eventually be cancelled without leaving any zombies. ''' -import trio +from contextlib import asynccontextmanager as acm +from functools import partial + from tractor import ( open_nursery, context, Context, + ContextCancelled, MsgStream, + _testing, ) +import trio +import pytest -async def break_channel_silently_then_error( +async def break_ipc( stream: MsgStream, -): - async for msg in stream: - await stream.send(msg) + method: str|None = None, + pre_close: bool = False, - # XXX: close the channel right after an error is raised - # purposely breaking the IPC transport to make sure the parent - # doesn't get stuck in debug or hang on the connection join. - # this more or less simulates an infinite msg-receive hang on - # the other end. - await stream._ctx.chan.send(None) - assert 0 + def_method: str = 'eof', +) -> None: + ''' + XXX: close the channel right after an error is raised + purposely breaking the IPC transport to make sure the parent + doesn't get stuck in debug or hang on the connection join. + this more or less simulates an infinite msg-receive hang on + the other end. -async def close_stream_and_error( - stream: MsgStream, -): - async for msg in stream: - await stream.send(msg) - - # wipe out channel right before raising - await stream._ctx.chan.send(None) + ''' + # close channel via IPC prot msging before + # any transport breakage + if pre_close: await stream.aclose() - assert 0 + + method: str = method or def_method + print( + '#################################\n' + 'Simulating CHILD-side IPC BREAK!\n' + f'method: {method}\n' + f'pre `.aclose()`: {pre_close}\n' + '#################################\n' + ) + + match method: + case 'trans_aclose': + await stream._ctx.chan.transport.stream.aclose() + + case 'eof': + await stream._ctx.chan.transport.stream.send_eof() + + case 'msg': + await stream._ctx.chan.send(None) + + # TODO: the actual real-world simulated cases like + # transport layer hangs and/or lower layer 2-gens type + # scenarios.. + # + # -[ ] already have some issues for this general testing + # area: + # - https://github.com/goodboy/tractor/issues/97 + # - https://github.com/goodboy/tractor/issues/124 + # - PR from @guille: + # https://github.com/goodboy/tractor/pull/149 + # case 'hang': + # TODO: framework research: + # + # - https://github.com/GuoTengda1993/pynetem + # - https://github.com/shopify/toxiproxy + # - https://manpages.ubuntu.com/manpages/trusty/man1/wirefilter.1.html + + case _: + raise RuntimeError( + f'IPC break method unsupported: {method}' + ) + + +async def break_ipc_then_error( + stream: MsgStream, + break_ipc_with: str|None = None, + pre_close: bool = False, +): + await break_ipc( + stream=stream, + method=break_ipc_with, + pre_close=pre_close, + ) + async for msg in stream: + await stream.send(msg) + + assert 0 + + +async def iter_ipc_stream( + stream: MsgStream, + break_ipc_with: str|None = None, + pre_close: bool = False, +): + async for msg in stream: + await stream.send(msg) @context async def recv_and_spawn_net_killers( ctx: Context, - break_ipc_after: bool | int = False, + break_ipc_after: bool|int = False, + pre_close: bool = False, ) -> None: ''' @@ -61,26 +129,53 @@ async def recv_and_spawn_net_killers( async for i in stream: print(f'child echoing {i}') await stream.send(i) + if ( break_ipc_after - and i > break_ipc_after + and + i >= break_ipc_after ): - '#################################\n' - 'Simulating child-side IPC BREAK!\n' - '#################################' - n.start_soon(break_channel_silently_then_error, stream) - n.start_soon(close_stream_and_error, stream) + n.start_soon( + iter_ipc_stream, + stream, + ) + n.start_soon( + partial( + break_ipc_then_error, + stream=stream, + pre_close=pre_close, + ) + ) + + +@acm +async def stuff_hangin_ctlc(timeout: float = 1) -> None: + + with trio.move_on_after(timeout) as cs: + yield timeout + + if cs.cancelled_caught: + # pretend to be a user seeing no streaming action + # thinking it's a hang, and then hitting ctl-c.. + print( + f"i'm a user on the PARENT side and thingz hangin " + f'after timeout={timeout} ???\n\n' + 'MASHING CTlR-C..!?\n' + ) + raise KeyboardInterrupt async def main( debug_mode: bool = False, start_method: str = 'trio', + loglevel: str = 'cancel', # by default we break the parent IPC first (if configured to break # at all), but this can be changed so the child does first (even if # both are set to break). - break_parent_ipc_after: int | bool = False, - break_child_ipc_after: int | bool = False, + break_parent_ipc_after: int|bool = False, + break_child_ipc_after: int|bool = False, + pre_close: bool = False, ) -> None: @@ -91,60 +186,123 @@ async def main( # NOTE: even debugger is used we shouldn't get # a hang since it never engages due to broken IPC debug_mode=debug_mode, - loglevel='warning', + loglevel=loglevel, ) as an, ): + sub_name: str = 'chitty_hijo' portal = await an.start_actor( - 'chitty_hijo', + sub_name, enable_modules=[__name__], ) - async with portal.open_context( - recv_and_spawn_net_killers, - break_ipc_after=break_child_ipc_after, + async with ( + stuff_hangin_ctlc(timeout=2) as timeout, + _testing.expect_ctxc( + yay=( + break_parent_ipc_after + or break_child_ipc_after + ), + # TODO: we CAN'T remove this right? + # since we need the ctxc to bubble up from either + # the stream API after the `None` msg is sent + # (which actually implicitly cancels all remote + # tasks in the hijo) or from simluated + # KBI-mash-from-user + # or should we expect that a KBI triggers the ctxc + # and KBI in an eg? + reraise=True, + ), - ) as (ctx, sent): + portal.open_context( + recv_and_spawn_net_killers, + break_ipc_after=break_child_ipc_after, + pre_close=pre_close, + ) as (ctx, sent), + ): + rx_eoc: bool = False + ipc_break_sent: bool = False async with ctx.open_stream() as stream: for i in range(1000): if ( break_parent_ipc_after - and i > break_parent_ipc_after + and + i > break_parent_ipc_after + and + not ipc_break_sent ): print( '#################################\n' - 'Simulating parent-side IPC BREAK!\n' - '#################################' + 'Simulating PARENT-side IPC BREAK!\n' + '#################################\n' ) - await stream._ctx.chan.send(None) + + # TODO: other methods? see break func above. + # await stream._ctx.chan.send(None) + # await stream._ctx.chan.transport.stream.send_eof() + await stream._ctx.chan.transport.stream.aclose() + + ipc_break_sent = True # it actually breaks right here in the # mp_spawn/forkserver backends and thus the zombie # reaper never even kicks in? print(f'parent sending {i}') - await stream.send(i) + try: + await stream.send(i) + except ContextCancelled as ctxc: + print( + 'parent received ctxc on `stream.send()`\n' + f'{ctxc}\n' + ) + assert 'root' in ctxc.canceller + assert sub_name in ctx.canceller - with trio.move_on_after(2) as cs: + # TODO: is this needed or no? + raise + # timeout: int = 1 + # with trio.move_on_after(timeout) as cs: + async with stuff_hangin_ctlc() as timeout: + print( + f'PARENT `stream.receive()` with timeout={timeout}\n' + ) # NOTE: in the parent side IPC failure case this # will raise an ``EndOfChannel`` after the child # is killed and sends a stop msg back to it's # caller/this-parent. - rx = await stream.receive() + try: + rx = await stream.receive() + print( + "I'm a happy PARENT user and echoed to me is\n" + f'{rx}\n' + ) + except trio.EndOfChannel: + rx_eoc: bool = True + print('MsgStream got EoC for PARENT') + raise - print(f"I'm a happy user and echoed to me is {rx}") + print( + 'Streaming finished and we got Eoc.\n' + 'Canceling `.open_context()` in root with\n' + 'CTlR-C..' + ) + if rx_eoc: + assert stream.closed + try: + await stream.send(i) + pytest.fail('stream not closed?') + except ( + trio.ClosedResourceError, + trio.EndOfChannel, + ) as send_err: + if rx_eoc: + assert send_err is stream._eoc + else: + assert send_err is stream._closed - if cs.cancelled_caught: - # pretend to be a user seeing no streaming action - # thinking it's a hang, and then hitting ctl-c.. - print("YOO i'm a user anddd thingz hangin..") - - print( - "YOO i'm mad send side dun but thingz hangin..\n" - 'MASHING CTlR-C Ctl-c..' - ) - raise KeyboardInterrupt + raise KeyboardInterrupt if __name__ == '__main__': diff --git a/examples/debugging/debug_mode_hang.py b/examples/debugging/debug_mode_hang.py new file mode 100644 index 00000000..a81890ed --- /dev/null +++ b/examples/debugging/debug_mode_hang.py @@ -0,0 +1,9 @@ +''' +Reproduce a bug where enabling debug mode for a sub-actor actually causes +a hang on teardown... + +''' +import asyncio + +import trio +import tractor diff --git a/examples/full_fledged_streaming_service.py b/examples/full_fledged_streaming_service.py index 1650b583..c93df242 100644 --- a/examples/full_fledged_streaming_service.py +++ b/examples/full_fledged_streaming_service.py @@ -65,21 +65,28 @@ async def aggregate(seed): print("AGGREGATOR COMPLETE!") -# this is the main actor and *arbiter* -async def main(): - # a nursery which spawns "actors" - async with tractor.open_nursery( - arbiter_addr=('127.0.0.1', 1616) - ) as nursery: +async def main() -> list[int]: + ''' + This is the "root" actor's main task's entrypoint. + + By default (and if not otherwise specified) that root process + also acts as a "registry actor" / "registrar" on the localhost + for the purposes of multi-actor "service discovery". + + ''' + # yes, a nursery which spawns `trio`-"actors" B) + nursery: tractor.ActorNursery + async with tractor.open_nursery() as nursery: seed = int(1e3) pre_start = time.time() - portal = await nursery.start_actor( + portal: tractor.Portal = await nursery.start_actor( name='aggregator', enable_modules=[__name__], ) + stream: tractor.MsgStream async with portal.open_stream_from( aggregate, seed=seed, diff --git a/examples/parallelism/concurrent_actors_primes.py b/examples/parallelism/concurrent_actors_primes.py index feaaca79..748861e6 100644 --- a/examples/parallelism/concurrent_actors_primes.py +++ b/examples/parallelism/concurrent_actors_primes.py @@ -8,7 +8,10 @@ This uses no extra threads, fancy semaphores or futures; all we need is ``tractor``'s channels. """ -from contextlib import asynccontextmanager +from contextlib import ( + asynccontextmanager as acm, + aclosing, +) from typing import Callable import itertools import math @@ -16,7 +19,6 @@ import time import tractor import trio -from async_generator import aclosing PRIMES = [ @@ -44,7 +46,7 @@ async def is_prime(n): return True -@asynccontextmanager +@acm async def worker_pool(workers=4): """Though it's a trivial special case for ``tractor``, the well known "worker pool" seems to be the defacto "but, I want this diff --git a/examples/rpc_bidir_streaming.py b/examples/rpc_bidir_streaming.py index 73200814..c961bf20 100644 --- a/examples/rpc_bidir_streaming.py +++ b/examples/rpc_bidir_streaming.py @@ -13,7 +13,7 @@ async def simple_rpc( ''' # signal to parent that we're up much like - # ``trio_typing.TaskStatus.started()`` + # ``trio.TaskStatus.started()`` await ctx.started(data + 1) async with ctx.open_stream() as stream: diff --git a/pyproject.toml b/pyproject.toml index e52aa476..84633806 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -26,3 +26,23 @@ all_bullets = true directory = "trivial" name = "Trivial/Internal Changes" showcontent = true + + +[tool.pytest.ini_options] +minversion = '6.0' +testpaths = [ + 'tests' +] +addopts = [ + # TODO: figure out why this isn't working.. + '--rootdir=./tests', + + '--import-mode=importlib', + # don't show frickin captured logs AGAIN in the report.. + '--show-capture=no', +] +log_cli = false + +# TODO: maybe some of these layout choices? +# https://docs.pytest.org/en/8.0.x/explanation/goodpractices.html#choosing-a-test-layout-import-rules +# pythonpath = "src" diff --git a/setup.py b/setup.py index d26deb9b..a4e5e1ed 100755 --- a/setup.py +++ b/setup.py @@ -36,18 +36,21 @@ setup( platforms=['linux', 'windows'], packages=[ 'tractor', - 'tractor.experimental', - 'tractor.trionics', + 'tractor.experimental', # wacky ideas + 'tractor.trionics', # trio extensions + 'tractor.msg', # lowlevel data types + 'tractor._testing', # internal cross-subsys suite utils ], install_requires=[ # trio related # proper range spec: # https://packaging.python.org/en/latest/discussions/install-requires-vs-requirements/#id5 - 'trio >= 0.22', - 'async_generator', - 'trio_typing', - 'exceptiongroup', + 'trio == 0.24', + + # 'async_generator', # in stdlib mostly! + # 'trio_typing', # trio==0.23.0 has type hints! + # 'exceptiongroup', # in stdlib as of 3.11! # tooling 'tricycle', @@ -73,7 +76,7 @@ setup( ], tests_require=['pytest'], - python_requires=">=3.10", + python_requires=">=3.11", keywords=[ 'trio', 'async', diff --git a/tests/conftest.py b/tests/conftest.py index 3363cf56..01811b56 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -7,94 +7,19 @@ import os import random import signal import platform -import pathlib import time -import inspect -from functools import partial, wraps import pytest -import trio import tractor +from tractor._testing import ( + examples_dir as examples_dir, + tractor_test as tractor_test, + expect_ctxc as expect_ctxc, +) +# TODO: include wtv plugin(s) we build in `._testing.pytest`? pytest_plugins = ['pytester'] - -def tractor_test(fn): - """ - Use: - - @tractor_test - async def test_whatever(): - await ... - - If fixtures: - - - ``arb_addr`` (a socket addr tuple where arbiter is listening) - - ``loglevel`` (logging level passed to tractor internals) - - ``start_method`` (subprocess spawning backend) - - are defined in the `pytest` fixture space they will be automatically - injected to tests declaring these funcargs. - """ - @wraps(fn) - def wrapper( - *args, - loglevel=None, - arb_addr=None, - start_method=None, - **kwargs - ): - # __tracebackhide__ = True - - if 'arb_addr' in inspect.signature(fn).parameters: - # injects test suite fixture value to test as well - # as `run()` - kwargs['arb_addr'] = arb_addr - - if 'loglevel' in inspect.signature(fn).parameters: - # allows test suites to define a 'loglevel' fixture - # that activates the internal logging - kwargs['loglevel'] = loglevel - - if start_method is None: - if platform.system() == "Windows": - start_method = 'trio' - - if 'start_method' in inspect.signature(fn).parameters: - # set of subprocess spawning backends - kwargs['start_method'] = start_method - - if kwargs: - - # use explicit root actor start - - async def _main(): - async with tractor.open_root_actor( - # **kwargs, - arbiter_addr=arb_addr, - loglevel=loglevel, - start_method=start_method, - - # TODO: only enable when pytest is passed --pdb - # debug_mode=True, - - ): - await fn(*args, **kwargs) - - main = _main - - else: - # use implicit root actor start - main = partial(fn, *args, **kwargs) - - return trio.run(main) - - return wrapper - - -_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 @@ -114,41 +39,45 @@ no_windows = pytest.mark.skipif( ) -def repodir() -> pathlib.Path: - ''' - Return the abspath to the repo directory. - - ''' - # 2 parents up to step up through tests/ - return pathlib.Path(__file__).parent.parent.absolute() - - -def examples_dir() -> pathlib.Path: - ''' - Return the abspath to the examples directory as `pathlib.Path`. - - ''' - return repodir() / 'examples' - - def pytest_addoption(parser): parser.addoption( - "--ll", action="store", dest='loglevel', + "--ll", + action="store", + dest='loglevel', default='ERROR', help="logging level to set when testing" ) parser.addoption( - "--spawn-backend", action="store", dest='spawn_backend', + "--spawn-backend", + action="store", + dest='spawn_backend', default='trio', help="Processing spawning backend to use for test run", ) + parser.addoption( + "--tpdb", "--debug-mode", + action="store_true", + dest='tractor_debug_mode', + # default=False, + help=( + 'Enable a flag that can be used by tests to to set the ' + '`debug_mode: bool` for engaging the internal ' + 'multi-proc debugger sys.' + ), + ) + def pytest_configure(config): backend = config.option.spawn_backend tractor._spawn.try_set_start_method(backend) +@pytest.fixture(scope='session') +def debug_mode(request): + return request.config.option.tractor_debug_mode + + @pytest.fixture(scope='session', autouse=True) def loglevel(request): orig = tractor.log._default_loglevel @@ -168,11 +97,26 @@ _ci_env: bool = os.environ.get('CI', False) @pytest.fixture(scope='session') def ci_env() -> bool: - """Detect CI envoirment. - """ + ''' + Detect CI envoirment. + + ''' return _ci_env +# TODO: also move this to `._testing` for now? +# -[ ] possibly generalize and re-use for multi-tree spawning +# along with the new stuff for multi-addrs in distribute_dis +# branch? +# +# choose randomly at import time +_reg_addr: tuple[str, int] = ( + '127.0.0.1', + random.randint(1000, 9999), +) +_arb_addr = _reg_addr + + @pytest.fixture(scope='session') def arb_addr(): return _arb_addr @@ -212,6 +156,7 @@ def sig_prog(proc, sig): assert ret +# TODO: factor into @cm and move to `._testing`? @pytest.fixture def daemon( loglevel: str, diff --git a/tests/test_advanced_faults.py b/tests/test_advanced_faults.py index a48866ea..5f73ac6c 100644 --- a/tests/test_advanced_faults.py +++ b/tests/test_advanced_faults.py @@ -3,22 +3,29 @@ Sketchy network blackoutz, ugly byzantine gens, puedes eschuchar la cancelacion?.. ''' +import itertools from functools import partial +from types import ModuleType import pytest from _pytest.pathlib import import_path import trio import tractor - -from conftest import ( +from tractor._testing import ( examples_dir, ) @pytest.mark.parametrize( - 'debug_mode', - [False, True], - ids=['no_debug_mode', 'debug_mode'], + 'pre_aclose_msgstream', + [ + False, + True, + ], + ids=[ + 'no_msgstream_aclose', + 'pre_aclose_msgstream', + ], ) @pytest.mark.parametrize( 'ipc_break', @@ -63,8 +70,10 @@ from conftest import ( ) def test_ipc_channel_break_during_stream( debug_mode: bool, + loglevel: str, spawn_backend: str, - ipc_break: dict | None, + ipc_break: dict|None, + pre_aclose_msgstream: bool, ): ''' Ensure we can have an IPC channel break its connection during @@ -83,70 +92,130 @@ def test_ipc_channel_break_during_stream( # requires the user to do ctl-c to cancel the actor tree. expect_final_exc = trio.ClosedResourceError - mod = import_path( + mod: ModuleType = import_path( examples_dir() / 'advanced_faults' / 'ipc_failure_during_stream.py', root=examples_dir(), ) - expect_final_exc = KeyboardInterrupt - - # when ONLY the child breaks we expect the parent to get a closed - # resource error on the next `MsgStream.receive()` and then fail out - # and cancel the child from there. + # by def we expect KBI from user after a simulated "hang + # period" wherein the user eventually hits ctl-c to kill the + # root-actor tree. + expect_final_exc: BaseException = KeyboardInterrupt if ( + # only expect EoC if trans is broken on the child side, + ipc_break['break_child_ipc_after'] is not False + # AND we tell the child to call `MsgStream.aclose()`. + and pre_aclose_msgstream + ): + # expect_final_exc = trio.EndOfChannel + # ^XXX NOPE! XXX^ since now `.open_stream()` absorbs this + # gracefully! + expect_final_exc = KeyboardInterrupt - # only child breaks - ( - ipc_break['break_child_ipc_after'] - and ipc_break['break_parent_ipc_after'] is False - ) - - # both break but, parent breaks first - or ( - ipc_break['break_child_ipc_after'] is not False - and ( - ipc_break['break_parent_ipc_after'] - > ipc_break['break_child_ipc_after'] - ) + # NOTE when ONLY the child breaks or it breaks BEFORE the + # parent we expect the parent to get a closed resource error + # on the next `MsgStream.receive()` and then fail out and + # cancel the child from there. + # + # ONLY CHILD breaks + if ( + ipc_break['break_child_ipc_after'] + and + ipc_break['break_parent_ipc_after'] is False + ): + # NOTE: we DO NOT expect this any more since + # the child side's channel will be broken silently + # and nothing on the parent side will indicate this! + # expect_final_exc = trio.ClosedResourceError + + # NOTE: child will send a 'stop' msg before it breaks + # the transport channel BUT, that will be absorbed by the + # `ctx.open_stream()` block and thus the `.open_context()` + # should hang, after which the test script simulates + # a user sending ctl-c by raising a KBI. + if pre_aclose_msgstream: + expect_final_exc = KeyboardInterrupt + + # XXX OLD XXX + # if child calls `MsgStream.aclose()` then expect EoC. + # ^ XXX not any more ^ since eoc is always absorbed + # gracefully and NOT bubbled to the `.open_context()` + # block! + # expect_final_exc = trio.EndOfChannel + + # BOTH but, CHILD breaks FIRST + elif ( + ipc_break['break_child_ipc_after'] is not False + and ( + ipc_break['break_parent_ipc_after'] + > ipc_break['break_child_ipc_after'] ) + ): + if pre_aclose_msgstream: + expect_final_exc = KeyboardInterrupt + # NOTE when the parent IPC side dies (even if the child's does as well + # but the child fails BEFORE the parent) we always expect the + # IPC layer to raise a closed-resource, NEVER do we expect + # a stop msg since the parent-side ctx apis will error out + # IMMEDIATELY before the child ever sends any 'stop' msg. + # + # ONLY PARENT breaks + elif ( + ipc_break['break_parent_ipc_after'] + and + ipc_break['break_child_ipc_after'] is False ): expect_final_exc = trio.ClosedResourceError - # when the parent IPC side dies (even if the child's does as well - # but the child fails BEFORE the parent) we expect the channel to be - # sent a stop msg from the child at some point which will signal the - # parent that the stream has been terminated. - # NOTE: when the parent breaks "after" the child you get this same - # case as well, the child breaks the IPC channel with a stop msg - # before any closure takes place. + # BOTH but, PARENT breaks FIRST elif ( - # only parent breaks - ( + ipc_break['break_parent_ipc_after'] is not False + and ( + ipc_break['break_child_ipc_after'] + > ipc_break['break_parent_ipc_after'] - and ipc_break['break_child_ipc_after'] is False - ) - - # both break but, child breaks first - or ( - ipc_break['break_parent_ipc_after'] is not False - and ( - ipc_break['break_child_ipc_after'] - > ipc_break['break_parent_ipc_after'] - ) ) ): - expect_final_exc = trio.EndOfChannel + expect_final_exc = trio.ClosedResourceError - with pytest.raises(expect_final_exc): - trio.run( - partial( - mod.main, - debug_mode=debug_mode, - start_method=spawn_backend, - **ipc_break, + with pytest.raises( + expected_exception=( + expect_final_exc, + ExceptionGroup, + ), + ) as excinfo: + try: + trio.run( + partial( + mod.main, + debug_mode=debug_mode, + start_method=spawn_backend, + loglevel=loglevel, + pre_close=pre_aclose_msgstream, + **ipc_break, + ) + ) + except KeyboardInterrupt as kbi: + _err = kbi + if expect_final_exc is not KeyboardInterrupt: + pytest.fail( + 'Rxed unexpected KBI !?\n' + f'{repr(kbi)}' + ) + + raise + + # get raw instance from pytest wrapper + value = excinfo.value + if isinstance(value, ExceptionGroup): + value = next( + itertools.dropwhile( + lambda exc: not isinstance(exc, expect_final_exc), + value.exceptions, ) ) + assert value @tractor.context @@ -169,25 +238,29 @@ def test_stream_closed_right_after_ipc_break_and_zombie_lord_engages(): ''' async def main(): - async with tractor.open_nursery() as n: - portal = await n.start_actor( - 'ipc_breaker', - enable_modules=[__name__], - ) + with trio.fail_after(3): + async with tractor.open_nursery() as n: + portal = await n.start_actor( + 'ipc_breaker', + enable_modules=[__name__], + ) - with trio.move_on_after(1): - async with ( - portal.open_context( - break_ipc_after_started - ) as (ctx, sent), - ): - async with ctx.open_stream(): - await trio.sleep(0.5) + with trio.move_on_after(1): + async with ( + portal.open_context( + break_ipc_after_started + ) as (ctx, sent), + ): + async with ctx.open_stream(): + await trio.sleep(0.5) - print('parent waiting on context') + print('parent waiting on context') - print('parent exited context') - raise KeyboardInterrupt + print( + 'parent exited context\n' + 'parent raising KBI..\n' + ) + raise KeyboardInterrupt with pytest.raises(KeyboardInterrupt): trio.run(main) diff --git a/tests/test_advanced_streaming.py b/tests/test_advanced_streaming.py index 799a0897..3134b9c2 100644 --- a/tests/test_advanced_streaming.py +++ b/tests/test_advanced_streaming.py @@ -6,6 +6,7 @@ from collections import Counter import itertools import platform +import pytest import trio import tractor @@ -143,8 +144,16 @@ def test_dynamic_pub_sub(): try: trio.run(main) - except trio.TooSlowError: - pass + except ( + trio.TooSlowError, + ExceptionGroup, + ) as err: + if isinstance(err, ExceptionGroup): + for suberr in err.exceptions: + if isinstance(suberr, trio.TooSlowError): + break + else: + pytest.fail('Never got a `TooSlowError` ?') @tractor.context @@ -298,44 +307,69 @@ async def inf_streamer( async with ( ctx.open_stream() as stream, - trio.open_nursery() as n, + trio.open_nursery() as tn, ): - async def bail_on_sentinel(): + async def close_stream_on_sentinel(): async for msg in stream: if msg == 'done': + print( + 'streamer RXed "done" sentinel msg!\n' + 'CLOSING `MsgStream`!' + ) await stream.aclose() else: print(f'streamer received {msg}') + else: + print('streamer exited recv loop') # start termination detector - n.start_soon(bail_on_sentinel) + tn.start_soon(close_stream_on_sentinel) - for val in itertools.count(): + cap: int = 10000 # so that we don't spin forever when bug.. + for val in range(cap): try: + print(f'streamer sending {val}') await stream.send(val) + if val > cap: + raise RuntimeError( + 'Streamer never cancelled by setinel?' + ) + await trio.sleep(0.001) + + # close out the stream gracefully except trio.ClosedResourceError: - # close out the stream gracefully + print('transport closed on streamer side!') + assert stream.closed break + else: + raise RuntimeError( + 'Streamer not cancelled before finished sending?' + ) - print('terminating streamer') + print('streamer exited .open_streamer() block') -def test_local_task_fanout_from_stream(): +def test_local_task_fanout_from_stream( + debug_mode: bool, +): ''' Single stream with multiple local consumer tasks using the ``MsgStream.subscribe()` api. - Ensure all tasks receive all values after stream completes sending. + Ensure all tasks receive all values after stream completes + sending. ''' - consumers = 22 + consumers: int = 22 async def main(): counts = Counter() - async with tractor.open_nursery() as tn: - p = await tn.start_actor( + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as tn: + p: tractor.Portal = await tn.start_actor( 'inf_streamer', enable_modules=[__name__], ) @@ -343,7 +377,6 @@ def test_local_task_fanout_from_stream(): p.open_context(inf_streamer) as (ctx, _), ctx.open_stream() as stream, ): - async def pull_and_count(name: str): # name = trio.lowlevel.current_task().name async with stream.subscribe() as recver: @@ -352,7 +385,7 @@ def test_local_task_fanout_from_stream(): tractor.trionics.BroadcastReceiver ) async for val in recver: - # print(f'{name}: {val}') + print(f'bx {name} rx: {val}') counts[name] += 1 print(f'{name} bcaster ended') @@ -362,10 +395,14 @@ def test_local_task_fanout_from_stream(): with trio.fail_after(3): async with trio.open_nursery() as nurse: for i in range(consumers): - nurse.start_soon(pull_and_count, i) + nurse.start_soon( + pull_and_count, + i, + ) + # delay to let bcast consumers pull msgs await trio.sleep(0.5) - print('\nterminating') + print('terminating nursery of bcast rxer consumers!') await stream.send('done') print('closed stream connection') diff --git a/tests/test_cancellation.py b/tests/test_cancellation.py index 657ab8e4..14e4d0ae 100644 --- a/tests/test_cancellation.py +++ b/tests/test_cancellation.py @@ -8,15 +8,13 @@ import platform import time from itertools import repeat -from exceptiongroup import ( - BaseExceptionGroup, - ExceptionGroup, -) import pytest import trio import tractor - -from conftest import tractor_test, no_windows +from tractor._testing import ( + tractor_test, +) +from .conftest import no_windows def is_win(): @@ -48,11 +46,13 @@ async def do_nuthin(): ids=['no_args', 'unexpected_args'], ) def test_remote_error(arb_addr, args_err): - """Verify an error raised in a subactor that is propagated + ''' + Verify an error raised in a subactor that is propagated to the parent nursery, contains the underlying boxed builtin error type info and causes cancellation and reraising all the way up the stack. - """ + + ''' args, errtype = args_err async def main(): @@ -65,7 +65,9 @@ def test_remote_error(arb_addr, args_err): # an exception group outside the nursery since the error # here and the far end task error are one in the same? portal = await nursery.run_in_actor( - assert_err, name='errorer', **args + assert_err, + name='errorer', + **args ) # get result(s) from main task diff --git a/tests/test_child_manages_service_nursery.py b/tests/test_child_manages_service_nursery.py index 806e6d7e..1dcbe031 100644 --- a/tests/test_child_manages_service_nursery.py +++ b/tests/test_child_manages_service_nursery.py @@ -6,14 +6,15 @@ sub-sub-actor daemons. ''' from typing import Optional import asyncio -from contextlib import asynccontextmanager as acm +from contextlib import ( + asynccontextmanager as acm, + aclosing, +) import pytest import trio -from trio_typing import TaskStatus import tractor from tractor import RemoteActorError -from async_generator import aclosing async def aio_streamer( diff --git a/tests/test_clustering.py b/tests/test_clustering.py index df3d8357..92362b58 100644 --- a/tests/test_clustering.py +++ b/tests/test_clustering.py @@ -5,9 +5,7 @@ import trio import tractor from tractor import open_actor_cluster from tractor.trionics import gather_contexts - -from conftest import tractor_test - +from tractor._testing import tractor_test MESSAGE = 'tractoring at full speed' @@ -49,7 +47,7 @@ async def worker( await ctx.started() async with ctx.open_stream( - backpressure=True, + allow_overruns=True, ) as stream: # TODO: this with the below assert causes a hang bug? diff --git a/tests/test_context_stream_semantics.py b/tests/test_context_stream_semantics.py index c92c4407..42b1f7d0 100644 --- a/tests/test_context_stream_semantics.py +++ b/tests/test_context_stream_semantics.py @@ -1,21 +1,34 @@ ''' ``async with ():`` inlined context-stream cancellation testing. -Verify the we raise errors when streams are opened prior to sync-opening -a ``tractor.Context`` beforehand. +Verify the we raise errors when streams are opened prior to +sync-opening a ``tractor.Context`` beforehand. ''' -from contextlib import asynccontextmanager as acm from itertools import count import platform -from typing import Optional +from pprint import pformat +from typing import ( + Callable, +) import pytest import trio import tractor -from tractor._exceptions import StreamOverrun +from tractor import ( + Actor, + Context, + current_actor, +) +from tractor._exceptions import ( + StreamOverrun, + ContextCancelled, +) -from conftest import tractor_test +from tractor._testing import ( + tractor_test, + expect_ctxc, +) # ``Context`` semantics are as follows, # ------------------------------------ @@ -61,7 +74,7 @@ _state: bool = False @tractor.context async def too_many_starteds( - ctx: tractor.Context, + ctx: Context, ) -> None: ''' Call ``Context.started()`` more then once (an error). @@ -76,7 +89,7 @@ async def too_many_starteds( @tractor.context async def not_started_but_stream_opened( - ctx: tractor.Context, + ctx: Context, ) -> None: ''' Enter ``Context.open_stream()`` without calling ``.started()``. @@ -91,14 +104,21 @@ async def not_started_but_stream_opened( @pytest.mark.parametrize( 'target', - [too_many_starteds, not_started_but_stream_opened], + [ + too_many_starteds, + not_started_but_stream_opened, + ], ids='misuse_type={}'.format, ) -def test_started_misuse(target): - +def test_started_misuse( + target: Callable, + debug_mode: bool, +): async def main(): - async with tractor.open_nursery() as n: - portal = await n.start_actor( + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.start_actor( target.__name__, enable_modules=[__name__], ) @@ -113,7 +133,7 @@ def test_started_misuse(target): @tractor.context async def simple_setup_teardown( - ctx: tractor.Context, + ctx: Context, data: int, block_forever: bool = False, @@ -159,6 +179,7 @@ def test_simple_context( error_parent, callee_blocks_forever, pointlessly_open_stream, + debug_mode: bool, ): timeout = 1.5 if not platform.system() == 'Windows' else 4 @@ -166,20 +187,22 @@ def test_simple_context( async def main(): with trio.fail_after(timeout): - async with tractor.open_nursery() as nursery: - - portal = await nursery.start_actor( + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.start_actor( 'simple_context', enable_modules=[__name__], ) try: - async with portal.open_context( - simple_setup_teardown, - data=10, - block_forever=callee_blocks_forever, - ) as (ctx, sent): - + async with ( + portal.open_context( + simple_setup_teardown, + data=10, + block_forever=callee_blocks_forever, + ) as (ctx, sent), + ): assert sent == 11 if callee_blocks_forever: @@ -187,9 +210,6 @@ def test_simple_context( else: assert await ctx.result() == 'yo' - if not error_parent: - await ctx.cancel() - if pointlessly_open_stream: async with ctx.open_stream(): if error_parent: @@ -202,10 +222,15 @@ def test_simple_context( # 'stop' msg to the far end which needs # to be ignored pass + else: if error_parent: raise error_parent + # cancel AFTER we open a stream + # to avoid a cancel raised inside + # `.open_stream()` + await ctx.cancel() finally: # after cancellation @@ -228,6 +253,177 @@ def test_simple_context( trio.run(main) +@pytest.mark.parametrize( + 'callee_returns_early', + [True, False], + ids=lambda item: f'callee_returns_early={item}' +) +@pytest.mark.parametrize( + 'cancel_method', + ['ctx', 'portal'], + ids=lambda item: f'cancel_method={item}' +) +@pytest.mark.parametrize( + 'chk_ctx_result_before_exit', + [True, False], + ids=lambda item: f'chk_ctx_result_before_exit={item}' +) +def test_caller_cancels( + cancel_method: str, + chk_ctx_result_before_exit: bool, + callee_returns_early: bool, + debug_mode: bool, +): + ''' + Verify that when the opening side of a context (aka the caller) + cancels that context, the ctx does not raise a cancelled when + either calling `.result()` or on context exit. + + ''' + async def check_canceller( + ctx: Context, + ) -> None: + actor: Actor = current_actor() + uid: tuple = actor.uid + _ctxc: ContextCancelled|None = None + + if ( + cancel_method == 'portal' + and not callee_returns_early + ): + try: + res = await ctx.result() + assert 0, 'Portal cancel should raise!' + + except ContextCancelled as ctxc: + # with trio.CancelScope(shield=True): + # await tractor.pause() + _ctxc = ctxc + assert ctx.chan._cancel_called + assert ctxc.canceller == uid + assert ctxc is ctx.maybe_error + + # NOTE: should not ever raise even in the `ctx` + # case since self-cancellation should swallow the ctxc + # silently! + else: + try: + res = await ctx.result() + except ContextCancelled as ctxc: + pytest.fail(f'should not have raised ctxc\n{ctxc}') + + # we actually get a result + if callee_returns_early: + assert res == 'yo' + assert ctx.outcome is res + assert ctx.maybe_error is None + + else: + err: Exception = ctx.outcome + assert isinstance(err, ContextCancelled) + assert ( + tuple(err.canceller) + == + uid + ) + assert ( + err + is ctx.maybe_error + is ctx._remote_error + ) + if le := ctx._local_error: + assert err is le + + # else: + # TODO: what should this be then? + # not defined until block closes right? + # + # await tractor.pause() + # assert ctx._local_error is None + + # TODO: don't need this right? + # if _ctxc: + # raise _ctxc + + + async def main(): + + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.start_actor( + 'simple_context', + enable_modules=[__name__], + ) + timeout: float = ( + 0.5 + if not callee_returns_early + else 2 + ) + with trio.fail_after(timeout): + async with ( + expect_ctxc( + yay=( + not callee_returns_early + and cancel_method == 'portal' + ) + ), + + portal.open_context( + simple_setup_teardown, + data=10, + block_forever=not callee_returns_early, + ) as (ctx, sent), + ): + + if callee_returns_early: + # ensure we block long enough before sending + # a cancel such that the callee has already + # returned it's result. + await trio.sleep(0.5) + + if cancel_method == 'ctx': + print('cancelling with `Context.cancel()`') + await ctx.cancel() + + elif cancel_method == 'portal': + print('cancelling with `Portal.cancel_actor()`') + await portal.cancel_actor() + + else: + pytest.fail( + f'Unknown `cancel_method={cancel_method} ?' + ) + + if chk_ctx_result_before_exit: + await check_canceller(ctx) + + if not chk_ctx_result_before_exit: + await check_canceller(ctx) + + if cancel_method != 'portal': + await portal.cancel_actor() + + # XXX NOTE XXX: non-normal yet purposeful + # test-specific ctxc suppression is implemented! + # + # WHY: the `.cancel_actor()` case (cancel_method='portal') + # will cause both: + # * the `ctx.result()` inside `.open_context().__aexit__()` + # * AND the `ctx.result()` inside `check_canceller()` + # to raise ctxc. + # + # which should in turn cause `ctx._scope` to + # catch any cancellation? + if ( + not callee_returns_early + and cancel_method != 'portal' + ): + assert not ctx._scope.cancelled_caught + + trio.run(main) + + # basic stream terminations: # - callee context closes without using stream # - caller context closes without using stream @@ -243,7 +439,7 @@ def test_simple_context( @tractor.context async def close_ctx_immediately( - ctx: tractor.Context, + ctx: Context, ) -> None: @@ -255,17 +451,33 @@ async def close_ctx_immediately( @tractor_test -async def test_callee_closes_ctx_after_stream_open(): - 'callee context closes without using stream' +async def test_callee_closes_ctx_after_stream_open( + debug_mode: bool, +): + ''' + callee context closes without using stream. - async with tractor.open_nursery() as n: + This should result in a msg sequence + |__ + |_ - portal = await n.start_actor( + <= {'started': , 'cid': } + <= {'stop': True, 'cid': } + <= {'result': Any, ..} + + (ignored by child) + => {'stop': True, 'cid': } + + ''' + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.start_actor( 'fast_stream_closer', enable_modules=[__name__], ) - with trio.fail_after(2): + with trio.fail_after(0.5): async with portal.open_context( close_ctx_immediately, @@ -273,10 +485,9 @@ async def test_callee_closes_ctx_after_stream_open(): # cancel_on_exit=True, ) as (ctx, sent): - assert sent is None - with trio.fail_after(0.5): + with trio.fail_after(0.4): async with ctx.open_stream() as stream: # should fall through since ``StopAsyncIteration`` @@ -284,12 +495,15 @@ async def test_callee_closes_ctx_after_stream_open(): # a ``trio.EndOfChannel`` by # ``trio.abc.ReceiveChannel.__anext__()`` async for _ in stream: + # trigger failure if we DO NOT + # get an EOC! assert 0 else: # verify stream is now closed try: - await stream.receive() + with trio.fail_after(0.3): + await stream.receive() except trio.EndOfChannel: pass @@ -309,8 +523,7 @@ async def test_callee_closes_ctx_after_stream_open(): @tractor.context async def expect_cancelled( - - ctx: tractor.Context, + ctx: Context, ) -> None: global _state @@ -324,12 +537,29 @@ async def expect_cancelled( await stream.send(msg) # echo server except trio.Cancelled: + + # on ctx.cancel() the internal RPC scope is cancelled but + # never caught until the func exits. + assert ctx._scope.cancel_called + assert not ctx._scope.cancelled_caught + + # should be the RPC cmd request for `._cancel_task()` + assert ctx._cancel_msg + # which, has not yet resolved to an error outcome + # since this rpc func has not yet exited. + assert not ctx.maybe_error + assert not ctx._final_result_is_set() + + # debug REPL if needed + # with trio.CancelScope(shield=True): + # await tractor.pause() + # expected case _state = False raise else: - assert 0, "Wasn't cancelled!?" + assert 0, "callee wasn't cancelled !?" @pytest.mark.parametrize( @@ -339,12 +569,18 @@ async def expect_cancelled( @tractor_test async def test_caller_closes_ctx_after_callee_opens_stream( use_ctx_cancel_method: bool, + debug_mode: bool, ): - 'caller context closes without using stream' + ''' + caller context closes without using/opening stream - async with tractor.open_nursery() as n: + ''' + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: - portal = await n.start_actor( + root: Actor = current_actor() + portal = await an.start_actor( 'ctx_cancelled', enable_modules=[__name__], ) @@ -352,22 +588,37 @@ async def test_caller_closes_ctx_after_callee_opens_stream( async with portal.open_context( expect_cancelled, ) as (ctx, sent): - await portal.run(assert_state, value=True) - assert sent is None - # call cancel explicitly - if use_ctx_cancel_method: + await portal.run(assert_state, value=True) + # call `ctx.cancel()` explicitly + if use_ctx_cancel_method: await ctx.cancel() + # NOTE: means the local side `ctx._scope` will + # have been cancelled by an ctxc ack and thus + # `._scope.cancelled_caught` should be set. try: async with ctx.open_stream() as stream: async for msg in stream: pass - except tractor.ContextCancelled: - raise # XXX: must be propagated to __aexit__ + except tractor.ContextCancelled as ctxc: + # XXX: the cause is US since we call + # `Context.cancel()` just above! + assert ( + ctxc.canceller + == + current_actor().uid + == + root.uid + ) + + # XXX: must be propagated to __aexit__ + # and should be silently absorbed there + # since we called `.cancel()` just above ;) + raise else: assert 0, "Should have context cancelled?" @@ -376,7 +627,10 @@ async def test_caller_closes_ctx_after_callee_opens_stream( assert portal.channel.connected() # ctx is closed here - await portal.run(assert_state, value=False) + await portal.run( + assert_state, + value=False, + ) else: try: @@ -384,7 +638,25 @@ async def test_caller_closes_ctx_after_callee_opens_stream( await ctx.result() assert 0, "Callee should have blocked!?" except trio.TooSlowError: + # NO-OP -> since already called above await ctx.cancel() + + # NOTE: local scope should have absorbed the cancellation since + # in this case we call `ctx.cancel()` and the local + # `._scope` does not get `.cancel_called` and thus + # `.cancelled_caught` neither will ever bet set. + if use_ctx_cancel_method: + assert not ctx._scope.cancelled_caught + + # rxed ctxc response from far end + assert ctx.cancel_acked + assert ( + ctx._remote_error + is ctx._local_error + is ctx.maybe_error + is ctx.outcome + ) + try: async with ctx.open_stream() as stream: async for msg in stream: @@ -406,11 +678,13 @@ async def test_caller_closes_ctx_after_callee_opens_stream( @tractor_test -async def test_multitask_caller_cancels_from_nonroot_task(): - - async with tractor.open_nursery() as n: - - portal = await n.start_actor( +async def test_multitask_caller_cancels_from_nonroot_task( + debug_mode: bool, +): + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.start_actor( 'ctx_cancelled', enable_modules=[__name__], ) @@ -457,25 +731,31 @@ async def test_multitask_caller_cancels_from_nonroot_task(): @tractor.context async def cancel_self( - ctx: tractor.Context, + ctx: Context, ) -> None: global _state _state = True + # since we call this the below `.open_stream()` should always + # error! await ctx.cancel() # should inline raise immediately try: async with ctx.open_stream(): pass - except tractor.ContextCancelled: + # except tractor.ContextCancelled: + except RuntimeError: # suppress for now so we can do checkpoint tests below - pass + print('Got expected runtime error for stream-after-cancel') + else: raise RuntimeError('Context didnt cancel itself?!') - # check a real ``trio.Cancelled`` is raised on a checkpoint + # check that``trio.Cancelled`` is now raised on any further + # checkpoints since the self cancel above will have cancelled + # the `Context._scope.cancel_scope: trio.CancelScope` try: with trio.fail_after(0.1): await trio.sleep_forever() @@ -486,17 +766,22 @@ async def cancel_self( # should never get here assert 0 + raise RuntimeError('Context didnt cancel itself?!') + @tractor_test -async def test_callee_cancels_before_started(): +async def test_callee_cancels_before_started( + debug_mode: bool, +): ''' Callee calls `Context.cancel()` while streaming and caller sees stream terminated in `ContextCancelled`. ''' - async with tractor.open_nursery() as n: - - portal = await n.start_actor( + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.start_actor( 'cancels_self', enable_modules=[__name__], ) @@ -506,7 +791,6 @@ async def test_callee_cancels_before_started(): cancel_self, ) as (ctx, sent): async with ctx.open_stream(): - await trio.sleep_forever() # raises a special cancel signal @@ -514,7 +798,7 @@ async def test_callee_cancels_before_started(): ce.type == trio.Cancelled # the traceback should be informative - assert 'cancelled itself' in ce.msgdata['tb_str'] + assert 'itself' in ce.msgdata['tb_str'] # teardown the actor await portal.cancel_actor() @@ -523,7 +807,7 @@ async def test_callee_cancels_before_started(): @tractor.context async def never_open_stream( - ctx: tractor.Context, + ctx: Context, ) -> None: ''' @@ -537,8 +821,8 @@ async def never_open_stream( @tractor.context async def keep_sending_from_callee( - ctx: tractor.Context, - msg_buffer_size: Optional[int] = None, + ctx: Context, + msg_buffer_size: int|None = None, ) -> None: ''' @@ -559,12 +843,14 @@ async def keep_sending_from_callee( 'overrun_by', [ ('caller', 1, never_open_stream), - ('cancel_caller_during_overrun', 1, never_open_stream), ('callee', 0, keep_sending_from_callee), ], ids='overrun_condition={}'.format, ) -def test_one_end_stream_not_opened(overrun_by): +def test_one_end_stream_not_opened( + overrun_by: tuple[str, int, Callable], + debug_mode: bool, +): ''' This should exemplify the bug from: https://github.com/goodboy/tractor/issues/265 @@ -575,8 +861,10 @@ def test_one_end_stream_not_opened(overrun_by): buf_size = buf_size_increase + Actor.msg_buffer_size async def main(): - async with tractor.open_nursery() as n: - portal = await n.start_actor( + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.start_actor( entrypoint.__name__, enable_modules=[__name__], ) @@ -589,14 +877,13 @@ def test_one_end_stream_not_opened(overrun_by): if 'caller' in overrunner: async with ctx.open_stream() as stream: + + # itersend +1 msg more then the buffer size + # to cause the most basic overrun. for i in range(buf_size): print(f'sending {i}') await stream.send(i) - if 'cancel' in overrunner: - # without this we block waiting on the child side - await ctx.cancel() - else: # expect overrun error to be relayed back # and this sleep interrupted @@ -610,7 +897,9 @@ def test_one_end_stream_not_opened(overrun_by): # 2 overrun cases and the no overrun case (which pushes right up to # the msg limit) - if overrunner == 'caller' or 'cance' in overrunner: + if ( + overrunner == 'caller' + ): with pytest.raises(tractor.RemoteActorError) as excinfo: trio.run(main) @@ -632,167 +921,280 @@ def test_one_end_stream_not_opened(overrun_by): @tractor.context async def echo_back_sequence( - ctx: tractor.Context, + ctx: Context, seq: list[int], - msg_buffer_size: Optional[int] = None, + wait_for_cancel: bool, + allow_overruns_side: str, + be_slow: bool = False, + msg_buffer_size: int = 1, ) -> None: ''' - Send endlessly on the calleee stream. + Send endlessly on the calleee stream using a small buffer size + setting on the contex to simulate backlogging that would normally + cause overruns. ''' + # NOTE: ensure that if the caller is expecting to cancel this task + # that we stay echoing much longer then they are so we don't + # return early instead of receive the cancel msg. + total_batches: int = ( + 1000 if wait_for_cancel + else 6 + ) + await ctx.started() + # await tractor.breakpoint() async with ctx.open_stream( msg_buffer_size=msg_buffer_size, + + # literally the point of this test XD + allow_overruns=(allow_overruns_side in {'child', 'both'}), ) as stream: - seq = list(seq) # bleh, `msgpack`... - count = 0 - while count < 3: + # ensure mem chan settings are correct + assert ( + ctx._send_chan._state.max_buffer_size + == + msg_buffer_size + ) + + seq = list(seq) # bleh, msgpack sometimes ain't decoded right + for i in range(total_batches): + print(f'starting new stream batch {i} iter in child') batch = [] + + # EoC case, delay a little instead of hot + # iter-stopping (since apparently py3.11+ can do that + # faster then a ctxc can be sent) on the async for + # loop when child was requested to ctxc. + if ( + stream.closed + or + ctx.cancel_called + ): + print('child stream already closed!?!') + await trio.sleep(0.05) + continue + async for msg in stream: batch.append(msg) if batch == seq: break + if be_slow: + await trio.sleep(0.05) + + print('callee waiting on next') + + print(f'callee echoing back latest batch\n{batch}') for msg in batch: - print(f'callee sending {msg}') + print(f'callee sending msg\n{msg}') await stream.send(msg) - count += 1 - + try: return 'yo' + finally: + print( + 'exiting callee with context:\n' + f'{pformat(ctx)}\n' + ) -def test_stream_backpressure(): +@pytest.mark.parametrize( + # aka the side that will / should raise + # and overrun under normal conditions. + 'allow_overruns_side', + ['parent', 'child', 'none', 'both'], + ids=lambda item: f'allow_overruns_side={item}' +) +@pytest.mark.parametrize( + # aka the side that will / should raise + # and overrun under normal conditions. + 'slow_side', + ['parent', 'child'], + ids=lambda item: f'slow_side={item}' +) +@pytest.mark.parametrize( + 'cancel_ctx', + [True, False], + ids=lambda item: f'cancel_ctx={item}' +) +def test_maybe_allow_overruns_stream( + cancel_ctx: bool, + slow_side: str, + allow_overruns_side: str, + + # conftest wide + loglevel: str, + debug_mode: bool, +): ''' Demonstrate small overruns of each task back and forth - on a stream not raising any errors by default. + on a stream not raising any errors by default by setting + the ``allow_overruns=True``. + + The original idea here was to show that if you set the feeder mem + chan to a size smaller then the # of msgs sent you could could not + get a `StreamOverrun` crash plus maybe get all the msgs that were + sent. The problem with the "real backpressure" case is that due to + the current arch it can result in the msg loop being blocked and thus + blocking cancellation - which is like super bad. So instead this test + had to be adjusted to more or less just "not send overrun errors" so + as to handle the case where the sender just moreso cares about not getting + errored out when it send to fast.. ''' async def main(): - async with tractor.open_nursery() as n: - portal = await n.start_actor( + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as an: + portal = await an.start_actor( 'callee_sends_forever', enable_modules=[__name__], + loglevel=loglevel, + debug_mode=debug_mode, ) - seq = list(range(3)) + seq = list(range(10)) async with portal.open_context( echo_back_sequence, seq=seq, - msg_buffer_size=1, + wait_for_cancel=cancel_ctx, + be_slow=(slow_side == 'child'), + allow_overruns_side=allow_overruns_side, + ) as (ctx, sent): assert sent is None - async with ctx.open_stream(msg_buffer_size=1) as stream: - count = 0 - while count < 3: + async with ctx.open_stream( + msg_buffer_size=1 if slow_side == 'parent' else None, + allow_overruns=(allow_overruns_side in {'parent', 'both'}), + ) as stream: + + total_batches: int = 2 + for _ in range(total_batches): for msg in seq: - print(f'caller sending {msg}') + # print(f'root tx {msg}') await stream.send(msg) - await trio.sleep(0.1) + if slow_side == 'parent': + # NOTE: we make the parent slightly + # slower, when it is slow, to make sure + # that in the overruns everywhere case + await trio.sleep(0.16) batch = [] async for msg in stream: + print(f'root rx {msg}') batch.append(msg) if batch == seq: break - count += 1 + if cancel_ctx: + # cancel the remote task + print('Requesting `ctx.cancel()` in parent!') + await ctx.cancel() - # here the context should return - assert await ctx.result() == 'yo' + res: str|ContextCancelled = await ctx.result() + + if cancel_ctx: + assert isinstance(res, ContextCancelled) + assert tuple(res.canceller) == current_actor().uid + + else: + print(f'RX ROOT SIDE RESULT {res}') + assert res == 'yo' # cancel the daemon await portal.cancel_actor() - trio.run(main) + if ( + allow_overruns_side == 'both' + or slow_side == allow_overruns_side + ): + trio.run(main) + elif ( + slow_side != allow_overruns_side + ): -@tractor.context -async def sleep_forever( - ctx: tractor.Context, -) -> None: - await ctx.started() - async with ctx.open_stream(): - await trio.sleep_forever() + with pytest.raises(tractor.RemoteActorError) as excinfo: + trio.run(main) + err = excinfo.value -@acm -async def attach_to_sleep_forever(): - ''' - Cancel a context **before** any underlying error is raised in order - to trigger a local reception of a ``ContextCancelled`` which **should not** - be re-raised in the local surrounding ``Context`` *iff* the cancel was - requested by **this** side of the context. - - ''' - async with tractor.wait_for_actor('sleeper') as p2: - async with ( - p2.open_context(sleep_forever) as (peer_ctx, first), - peer_ctx.open_stream(), + if ( + allow_overruns_side == 'none' ): - try: - yield - finally: - # XXX: previously this would trigger local - # ``ContextCancelled`` to be received and raised in the - # local context overriding any local error due to - # logic inside ``_invoke()`` which checked for - # an error set on ``Context._error`` and raised it in - # under a cancellation scenario. + # depends on timing is is racy which side will + # overrun first :sadkitty: - # The problem is you can have a remote cancellation - # that is part of a local error and we shouldn't raise - # ``ContextCancelled`` **iff** we weren't the side of - # the context to initiate it, i.e. - # ``Context._cancel_called`` should **NOT** have been - # set. The special logic to handle this case is now - # inside ``Context._may_raise_from_remote_msg()`` XD - await peer_ctx.cancel() + # NOTE: i tried to isolate to a deterministic case here + # based on timeing, but i was kinda wasted, and i don't + # think it's sane to catch them.. + assert err.type in ( + tractor.RemoteActorError, + StreamOverrun, + ) + + elif ( + slow_side == 'child' + ): + assert err.type == StreamOverrun + + elif slow_side == 'parent': + assert err.type == tractor.RemoteActorError + assert 'StreamOverrun' in err.msgdata['tb_str'] + + else: + # if this hits the logic blocks from above are not + # exhaustive.. + pytest.fail('PARAMETRIZED CASE GEN PROBLEM YO') -@tractor.context -async def error_before_started( - ctx: tractor.Context, -) -> None: +def test_ctx_with_self_actor( + loglevel: str, + debug_mode: bool, +): ''' - This simulates exactly an original bug discovered in: - https://github.com/pikers/piker/issues/244 + NOTE: for now this is an INVALID OP! - ''' - async with attach_to_sleep_forever(): - # send an unserializable type which should raise a type error - # here and **NOT BE SWALLOWED** by the surrounding acm!!?! - await ctx.started(object()) + BUT, eventually presuming we add a "side" key to `Actor.get_context()`, + we might be able to get this working symmetrically, but should we?? - -def test_do_not_swallow_error_before_started_by_remote_contextcancelled(): - ''' - Verify that an error raised in a remote context which itself opens another - remote context, which it cancels, does not ovverride the original error that - caused the cancellation of the secondardy context. + Open a context back to the same actor and ensure all cancellation + and error semantics hold the same. ''' async def main(): - async with tractor.open_nursery() as n: - portal = await n.start_actor( - 'errorer', - enable_modules=[__name__], - ) - await n.start_actor( - 'sleeper', - enable_modules=[__name__], - ) - + async with tractor.open_nursery( + debug_mode=debug_mode, + enable_modules=[__name__], + ) as an: + assert an async with ( + tractor.find_actor('root') as portal, portal.open_context( - error_before_started - ) as (ctx, sent), - ): - await trio.sleep_forever() + expect_cancelled, + # echo_back_sequence, + # seq=seq, + # wait_for_cancel=cancel_ctx, + # be_slow=(slow_side == 'child'), + # allow_overruns_side=allow_overruns_side, - with pytest.raises(tractor.RemoteActorError) as excinfo: + ) as (ctx, sent), + ctx.open_stream() as ipc, + ): + assert sent is None + + seq = list(range(10)) + for i in seq: + await ipc.send(i) + rx: int = await ipc.receive() + assert rx == i + + await ctx.cancel() + + with pytest.raises(RuntimeError) as excinfo: trio.run(main) - assert excinfo.value.type == TypeError + assert 'Invalid Operation' in repr(excinfo.value) diff --git a/tests/test_debugger.py b/tests/test_debugger.py index a44a3138..889e7c74 100644 --- a/tests/test_debugger.py +++ b/tests/test_debugger.py @@ -11,11 +11,9 @@ TODO: """ import itertools -from os import path from typing import Optional import platform import pathlib -import sys import time import pytest @@ -25,8 +23,10 @@ from pexpect.exceptions import ( EOF, ) -from conftest import ( +from tractor._testing import ( examples_dir, +) +from .conftest import ( _ci_env, ) diff --git a/tests/test_discovery.py b/tests/test_discovery.py index 8ba4ebee..b56c3a2e 100644 --- a/tests/test_discovery.py +++ b/tests/test_discovery.py @@ -9,10 +9,9 @@ import itertools import pytest import tractor +from tractor._testing import tractor_test import trio -from conftest import tractor_test - @tractor_test async def test_reg_then_unreg(arb_addr): diff --git a/tests/test_docs_examples.py b/tests/test_docs_examples.py index f134c71b..7a923343 100644 --- a/tests/test_docs_examples.py +++ b/tests/test_docs_examples.py @@ -11,8 +11,7 @@ import platform import shutil import pytest - -from conftest import ( +from tractor._testing import ( examples_dir, ) @@ -89,6 +88,7 @@ def run_example_in_subproc( and 'debugging' not in p[0] and 'integration' not in p[0] and 'advanced_faults' not in p[0] + and 'multihost' not in p[0] ], ids=lambda t: t[1], diff --git a/tests/test_infected_asyncio.py b/tests/test_infected_asyncio.py index 775ee98d..f9670225 100644 --- a/tests/test_infected_asyncio.py +++ b/tests/test_infected_asyncio.py @@ -8,15 +8,16 @@ import builtins import itertools import importlib -from exceptiongroup import BaseExceptionGroup import pytest import trio import tractor from tractor import ( to_asyncio, RemoteActorError, + ContextCancelled, ) from tractor.trionics import BroadcastReceiver +from tractor._testing import expect_ctxc async def sleep_and_err( @@ -67,7 +68,7 @@ def test_trio_cancels_aio_on_actor_side(arb_addr): async def asyncio_actor( target: str, - expect_err: Optional[Exception] = None + expect_err: Exception|None = None ) -> None: @@ -111,10 +112,21 @@ def test_aio_simple_error(arb_addr): infect_asyncio=True, ) - with pytest.raises(RemoteActorError) as excinfo: + with pytest.raises( + expected_exception=(RemoteActorError, ExceptionGroup), + ) as excinfo: trio.run(main) err = excinfo.value + + # might get multiple `trio.Cancelled`s as well inside an inception + if isinstance(err, ExceptionGroup): + err = next(itertools.dropwhile( + lambda exc: not isinstance(exc, tractor.RemoteActorError), + err.exceptions + )) + assert err + assert isinstance(err, RemoteActorError) assert err.type == AssertionError @@ -189,7 +201,8 @@ async def trio_ctx( @pytest.mark.parametrize( - 'parent_cancels', [False, True], + 'parent_cancels', + ['context', 'actor', False], ids='parent_actor_cancels_child={}'.format ) def test_context_spawns_aio_task_that_errors( @@ -213,25 +226,52 @@ def test_context_spawns_aio_task_that_errors( # debug_mode=True, loglevel='cancel', ) - async with p.open_context( - trio_ctx, - ) as (ctx, first): + async with ( + expect_ctxc( + yay=parent_cancels == 'actor', + ), + p.open_context( + trio_ctx, + ) as (ctx, first), + ): assert first == 'start' - if parent_cancels: + if parent_cancels == 'actor': await p.cancel_actor() - await trio.sleep_forever() + elif parent_cancels == 'context': + await ctx.cancel() - with pytest.raises(RemoteActorError) as excinfo: - trio.run(main) + else: + await trio.sleep_forever() + + async with expect_ctxc( + yay=parent_cancels == 'actor', + ): + await ctx.result() + + if parent_cancels == 'context': + # to tear down sub-acor + await p.cancel_actor() + + return ctx.outcome - err = excinfo.value - assert isinstance(err, RemoteActorError) if parent_cancels: - assert err.type == trio.Cancelled + # bc the parent made the cancel request, + # the error is not raised locally but instead + # the context is exited silently + res = trio.run(main) + assert isinstance(res, ContextCancelled) + assert 'root' in res.canceller[0] + else: + expect = RemoteActorError + with pytest.raises(expect) as excinfo: + trio.run(main) + + err = excinfo.value + assert isinstance(err, expect) assert err.type == AssertionError @@ -259,11 +299,22 @@ def test_aio_cancelled_from_aio_causes_trio_cancelled(arb_addr): infect_asyncio=True, ) - with pytest.raises(RemoteActorError) as excinfo: + with pytest.raises( + expected_exception=(RemoteActorError, ExceptionGroup), + ) as excinfo: trio.run(main) + # might get multiple `trio.Cancelled`s as well inside an inception + err = excinfo.value + if isinstance(err, ExceptionGroup): + err = next(itertools.dropwhile( + lambda exc: not isinstance(exc, tractor.RemoteActorError), + err.exceptions + )) + assert err + # ensure boxed error is correct - assert excinfo.value.type == to_asyncio.AsyncioCancelled + assert err.type == to_asyncio.AsyncioCancelled # TODO: verify open_channel_from will fail on this.. diff --git a/tests/test_inter_peer_cancellation.py b/tests/test_inter_peer_cancellation.py new file mode 100644 index 00000000..c3d9e4fd --- /dev/null +++ b/tests/test_inter_peer_cancellation.py @@ -0,0 +1,1057 @@ +''' +Codify the cancellation request semantics in terms +of one remote actor cancelling another. + +''' +# from contextlib import asynccontextmanager as acm +import itertools + +import pytest +import trio +import tractor +from tractor import ( # typing + Actor, + current_actor, + open_nursery, + Portal, + Context, + ContextCancelled, +) + +# XXX TODO cases: +# - [ ] peer cancelled itself - so other peers should +# get errors reflecting that the peer was itself the .canceller? + +# - [x] WE cancelled the peer and thus should not see any raised +# `ContextCancelled` as it should be reaped silently? +# => pretty sure `test_context_stream_semantics::test_caller_cancels()` +# already covers this case? + +# - [x] INTER-PEER: some arbitrary remote peer cancels via +# Portal.cancel_actor(). +# => all other connected peers should get that cancel requesting peer's +# uid in the ctx-cancelled error msg raised in all open ctxs +# with that peer. + +# - [ ] PEER-FAILS-BY-CHILD-ERROR: peer spawned a sub-actor which +# (also) spawned a failing task which was unhandled and +# propagated up to the immediate parent - the peer to the actor +# that also spawned a remote task task in that same peer-parent. + + +# def test_self_cancel(): +# ''' +# 2 cases: +# - calls `Actor.cancel()` locally in some task +# - calls LocalPortal.cancel_actor()` ? + +# ''' +# ... + + +@tractor.context +async def sleep_forever( + ctx: Context, + expect_ctxc: bool = False, +) -> None: + ''' + Sync the context, open a stream then just sleep. + + Allow checking for (context) cancellation locally. + + ''' + try: + await ctx.started() + async with ctx.open_stream(): + await trio.sleep_forever() + + except BaseException as berr: + + # TODO: it'd sure be nice to be able to inject our own + # `ContextCancelled` here instead of of `trio.Cancelled` + # so that our runtime can expect it and this "user code" + # would be able to tell the diff between a generic trio + # cancel and a tractor runtime-IPC cancel. + if expect_ctxc: + assert isinstance(berr, trio.Cancelled) + + raise + + +@tractor.context +async def error_before_started( + ctx: Context, +) -> None: + ''' + This simulates exactly an original bug discovered in: + https://github.com/pikers/piker/issues/244 + + Cancel a context **before** any underlying error is raised so + as to trigger a local reception of a ``ContextCancelled`` which + SHOULD NOT be re-raised in the local surrounding ``Context`` + *iff* the cancel was requested by **this** (callee) side of + the context. + + ''' + async with tractor.wait_for_actor('sleeper') as p2: + async with ( + p2.open_context(sleep_forever) as (peer_ctx, first), + peer_ctx.open_stream(), + ): + # NOTE: this WAS inside an @acm body but i factored it + # out and just put it inline here since i don't think + # the mngr part really matters, though maybe it could? + try: + # XXX NOTE XXX: THIS sends an UNSERIALIZABLE TYPE which + # should raise a `TypeError` and **NOT BE SWALLOWED** by + # the surrounding try/finally (normally inside the + # body of some acm).. + await ctx.started(object()) + # yield + finally: + # XXX: previously this would trigger local + # ``ContextCancelled`` to be received and raised in the + # local context overriding any local error due to logic + # inside ``_invoke()`` which checked for an error set on + # ``Context._error`` and raised it in a cancellation + # scenario. + # ------ + # The problem is you can have a remote cancellation that + # is part of a local error and we shouldn't raise + # ``ContextCancelled`` **iff** we **were not** the side + # of the context to initiate it, i.e. + # ``Context._cancel_called`` should **NOT** have been + # set. The special logic to handle this case is now + # inside ``Context._maybe_raise_from_remote_msg()`` XD + await peer_ctx.cancel() + + +def test_do_not_swallow_error_before_started_by_remote_contextcancelled( + debug_mode: bool, +): + ''' + Verify that an error raised in a remote context which itself + opens YET ANOTHER remote context, which it then cancels, does not + override the original error that caused the cancellation of the + secondary context. + + ''' + async def main(): + async with tractor.open_nursery( + debug_mode=debug_mode, + ) as n: + portal = await n.start_actor( + 'errorer', + enable_modules=[__name__], + ) + await n.start_actor( + 'sleeper', + enable_modules=[__name__], + ) + + async with ( + portal.open_context( + error_before_started + ) as (ctx, sent), + ): + await trio.sleep_forever() + + with pytest.raises(tractor.RemoteActorError) as excinfo: + trio.run(main) + + assert excinfo.value.type == TypeError + + +@tractor.context +async def sleep_a_bit_then_cancel_peer( + ctx: Context, + peer_name: str = 'sleeper', + cancel_after: float = .5, + +) -> None: + ''' + Connect to peer, sleep as per input delay, cancel the peer. + + ''' + peer: Portal + async with tractor.wait_for_actor(peer_name) as peer: + await ctx.started() + await trio.sleep(cancel_after) + await peer.cancel_actor() + + +@tractor.context +async def stream_ints( + ctx: Context, +): + await ctx.started() + async with ctx.open_stream() as stream: + for i in itertools.count(): + await stream.send(i) + await trio.sleep(0.01) + + +@tractor.context +async def stream_from_peer( + ctx: Context, + peer_name: str = 'sleeper', +) -> None: + + peer: Portal + try: + async with ( + tractor.wait_for_actor(peer_name) as peer, + peer.open_context(stream_ints) as (peer_ctx, first), + peer_ctx.open_stream() as stream, + ): + await ctx.started() + # XXX QUESTIONS & TODO: for further details around this + # in the longer run.. + # https://github.com/goodboy/tractor/issues/368 + # - should we raise `ContextCancelled` or `Cancelled` (rn + # it does latter) and should/could it be implemented + # as a general injection override for `trio` such + # that ANY next checkpoint would raise the "cancel + # error type" of choice? + # - should the `ContextCancelled` bubble from + # all `Context` and `MsgStream` apis wherein it + # prolly makes the most sense to make it + # a `trio.Cancelled` subtype? + # - what about IPC-transport specific errors, should + # they bubble from the async for and trigger + # other special cases? + # + # NOTE: current ctl flow: + # - stream raises `trio.EndOfChannel` and + # exits the loop + # - `.open_context()` will raise the ctxc received + # from the sleeper. + async for msg in stream: + assert msg is not None + print(msg) + + # NOTE: cancellation of the (sleeper) peer should always + # cause a `ContextCancelled` raise in this streaming + # actor. + except ContextCancelled as ctxc: + ctxerr = ctxc + + assert peer_ctx._remote_error is ctxerr + assert peer_ctx._remote_error.msgdata == ctxerr.msgdata + + # the peer ctx is the canceller even though it's canceller + # is the "canceller" XD + assert peer_name in peer_ctx.canceller + + assert "canceller" in ctxerr.canceller + + # caller peer should not be the cancel requester + assert not ctx.cancel_called + assert not ctx.cancel_acked + + # XXX can NEVER BE TRUE since `._invoke` only + # sets this AFTER the nursery block this task + # was started in, exits. + assert not ctx._scope.cancelled_caught + + # we never requested cancellation, it was the 'canceller' + # peer. + assert not peer_ctx.cancel_called + assert not peer_ctx.cancel_acked + + # the `.open_context()` exit definitely caught + # a cancellation in the internal `Context._scope` since + # likely the runtime called `_deliver_msg()` after + # receiving the remote error from the streaming task. + assert not peer_ctx._scope.cancelled_caught + + # TODO / NOTE `.canceller` won't have been set yet + # here because that machinery is inside + # `.open_context().__aexit__()` BUT, if we had + # a way to know immediately (from the last + # checkpoint) that cancellation was due to + # a remote, we COULD assert this here..see, + # https://github.com/goodboy/tractor/issues/368 + # + # assert 'canceller' in ctx.canceller + + # root/parent actor task should NEVER HAVE cancelled us! + assert not ctx.canceller + + raise + # TODO: IN THEORY we could have other cases depending on + # who cancels first, the root actor or the canceller peer?. + # + # 1- when the peer request is first then the `.canceller` + # field should obvi be set to the 'canceller' uid, + # + # 2-if the root DOES req cancel then we should see the same + # `trio.Cancelled` implicitly raised + # assert ctx.canceller[0] == 'root' + # assert peer_ctx.canceller[0] == 'sleeper' + + raise RuntimeError('Never triggered local `ContextCancelled` ?!?') + + +@pytest.mark.parametrize( + 'error_during_ctxerr_handling', + [False, True], + ids=lambda item: f'rte_during_ctxerr={item}', +) +def test_peer_canceller( + error_during_ctxerr_handling: bool, + debug_mode: bool, +): + ''' + Verify that a cancellation triggered by an in-actor-tree peer + results in a cancelled errors with all other actors which have + opened contexts to that same actor. + + legend: + name> + a "play button" that indicates a new runtime instance, + an individual actor with `name`. + + .subname> + a subactor who's parent should be on some previous + line and be less indented. + + .actor0> ()-> .actor1> + a inter-actor task context opened (by `async with + `Portal.open_context()`) from actor0 *into* actor1. + + .actor0> ()<=> .actor1> + a inter-actor task context opened (as above) + from actor0 *into* actor1 which INCLUDES an additional + stream open using `async with Context.open_stream()`. + + + ------ - ------ + supervision view + ------ - ------ + root> + .sleeper> TODO: SOME SYNTAX SHOWING JUST SLEEPING + .just_caller> ()=> .sleeper> + .canceller> ()-> .sleeper> + TODO: how define calling `Portal.cancel_actor()` + + In this case a `ContextCancelled` with `.errorer` set to the + requesting actor, in this case 'canceller', should be relayed + to all other actors who have also opened a (remote task) + context with that now cancelled actor. + + ------ - ------ + task view + ------ - ------ + So there are 5 context open in total with 3 from the root to + its children and 2 from children to their peers: + 1. root> ()-> .sleeper> + 2. root> ()-> .streamer> + 3. root> ()-> .canceller> + + 4. .streamer> ()<=> .sleep> + 5. .canceller> ()-> .sleeper> + - calls `Portal.cancel_actor()` + + ''' + async def main(): + async with tractor.open_nursery( + # NOTE: to halt the peer tasks on ctxc, uncomment this. + debug_mode=debug_mode, + ) as an: + canceller: Portal = await an.start_actor( + 'canceller', + enable_modules=[__name__], + ) + sleeper: Portal = await an.start_actor( + 'sleeper', + enable_modules=[__name__], + ) + just_caller: Portal = await an.start_actor( + 'just_caller', # but i just met her? + enable_modules=[__name__], + ) + root: Actor = current_actor() + + try: + async with ( + sleeper.open_context( + sleep_forever, + expect_ctxc=True, + ) as (sleeper_ctx, sent), + + just_caller.open_context( + stream_from_peer, + ) as (caller_ctx, sent), + + canceller.open_context( + sleep_a_bit_then_cancel_peer, + ) as (canceller_ctx, sent), + + ): + ctxs: dict[str, Context] = { + 'sleeper': sleeper_ctx, + 'caller': caller_ctx, + 'canceller': canceller_ctx, + } + + try: + print('PRE CONTEXT RESULT') + res = await sleeper_ctx.result() + assert res + + # should never get here + pytest.fail( + 'Context.result() did not raise ctx-cancelled?' + ) + + # should always raise since this root task does + # not request the sleeper cancellation ;) + except ContextCancelled as ctxerr: + print( + 'CAUGHT REMOTE CONTEXT CANCEL\n\n' + f'{ctxerr}\n' + ) + + # canceller and caller peers should not + # have been remotely cancelled. + assert canceller_ctx.canceller is None + assert caller_ctx.canceller is None + + # we were not the actor, our peer was + assert not sleeper_ctx.cancel_acked + + assert ctxerr.canceller[0] == 'canceller' + + # XXX NOTE XXX: since THIS `ContextCancelled` + # HAS NOT YET bubbled up to the + # `sleeper.open_context().__aexit__()` this + # value is not yet set, however outside this + # block it should be. + assert not sleeper_ctx._scope.cancelled_caught + + # CASE_1: error-during-ctxc-handling, + if error_during_ctxerr_handling: + raise RuntimeError('Simulated error during teardown') + + # CASE_2: standard teardown inside in `.open_context()` block + raise + + # XXX SHOULD NEVER EVER GET HERE XXX + except BaseException as berr: + raise + + # XXX if needed to debug failure + # _err = berr + # await tractor.pause() + # await trio.sleep_forever() + + pytest.fail( + 'did not rx ctxc ?!?\n\n' + + f'{berr}\n' + ) + + else: + pytest.fail( + 'did not rx ctxc ?!?\n\n' + f'{ctxs}\n' + ) + + except ( + ContextCancelled, + RuntimeError, + )as loc_err: + _loc_err = loc_err + + # NOTE: the main state to check on `Context` is: + # - `.cancel_called` (bool of whether this side + # requested) + # - `.cancel_acked` (bool of whether a ctxc + # response was received due to cancel req). + # - `.maybe_error` (highest prio error to raise + # locally) + # - `.outcome` (final error or result value) + # - `.canceller` (uid of cancel-causing actor-task) + # - `._remote_error` (any `RemoteActorError` + # instance from other side of context) + # - `._local_error` (any error caught inside the + # `.open_context()` block). + # + # XXX: Deprecated and internal only + # - `.cancelled_caught` (maps to nursery cs) + # - now just use `._scope.cancelled_caught` + # since it maps to the internal (maps to nursery cs) + # + # TODO: are we really planning to use this tho? + # - `._cancel_msg` (any msg that caused the + # cancel) + + # CASE_1: error-during-ctxc-handling, + # - far end cancels due to peer 'canceller', + # - `ContextCancelled` relayed to this scope, + # - inside `.open_context()` ctxc is caught and + # a rte raised instead + # + # => block should raise the rte but all peers + # should be cancelled by US. + # + if error_during_ctxerr_handling: + # since we do a rte reraise above, the + # `.open_context()` error handling should have + # raised a local rte, thus the internal + # `.open_context()` enterer task's + # cancel-scope should have raised the RTE, NOT + # a `trio.Cancelled` due to a local + # `._scope.cancel()` call. + assert not sleeper_ctx._scope.cancelled_caught + + assert isinstance(loc_err, RuntimeError) + print(f'_loc_err: {_loc_err}\n') + # assert sleeper_ctx._local_error is _loc_err + # assert sleeper_ctx._local_error is _loc_err + assert not ( + loc_err + is sleeper_ctx.maybe_error + is sleeper_ctx.outcome + is sleeper_ctx._remote_error + ) + + # NOTE: this root actor task should have + # called `Context.cancel()` on the + # `.__aexit__()` to every opened ctx. + for name, ctx in ctxs.items(): + + # this root actor task should have + # cancelled all opened contexts except the + # sleeper which is obvi by the "canceller" + # peer. + re = ctx._remote_error + le = ctx._local_error + + assert ctx.cancel_called + + if ctx is sleeper_ctx: + assert 'canceller' in re.canceller + assert 'sleeper' in ctx.canceller + + if ctx is canceller_ctx: + assert ( + re.canceller + == + root.uid + ) + + else: # the other 2 ctxs + assert ( + re.canceller + == + canceller.channel.uid + ) + + # since the sleeper errors while handling a + # peer-cancelled (by ctxc) scenario, we expect + # that the `.open_context()` block DOES call + # `.cancel() (despite in this test case it + # being unecessary). + assert ( + sleeper_ctx.cancel_called + and + not sleeper_ctx.cancel_acked + ) + + # CASE_2: standard teardown inside in `.open_context()` block + # - far end cancels due to peer 'canceller', + # - `ContextCancelled` relayed to this scope and + # raised locally without any raise-during-handle, + # + # => inside `.open_context()` ctxc is raised and + # propagated + # + else: + # since sleeper_ctx.result() IS called above + # we should have (silently) absorbed the + # corresponding `ContextCancelled` for it and + # `._scope.cancel()` should never have been + # called. + assert not sleeper_ctx._scope.cancelled_caught + + assert isinstance(loc_err, ContextCancelled) + + # the received remote error's `.canceller` + # will of course be the "canceller" actor BUT + # the canceller set on the local handle to + # `sleeper_ctx` will be the "sleeper" uid + # since it's the actor that relayed us the + # error which was **caused** by the + # "canceller". + assert 'sleeper' in sleeper_ctx.canceller + assert 'canceller' == loc_err.canceller[0] + + # the sleeper's remote error is the error bubbled + # out of the context-stack above! + final_err = sleeper_ctx.outcome + assert ( + final_err is loc_err + is sleeper_ctx.maybe_error + is sleeper_ctx._remote_error + ) + + for name, ctx in ctxs.items(): + + re: BaseException|None = ctx._remote_error + le: BaseException|None = ctx._local_error + err = ctx.maybe_error + out = ctx.outcome + + # every ctx should error! + assert out is err + + # the recorded local erro should always be + # the same as the one raised by the + # `sleeper_ctx.result()` call + assert ( + le + and + le is loc_err + ) + + # root doesn't cancel sleeper since it's + # cancelled by its peer. + if ctx is sleeper_ctx: + assert re + assert ( + ctx._remote_error + is ctx.maybe_error + is ctx.outcome + is ctx._local_error + ) + + assert not ctx.cancel_called + assert not ctx.cancel_acked + + # since sleeper_ctx.result() IS called + # above we should have (silently) + # absorbed the corresponding + # `ContextCancelled` for it and thus + # the logic inside `.cancelled_caught` + # should trigger! + assert not ctx._scope.cancelled_caught + + elif ctx in ( + caller_ctx, + canceller_ctx, + ): + + assert not ctx._remote_error + + # neither of the `caller/canceller_ctx` should + # have called `ctx.cancel()` bc the + # canceller's task internally issues + # a `Portal.cancel_actor()` to the + # sleeper and thus never should call + # `ctx.cancel()` per say UNLESS the + # sleeper's `.result()` call above + # ctxc exception results in the + # canceller's + # `.open_context().__aexit__()` error + # handling to kick in BEFORE a remote + # error is delivered - which since + # we're asserting what we are above, + # that should normally be the case + # right? + # + assert not ctx.cancel_called + # + # assert ctx.cancel_called + # orig ^ + + # TODO: figure out the details of this..? + # if you look the `._local_error` here + # is a multi of ctxc + 2 Cancelleds? + # assert not ctx._scope.cancelled_caught + + assert ( + not ctx.cancel_called + and not ctx.cancel_acked + ) + assert not ctx._scope.cancelled_caught + + # elif ctx is canceller_ctx: + # assert not ctx._remote_error + + # XXX NOTE XXX: ONLY the canceller + # will get a self-cancelled outcome + # whilst everyone else gets + # a peer-caused cancellation! + # + # TODO: really we should avoid calling + # .cancel() whenever an interpeer + # cancel takes place since each + # reception of a ctxc + + else: + pytest.fail( + 'Uhh wut ctx is this?\n' + f'{ctx}\n' + ) + + # TODO: do we even need this flag? + # -> each context should have received + # a silently absorbed context cancellation + # in its remote nursery scope. + # assert ctx.chan.uid == ctx.canceller + + # NOTE: when an inter-peer cancellation + # occurred, we DO NOT expect this + # root-actor-task to have requested a cancel of + # the context since cancellation was caused by + # the "canceller" peer and thus + # `Context.cancel()` SHOULD NOT have been + # called inside + # `Portal.open_context().__aexit__()`. + assert not ( + sleeper_ctx.cancel_called + or + sleeper_ctx.cancel_acked + ) + + # XXX NOTE XXX: and see matching comment above but, + # the `._scope` is only set by `trio` AFTER the + # `.open_context()` block has exited and should be + # set in both outcomes including the case where + # ctx-cancel handling itself errors. + assert not sleeper_ctx._scope.cancelled_caught + assert _loc_err is sleeper_ctx._local_error + assert ( + sleeper_ctx.outcome + is sleeper_ctx.maybe_error + is sleeper_ctx._remote_error + ) + + raise # always to ensure teardown + + if error_during_ctxerr_handling: + with pytest.raises(RuntimeError) as excinfo: + trio.run(main) + else: + + with pytest.raises(ContextCancelled) as excinfo: + trio.run(main) + + assert excinfo.value.type == ContextCancelled + assert excinfo.value.canceller[0] == 'canceller' + + +@tractor.context +async def basic_echo_server( + ctx: Context, + peer_name: str = 'stepbro', + +) -> None: + ''' + Just the simplest `MsgStream` echo server which resays what + you told it but with its uid in front ;) + + ''' + actor: Actor = tractor.current_actor() + uid: tuple = actor.uid + await ctx.started(uid) + async with ctx.open_stream() as ipc: + async for msg in ipc: + + # repack msg pair with our uid + # as first element. + ( + client_uid, + i, + ) = msg + resp: tuple = ( + uid, + i, + ) + # OOF! looks like my runtime-error is causing a lockup + # assert 0 + await ipc.send(resp) + + +@tractor.context +async def serve_subactors( + ctx: Context, + peer_name: str, + +) -> None: + async with open_nursery() as an: + await ctx.started(peer_name) + async with ctx.open_stream() as reqs: + async for msg in reqs: + peer_name: str = msg + peer: Portal = await an.start_actor( + name=peer_name, + enable_modules=[__name__], + ) + print( + 'Spawning new subactor\n' + f'{peer_name}\n' + f'|_{peer}\n' + ) + await reqs.send(( + peer.chan.uid, + peer.chan.raddr, + )) + + print('Spawner exiting spawn serve loop!') + + +@tractor.context +async def client_req_subactor( + ctx: Context, + peer_name: str, + + # used to simulate a user causing an error to be raised + # directly in thread (like a KBI) to better replicate the + # case where a `modden` CLI client would hang afer requesting + # a `Context.cancel()` to `bigd`'s wks spawner. + reraise_on_cancel: str|None = None, + +) -> None: + # TODO: other cases to do with sub lifetimes: + # -[ ] test that we can have the server spawn a sub + # that lives longer then ctx with this client. + # -[ ] test that + + # open ctx with peer spawn server and ask it to spawn a little + # bro which we'll then connect and stream with. + async with ( + tractor.find_actor( + name='spawn_server', + raise_on_none=True, + + # TODO: we should be isolating this from other runs! + # => ideally so we can eventually use something like + # `pytest-xdist` Bo + # registry_addrs=bigd._reg_addrs, + ) as spawner, + + spawner.open_context( + serve_subactors, + peer_name=peer_name, + ) as (spawner_ctx, first), + ): + assert first == peer_name + await ctx.started( + 'yup i had brudder', + ) + + async with spawner_ctx.open_stream() as reqs: + + # send single spawn request to the server + await reqs.send(peer_name) + with trio.fail_after(3): + ( + sub_uid, + sub_raddr, + ) = await reqs.receive() + + + await tell_little_bro( + actor_name=sub_uid[0], + caller='client', + ) + + # TODO: test different scope-layers of + # cancellation? + # with trio.CancelScope() as cs: + try: + await trio.sleep_forever() + + # TODO: would be super nice to have a special injected + # cancel type here (maybe just our ctxc) but using + # some native mechanism in `trio` :p + except ( + trio.Cancelled + ) as err: + _err = err + if reraise_on_cancel: + errtype = globals()['__builtins__'][reraise_on_cancel] + assert errtype + to_reraise: BaseException = errtype() + print(f'client re-raising on cancel: {repr(to_reraise)}') + raise err + + raise + + # if cs.cancelled_caught: + # print('client handling expected KBI!') + # await ctx. + # await trio.sleep( + # await tractor.pause() + # await spawner_ctx.cancel() + + # cancel spawned sub-actor directly? + # await sub_ctx.cancel() + + # maybe cancel runtime? + # await sub.cancel_actor() + + +async def tell_little_bro( + actor_name: str, + caller: str = '' +): + # contact target actor, do a stream dialog. + async with ( + tractor.wait_for_actor( + name=actor_name + ) as lb, + lb.open_context( + basic_echo_server, + ) as (sub_ctx, first), + sub_ctx.open_stream( + basic_echo_server, + ) as echo_ipc, + ): + actor: Actor = current_actor() + uid: tuple = actor.uid + for i in range(100): + msg: tuple = ( + uid, + i, + ) + await echo_ipc.send(msg) + resp = await echo_ipc.receive() + print( + f'{caller} => {actor_name}: {msg}\n' + f'{caller} <= {actor_name}: {resp}\n' + ) + ( + sub_uid, + _i, + ) = resp + assert sub_uid != uid + assert _i == i + + +@pytest.mark.parametrize( + 'raise_client_error', + [None, 'KeyboardInterrupt'], +) +def test_peer_spawns_and_cancels_service_subactor( + debug_mode: bool, + raise_client_error: str, +): + # NOTE: this tests for the modden `mod wks open piker` bug + # discovered as part of implementing workspace ctx + # open-.pause()-ctx.cancel() as part of the CLI.. + + # -> start actor-tree (server) that offers sub-actor spawns via + # context API + # -> start another full actor-tree (client) which requests to the first to + # spawn over its `@context` ep / api. + # -> client actor cancels the context and should exit gracefully + # and the server's spawned child should cancel and terminate! + peer_name: str = 'little_bro' + + async def main(): + async with tractor.open_nursery( + # NOTE: to halt the peer tasks on ctxc, uncomment this. + debug_mode=debug_mode, + ) as an: + server: Portal = await an.start_actor( + (server_name := 'spawn_server'), + enable_modules=[__name__], + ) + print(f'Spawned `{server_name}`') + + client: Portal = await an.start_actor( + client_name := 'client', + enable_modules=[__name__], + ) + print(f'Spawned `{client_name}`') + + try: + async with ( + server.open_context( + serve_subactors, + peer_name=peer_name, + ) as (spawn_ctx, first), + + client.open_context( + client_req_subactor, + peer_name=peer_name, + reraise_on_cancel=raise_client_error, + ) as (client_ctx, client_says), + ): + print( + f'Server says: {first}\n' + f'Client says: {client_says}\n' + ) + + # attach to client-requested-to-spawn + # (grandchild of this root actor) "little_bro" + # and ensure we can also use it as an echo + # server. + async with tractor.wait_for_actor( + name=peer_name, + ) as sub: + assert sub + + print( + 'Sub-spawn came online\n' + f'portal: {sub}\n' + f'.uid: {sub.actor.uid}\n' + f'chan.raddr: {sub.chan.raddr}\n' + ) + await tell_little_bro( + actor_name=peer_name, + caller='root', + ) + + # signal client to raise a KBI + await client_ctx.cancel() + print('root cancelled client, checking that sub-spawn is down') + + async with tractor.find_actor( + name=peer_name, + ) as sub: + assert not sub + + print('root cancelling server/client sub-actors') + + # await tractor.pause() + res = await client_ctx.result(hide_tb=False) + assert isinstance(res, ContextCancelled) + assert client_ctx.cancel_acked + assert res.canceller == current_actor().uid + + await spawn_ctx.cancel() + # await server.cancel_actor() + + # since we called `.cancel_actor()`, `.cancel_ack` + # will not be set on the ctx bc `ctx.cancel()` was not + # called directly fot this confext. + except ContextCancelled as ctxc: + print('caught ctxc from contexts!') + assert ctxc.canceller == current_actor().uid + assert ctxc is spawn_ctx.outcome + assert ctxc is spawn_ctx.maybe_error + raise + + # assert spawn_ctx.cancel_acked + assert spawn_ctx.cancel_acked + assert client_ctx.cancel_acked + + await client.cancel_actor() + await server.cancel_actor() + + # WOA WOA WOA! we need this to close..!!!?? + # that's super bad XD + + # TODO: why isn't this working!?!? + # we're now outside the `.open_context()` block so + # the internal `Context._scope: CancelScope` should be + # gracefully "closed" ;) + + # assert spawn_ctx.cancelled_caught + + trio.run(main) diff --git a/tests/test_legacy_one_way_streaming.py b/tests/test_legacy_one_way_streaming.py index 17e94ba3..5d7787fa 100644 --- a/tests/test_legacy_one_way_streaming.py +++ b/tests/test_legacy_one_way_streaming.py @@ -9,7 +9,7 @@ import trio import tractor import pytest -from conftest import tractor_test +from tractor._testing import tractor_test def test_must_define_ctx(): diff --git a/tests/test_local.py b/tests/test_local.py index 97a83285..bb013043 100644 --- a/tests/test_local.py +++ b/tests/test_local.py @@ -7,7 +7,7 @@ import pytest import trio import tractor -from conftest import tractor_test +from tractor._testing import tractor_test @pytest.mark.trio diff --git a/tests/test_multi_program.py b/tests/test_multi_program.py index e7a3ac5c..d3eadabf 100644 --- a/tests/test_multi_program.py +++ b/tests/test_multi_program.py @@ -7,8 +7,10 @@ import time import pytest import trio import tractor -from conftest import ( +from tractor._testing import ( tractor_test, +) +from .conftest import ( sig_prog, _INT_SIGNAL, _INT_RETURN_CODE, diff --git a/tests/test_pubsub.py b/tests/test_pubsub.py index ababcb51..69f4c513 100644 --- a/tests/test_pubsub.py +++ b/tests/test_pubsub.py @@ -5,8 +5,7 @@ import pytest import trio import tractor from tractor.experimental import msgpub - -from conftest import tractor_test +from tractor._testing import tractor_test def test_type_checks(): diff --git a/tests/test_resource_cache.py b/tests/test_resource_cache.py index 0e7ad74d..d3859814 100644 --- a/tests/test_resource_cache.py +++ b/tests/test_resource_cache.py @@ -34,7 +34,6 @@ def test_resource_only_entered_once(key_on): global _resource _resource = 0 - kwargs = {} key = None if key_on == 'key_value': key = 'some_common_key' @@ -139,7 +138,7 @@ def test_open_local_sub_to_stream(): N local tasks using ``trionics.maybe_open_context():``. ''' - timeout = 3 if platform.system() != "Windows" else 10 + timeout: float = 3.6 if platform.system() != "Windows" else 10 async def main(): diff --git a/tests/test_rpc.py b/tests/test_rpc.py index 6d158961..b16f2f1d 100644 --- a/tests/test_rpc.py +++ b/tests/test_rpc.py @@ -1,6 +1,8 @@ -""" -RPC related -""" +''' +RPC (or maybe better labelled as "RTS: remote task scheduling"?) +related API and error checks. + +''' import itertools import pytest @@ -42,8 +44,13 @@ async def short_sleep(): (['tmp_mod'], 'import doggy', ModuleNotFoundError), (['tmp_mod'], '4doggy', SyntaxError), ], - ids=['no_mods', 'this_mod', 'this_mod_bad_func', 'fail_to_import', - 'fail_on_syntax'], + ids=[ + 'no_mods', + 'this_mod', + 'this_mod_bad_func', + 'fail_to_import', + 'fail_on_syntax', + ], ) def test_rpc_errors(arb_addr, to_call, testdir): """Test errors when making various RPC requests to an actor @@ -105,14 +112,16 @@ def test_rpc_errors(arb_addr, to_call, testdir): run() else: # underlying errors aren't propagated upwards (yet) - with pytest.raises(remote_err) as err: + with pytest.raises( + expected_exception=(remote_err, ExceptionGroup), + ) as err: run() # get raw instance from pytest wrapper value = err.value # might get multiple `trio.Cancelled`s as well inside an inception - if isinstance(value, trio.MultiError): + if isinstance(value, ExceptionGroup): value = next(itertools.dropwhile( lambda exc: not isinstance(exc, tractor.RemoteActorError), value.exceptions diff --git a/tests/test_runtime.py b/tests/test_runtime.py index 127138c2..55553dd9 100644 --- a/tests/test_runtime.py +++ b/tests/test_runtime.py @@ -8,7 +8,7 @@ import pytest import trio import tractor -from conftest import tractor_test +from tractor._testing import tractor_test _file_path: str = '' @@ -64,7 +64,8 @@ async def test_lifetime_stack_wipes_tmpfile( except ( tractor.RemoteActorError, - tractor.BaseExceptionGroup, + # tractor.BaseExceptionGroup, + BaseExceptionGroup, ): pass diff --git a/tests/test_spawning.py b/tests/test_spawning.py index 17798c09..3f4772e9 100644 --- a/tests/test_spawning.py +++ b/tests/test_spawning.py @@ -8,7 +8,7 @@ import pytest import trio import tractor -from conftest import tractor_test +from tractor._testing import tractor_test data_to_pass_down = {'doggy': 10, 'kitty': 4} diff --git a/tests/test_task_broadcasting.py b/tests/test_task_broadcasting.py index 9f4a1fed..5e18e10a 100644 --- a/tests/test_task_broadcasting.py +++ b/tests/test_task_broadcasting.py @@ -86,7 +86,7 @@ async def open_sequence_streamer( ) as (ctx, first): assert first is None - async with ctx.open_stream(backpressure=True) as stream: + async with ctx.open_stream(allow_overruns=True) as stream: yield stream await portal.cancel_actor() @@ -413,8 +413,8 @@ def test_ensure_slow_consumers_lag_out( seq = brx._state.subs[brx.key] assert seq == len(brx._state.queue) - 1 - # all backpressured entries in the underlying - # channel should have been copied into the caster + # all no_overruns entries in the underlying + # channel should have been copied into the bcaster # queue trailing-window async for i in rx: print(f'bped: {i}') diff --git a/tests/test_trioisms.py b/tests/test_trioisms.py index 5b19f50d..27dc6c34 100644 --- a/tests/test_trioisms.py +++ b/tests/test_trioisms.py @@ -5,7 +5,7 @@ want to see changed. ''' import pytest import trio -from trio_typing import TaskStatus +from trio import TaskStatus @pytest.mark.parametrize( diff --git a/tractor/__init__.py b/tractor/__init__.py index 12123a2d..7af40c6e 100644 --- a/tractor/__init__.py +++ b/tractor/__init__.py @@ -15,18 +15,18 @@ # along with this program. If not, see . """ -tractor: structured concurrent "actors". +tractor: structured concurrent ``trio``-"actors". """ -from exceptiongroup import BaseExceptionGroup - from ._clustering import open_actor_cluster from ._ipc import Channel +from ._context import ( + Context, # the type + context, # a func-decorator +) from ._streaming import ( - Context, MsgStream, stream, - context, ) from ._discovery import ( get_arbiter, diff --git a/tractor/_child.py b/tractor/_child.py index 91aaec4f..bd1e830e 100644 --- a/tractor/_child.py +++ b/tractor/_child.py @@ -18,8 +18,6 @@ This is the "bootloader" for actors started using the native trio backend. """ -import sys -import trio import argparse from ast import literal_eval @@ -37,8 +35,6 @@ def parse_ipaddr(arg): return (str(host), int(port)) -from ._entry import _trio_main - if __name__ == "__main__": parser = argparse.ArgumentParser() diff --git a/tractor/_context.py b/tractor/_context.py new file mode 100644 index 00000000..50f7bfa5 --- /dev/null +++ b/tractor/_context.py @@ -0,0 +1,2472 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +''' +The fundamental cross process SC abstraction: an inter-actor, +cancel-scope linked task "context". + +A ``Context`` is very similar to the ``trio.Nursery.cancel_scope`` built +into each ``trio.Nursery`` except it links the lifetimes of memory space +disjoint, parallel executing tasks in separate actors. + +''' +from __future__ import annotations +from collections import deque +from contextlib import asynccontextmanager as acm +from dataclasses import ( + dataclass, + field, +) +from functools import partial +import inspect +from pprint import pformat +from typing import ( + Any, + Callable, + AsyncGenerator, + TYPE_CHECKING, +) +import warnings + +import trio + +from ._exceptions import ( + ContextCancelled, + InternalError, + RemoteActorError, + StreamOverrun, + pack_error, + unpack_error, + _raise_from_no_key_in_msg, +) +from .log import get_logger +from .msg import NamespacePath +from ._ipc import Channel +from ._streaming import MsgStream +from ._state import ( + current_actor, + debug_mode, +) + +if TYPE_CHECKING: + from ._portal import Portal + from ._runtime import Actor + from ._ipc import MsgTransport + + +log = get_logger(__name__) + + +async def _drain_to_final_msg( + ctx: Context, + + hide_tb: bool = True, + msg_limit: int = 6, + +) -> list[dict]: + ''' + Drain IPC msgs delivered to the underlying rx-mem-chan + `Context._recv_chan` from the runtime in search for a final + result or error msg. + + The motivation here is to ideally capture errors during ctxc + conditions where a canc-request/or local error is sent but the + local task also excepts and enters the + `Portal.open_context().__aexit__()` block wherein we prefer to + capture and raise any remote error or ctxc-ack as part of the + `ctx.result()` cleanup and teardown sequence. + + ''' + __tracebackhide__: bool = hide_tb + raise_overrun: bool = not ctx._allow_overruns + + # wait for a final context result by collecting (but + # basically ignoring) any bi-dir-stream msgs still in transit + # from the far end. + pre_result_drained: list[dict] = [] + while not ( + ctx.maybe_error + and not ctx._final_result_is_set() + ): + try: + # TODO: can remove? + # await trio.lowlevel.checkpoint() + + # NOTE: this REPL usage actually works here dawg! Bo + # from .devx._debug import pause + # await pause() + + # TODO: bad idea? + # -[ ] wrap final outcome channel wait in a scope so + # it can be cancelled out of band if needed? + # + # with trio.CancelScope() as res_cs: + # ctx._res_scope = res_cs + # msg: dict = await ctx._recv_chan.receive() + # if res_cs.cancelled_caught: + + # TODO: ensure there's no more hangs, debugging the + # runtime pretty preaase! + # from .devx._debug import pause + # await pause() + + # TODO: can remove this finally? + # we have no more need for the sync draining right + # since we're can kinda guarantee the async + # `.receive()` below will never block yah? + # + # if ( + # ctx._cancel_called and ( + # ctx.cancel_acked + # # or ctx.chan._cancel_called + # ) + # # or not ctx._final_result_is_set() + # # ctx.outcome is not + # # or ctx.chan._closed + # ): + # try: + # msg: dict = await ctx._recv_chan.receive_nowait()() + # except trio.WouldBlock: + # log.warning( + # 'When draining already `.cancel_called` ctx!\n' + # 'No final msg arrived..\n' + # ) + # break + # else: + # msg: dict = await ctx._recv_chan.receive() + + # TODO: don't need it right jefe? + # with trio.move_on_after(1) as cs: + # if cs.cancelled_caught: + # from .devx._debug import pause + # await pause() + + # pray to the `trio` gawds that we're corrent with this + msg: dict = await ctx._recv_chan.receive() + + # NOTE: we get here if the far end was + # `ContextCancelled` in 2 cases: + # 1. we requested the cancellation and thus + # SHOULD NOT raise that far end error, + # 2. WE DID NOT REQUEST that cancel and thus + # SHOULD RAISE HERE! + except trio.Cancelled: + + # CASE 2: mask the local cancelled-error(s) + # only when we are sure the remote error is + # the source cause of this local task's + # cancellation. + ctx.maybe_raise() + + # CASE 1: we DID request the cancel we simply + # continue to bubble up as normal. + raise + + try: + ctx._result: Any = msg['return'] + log.runtime( + 'Context delivered final draining msg:\n' + f'{pformat(msg)}' + ) + # XXX: only close the rx mem chan AFTER + # a final result is retreived. + # if ctx._recv_chan: + # await ctx._recv_chan.aclose() + # TODO: ^ we don't need it right? + break + + except KeyError: + # always capture unexpected/non-result msgs + pre_result_drained.append(msg) + + if 'yield' in msg: + # far end task is still streaming to us so discard + # and report per local context state. + if ( + (ctx._stream.closed + and (reason := 'stream was already closed') + ) + or (ctx.cancel_acked + and (reason := 'ctx cancelled other side') + ) + or (ctx._cancel_called + and (reason := 'ctx called `.cancel()`') + ) + or (len(pre_result_drained) > msg_limit + and (reason := f'"yield" limit={msg_limit}') + ) + ): + log.cancel( + 'Cancelling `MsgStream` drain since ' + f'{reason}\n\n' + f'<= {ctx.chan.uid}\n' + f' |_{ctx._nsf}()\n\n' + f'=> {ctx._task}\n' + f' |_{ctx._stream}\n\n' + + f'{pformat(msg)}\n' + ) + return pre_result_drained + + # drain up to the `msg_limit` hoping to get + # a final result or error/ctxc. + else: + log.warning( + 'Ignoring "yield" msg during `ctx.result()` drain..\n' + f'<= {ctx.chan.uid}\n' + f' |_{ctx._nsf}()\n\n' + f'=> {ctx._task}\n' + f' |_{ctx._stream}\n\n' + + f'{pformat(msg)}\n' + ) + continue + + # TODO: work out edge cases here where + # a stream is open but the task also calls + # this? + # -[ ] should be a runtime error if a stream is open + # right? + elif 'stop' in msg: + log.cancel( + 'Remote stream terminated due to "stop" msg:\n\n' + f'{pformat(msg)}\n' + ) + continue + + # It's an internal error if any other msg type without + # a`'cid'` field arrives here! + if not msg.get('cid'): + raise InternalError( + 'Unexpected cid-missing msg?\n\n' + f'{msg}\n' + ) + + # XXX fallthrough to handle expected error XXX + # TODO: replace this with `ctx.maybe_raise()` + # + # TODO: would this be handier for this case maybe? + # async with maybe_raise_on_exit() as raises: + # if raises: + # log.error('some msg about raising..') + + re: Exception|None = ctx._remote_error + if re: + log.critical( + 'Remote ctx terminated due to "error" msg:\n' + f'{re}' + ) + assert msg is ctx._cancel_msg + # NOTE: this solved a super dupe edge case XD + # this was THE super duper edge case of: + # - local task opens a remote task, + # - requests remote cancellation of far end + # ctx/tasks, + # - needs to wait for the cancel ack msg + # (ctxc) or some result in the race case + # where the other side's task returns + # before the cancel request msg is ever + # rxed and processed, + # - here this surrounding drain loop (which + # iterates all ipc msgs until the ack or + # an early result arrives) was NOT exiting + # since we are the edge case: local task + # does not re-raise any ctxc it receives + # IFF **it** was the cancellation + # requester.. + # will raise if necessary, ow break from + # loop presuming any error terminates the + # context! + ctx._maybe_raise_remote_err( + re, + # NOTE: obvi we don't care if we + # overran the far end if we're already + # waiting on a final result (msg). + # raise_overrun_from_self=False, + raise_overrun_from_self=raise_overrun, + ) + + break # OOOOOF, yeah obvi we need this.. + + # XXX we should never really get here + # right! since `._deliver_msg()` should + # always have detected an {'error': ..} + # msg and already called this right!?! + elif error := unpack_error( + msg=msg, + chan=ctx._portal.channel, + hide_tb=False, + ): + log.critical('SHOULD NEVER GET HERE!?') + assert msg is ctx._cancel_msg + assert error.msgdata == ctx._remote_error.msgdata + from ._debug import pause + await pause() + ctx._maybe_cancel_and_set_remote_error(error) + ctx._maybe_raise_remote_err(error) + + else: + # bubble the original src key error + raise + else: + log.cancel( + 'Skipping `MsgStream` drain since final outcome is set\n\n' + f'{ctx.outcome}\n' + ) + + return pre_result_drained + + +class Unresolved: + ''' + Placeholder value for `Context._result` until + a final return value or raised error is resolved. + + ''' + ... + + +# TODO: make this a .msg.types.Struct! +# -[ ] ideally we can freeze it +# -[ ] let's us do field diffing nicely in tests Bo +@dataclass +class Context: + ''' + An inter-actor, SC transitive, `trio.Task` communication context. + + NB: This class should **never be instatiated directly**, it is allocated + by the runtime in 2 ways: + - by entering ``Portal.open_context()`` which is the primary + public API for any "caller" task or, + - by the RPC machinery's `._runtime._invoke()` as a `ctx` arg + to a remotely scheduled "callee" function. + + AND is always constructed using the below ``mk_context()``. + + Allows maintaining task or protocol specific state between + 2 cancel-scope-linked, communicating and parallel executing + `trio.Task`s. Contexts are allocated on each side of any task + RPC-linked msg dialog, i.e. for every request to a remote + actor from a `Portal`. On the "callee" side a context is + always allocated inside ``._runtime._invoke()``. + + # TODO: more detailed writeup on cancellation, error and + # streaming semantics.. + + A context can be cancelled and (possibly eventually restarted) from + either side of the underlying IPC channel, it can also open task + oriented message streams, and acts more or less as an IPC aware + inter-actor-task ``trio.CancelScope``. + + ''' + chan: Channel + cid: str # "context id", more or less a unique linked-task-pair id + + _actor: Actor + + # the "feeder" channels for delivering message values to the + # local task from the runtime's msg processing loop. + _recv_chan: trio.MemoryReceiveChannel + _send_chan: trio.MemorySendChannel + + # full "namespace-path" to target RPC function + _nsf: NamespacePath + + # the "invocation type" of the far end task-entry-point + # function, normally matching a logic block inside + # `._runtime.invoke()`. + _remote_func_type: str | None = None + + # NOTE: (for now) only set (a portal) on the caller side since + # the callee doesn't generally need a ref to one and should + # normally need to explicitly ask for handle to its peer if + # more the the `Context` is needed? + _portal: Portal | None = None + + # NOTE: each side of the context has its own cancel scope + # which is exactly the primitive that allows for + # cross-actor-task-supervision and thus SC. + _scope: trio.CancelScope|None = None + _task: trio.lowlevel.Task|None = None + + # TODO: cs around result waiting so we can cancel any + # permanently blocking `._recv_chan.receive()` call in + # a drain loop? + # _res_scope: trio.CancelScope|None = None + + # on a clean exit there should be a final value + # delivered from the far end "callee" task, so + # this value is only set on one side. + # _result: Any | int = None + _result: Any|Unresolved = Unresolved + + # if the local "caller" task errors this value is always set + # to the error that was captured in the + # `Portal.open_context().__aexit__()` teardown block OR, in + # 2 special cases when an (maybe) expected remote error + # arrives that we purposely swallow silently: + # - `ContextCancelled` with `.canceller` set to our uid: + # a self-cancel, + # - `RemoteActorError[StreamOverrun]` which was caught during + # a self-cancellation teardown msg drain. + _local_error: BaseException|None = None + + # if the either side gets an error from the other + # this value is set to that error unpacked from an + # IPC msg. + _remote_error: BaseException|None = None + + # only set if an actor-local task called `.cancel()` + _cancel_called: bool = False # did WE request cancel of the far end? + + # TODO: do we even need this? we can assume that if we're + # cancelled that the other side is as well, so maybe we should + # instead just have a `.canceller` pulled from the + # `ContextCancelled`? + _canceller: tuple[str, str] | None = None + + # NOTE: we try to ensure assignment of a "cancel msg" since + # there's always going to be an "underlying reason" that any + # context was closed due to either a remote side error or + # a call to `.cancel()` which triggers `ContextCancelled`. + _cancel_msg: str|dict|None = None + + # NOTE: this state var used by the runtime to determine if the + # `pdbp` REPL is allowed to engage on contexts terminated via + # a `ContextCancelled` due to a call to `.cancel()` triggering + # "graceful closure" on either side: + # - `._runtime._invoke()` will check this flag before engaging + # the crash handler REPL in such cases where the "callee" + # raises the cancellation, + # - `.devx._debug.lock_tty_for_child()` will set it to `False` if + # the global tty-lock has been configured to filter out some + # actors from being able to acquire the debugger lock. + _enter_debugger_on_cancel: bool = True + + # init and streaming state + _started_called: bool = False + _stream_opened: bool = False + _stream: MsgStream|None = None + + # overrun handling machinery + # NOTE: none of this provides "backpressure" to the remote + # task, only an ability to not lose messages when the local + # task is configured to NOT transmit ``StreamOverrun``s back + # to the other side. + _overflow_q: deque[dict] = field( + default_factory=partial( + deque, + maxlen=616, + ) + ) + + # NOTE: this was originally a legacy interface from when we + # were raising remote errors (set to `._remote_error`) by + # starting a task inside this nursery that simply raised the + # boxed exception. NOW, it's used for spawning overrun queuing + # tasks when `.allow_overruns == True` !!! + _scope_nursery: trio.Nursery|None = None + + # streaming overrun state tracking + _in_overrun: bool = False + _allow_overruns: bool = False + + + def __str__(self) -> str: + ds: str = '=' + # ds: str = ': ' + + # only show if opened + maybe_stream_repr: str = '' + if stream := self._stream: + # TODO: a `MsgStream.reprol()` !! + # f' stream{ds}{self._stream}\n' + # f' {self._stream}\n' + maybe_stream_repr: str = ( + f' {stream}\n' + ) + + outcome_str: str = self.repr_outcome( + show_error_fields=True + ) + outcome_typ_str: str = self.repr_outcome( + type_only=True + ) + + return ( + f'\n' + ) + # NOTE: making this return a value that can be passed to + # `eval()` is entirely **optional** dawggg B) + # https://docs.python.org/3/library/functions.html#repr + # https://docs.python.org/3/reference/datamodel.html#object.__repr__ + # + # XXX: Currently we target **readability** from a (console) + # logging perspective over `eval()`-ability since we do NOT + # target serializing non-struct instances! + # def __repr__(self) -> str: + __repr__ = __str__ + + @property + def cancel_called(self) -> bool: + ''' + Records whether cancellation has been requested for this context + by a call to `.cancel()` either due to, + - either an explicit call by some local task, + - or an implicit call due to an error caught inside + the ``Portal.open_context()`` block. + + ''' + return self._cancel_called + + @property + def canceller(self) -> tuple[str, str]|None: + ''' + ``Actor.uid: tuple[str, str]`` of the (remote) + actor-process who's task was cancelled thus causing this + (side of the) context to also be cancelled. + + ''' + if canc := self._canceller: + return tuple(canc) + + return None + + def _is_self_cancelled( + self, + remote_error: Exception|None = None, + + ) -> bool: + + if not self._cancel_called: + return False + + re: BaseException|None = ( + remote_error + or self._remote_error + ) + if not re: + return False + + if from_uid := re.src_uid: + from_uid: tuple = tuple(from_uid) + + our_uid: tuple = self._actor.uid + our_canceller = self.canceller + + return bool( + isinstance(re, ContextCancelled) + and from_uid == self.chan.uid + and re.canceller == our_uid + and our_canceller == from_uid + ) + + @property + def cancel_acked(self) -> bool: + ''' + Records whether the task on the remote side of this IPC + context acknowledged a cancel request via a relayed + `ContextCancelled` with the `.canceller` attr set to the + `Actor.uid` of the local actor who's task entered + `Portal.open_context()`. + + This will only be `True` when `.cancel()` is called and + the ctxc response contains a `.canceller: tuple` field + equal to the uid of the calling task's actor. + + ''' + return self._is_self_cancelled() + + @property + def cancelled_caught(self) -> bool: + ''' + Exactly the value of `self._scope.cancelled_caught` + (delegation) and should only be (able to be read as) + `True` for a `.side == "caller"` ctx wherein the + `Portal.open_context()` block was exited due to a call to + `._scope.cancel()` - which should only ocurr in 2 cases: + + - a caller side calls `.cancel()`, the far side cancels + and delivers back a `ContextCancelled` (making + `.cancel_acked == True`) and `._scope.cancel()` is + called by `._maybe_cancel_and_set_remote_error()` which + in turn cancels all `.open_context()` started tasks + (including any overrun queuing ones). + => `._scope.cancelled_caught == True` by normal `trio` + cs semantics. + + - a caller side is delivered a `._remote_error: + RemoteActorError` via `._deliver_msg()` and a transitive + call to `_maybe_cancel_and_set_remote_error()` calls + `._scope.cancel()` and that cancellation eventually + results in `trio.Cancelled`(s) caught in the + `.open_context()` handling around the @acm's `yield`. + + Only as an FYI, in the "callee" side case it can also be + set but never is readable by any task outside the RPC + machinery in `._invoke()` since,: + - when a callee side calls `.cancel()`, `._scope.cancel()` + is called immediately and handled specially inside + `._invoke()` to raise a `ContextCancelled` which is then + sent to the caller side. + + However, `._scope.cancelled_caught` can NEVER be + accessed/read as `True` by any RPC invoked task since it + will have terminated before the cs block exit. + + ''' + return bool( + # the local scope was cancelled either by + # remote error or self-request + (self._scope and self._scope.cancelled_caught) + + # the local scope was never cancelled + # and instead likely we received a remote side + # # cancellation that was raised inside `.result()` + # or ( + # (se := self._local_error) + # and se is re + # ) + ) + + # @property + # def is_waiting_result(self) -> bool: + # return bool(self._res_scope) + + @property + def side(self) -> str: + ''' + Return string indicating which task this instance is wrapping. + + ''' + return 'caller' if self._portal else 'callee' + + async def send_yield( + self, + data: Any, + + ) -> None: + + warnings.warn( + "`Context.send_yield()` is now deprecated. " + "Use ``MessageStream.send()``. ", + DeprecationWarning, + stacklevel=2, + ) + await self.chan.send({'yield': data, 'cid': self.cid}) + + async def send_stop(self) -> None: + # await pause() + await self.chan.send({ + 'stop': True, + 'cid': self.cid + }) + + def _maybe_cancel_and_set_remote_error( + self, + error: BaseException, + + ) -> None: + ''' + (Maybe) cancel this local scope due to a received remote + error (normally via an IPC msg) which the actor runtime + routes to this context. + + Acts as a form of "relay" for a remote error raised in the + corresponding remote task's `Context` wherein the next time + the local task exectutes a checkpoint, a `trio.Cancelled` + will be raised and depending on the type and source of the + original remote error, and whether or not the local task + called `.cancel()` itself prior, an equivalent + `ContextCancelled` or `RemoteActorError` wrapping the + remote error may be raised here by any of, + + - `Portal.open_context()` + - `Portal.result()` + - `Context.open_stream()` + - `Context.result()` + + when called/closed by actor local task(s). + + NOTEs: + - It is expected that the caller has previously unwrapped + the remote error using a call to `unpack_error()` and + provides that output exception value as the input + `error` argument *here*. + + TODOs: + - If this is an error message from a context opened by + `Portal.open_context()` (ideally) we want to interrupt + any ongoing local tasks operating within that + `Context`'s cancel-scope so as to be notified ASAP of + the remote error and engage any caller handling (eg. + for cross-process task supervision). + + - In some cases we may want to raise the remote error + immediately since there is no guarantee the locally + operating task(s) will attempt to execute a checkpoint + any time soon; in such cases there are 2 possible + approaches depending on the current task's work and + wrapping "thread" type: + + - Currently we only support + a `trio`-native-and-graceful approach: we only ever + wait for local tasks to exec a next + `trio.lowlevel.checkpoint()` assuming that any such + task must do so to interact with the actor runtime + and IPC interfaces and will then be cancelled by + the internal `._scope` block. + + - (NOT IMPLEMENTED) system-level-aggressive: maybe we + could eventually interrupt sync code (invoked using + `trio.to_thread` or some other adapter layer) with + a signal (a custom unix one for example? + https://stackoverflow.com/a/5744185) depending on the + task's wrapping thread-type such that long running + sync code should never cause the delay of actor + supervision tasks such as cancellation and respawn + logic. + + ''' + # XXX: currently this should only be used when + # `Portal.open_context()` has been opened since it's + # assumed that other portal APIs like, + # - `Portal.run()`, + # - `ActorNursery.run_in_actor()` + # do their own error checking at their own call points and + # result processing. + + # TODO: never do this right? + # if self._remote_error: + # return + + # XXX: denote and set the remote side's error so that + # after we cancel whatever task is the opener of this + # context, it can raise or swallow that error + # appropriately. + log.runtime( + 'Setting remote error for ctx\n\n' + f'<= remote ctx uid: {self.chan.uid}\n' + f'=>{error}' + ) + self._remote_error: BaseException = error + + # self-cancel (ack) or, + # peer propagated remote cancellation. + if isinstance(error, ContextCancelled): + + whom: str = ( + 'us' if error.canceller == self._actor.uid + else 'peer' + ) + log.cancel( + f'IPC context cancelled by {whom}!\n\n' + f'{error}' + ) + + else: + log.error( + f'Remote context error:\n\n' + + f'{error}\n' + f'{pformat(self)}\n' + ) + + # always record the cancelling actor's uid since its + # cancellation state is linked and we want to know + # which process was the cause / requester of the + # cancellation. + maybe_error_src: tuple = getattr( + error, + 'src_uid', + None, + ) + self._canceller = ( + maybe_error_src + or + # XXX: in the case we get a non-boxed error? + # -> wait but this should never happen right? + self.chan.uid + ) + + # Cancel the local `._scope`, catch that + # `._scope.cancelled_caught` and re-raise any remote error + # once exiting (or manually calling `.result()`) the + # `.open_context()` block. + cs: trio.CancelScope = self._scope + if ( + cs + + # XXX this is an expected cancel request response + # message and we **don't need to raise it** in the + # local cancel `._scope` since it will potentially + # override a real error. After this method returns + # if `._cancel_called` then `.cancel_acked and .cancel_called` + # always should be set. + and not self._is_self_cancelled() + + and not cs.cancel_called + and not cs.cancelled_caught + ): + # TODO: it'd sure be handy to inject our own + # `trio.Cancelled` subtype here ;) + # https://github.com/goodboy/tractor/issues/368 + self._scope.cancel() + + # TODO: maybe we should also call `._res_scope.cancel()` if it + # exists to support cancelling any drain loop hangs? + + # TODO: add to `Channel`? + @property + def dst_maddr(self) -> str: + chan: Channel = self.chan + dst_addr, dst_port = chan.raddr + trans: MsgTransport = chan.transport + # cid: str = self.cid + # cid_head, cid_tail = cid[:6], cid[-6:] + return ( + f'/ipv4/{dst_addr}' + f'/{trans.name_key}/{dst_port}' + # f'/{self.chan.uid[0]}' + # f'/{self.cid}' + + # f'/cid={cid_head}..{cid_tail}' + # TODO: ? not use this ^ right ? + ) + + dmaddr = dst_maddr + + @property + def repr_rpc( + self, + ) -> str: + # TODO: how to show the transport interchange fmt? + # codec: str = self.chan.transport.codec_key + outcome_str: str = self.repr_outcome( + show_error_fields=True, + type_only=True, + ) + return ( + # f'{self._nsf}() -{{{codec}}}-> {repr(self.outcome)}:' + f'{self._nsf}() -> {outcome_str}:' + ) + + async def cancel( + self, + timeout: float = 0.616, + + ) -> None: + ''' + Cancel this inter-actor IPC context by requestng the + remote side's cancel-scope-linked `trio.Task` by calling + `._scope.cancel()` and delivering an `ContextCancelled` + ack msg in reponse. + + Behaviour: + --------- + - after the far end cancels, the `.cancel()` calling side + should receive a `ContextCancelled` with the + `.canceller: tuple` uid set to the current `Actor.uid`. + + - timeout (quickly) on failure to rx this ACK error-msg in + an attempt to sidestep 2-generals when the transport + layer fails. + + Note, that calling this method DOES NOT also necessarily + result in `Context._scope.cancel()` being called + **locally**! + + => That is, an IPC `Context` (this) **does not** + have the same semantics as a `trio.CancelScope`. + + If the caller (who entered the `Portal.open_context()`) + desires that the internal block's cancel-scope be + cancelled it should open its own `trio.CancelScope` and + manage it as needed. + + ''' + side: str = self.side + self._cancel_called: bool = True + + header: str = ( + f'Cancelling ctx with peer from {side.upper()} side\n\n' + ) + reminfo: str = ( + # ' =>\n' + f'Context.cancel() => {self.chan.uid}\n' + # f'{self.chan.uid}\n' + f' |_ @{self.dst_maddr}\n' + f' >> {self.repr_rpc}\n' + # f' >> {self._nsf}() -> {codec}[dict]:\n\n' + # TODO: pull msg-type from spec re #320 + ) + + # CALLER side task + # ------ - ------ + # Aka the one that entered `Portal.open_context()` + # + # NOTE: on the call side we never manually call + # `._scope.cancel()` since we expect the eventual + # `ContextCancelled` from the other side to trigger this + # when the runtime finally receives it during teardown + # (normally in `.result()` called from + # `Portal.open_context().__aexit__()`) + if side == 'caller': + if not self._portal: + raise InternalError( + 'No portal found!?\n' + 'Why is this supposed caller context missing it?' + ) + + cid: str = self.cid + with trio.move_on_after(timeout) as cs: + cs.shield = True + log.cancel( + header + + + reminfo + ) + + # NOTE: we're telling the far end actor to cancel a task + # corresponding to *this actor*. The far end local channel + # instance is passed to `Actor._cancel_task()` implicitly. + await self._portal.run_from_ns( + 'self', + '_cancel_task', + cid=cid, + ) + + if cs.cancelled_caught: + # XXX: there's no way to know if the remote task was indeed + # cancelled in the case where the connection is broken or + # some other network error occurred. + # if not self._portal.channel.connected(): + if not self.chan.connected(): + log.cancel( + 'May have failed to cancel remote task?\n' + f'{reminfo}' + ) + else: + log.cancel( + 'Timed out on cancel request of remote task?\n' + f'{reminfo}' + ) + + # CALLEE side task + # ------ - ------ + # Aka the one that DID NOT EVER enter a `Portal.open_context()` + # and instead was constructed and scheduled as an + # `_invoke()` RPC task. + # + # NOTE: on this side we ALWAYS cancel the local scope + # since the caller expects a `ContextCancelled` to be sent + # from `._runtime._invoke()` back to the other side. The + # logic for catching the result of the below + # `._scope.cancel()` is inside the `._runtime._invoke()` + # context RPC handling block. + else: + log.cancel( + header + + + reminfo + ) + # TODO: should we have an explicit cancel message + # or is relaying the local `trio.Cancelled` as an + # {'error': trio.Cancelled, cid: "blah"} enough? + # This probably gets into the discussion in + # https://github.com/goodboy/tractor/issues/36 + assert self._scope + self._scope.cancel() + + # TODO? should we move this to `._streaming` much like we + # moved `Portal.open_context()`'s def to this mod? + @acm + async def open_stream( + self, + allow_overruns: bool|None = False, + msg_buffer_size: int|None = None, + + ) -> AsyncGenerator[MsgStream, None]: + ''' + Open a ``MsgStream``, a bi-directional stream connected to the + cross-actor (far end) task for this ``Context``. + + This context manager must be entered on both the caller and + callee for the stream to logically be considered "connected". + + A ``MsgStream`` is currently "one-shot" use, meaning if you + close it you can not "re-open" it for streaming and instead you + must re-establish a new surrounding ``Context`` using + ``Portal.open_context()``. In the future this may change but + currently there seems to be no obvious reason to support + "re-opening": + - pausing a stream can be done with a message. + - task errors will normally require a restart of the entire + scope of the inter-actor task context due to the nature of + ``trio``'s cancellation system. + + ''' + actor: Actor = self._actor + + # If the surrounding context has been cancelled by some + # task with a handle to THIS, we error here immediately + # since it likely means the surrounding lexical-scope has + # errored, been `trio.Cancelled` or at the least + # `Context.cancel()` was called by some task. + if self._cancel_called: + + # XXX NOTE: ALWAYS RAISE any remote error here even if + # it's an expected `ContextCancelled` due to a local + # task having called `.cancel()`! + # + # WHY: we expect the error to always bubble up to the + # surrounding `Portal.open_context()` call and be + # absorbed there (silently) and we DO NOT want to + # actually try to stream - a cancel msg was already + # sent to the other side! + self.maybe_raise( + raise_ctxc_from_self_call=True, + ) + # NOTE: this is diff then calling + # `._maybe_raise_remote_err()` specifically + # because we want to raise a ctxc on any task entering this `.open_stream()` + # AFTER cancellation was already been requested, + # we DO NOT want to absorb any ctxc ACK silently! + # if self._remote_error: + # raise self._remote_error + + # XXX NOTE: if no `ContextCancelled` has been responded + # back from the other side (yet), we raise a different + # runtime error indicating that this task's usage of + # `Context.cancel()` and then `.open_stream()` is WRONG! + task: str = trio.lowlevel.current_task().name + raise RuntimeError( + 'Stream opened after `Context.cancel()` called..?\n' + f'task: {actor.uid[0]}:{task}\n' + f'{self}' + ) + + if ( + not self._portal + and not self._started_called + ): + raise RuntimeError( + 'Context.started()` must be called before opening a stream' + ) + + # NOTE: in one way streaming this only happens on the + # caller side inside `Actor.start_remote_task()` 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. + ctx: Context = actor.get_context( + chan=self.chan, + cid=self.cid, + nsf=self._nsf, + # side=self.side, + + msg_buffer_size=msg_buffer_size, + allow_overruns=allow_overruns, + ) + ctx._allow_overruns: bool = allow_overruns + assert ctx is self + + # XXX: If the underlying channel feeder receive mem chan has + # been closed then likely client code has already exited + # a ``.open_stream()`` block prior or there was some other + # unanticipated error or cancellation from ``trio``. + + if ctx._recv_chan._closed: + raise trio.ClosedResourceError( + 'The underlying channel for this stream was already closed!\n' + ) + + # NOTE: implicitly this will call `MsgStream.aclose()` on + # `.__aexit__()` due to stream's parent `Channel` type! + # + # XXX NOTE XXX: ensures the stream is "one-shot use", + # which specifically means that on exit, + # - signal ``trio.EndOfChannel``/``StopAsyncIteration`` to + # the far end indicating that the caller exited + # the streaming context purposefully by letting + # the exit block exec. + # - this is diff from the cancel/error case where + # a cancel request from this side or an error + # should be sent to the far end indicating the + # stream WAS NOT just closed normally/gracefully. + async with MsgStream( + ctx=self, + rx_chan=ctx._recv_chan, + ) as stream: + + # NOTE: we track all existing streams per portal for + # the purposes of attempting graceful closes on runtime + # cancel requests. + if self._portal: + self._portal._streams.add(stream) + + try: + self._stream_opened: bool = True + self._stream = stream + + # XXX: do we need this? + # ensure we aren't cancelled before yielding the stream + # await trio.lowlevel.checkpoint() + yield stream + + # XXX: (MEGA IMPORTANT) if this is a root opened process we + # wait for any immediate child in debug before popping the + # context from the runtime msg loop otherwise inside + # ``Actor._push_result()`` the msg will be discarded and in + # the case where that msg is global debugger unlock (via + # a "stop" msg for a stream), this can result in a deadlock + # where the root is waiting on the lock to clear but the + # child has already cleared it and clobbered IPC. + # + # await maybe_wait_for_debugger() + + # XXX TODO: pretty sure this isn't needed (see + # note above this block) AND will result in + # a double `.send_stop()` call. The only reason to + # put it here would be to due with "order" in + # terms of raising any remote error (as per + # directly below) or bc the stream's + # `.__aexit__()` block might not get run + # (doubtful)? Either way if we did put this back + # in we also need a state var to avoid the double + # stop-msg send.. + # + # await stream.aclose() + + # NOTE: absorb and do not raise any + # EoC received from the other side such that + # it is not raised inside the surrounding + # context block's scope! + except trio.EndOfChannel as eoc: + if ( + eoc + and stream.closed + ): + # sanity, can remove? + assert eoc is stream._eoc + # from .devx import pause + # await pause() + log.warning( + 'Stream was terminated by EoC\n\n' + f'{repr(eoc)}\n' + ) + + finally: + if self._portal: + try: + self._portal._streams.remove(stream) + except KeyError: + log.warning( + f'Stream was already destroyed?\n' + f'actor: {self.chan.uid}\n' + f'ctx id: {self.cid}' + ) + + # TODO: replace all the instances of this!! XD + def maybe_raise( + self, + hide_tb: bool = True, + **kwargs, + + ) -> Exception|None: + __tracebackhide__: bool = hide_tb + if re := self._remote_error: + return self._maybe_raise_remote_err( + re, + **kwargs, + ) + + def _maybe_raise_remote_err( + self, + remote_error: Exception, + + raise_ctxc_from_self_call: bool = False, + raise_overrun_from_self: bool = True, + hide_tb: bool = True, + + ) -> ( + ContextCancelled # `.cancel()` request to far side + |RemoteActorError # stream overrun caused and ignored by us + ): + ''' + Maybe raise a remote error depending on the type of error + and *who* (i.e. which task from which actor) requested + a cancellation (if any). + + ''' + __tracebackhide__: bool = hide_tb + our_uid: tuple = self.chan.uid + + # XXX NOTE XXX: `ContextCancelled`/`StreamOverrun` absorption + # for "graceful cancellation" case: + # + # Whenever a "side" of a context (a `trio.Task` running in + # an actor) **is** the side which requested ctx + # cancellation (likekly via ``Context.cancel()``), we + # **don't** want to re-raise any eventually received + # `ContextCancelled` response locally (would be akin to + # a `trio.Nursery` nursery raising `trio.Cancelled` + # whenever `CancelScope.cancel()` was called). + # + # Instead, silently reap the remote delivered ctxc + # (`ContextCancelled`) as an expected + # error-msg-is-cancellation-ack IFF said + # `remote_error: ContextCancelled` has `.canceller` + # set to the `Actor.uid` of THIS task (i.e. the + # cancellation requesting task's actor is the actor + # checking whether it should absorb the ctxc). + if ( + not raise_ctxc_from_self_call + and self._is_self_cancelled(remote_error) + + # TODO: ?potentially it is useful to emit certain + # warning/cancel logs for the cases where the + # cancellation is due to a lower level cancel + # request, such as `Portal.cancel_actor()`, since in + # that case it's not actually this specific ctx that + # made a `.cancel()` call, but it is the same + # actor-process? + # or self.chan._cancel_called + # XXX: ^ should we have a special separate case + # for this ^, NO right? + + ) or ( + # NOTE: whenever this context is the cause of an + # overrun on the remote side (aka we sent msgs too + # fast that the remote task was overrun according + # to `MsgStream` buffer settings) AND the caller + # has requested to not raise overruns this side + # caused, we also silently absorb any remotely + # boxed `StreamOverrun`. This is mostly useful for + # supressing such faults during + # cancellation/error/final-result handling inside + # `_drain_to_final_msg()` such that we do not + # raise such errors particularly in the case where + # `._cancel_called == True`. + not raise_overrun_from_self + and isinstance(remote_error, RemoteActorError) + and remote_error.msgdata['type_str'] == 'StreamOverrun' + and tuple(remote_error.msgdata['sender']) == our_uid + ): + # NOTE: we set the local scope error to any "self + # cancellation" error-response thus "absorbing" + # the error silently B) + if self._local_error is None: + self._local_error = remote_error + + else: + log.warning( + 'Local error already set for ctx?\n' + f'{self._local_error}\n' + ) + + return remote_error + + # NOTE: currently we are hiding underlying runtime errors + # which are often superfluous to user handler code. not + # sure if this is still needed / desired for all operation? + # TODO: maybe we can only NOT mask if: + # - [ ] debug mode is enabled or, + # - [ ] a certain log level is set? + # - [ ] consider using `.with_traceback()` to filter out + # runtime frames from the tb explicitly? + # https://docs.python.org/3/reference/simple_stmts.html#the-raise-statement + # https://stackoverflow.com/a/24752607 + __tracebackhide__: bool = True + raise remote_error # from None + + # TODO: change to `.wait_for_result()`? + async def result( + self, + hide_tb: bool = True, + + ) -> Any|Exception: + ''' + From some (caller) side task, wait for and return the final + result from the remote (callee) side's task. + + This provides a mechanism for one task running in some actor to wait + on another task at the other side, in some other actor, to terminate. + + If the remote task is still in a streaming state (it is delivering + values from inside a ``Context.open_stream():`` block, then those + msgs are drained but discarded since it is presumed this side of + the context has already finished with its own streaming logic. + + If the remote context (or its containing actor runtime) was + canceled, either by a local task calling one of + ``Context.cancel()`` or `Portal.cancel_actor()``, we ignore the + received ``ContextCancelled`` exception if the context or + underlying IPC channel is marked as having been "cancel called". + This is similar behavior to using ``trio.Nursery.cancel()`` + wherein tasks which raise ``trio.Cancel`` are silently reaped; + the main different in this API is in the "cancel called" case, + instead of just not raising, we also return the exception *as + the result* since client code may be interested in the details + of the remote cancellation. + + ''' + __tracebackhide__ = hide_tb + assert self._portal, ( + "Context.result() can not be called from callee side!" + ) + if self._final_result_is_set(): + return self._result + + assert self._recv_chan + raise_overrun: bool = not self._allow_overruns + # res_placeholder: int = id(self) + if ( + # self._result == res_placeholder + # and not self._remote_error + self.maybe_error is None + # not self._remote_error + # and not self._local_error + and not self._recv_chan._closed # type: ignore + ): + + # wait for a final context result/error by "draining" + # (by more or less ignoring) any bi-dir-stream "yield" + # msgs still in transit from the far end. + drained_msgs: list[dict] = await _drain_to_final_msg( + ctx=self, + hide_tb=hide_tb, + ) + for msg in drained_msgs: + + # TODO: mask this by default.. + if 'return' in msg: + # from .devx import pause + # await pause() + raise InternalError( + 'Final `return` msg should never be drained !?!?\n\n' + f'{msg}\n' + ) + + log.cancel( + 'Ctx drained pre-result msgs:\n' + f'{drained_msgs}' + ) + + self.maybe_raise( + raise_overrun_from_self=( + raise_overrun + and + # only when we ARE NOT the canceller + # should we raise overruns, bc ow we're + # raising something we know might happen + # during cancellation ;) + (not self._cancel_called) + ) + ) + # if ( + # (re := self._remote_error) + # # and self._result == res_placeholder + # ): + # self._maybe_raise_remote_err( + # re, + # # NOTE: obvi we don't care if we + # # overran the far end if we're already + # # waiting on a final result (msg). + # # raise_overrun_from_self=False, + # raise_overrun_from_self=( + # raise_overrun + # and + # # only when we ARE NOT the canceller + # # should we raise overruns, bc ow we're + # # raising something we know might happen + # # during cancellation ;) + # (not self._cancel_called) + # ), + # ) + # if maybe_err: + # self._result = maybe_err + + return self.outcome + + # TODO: switch this with above which should be named + # `.wait_for_outcome()` and instead do + # a `.outcome.Outcome.unwrap()` ? + # @property + # def result(self) -> Any|None: + # if self._final_result_is_set(): + # return self._result + + # raise RuntimeError('No result is available!') + + @property + def maybe_error(self) -> BaseException|None: + le: Exception|None = self._local_error + re: RemoteActorError|ContextCancelled|None = self._remote_error + + match (le, re): + # NOTE: remote errors always get precedence since even + # in the cases where a local error was the cause, the + # received boxed ctxc should include the src info + # caused by us right? + case ( + _, + RemoteActorError(), + ): + # give precedence to remote error if it's + # NOT a cancel ack (ctxc). + return ( + re or le + ) + + # TODO: extra logic to handle ctxc ack case(s)? + # -[ ] eg. we error, call .cancel(), rx ack but should + # raise the _local_error instead? + # -[ ] are there special error conditions where local vs. + # remote should take precedence? + # case ( + # _, + # ContextCancelled(canceller=), + # ): + + error: Exception|None = le or re + if error: + return error + + if cancmsg := self._cancel_msg: + # NOTE: means we're prolly in the process of + # processing the cancellation caused by + # this msg (eg. logging from `Actor._cancel_task()` + # method after receiving a `Context.cancel()` RPC) + # though there shouldn't ever be a `._cancel_msg` + # without it eventually resulting in this property + # delivering a value! + log.debug( + '`Context._cancel_msg` is set but has not yet resolved to `.maybe_error`?\n\n' + f'{cancmsg}\n' + ) + + # assert not self._cancel_msg + return None + + def _final_result_is_set(self) -> bool: + # return not (self._result == id(self)) + return self._result is not Unresolved + + # def get_result_nowait(self) -> Any|None: + # TODO: use `outcome.Outcome` here instead? + @property + def outcome(self) -> ( + Any| + RemoteActorError| + ContextCancelled + ): + ''' + The final "outcome" from an IPC context which can either be + some Value returned from the target `@context`-decorated + remote task-as-func, or an `Error` wrapping an exception + raised from an RPC task fault or cancellation. + + Note that if the remote task has not terminated then this + field always resolves to the module defined `Unresolved` handle. + + TODO: implement this using `outcome.Outcome` types? + + ''' + return ( + self.maybe_error + or + self._result + ) + + # @property + def repr_outcome( + self, + show_error_fields: bool = False, + type_only: bool = False, + + ) -> str: + ''' + Deliver a (simplified) `str` representation (as in + `.__repr__()`) of the final `.outcome` + + ''' + merr: Exception|None = self.maybe_error + if merr: + if type_only: + return type(merr).__name__ + + # if the error-type is one of ours and has the custom + # defined "repr-(in)-one-line" method call it, ow + # just deliver the type name. + if ( + (reprol := getattr(merr, 'reprol', False)) + and show_error_fields + ): + return reprol() + + elif isinstance(merr, BaseExceptionGroup): + # TODO: maybe for multis we should just show + # a one-line count per error type, like with + # `collections.Counter`? + # + # just the type name for now to avoid long lines + # when tons of cancels.. + return ( + str(type(merr).__name__) + or + repr(merr) + ) + + # just the type name + # else: # but wen? + # return type(merr).__name__ + + # for all other errors show their regular output + return ( + str(merr) + or + repr(merr) + ) + + return ( + str(self._result) + or + repr(self._result) + ) + + async def started( + self, + value: Any | None = None + + ) -> None: + ''' + Indicate to calling actor's task that this linked context + has started and send ``value`` to the other side via IPC. + + On the calling side ``value`` is the second item delivered + in the tuple returned by ``Portal.open_context()``. + + ''' + if self._portal: + raise RuntimeError( + f'Caller side context {self} can not call started!' + ) + + elif self._started_called: + raise RuntimeError( + f'called `.started()` twice on context with {self.chan.uid}' + ) + + await self.chan.send({'started': value, 'cid': self.cid}) + self._started_called = True + + async def _drain_overflows( + self, + ) -> None: + ''' + Private task spawned to push newly received msgs to the local + task which getting overrun by the remote side. + + In order to not block the rpc msg loop, but also not discard + msgs received in this context, we need to async push msgs in + a new task which only runs for as long as the local task is in + an overrun state. + + ''' + self._in_overrun = True + try: + while self._overflow_q: + # NOTE: these msgs should never be errors since we always do + # the check prior to checking if we're in an overrun state + # inside ``._deliver_msg()``. + msg = self._overflow_q.popleft() + try: + await self._send_chan.send(msg) + except trio.BrokenResourceError: + log.warning( + f"{self._send_chan} consumer is already closed" + ) + return + except trio.Cancelled: + # we are obviously still in overrun + # but the context is being closed anyway + # so we just warn that there are un received + # msgs still.. + self._overflow_q.appendleft(msg) + fmt_msgs = '' + for msg in self._overflow_q: + fmt_msgs += f'{pformat(msg)}\n' + + log.warning( + f'Context for {self.cid} is being closed while ' + 'in an overrun state!\n' + 'Discarding the following msgs:\n' + f'{fmt_msgs}\n' + ) + raise + + finally: + # task is now finished with the backlog so mark us as + # no longer in backlog. + self._in_overrun = False + + async def _deliver_msg( + self, + msg: dict, + + ) -> bool: + ''' + Deliver an IPC msg received from a transport-channel to + this context's underlying mem chan for handling by local + user application tasks; deliver `bool` indicating whether + the msg was able to be delivered. + + If `._allow_overruns == True` (maybe) append the msg to an + "overflow queue" and start a "drainer task" (inside the + `._scope_nursery: trio.Nursery`) which ensures that such + messages are queued up and eventually sent if possible. + + ''' + cid: str = self.cid + chan: Channel = self.chan + from_uid: tuple[str, str] = chan.uid + send_chan: trio.MemorySendChannel = self._send_chan + nsf: NamespacePath = self._nsf + + re: Exception|None + if re := unpack_error( + msg, + self.chan, + ): + if not isinstance(re, ContextCancelled): + log_meth = log.error + else: + log_meth = log.runtime + + log_meth( + f'Delivering error-msg to caller\n\n' + + f'<= peer: {from_uid}\n' + f' |_ {nsf}()\n\n' + + f'=> cid: {cid}\n' + f' |_{self._task}\n\n' + + f'{pformat(re)}\n' + ) + self._cancel_msg: dict = msg + + # NOTE: this will not raise an error, merely set + # `._remote_error` and maybe cancel any task currently + # entered in `Portal.open_context()` presuming the + # error is "cancel causing" (i.e. a `ContextCancelled` + # or `RemoteActorError`). + self._maybe_cancel_and_set_remote_error(re) + + # XXX NEVER do this XXX..!! + # bc if the error is a ctxc and there is a task + # waiting on `.result()` we need the msg to be sent + # over the `send_chan`/`._recv_chan` so that the error + # is relayed to that waiter task.. + # return True + # + # XXX ALSO NO!! XXX + # => NEVER raise remote errors from the calling + # runtime task, they should always be raised by + # consumer side tasks operating on the + # `Portal`/`Context` APIs. + # if self._remote_error: + # self._maybe_raise_remote_err(error) + + if self._in_overrun: + log.warning( + f'Queueing OVERRUN msg on caller task:\n' + f'<= peer: {from_uid}\n' + f' |_ {nsf}()\n\n' + + f'=> cid: {cid}\n' + f' |_{self._task}\n\n' + + f'{pformat(msg)}\n' + ) + self._overflow_q.append(msg) + return False + + try: + log.runtime( + f'Delivering msg from IPC ctx:\n' + f'<= {from_uid}\n' + f' |_ {nsf}()\n\n' + + f'=> {self._task}\n' + f' |_cid={self.cid}\n\n' + + f'{pformat(msg)}\n' + ) + + # NOTE: if an error is deteced we should always still + # send it through the feeder-mem-chan and expect + # it to be raised by any context (stream) consumer + # task via the consumer APIs on both the `Context` and + # `MsgStream`! + # + # XXX the reason is that this method is always called + # by the IPC msg handling runtime task and that is not + # normally the task that should get cancelled/error + # from some remote fault! + send_chan.send_nowait(msg) + return True + + except trio.BrokenResourceError: + # TODO: what is the right way to handle the case where the + # local task has already sent a 'stop' / StopAsyncInteration + # to the other side but and possibly has closed the local + # feeder mem chan? Do we wait for some kind of ack or just + # let this fail silently and bubble up (currently)? + + # XXX: local consumer has closed their side + # so cancel the far end streaming task + log.warning( + 'Rx chan for `Context` alfready closed?\n' + f'cid: {self.cid}\n' + 'Failed to deliver msg:\n' + f'send_chan: {send_chan}\n\n' + f'{pformat(msg)}\n' + ) + return False + + # NOTE XXX: by default we do **not** maintain context-stream + # backpressure and instead opt to relay stream overrun errors to + # the sender; the main motivation is that using bp can block the + # msg handling loop which calls into this method! + except trio.WouldBlock: + + # XXX: always push an error even if the local receiver + # is in overrun state - i.e. if an 'error' msg is + # delivered then + # `._maybe_cancel_and_set_remote_error(msg)` should + # have already been called above! + # + # XXX QUESTION XXX: if we rx an error while in an + # overrun state and that msg isn't stuck in an + # overflow queue what happens?!? + + local_uid = self._actor.uid + txt: str = ( + 'on IPC context:\n' + + f'<= sender: {from_uid}\n' + f' |_ {self._nsf}()\n\n' + + f'=> overrun: {local_uid}\n' + f' |_cid: {cid}\n' + f' |_task: {self._task}\n' + ) + if not self._stream_opened: + txt += ( + f'\n*** No stream open on `{local_uid[0]}` side! ***\n\n' + f'{msg}\n' + ) + + # XXX: lul, this really can't be backpressure since any + # blocking here will block the entire msg loop rpc sched for + # a whole channel.. maybe we should rename it? + if self._allow_overruns: + txt += ( + '\n*** Starting overflow queuing task on msg ***\n\n' + f'{msg}\n' + ) + log.warning(txt) + if ( + not self._in_overrun + ): + self._overflow_q.append(msg) + tn: trio.Nursery = self._scope_nursery + assert not tn.child_tasks + try: + tn.start_soon( + self._drain_overflows, + ) + return True + + except RuntimeError: + # if the nursery is already cancelled due to + # this context exiting or in error, we ignore + # the nursery error since we never expected + # anything different. + return False + else: + txt += f'\n{msg}\n' + # raise local overrun and immediately pack as IPC + # msg for far end. + try: + raise StreamOverrun( + txt, + sender=from_uid, + ) + except StreamOverrun as err: + err_msg: dict[str, dict] = pack_error( + err, + cid=cid, + ) + try: + # relay condition to sender side remote task + await chan.send(err_msg) + return True + + except trio.BrokenResourceError: + # XXX: local consumer has closed their side + # so cancel the far end streaming task + log.warning( + 'Channel for ctx is already closed?\n' + f'|_{chan}\n' + ) + + # ow, indicate unable to deliver by default + return False + + +@acm +async def open_context_from_portal( + portal: Portal, + func: Callable, + + allow_overruns: bool = False, + + # TODO: if we set this the wrapping `@acm` body will + # still be shown (awkwardly) on pdb REPL entry. Ideally + # we can similarly annotate that frame to NOT show? for now + # we DO SHOW this frame since it's awkward ow.. + hide_tb: bool = False, + + # proxied to RPC + **kwargs, + +) -> AsyncGenerator[tuple[Context, Any], None]: + ''' + Open an inter-actor "task context"; a remote task is + scheduled and cancel-scope-state-linked to a `trio.run()` across + memory boundaries in another actor's runtime. + + This is an `@acm` API bound as `Portal.open_context()` which + allows for deterministic setup and teardown of a remotely + scheduled task in another remote actor. Once opened, the 2 now + "linked" tasks run completely in parallel in each actor's + runtime with their enclosing `trio.CancelScope`s kept in + a synced state wherein if either side errors or cancels an + equivalent error is relayed to the other side via an SC-compat + IPC protocol. + + The yielded `tuple` is a pair delivering a `tractor.Context` + and any first value "sent" by the "callee" task via a call + to `Context.started()`; this side of the + context does not unblock until the "callee" task calls + `.started()` in similar style to `trio.Nursery.start()`. + When the "callee" (side that is "called"/started by a call + to *this* method) returns, the caller side (this) unblocks + and any final value delivered from the other end can be + retrieved using the `Contex.result()` api. + + The yielded ``Context`` instance further allows for opening + bidirectional streams, explicit cancellation and + structurred-concurrency-synchronized final result-msg + collection. See ``tractor.Context`` for more details. + + ''' + __tracebackhide__: bool = hide_tb + + # conduct target func method structural checks + if not inspect.iscoroutinefunction(func) and ( + getattr(func, '_tractor_contex_function', False) + ): + raise TypeError( + f'{func} must be an async generator function!') + + # TODO: i think from here onward should probably + # just be factored into an `@acm` inside a new + # a new `_context.py` mod. + nsf = NamespacePath.from_ref(func) + + # XXX NOTE XXX: currenly we do NOT allow opening a contex + # with "self" since the local feeder mem-chan processing + # is not built for it. + if portal.channel.uid == portal.actor.uid: + raise RuntimeError( + '** !! Invalid Operation !! **\n' + 'Can not open an IPC ctx with the local actor!\n' + f'|_{portal.actor}\n' + ) + + ctx: Context = await portal.actor.start_remote_task( + portal.channel, + nsf=nsf, + kwargs=kwargs, + + # NOTE: it's imporant to expose this since you might + # get the case where the parent who opened the context does + # not open a stream until after some slow startup/init + # period, in which case when the first msg is read from + # the feeder mem chan, say when first calling + # `Context.open_stream(allow_overruns=True)`, the overrun condition will be + # raised before any ignoring of overflow msgs can take + # place.. + allow_overruns=allow_overruns, + ) + + assert ctx._remote_func_type == 'context' + msg: dict = await ctx._recv_chan.receive() + + try: + # the "first" value here is delivered by the callee's + # ``Context.started()`` call. + first: Any = msg['started'] + ctx._started_called: bool = True + + except KeyError as src_error: + _raise_from_no_key_in_msg( + ctx=ctx, + msg=msg, + src_err=src_error, + log=log, + expect_key='started', + ) + + ctx._portal: Portal = portal + uid: tuple = portal.channel.uid + cid: str = ctx.cid + + # placeholder for any exception raised in the runtime + # or by user tasks which cause this context's closure. + scope_err: BaseException|None = None + ctxc_from_callee: ContextCancelled|None = None + try: + async with trio.open_nursery() as nurse: + + # NOTE: used to start overrun queuing tasks + ctx._scope_nursery: trio.Nursery = nurse + ctx._scope: trio.CancelScope = nurse.cancel_scope + + # deliver context instance and .started() msg value + # in enter tuple. + yield ctx, first + + # ??TODO??: do we still want to consider this or is + # the `else:` block handling via a `.result()` + # call below enough?? + # -[ ] pretty sure `.result()` internals do the + # same as our ctxc handler below so it ended up + # being same (repeated?) behaviour, but ideally we + # wouldn't have that duplication either by somehow + # factoring the `.result()` handler impl in a way + # that we can re-use it around the `yield` ^ here + # or vice versa? + # + # NOTE: between the caller exiting and arriving + # here the far end may have sent a ctxc-msg or + # other error, so check for it here immediately + # and maybe raise so as to engage the ctxc + # handling block below! + # + # if re := ctx._remote_error: + # maybe_ctxc: ContextCancelled|None = ctx._maybe_raise_remote_err( + # re, + # # TODO: do we want this to always raise? + # # - means that on self-ctxc, if/when the + # # block is exited before the msg arrives + # # but then the msg during __exit__ + # # calling we may not activate the + # # ctxc-handler block below? should we + # # be? + # # - if there's a remote error that arrives + # # after the child has exited, we won't + # # handle until the `finally:` block + # # where `.result()` is always called, + # # again in which case we handle it + # # differently then in the handler block + # # that would normally engage from THIS + # # block? + # raise_ctxc_from_self_call=True, + # ) + # ctxc_from_callee = maybe_ctxc + + # when in allow_overruns mode there may be + # lingering overflow sender tasks remaining? + if nurse.child_tasks: + # XXX: ensure we are in overrun state + # with ``._allow_overruns=True`` bc otherwise + # there should be no tasks in this nursery! + if ( + not ctx._allow_overruns + or len(nurse.child_tasks) > 1 + ): + raise InternalError( + 'Context has sub-tasks but is ' + 'not in `allow_overruns=True` mode!?' + ) + + # ensure we cancel all overflow sender + # tasks started in the nursery when + # `._allow_overruns == True`. + # + # NOTE: this means `._scope.cancelled_caught` + # will prolly be set! not sure if that's + # non-ideal or not ??? + ctx._scope.cancel() + + # XXX NOTE XXX: maybe shield against + # self-context-cancellation (which raises a local + # `ContextCancelled`) when requested (via + # `Context.cancel()`) by the same task (tree) which entered + # THIS `.open_context()`. + # + # NOTE: There are 2 operating cases for a "graceful cancel" + # of a `Context`. In both cases any `ContextCancelled` + # raised in this scope-block came from a transport msg + # relayed from some remote-actor-task which our runtime set + # as to `Context._remote_error` + # + # the CASES: + # + # - if that context IS THE SAME ONE that called + # `Context.cancel()`, we want to absorb the error + # silently and let this `.open_context()` block to exit + # without raising, ideally eventually receiving the ctxc + # ack msg thus resulting in `ctx.cancel_acked == True`. + # + # - if it is from some OTHER context (we did NOT call + # `.cancel()`), we want to re-RAISE IT whilst also + # setting our own ctx's "reason for cancel" to be that + # other context's cancellation condition; we set our + # `.canceller: tuple[str, str]` to be same value as + # caught here in a `ContextCancelled.canceller`. + # + # AGAIN to restate the above, there are 2 cases: + # + # 1-some other context opened in this `.open_context()` + # block cancelled due to a self or peer cancellation + # request in which case we DO let the error bubble to the + # opener. + # + # 2-THIS "caller" task somewhere invoked `Context.cancel()` + # and received a `ContextCanclled` from the "callee" + # task, in which case we mask the `ContextCancelled` from + # bubbling to this "caller" (much like how `trio.Nursery` + # swallows any `trio.Cancelled` bubbled by a call to + # `Nursery.cancel_scope.cancel()`) + except ContextCancelled as ctxc: + scope_err = ctxc + ctx._local_error: BaseException = scope_err + ctxc_from_callee = ctxc + + # XXX TODO XXX: FIX THIS debug_mode BUGGGG!!! + # using this code and then resuming the REPL will + # cause a SIGINT-ignoring HANG! + # -> prolly due to a stale debug lock entry.. + # -[ ] USE `.stackscope` to demonstrate that (possibly + # documenting it as a definittive example of + # debugging the tractor-runtime itself using it's + # own `.devx.` tooling! + # + # await _debug.pause() + + # CASE 2: context was cancelled by local task calling + # `.cancel()`, we don't raise and the exit block should + # exit silently. + if ( + ctx._cancel_called + and + ctxc is ctx._remote_error + and + ctxc.canceller == portal.actor.uid + ): + log.cancel( + f'Context (cid=[{ctx.cid[-6:]}..] cancelled gracefully with:\n' + f'{ctxc}' + ) + # CASE 1: this context was never cancelled via a local + # task (tree) having called `Context.cancel()`, raise + # the error since it was caused by someone else + # -> probably a remote peer! + else: + raise + + # the above `._scope` can be cancelled due to: + # 1. an explicit self cancel via `Context.cancel()` or + # `Actor.cancel()`, + # 2. any "callee"-side remote error, possibly also a cancellation + # request by some peer, + # 3. any "caller" (aka THIS scope's) local error raised in the above `yield` + except ( + # CASE 3: standard local error in this caller/yieldee + Exception, + + # CASES 1 & 2: can manifest as a `ctx._scope_nursery` + # exception-group of, + # + # 1.-`trio.Cancelled`s, since + # `._scope.cancel()` will have been called + # (transitively by the runtime calling + # `._deliver_msg()`) and any `ContextCancelled` + # eventually absorbed and thus absorbed/supressed in + # any `Context._maybe_raise_remote_err()` call. + # + # 2.-`BaseExceptionGroup[ContextCancelled | RemoteActorError]` + # from any error delivered from the "callee" side + # AND a group-exc is only raised if there was > 1 + # tasks started *here* in the "caller" / opener + # block. If any one of those tasks calls + # `.result()` or `MsgStream.receive()` + # `._maybe_raise_remote_err()` will be transitively + # called and the remote error raised causing all + # tasks to be cancelled. + # NOTE: ^ this case always can happen if any + # overrun handler tasks were spawned! + BaseExceptionGroup, + + trio.Cancelled, # NOTE: NOT from inside the ctx._scope + KeyboardInterrupt, + + ) as caller_err: + scope_err = caller_err + ctx._local_error: BaseException = scope_err + + # XXX: ALWAYS request the context to CANCEL ON any ERROR. + # NOTE: `Context.cancel()` is conversely NEVER CALLED in + # the `ContextCancelled` "self cancellation absorbed" case + # handled in the block above ^^^ !! + # await _debug.pause() + log.cancel( + 'Context terminated due to\n\n' + f'.outcome => {ctx.repr_outcome()}\n' + ) + + if debug_mode(): + # async with _debug.acquire_debug_lock(portal.actor.uid): + # pass + # TODO: factor ^ into below for non-root cases? + # + from ._debug import maybe_wait_for_debugger + was_acquired: bool = await maybe_wait_for_debugger( + # header_msg=( + # 'Delaying `ctx.cancel()` until debug lock ' + # 'acquired..\n' + # ), + ) + if was_acquired: + log.pdb( + 'Acquired debug lock! ' + 'Calling `ctx.cancel()`!\n' + ) + + # we don't need to cancel the callee if it already + # told us it's cancelled ;p + if ctxc_from_callee is None: + try: + await ctx.cancel() + except ( + trio.BrokenResourceError, + trio.ClosedResourceError, + ): + log.warning( + 'IPC connection for context is broken?\n' + f'task:{cid}\n' + f'actor:{uid}' + ) + + raise # duh + + # no local scope error, the "clean exit with a result" case. + else: + if ctx.chan.connected(): + log.runtime( + 'Waiting on final context result for\n' + f'peer: {uid}\n' + f'|_{ctx._task}\n' + ) + # XXX NOTE XXX: the below call to + # `Context.result()` will ALWAYS raise + # a `ContextCancelled` (via an embedded call to + # `Context._maybe_raise_remote_err()`) IFF + # a `Context._remote_error` was set by the runtime + # via a call to + # `Context._maybe_cancel_and_set_remote_error()`. + # As per `Context._deliver_msg()`, that error IS + # ALWAYS SET any time "callee" side fails and causes "caller + # side" cancellation via a `ContextCancelled` here. + try: + result_or_err: Exception|Any = await ctx.result() + except BaseException as berr: + # on normal teardown, if we get some error + # raised in `Context.result()` we still want to + # save that error on the ctx's state to + # determine things like `.cancelled_caught` for + # cases where there was remote cancellation but + # this task didn't know until final teardown + # / value collection. + scope_err = berr + ctx._local_error: BaseException = scope_err + raise + + # yes! this worx Bp + # from .devx import _debug + # await _debug.pause() + + # an exception type boxed in a `RemoteActorError` + # is returned (meaning it was obvi not raised) + # that we want to log-report on. + msgdata: str|None = getattr( + result_or_err, + 'msgdata', + None + ) + match (msgdata, result_or_err): + case ( + {'tb_str': tbstr}, + ContextCancelled(), + ): + log.cancel(tbstr) + + case ( + {'tb_str': tbstr}, + RemoteActorError(), + ): + log.exception( + 'Context remotely errored!\n' + f'<= peer: {uid}\n' + f' |_ {nsf}()\n\n' + + f'{tbstr}' + ) + case (None, _): + log.runtime( + 'Context returned final result from callee task:\n' + f'<= peer: {uid}\n' + f' |_ {nsf}()\n\n' + + f'`{result_or_err}`\n' + ) + + finally: + # XXX: (MEGA IMPORTANT) if this is a root opened process we + # wait for any immediate child in debug before popping the + # context from the runtime msg loop otherwise inside + # ``Actor._push_result()`` the msg will be discarded and in + # the case where that msg is global debugger unlock (via + # a "stop" msg for a stream), this can result in a deadlock + # where the root is waiting on the lock to clear but the + # child has already cleared it and clobbered IPC. + if debug_mode(): + from ._debug import maybe_wait_for_debugger + await maybe_wait_for_debugger() + + # though it should be impossible for any tasks + # operating *in* this scope to have survived + # we tear down the runtime feeder chan last + # to avoid premature stream clobbers. + if ( + (rxchan := ctx._recv_chan) + + # maybe TODO: yes i know the below check is + # touching `trio` memchan internals..BUT, there are + # only a couple ways to avoid a `trio.Cancelled` + # bubbling from the `.aclose()` call below: + # + # - catch and mask it via the cancel-scope-shielded call + # as we are rn (manual and frowned upon) OR, + # - specially handle the case where `scope_err` is + # one of {`BaseExceptionGroup`, `trio.Cancelled`} + # and then presume that the `.aclose()` call will + # raise a `trio.Cancelled` and just don't call it + # in those cases.. + # + # that latter approach is more logic, LOC, and more + # convoluted so for now stick with the first + # psuedo-hack-workaround where we just try to avoid + # the shielded call as much as we can detect from + # the memchan's `._closed` state.. + # + # XXX MOTIVATION XXX-> we generally want to raise + # any underlying actor-runtime/internals error that + # surfaces from a bug in tractor itself so it can + # be easily detected/fixed AND, we also want to + # minimize noisy runtime tracebacks (normally due + # to the cross-actor linked task scope machinery + # teardown) displayed to user-code and instead only + # displaying `ContextCancelled` traces where the + # cause of crash/exit IS due to something in + # user/app code on either end of the context. + and not rxchan._closed + ): + # XXX NOTE XXX: and again as per above, we mask any + # `trio.Cancelled` raised here so as to NOT mask + # out any exception group or legit (remote) ctx + # error that sourced from the remote task or its + # runtime. + # + # NOTE: further, this should be the only place the + # underlying feeder channel is + # once-and-only-CLOSED! + with trio.CancelScope(shield=True): + await ctx._recv_chan.aclose() + + # XXX: we always raise remote errors locally and + # generally speaking mask runtime-machinery related + # multi-`trio.Cancelled`s. As such, any `scope_error` + # which was the underlying cause of this context's exit + # should be stored as the `Context._local_error` and + # used in determining `Context.cancelled_caught: bool`. + if scope_err is not None: + # sanity, tho can remove? + assert ctx._local_error is scope_err + # ctx._local_error: BaseException = scope_err + # etype: Type[BaseException] = type(scope_err) + + # CASE 2 + if ( + ctx._cancel_called + and ctx.cancel_acked + ): + log.cancel( + 'Context cancelled by caller task\n' + f'|_{ctx._task}\n\n' + + f'{repr(scope_err)}\n' + ) + + # TODO: should we add a `._cancel_req_received` + # flag to determine if the callee manually called + # `ctx.cancel()`? + # -[ ] going to need a cid check no? + + # CASE 1 + else: + outcome_str: str = ctx.repr_outcome( + show_error_fields=True, + # type_only=True, + ) + log.cancel( + f'Context terminated due to local scope error:\n\n' + f'{ctx.chan.uid} => {outcome_str}\n' + ) + + # FINALLY, remove the context from runtime tracking and + # exit! + log.runtime( + 'Removing IPC ctx opened with peer\n' + f'{uid}\n' + f'|_{ctx}\n' + ) + portal.actor._contexts.pop( + (uid, cid), + None, + ) + + +def mk_context( + chan: Channel, + cid: str, + nsf: NamespacePath, + + msg_buffer_size: int = 2**6, + + **kwargs, + +) -> Context: + ''' + Internal factory to create an inter-actor task ``Context``. + + This is called by internals and should generally never be called + by user code. + + ''' + send_chan: trio.MemorySendChannel + recv_chan: trio.MemoryReceiveChannel + send_chan, recv_chan = trio.open_memory_channel(msg_buffer_size) + + ctx = Context( + chan=chan, + cid=cid, + _actor=current_actor(), + _send_chan=send_chan, + _recv_chan=recv_chan, + _nsf=nsf, + _task=trio.lowlevel.current_task(), + **kwargs, + ) + # TODO: we can drop the old placeholder yah? + # ctx._result: int | Any = id(ctx) + ctx._result = Unresolved + return ctx + + +def context(func: Callable) -> Callable: + ''' + Mark an async function as a streaming routine with ``@context``. + + ''' + # TODO: apply whatever solution ``mypy`` ends up picking for this: + # https://github.com/python/mypy/issues/2087#issuecomment-769266912 + func._tractor_context_function = True # type: ignore + + sig = inspect.signature(func) + params = sig.parameters + if 'ctx' not in params: + raise TypeError( + "The first argument to the context function " + f"{func.__name__} must be `ctx: tractor.Context`" + ) + return func diff --git a/tractor/_discovery.py b/tractor/_discovery.py index b6957ba3..03775ac2 100644 --- a/tractor/_discovery.py +++ b/tractor/_discovery.py @@ -41,8 +41,10 @@ async def get_arbiter( port: int, ) -> AsyncGenerator[Union[Portal, LocalPortal], None]: - '''Return a portal instance connected to a local or remote + ''' + Return a portal instance connected to a local or remote arbiter. + ''' actor = current_actor() @@ -134,12 +136,16 @@ async def find_actor( @acm async def wait_for_actor( name: str, - arbiter_sockaddr: tuple[str, int] | None = None + arbiter_sockaddr: tuple[str, int] | None = None, + # registry_addr: tuple[str, int] | None = None, + ) -> AsyncGenerator[Portal, None]: - """Wait on an actor to register with the arbiter. + ''' + Wait on an actor to register with the arbiter. A portal to the first registered actor is returned. - """ + + ''' actor = current_actor() async with get_arbiter( diff --git a/tractor/_entry.py b/tractor/_entry.py index 1e7997e8..a59975ce 100644 --- a/tractor/_entry.py +++ b/tractor/_entry.py @@ -106,25 +106,29 @@ def _trio_main( Entry point for a `trio_run_in_process` subactor. ''' - log.info(f"Started new trio process for {actor.uid}") - - if actor.loglevel is not None: - log.info( - f"Setting loglevel for {actor.uid} to {actor.loglevel}") - get_console_log(actor.loglevel) - - log.info( - f"Started {actor.uid}") - _state._current_actor = actor - - log.debug(f"parent_addr is {parent_addr}") trio_main = partial( async_main, actor, parent_addr=parent_addr ) + if actor.loglevel is not None: + get_console_log(actor.loglevel) + import os + actor_info: str = ( + f'|_{actor}\n' + f' uid: {actor.uid}\n' + f' pid: {os.getpid()}\n' + f' parent_addr: {parent_addr}\n' + f' loglevel: {actor.loglevel}\n' + ) + log.info( + 'Started new trio process:\n' + + + actor_info + ) + try: if infect_asyncio: actor._infected_aio = True @@ -132,7 +136,15 @@ def _trio_main( else: trio.run(trio_main) except KeyboardInterrupt: - log.warning(f"Actor {actor.uid} received KBI") + log.cancel( + 'Actor received KBI\n' + + + actor_info + ) finally: - log.info(f"Actor {actor.uid} terminated") + log.info( + 'Actor terminated\n' + + + actor_info + ) diff --git a/tractor/_exceptions.py b/tractor/_exceptions.py index 5440cad0..344f0c33 100644 --- a/tractor/_exceptions.py +++ b/tractor/_exceptions.py @@ -14,22 +14,34 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . -""" +''' Our classy exception set. -""" +''' +from __future__ import annotations +import builtins +import importlib +from pprint import pformat from typing import ( Any, - Optional, Type, + TYPE_CHECKING, ) -import importlib -import builtins +import textwrap import traceback -import exceptiongroup as eg import trio +from tractor._state import current_actor +from tractor.log import get_logger + +if TYPE_CHECKING: + from ._context import Context + from .log import StackLevelAdapter + from ._stream import MsgStream + from ._ipc import Channel + +log = get_logger('tractor') _this_mod = importlib.import_module(__name__) @@ -38,36 +50,196 @@ class ActorFailure(Exception): "General actor failure" +class InternalError(RuntimeError): + ''' + Entirely unexpected internal machinery error indicating + a completely invalid state or interface. + + ''' + +_body_fields: list[str] = [ + 'src_actor_uid', + 'canceller', + 'sender', +] + +_msgdata_keys: list[str] = [ + 'type_str', +] + _body_fields + + + +# TODO: rename to just `RemoteError`? class RemoteActorError(Exception): - # TODO: local recontruction of remote exception deats - "Remote actor exception bundled locally" + ''' + A box(ing) type which bundles a remote actor `BaseException` for + (near identical, and only if possible,) local object/instance + re-construction in the local process memory domain. + + Normally each instance is expected to be constructed from + a special "error" IPC msg sent by some remote actor-runtime. + + ''' + reprol_fields: list[str] = [ + 'src_actor_uid', + ] + def __init__( self, message: str, - suberror_type: Optional[Type[BaseException]] = None, + suberror_type: Type[BaseException] | None = None, **msgdata ) -> None: super().__init__(message) - self.type = suberror_type - self.msgdata = msgdata + # TODO: maybe a better name? + # - .errtype + # - .retype + # - .boxed_errtype + # - .boxed_type + # - .remote_type + # also pertains to our long long oustanding issue XD + # https://github.com/goodboy/tractor/issues/5 + self.boxed_type: str = suberror_type + self.msgdata: dict[str, Any] = msgdata + + @property + def type(self) -> str: + return self.boxed_type + + @property + def type_str(self) -> str: + return str(type(self.boxed_type).__name__) + + @property + def src_actor_uid(self) -> tuple[str, str]|None: + return self.msgdata.get('src_actor_uid') + + @property + def tb_str( + self, + indent: str = ' '*3, + ) -> str: + if remote_tb := self.msgdata.get('tb_str'): + return textwrap.indent( + remote_tb, + prefix=indent, + ) + + return '' + + def reprol(self) -> str: + ''' + Represent this error for "one line" display, like in + a field of our `Context.__repr__()` output. + + ''' + _repr: str = f'{type(self).__name__}(' + for key in self.reprol_fields: + val: Any|None = self.msgdata.get(key) + if val: + _repr += f'{key}={repr(val)} ' + + return _repr + + def __repr__(self) -> str: + + fields: str = '' + for key in _body_fields: + val: str|None = self.msgdata.get(key) + if val: + fields += f'{key}={val}\n' + + fields: str = textwrap.indent( + fields, + # prefix=' '*2, + prefix=' |_', + ) + indent: str = ''*1 + body: str = ( + f'{fields}' + f' |\n' + f' ------ - ------\n\n' + f'{self.tb_str}\n' + f' ------ - ------\n' + f' _|\n' + ) + # f'|\n' + # f' |\n' + if indent: + body: str = textwrap.indent( + body, + prefix=indent, + ) + return ( + f'<{type(self).__name__}(\n' + f'{body}' + ')>' + ) + + # TODO: local recontruction of remote exception deats + # def unbox(self) -> BaseException: + # ... class InternalActorError(RemoteActorError): - """Remote internal ``tractor`` error indicating - failure of some primitive or machinery. - """ + ''' + (Remote) internal `tractor` error indicating failure of some + primitive, machinery state or lowlevel task that should never + occur. + + ''' + + +class ContextCancelled(RemoteActorError): + ''' + Inter-actor task context was cancelled by either a call to + ``Portal.cancel_actor()`` or ``Context.cancel()``. + + ''' + reprol_fields: list[str] = [ + 'canceller', + ] + @property + def canceller(self) -> tuple[str, str]|None: + ''' + Return the (maybe) `Actor.uid` for the requesting-author + of this ctxc. + + Emit a warning msg when `.canceller` has not been set, + which usually idicates that a `None` msg-loop setinel was + sent before expected in the runtime. This can happen in + a few situations: + + - (simulating) an IPC transport network outage + - a (malicious) pkt sent specifically to cancel an actor's + runtime non-gracefully without ensuring ongoing RPC tasks are + incrementally cancelled as is done with: + `Actor` + |_`.cancel()` + |_`.cancel_soon()` + |_`._cancel_task()` + + ''' + value = self.msgdata.get('canceller') + if value: + return tuple(value) + + log.warning( + 'IPC Context cancelled without a requesting actor?\n' + 'Maybe the IPC transport ended abruptly?\n\n' + f'{self}' + ) + + # to make `.__repr__()` work uniformly + # src_actor_uid = canceller class TransportClosed(trio.ClosedResourceError): "Underlying channel transport was closed prior to use" -class ContextCancelled(RemoteActorError): - "Inter-actor task context cancelled itself on the callee side." - - class NoResult(RuntimeError): "No final result is expected for this actor" @@ -80,8 +252,22 @@ class NoRuntime(RuntimeError): "The root actor has not been initialized yet" -class StreamOverrun(trio.TooSlowError): - "This stream was overrun by sender" +class StreamOverrun( + RemoteActorError, + trio.TooSlowError, +): + reprol_fields: list[str] = [ + 'sender', + ] + ''' + This stream was overrun by sender + + ''' + @property + def sender(self) -> tuple[str, str] | None: + value = self.msgdata.get('sender') + if value: + return tuple(value) class AsyncioCancelled(Exception): @@ -92,71 +278,115 @@ class AsyncioCancelled(Exception): ''' +class MessagingError(Exception): + 'Some kind of unexpected SC messaging dialog issue' + def pack_error( exc: BaseException, - tb=None, + tb: str|None = None, + cid: str|None = None, -) -> dict[str, Any]: - """Create an "error message" for tranmission over - a channel (aka the wire). - """ +) -> dict[str, dict]: + ''' + Create an "error message" which boxes a locally caught + exception's meta-data and encodes it for wire transport via an + IPC `Channel`; expected to be unpacked (and thus unboxed) on + the receiver side using `unpack_error()` below. + + ''' if tb: tb_str = ''.join(traceback.format_tb(tb)) else: tb_str = traceback.format_exc() - return { - 'error': { - 'tb_str': tb_str, - 'type_str': type(exc).__name__, - } + error_msg: dict[ + str, + str | tuple[str, str] + ] = { + 'tb_str': tb_str, + 'type_str': type(exc).__name__, + 'boxed_type': type(exc).__name__, + 'src_actor_uid': current_actor().uid, } + # TODO: ?just wholesale proxy `.msgdata: dict`? + # XXX WARNING, when i swapped these ctx-semantics + # tests started hanging..???!!!??? + # if msgdata := exc.getattr('msgdata', {}): + # error_msg.update(msgdata) + if ( + isinstance(exc, ContextCancelled) + or isinstance(exc, StreamOverrun) + ): + error_msg.update(exc.msgdata) + + pkt: dict = {'error': error_msg} + if cid: + pkt['cid'] = cid + + return pkt + def unpack_error( msg: dict[str, Any], - chan=None, - err_type=RemoteActorError -) -> Exception: + chan: Channel|None = None, + box_type: RemoteActorError = RemoteActorError, + + hide_tb: bool = True, + +) -> None|Exception: ''' Unpack an 'error' message from the wire - into a local ``RemoteActorError``. + into a local `RemoteActorError` (subtype). + + NOTE: this routine DOES not RAISE the embedded remote error, + which is the responsibilitiy of the caller. ''' - __tracebackhide__ = True - error = msg['error'] + __tracebackhide__: bool = hide_tb - tb_str = error.get('tb_str', '') - message = f"{chan.uid}\n" + tb_str - type_name = error['type_str'] + error_dict: dict[str, dict] | None + if ( + error_dict := msg.get('error') + ) is None: + # no error field, nothing to unpack. + return None + + # retrieve the remote error's msg encoded details + tb_str: str = error_dict.get('tb_str', '') + message: str = f'{chan.uid}\n' + tb_str + type_name: str = ( + error_dict.get('type_str') + or error_dict['boxed_type'] + ) suberror_type: Type[BaseException] = Exception if type_name == 'ContextCancelled': - err_type = ContextCancelled - suberror_type = trio.Cancelled + box_type = ContextCancelled + suberror_type = box_type else: # try to lookup a suitable local error type for ns in [ builtins, _this_mod, - eg, trio, ]: - try: - suberror_type = getattr(ns, type_name) + if suberror_type := getattr( + ns, + type_name, + False, + ): break - except AttributeError: - continue - exc = err_type( + exc = box_type( message, suberror_type=suberror_type, # unpack other fields into error type init - **msg['error'], + **error_dict, ) return exc @@ -164,14 +394,127 @@ def unpack_error( def is_multi_cancelled(exc: BaseException) -> bool: ''' - Predicate to determine if a possible ``eg.BaseExceptionGroup`` contains + Predicate to determine if a possible ``BaseExceptionGroup`` contains only ``trio.Cancelled`` sub-exceptions (and is likely the result of cancelling a collection of subtasks. ''' - if isinstance(exc, eg.BaseExceptionGroup): + # if isinstance(exc, eg.BaseExceptionGroup): + if isinstance(exc, BaseExceptionGroup): return exc.subgroup( lambda exc: isinstance(exc, trio.Cancelled) ) is not None return False + + +def _raise_from_no_key_in_msg( + ctx: Context, + msg: dict, + src_err: KeyError, + log: StackLevelAdapter, # caller specific `log` obj + + expect_key: str = 'yield', + stream: MsgStream | None = None, + + # allow "deeper" tbs when debugging B^o + hide_tb: bool = True, + +) -> bool: + ''' + Raise an appopriate local error when a + `MsgStream` msg arrives which does not + contain the expected (at least under normal + operation) `'yield'` field. + + `Context` and any embedded `MsgStream` termination, + as well as remote task errors are handled in order + of priority as: + + - any 'error' msg is re-boxed and raised locally as + -> `RemoteActorError`|`ContextCancelled` + + - a `MsgStream` 'stop' msg is constructed, assigned + and raised locally as -> `trio.EndOfChannel` + + - All other mis-keyed msgss (like say a "final result" + 'return' msg, normally delivered from `Context.result()`) + are re-boxed inside a `MessagingError` with an explicit + exc content describing the missing IPC-msg-key. + + ''' + __tracebackhide__: bool = hide_tb + + # an internal error should never get here + try: + cid: str = msg['cid'] + except KeyError as src_err: + raise MessagingError( + f'IPC `Context` rx-ed msg without a ctx-id (cid)!?\n' + f'cid: {cid}\n\n' + + f'{pformat(msg)}\n' + ) from src_err + + # TODO: test that shows stream raising an expected error!!! + + # raise the error message in a boxed exception type! + if msg.get('error'): + raise unpack_error( + msg, + ctx.chan, + hide_tb=hide_tb, + + ) from None + + # `MsgStream` termination msg. + # TODO: does it make more sense to pack + # the stream._eoc outside this in the calleer always? + elif ( + msg.get('stop') + or ( + stream + and stream._eoc + ) + ): + log.debug( + f'Context[{cid}] stream was stopped by remote side\n' + f'cid: {cid}\n' + ) + + # TODO: if the a local task is already blocking on + # a `Context.result()` and thus a `.receive()` on the + # rx-chan, we close the chan and set state ensuring that + # an eoc is raised! + + # XXX: this causes ``ReceiveChannel.__anext__()`` to + # raise a ``StopAsyncIteration`` **and** in our catch + # block below it will trigger ``.aclose()``. + eoc = trio.EndOfChannel( + f'Context stream ended due to msg:\n\n' + f'{pformat(msg)}\n' + ) + # XXX: important to set so that a new `.receive()` + # call (likely by another task using a broadcast receiver) + # doesn't accidentally pull the `return` message + # value out of the underlying feed mem chan which is + # destined for the `Context.result()` call during ctx-exit! + stream._eoc: Exception = eoc + + raise eoc from src_err + + if ( + stream + and stream._closed + ): + raise trio.ClosedResourceError('This stream was closed') + + + # always re-raise the source error if no translation error case + # is activated above. + _type: str = 'Stream' if stream else 'Context' + raise MessagingError( + f"{_type} was expecting a '{expect_key}' message" + " BUT received a non-error msg:\n" + f'{pformat(msg)}' + ) from src_err diff --git a/tractor/_ipc.py b/tractor/_ipc.py index ebfd261c..e80a1c35 100644 --- a/tractor/_ipc.py +++ b/tractor/_ipc.py @@ -19,34 +19,33 @@ Inter-process comms abstractions """ from __future__ import annotations -import platform -import struct -import typing from collections.abc import ( AsyncGenerator, AsyncIterator, ) +from contextlib import asynccontextmanager as acm +import platform +from pprint import pformat +import struct +import typing from typing import ( Any, runtime_checkable, - Optional, Protocol, Type, TypeVar, ) -from tricycle import BufferedReceiveStream import msgspec +from tricycle import BufferedReceiveStream import trio -from async_generator import asynccontextmanager -from .log import get_logger -from ._exceptions import TransportClosed +from tractor.log import get_logger +from tractor._exceptions import TransportClosed + log = get_logger(__name__) - _is_windows = platform.system() == 'Windows' -log = get_logger(__name__) def get_stream_addrs(stream: trio.SocketStream) -> tuple: @@ -112,6 +111,13 @@ class MsgpackTCPStream(MsgTransport): using the ``msgspec`` codec lib. ''' + layer_key: int = 4 + name_key: str = 'tcp' + + # TODO: better naming for this? + # -[ ] check how libp2p does naming for such things? + codec_key: str = 'msgpack' + def __init__( self, stream: trio.SocketStream, @@ -199,7 +205,17 @@ class MsgpackTCPStream(MsgTransport): else: raise - async def send(self, msg: Any) -> None: + async def send( + self, + msg: Any, + + # hide_tb: bool = False, + ) -> None: + ''' + Send a msgpack coded blob-as-msg over TCP. + + ''' + # __tracebackhide__: bool = hide_tb async with self._send_lock: bytes_data: bytes = self.encode(msg) @@ -267,7 +283,7 @@ class Channel: def __init__( self, - destaddr: Optional[tuple[str, int]], + destaddr: tuple[str, int]|None, msg_transport_type_key: tuple[str, str] = ('msgpack', 'tcp'), @@ -285,18 +301,29 @@ class Channel: # Either created in ``.connect()`` or passed in by # user in ``.from_stream()``. - self._stream: Optional[trio.SocketStream] = None - self.msgstream: Optional[MsgTransport] = None + self._stream: trio.SocketStream|None = None + self._transport: MsgTransport|None = None # set after handshake - always uid of far end - self.uid: Optional[tuple[str, str]] = None + self.uid: tuple[str, str]|None = None self._agen = self._aiter_recv() - self._exc: Optional[Exception] = None # set if far end actor errors + self._exc: Exception|None = None # set if far end actor errors self._closed: bool = False - # flag set on ``Portal.cancel_actor()`` indicating - # remote (peer) cancellation of the far end actor runtime. - self._cancel_called: bool = False # set on ``Portal.cancel_actor()`` + + # flag set by ``Portal.cancel_actor()`` indicating remote + # (possibly peer) cancellation of the far end actor + # runtime. + self._cancel_called: bool = False + + @property + def msgstream(self) -> MsgTransport: + log.info('`Channel.msgstream` is an old name, use `._transport`') + return self._transport + + @property + def transport(self) -> MsgTransport: + return self._transport @classmethod def from_stream( @@ -307,37 +334,44 @@ class Channel: ) -> Channel: src, dst = get_stream_addrs(stream) - chan = Channel(destaddr=dst, **kwargs) + chan = Channel( + destaddr=dst, + **kwargs, + ) # set immediately here from provided instance - chan._stream = stream + chan._stream: trio.SocketStream = stream chan.set_msg_transport(stream) return chan def set_msg_transport( self, stream: trio.SocketStream, - type_key: Optional[tuple[str, str]] = None, + type_key: tuple[str, str]|None = None, ) -> MsgTransport: type_key = type_key or self._transport_key - self.msgstream = get_msg_transport(type_key)(stream) - return self.msgstream + self._transport = get_msg_transport(type_key)(stream) + return self._transport def __repr__(self) -> str: - if self.msgstream: - return repr( - self.msgstream.stream.socket._sock).replace( # type: ignore - "socket.socket", "Channel") - return object.__repr__(self) + if not self._transport: + return '' + + return repr( + self._transport.stream.socket._sock + ).replace( # type: ignore + "socket.socket", + "Channel", + ) @property - def laddr(self) -> Optional[tuple[str, int]]: - return self.msgstream.laddr if self.msgstream else None + def laddr(self) -> tuple[str, int]|None: + return self._transport.laddr if self._transport else None @property - def raddr(self) -> Optional[tuple[str, int]]: - return self.msgstream.raddr if self.msgstream else None + def raddr(self) -> tuple[str, int]|None: + return self._transport.raddr if self._transport else None async def connect( self, @@ -356,26 +390,42 @@ class Channel: *destaddr, **kwargs ) - msgstream = self.set_msg_transport(stream) + transport = self.set_msg_transport(stream) log.transport( - f'Opened channel[{type(msgstream)}]: {self.laddr} -> {self.raddr}' + f'Opened channel[{type(transport)}]: {self.laddr} -> {self.raddr}' ) - return msgstream + return transport - async def send(self, item: Any) -> None: + async def send( + self, + payload: Any, - log.transport(f"send `{item}`") # type: ignore - assert self.msgstream + # hide_tb: bool = False, - await self.msgstream.send(item) + ) -> None: + ''' + Send a coded msg-blob over the transport. + + ''' + # __tracebackhide__: bool = hide_tb + log.transport( + '=> send IPC msg:\n\n' + f'{pformat(payload)}\n' + ) # type: ignore + assert self._transport + + await self._transport.send( + payload, + # hide_tb=hide_tb, + ) async def recv(self) -> Any: - assert self.msgstream - return await self.msgstream.recv() + assert self._transport + return await self._transport.recv() # try: - # return await self.msgstream.recv() + # return await self._transport.recv() # except trio.BrokenResourceError: # if self._autorecon: # await self._reconnect() @@ -388,8 +438,8 @@ class Channel: f'Closing channel to {self.uid} ' f'{self.laddr} -> {self.raddr}' ) - assert self.msgstream - await self.msgstream.stream.aclose() + assert self._transport + await self._transport.stream.aclose() self._closed = True async def __aenter__(self): @@ -440,16 +490,16 @@ class Channel: Async iterate items from underlying stream. ''' - assert self.msgstream + assert self._transport while True: try: - async for item in self.msgstream: + async for item in self._transport: yield item # sent = yield item # if sent is not None: # # optimization, passing None through all the # # time is pointless - # await self.msgstream.send(sent) + # await self._transport.send(sent) except trio.BrokenResourceError: # if not self._autorecon: @@ -462,10 +512,10 @@ class Channel: # continue def connected(self) -> bool: - return self.msgstream.connected() if self.msgstream else False + return self._transport.connected() if self._transport else False -@asynccontextmanager +@acm async def _connect_chan( host: str, port: int ) -> typing.AsyncGenerator[Channel, None]: diff --git a/tractor/_portal.py b/tractor/_portal.py index 17871aa2..0ca44483 100644 --- a/tractor/_portal.py +++ b/tractor/_portal.py @@ -15,38 +15,46 @@ # along with this program. If not, see . ''' -Memory boundary "Portals": an API for structured -concurrency linked tasks running in disparate memory domains. +Memory "portal" contruct. + +"Memory portals" are both an API and set of IPC wrapping primitives +for managing structured concurrency "cancel-scope linked" tasks +running in disparate virtual memory domains - at least in different +OS processes, possibly on different (hardware) hosts. ''' from __future__ import annotations +from contextlib import asynccontextmanager as acm import importlib import inspect from typing import ( - Any, Optional, - Callable, AsyncGenerator, - Type, + Any, + Callable, + AsyncGenerator, + # Type, ) from functools import partial from dataclasses import dataclass -from pprint import pformat import warnings import trio -from async_generator import asynccontextmanager from .trionics import maybe_open_nursery -from ._state import current_actor +from ._state import ( + current_actor, +) from ._ipc import Channel from .log import get_logger from .msg import NamespacePath from ._exceptions import ( unpack_error, NoResult, - ContextCancelled, +) +from ._context import ( + Context, + open_context_from_portal, ) from ._streaming import ( - Context, MsgStream, ) @@ -54,34 +62,47 @@ from ._streaming import ( log = get_logger(__name__) +# TODO: rename to `unwrap_result()` and use +# `._raise_from_no_key_in_msg()` (after tweak to +# accept a `chan: Channel` arg) in key block! def _unwrap_msg( msg: dict[str, Any], - channel: Channel + channel: Channel, + + hide_tb: bool = True, ) -> Any: - __tracebackhide__ = True + ''' + Unwrap a final result from a `{return: }` IPC msg. + + ''' + __tracebackhide__: bool = hide_tb + try: return msg['return'] - except KeyError: + except KeyError as ke: + # internal error should never get here - assert msg.get('cid'), "Received internal error at portal?" - raise unpack_error(msg, channel) from None + assert msg.get('cid'), ( + "Received internal error at portal?" + ) - -class MessagingError(Exception): - 'Some kind of unexpected SC messaging dialog issue' + raise unpack_error( + msg, + channel + ) from ke class Portal: ''' - A 'portal' to a(n) (remote) ``Actor``. + A 'portal' to a memory-domain-separated `Actor`. A portal is "opened" (and eventually closed) by one side of an inter-actor communication context. The side which opens the portal is equivalent to a "caller" in function parlance and usually is either the called actor's parent (in process tree hierarchy terms) or a client interested in scheduling work to be done remotely in a - far process. + process which has a separate (virtual) memory domain. The portal api allows the "caller" actor to invoke remote routines and receive results through an underlying ``tractor.Channel`` as @@ -91,22 +112,34 @@ class Portal: like having a "portal" between the seperate actor memory spaces. ''' - # the timeout for a remote cancel request sent to - # a(n) (peer) actor. - cancel_timeout = 0.5 + # global timeout for remote cancel requests sent to + # connected (peer) actors. + cancel_timeout: float = 0.5 def __init__(self, channel: Channel) -> None: - self.channel = channel + self.chan = channel # during the portal's lifetime - self._result_msg: Optional[dict] = None + self._result_msg: dict|None = None # When set to a ``Context`` (when _submit_for_result is called) # it is expected that ``result()`` will be awaited at some # point. - self._expect_result: Optional[Context] = None + self._expect_result: Context | None = None self._streams: set[MsgStream] = set() self.actor = current_actor() + @property + def channel(self) -> Channel: + ''' + Proxy to legacy attr name.. + + Consider the shorter `Portal.chan` instead of `.channel` ;) + ''' + log.debug( + 'Consider the shorter `Portal.chan` instead of `.channel` ;)' + ) + return self.chan + async def _submit_for_result( self, ns: str, @@ -114,14 +147,14 @@ class Portal: **kwargs ) -> None: - assert self._expect_result is None, \ - "A pending main result has already been submitted" + assert self._expect_result is None, ( + "A pending main result has already been submitted" + ) self._expect_result = await self.actor.start_remote_task( self.channel, - ns, - func, - kwargs + nsf=NamespacePath(f'{ns}:{func}'), + kwargs=kwargs ) async def _return_once( @@ -131,7 +164,7 @@ class Portal: ) -> dict[str, Any]: assert ctx._remote_func_type == 'asyncfunc' # single response - msg = await ctx._recv_chan.receive() + msg: dict = await ctx._recv_chan.receive() return msg async def result(self) -> Any: @@ -162,7 +195,10 @@ class Portal: self._expect_result ) - return _unwrap_msg(self._result_msg, self.channel) + return _unwrap_msg( + self._result_msg, + self.channel, + ) async def _cancel_streams(self): # terminate all locally running async generator @@ -193,30 +229,57 @@ class Portal: ) -> bool: ''' - Cancel the actor on the other end of this portal. + Cancel the actor runtime (and thus process) on the far + end of this portal. + + **NOTE** THIS CANCELS THE ENTIRE RUNTIME AND THE + SUBPROCESS, it DOES NOT just cancel the remote task. If you + want to have a handle to cancel a remote ``tri.Task`` look + at `.open_context()` and the definition of + `._context.Context.cancel()` which CAN be used for this + purpose. ''' - if not self.channel.connected(): - log.cancel("This channel is already closed can't cancel") + chan: Channel = self.channel + if not chan.connected(): + log.runtime( + 'This channel is already closed, skipping cancel request..' + ) return False + reminfo: str = ( + f'`Portal.cancel_actor()` => {self.channel.uid}\n' + f' |_{chan}\n' + ) log.cancel( - f"Sending actor cancel request to {self.channel.uid} on " - f"{self.channel}") - - self.channel._cancel_called = True + f'Sending runtime `.cancel()` request to peer\n\n' + f'{reminfo}' + ) + self.channel._cancel_called: bool = True try: # send cancel cmd - might not get response - # XXX: sure would be nice to make this work with a proper shield - with trio.move_on_after(timeout or self.cancel_timeout) as cs: - cs.shield = True - - await self.run_from_ns('self', 'cancel') + # XXX: sure would be nice to make this work with + # a proper shield + with trio.move_on_after( + timeout + or + self.cancel_timeout + ) as cs: + cs.shield: bool = True + await self.run_from_ns( + 'self', + 'cancel', + ) return True if cs.cancelled_caught: - log.cancel(f"May have failed to cancel {self.channel.uid}") + # may timeout and we never get an ack (obvi racy) + # but that doesn't mean it wasn't cancelled. + log.debug( + 'May have failed to cancel peer?\n' + f'{reminfo}' + ) # if we get here some weird cancellation case happened return False @@ -225,9 +288,11 @@ class Portal: trio.ClosedResourceError, trio.BrokenResourceError, ): - log.cancel( - f"{self.channel} for {self.channel.uid} was already " - "closed or broken?") + log.debug( + 'IPC chan for actor already closed or broken?\n\n' + f'{self.channel.uid}\n' + f' |_{self.channel}\n' + ) return False async def run_from_ns( @@ -246,27 +311,33 @@ class Portal: Note:: - A special namespace `self` can be used to invoke `Actor` - instance methods in the remote runtime. Currently this - should only be used solely for ``tractor`` runtime - internals. + A special namespace `self` can be used to invoke `Actor` + instance methods in the remote runtime. Currently this + should only ever be used for `Actor` (method) runtime + internals! ''' + nsf = NamespacePath( + f'{namespace_path}:{function_name}' + ) ctx = await self.actor.start_remote_task( - self.channel, - namespace_path, - function_name, - kwargs, + chan=self.channel, + nsf=nsf, + kwargs=kwargs, ) ctx._portal = self msg = await self._return_once(ctx) - return _unwrap_msg(msg, self.channel) + return _unwrap_msg( + msg, + self.channel, + ) async def run( self, func: str, - fn_name: Optional[str] = None, + fn_name: str|None = None, **kwargs + ) -> Any: ''' Submit a remote function to be scheduled and run by actor, in @@ -285,8 +356,9 @@ class Portal: DeprecationWarning, stacklevel=2, ) - fn_mod_path = func + fn_mod_path: str = func assert isinstance(fn_name, str) + nsf = NamespacePath(f'{fn_mod_path}:{fn_name}') else: # function reference was passed directly if ( @@ -299,13 +371,12 @@ class Portal: raise TypeError( f'{func} must be a non-streaming async function!') - fn_mod_path, fn_name = NamespacePath.from_ref(func).to_tuple() + nsf = NamespacePath.from_ref(func) ctx = await self.actor.start_remote_task( self.channel, - fn_mod_path, - fn_name, - kwargs, + nsf=nsf, + kwargs=kwargs, ) ctx._portal = self return _unwrap_msg( @@ -313,7 +384,7 @@ class Portal: self.channel, ) - @asynccontextmanager + @acm async def open_stream_from( self, async_gen_func: Callable, # typing: ignore @@ -329,13 +400,10 @@ class Portal: raise TypeError( f'{async_gen_func} must be an async generator function!') - fn_mod_path, fn_name = NamespacePath.from_ref( - async_gen_func).to_tuple() - ctx = await self.actor.start_remote_task( + ctx: Context = await self.actor.start_remote_task( self.channel, - fn_mod_path, - fn_name, - kwargs + nsf=NamespacePath.from_ref(async_gen_func), + kwargs=kwargs, ) ctx._portal = self @@ -345,7 +413,8 @@ class Portal: try: # deliver receive only stream async with MsgStream( - ctx, ctx._recv_chan, + ctx=ctx, + rx_chan=ctx._recv_chan, ) as rchan: self._streams.add(rchan) yield rchan @@ -372,175 +441,12 @@ class Portal: # await recv_chan.aclose() self._streams.remove(rchan) - @asynccontextmanager - async def open_context( - - self, - func: Callable, - **kwargs, - - ) -> AsyncGenerator[tuple[Context, Any], None]: - ''' - Open an inter-actor task context. - - This is a synchronous API which allows for deterministic - setup/teardown of a remote task. The yielded ``Context`` further - allows for opening bidirectional streams, explicit cancellation - and synchronized final result collection. See ``tractor.Context``. - - ''' - # conduct target func method structural checks - if not inspect.iscoroutinefunction(func) and ( - getattr(func, '_tractor_contex_function', False) - ): - raise TypeError( - f'{func} must be an async generator function!') - - fn_mod_path, fn_name = NamespacePath.from_ref(func).to_tuple() - - ctx = await self.actor.start_remote_task( - self.channel, - fn_mod_path, - fn_name, - kwargs - ) - - assert ctx._remote_func_type == 'context' - msg = await ctx._recv_chan.receive() - - try: - # the "first" value here is delivered by the callee's - # ``Context.started()`` call. - first = msg['started'] - ctx._started_called = True - - except KeyError: - assert msg.get('cid'), ("Received internal error at context?") - - if msg.get('error'): - # raise kerr from unpack_error(msg, self.channel) - raise unpack_error(msg, self.channel) from None - else: - raise MessagingError( - f'Context for {ctx.cid} was expecting a `started` message' - f' but received a non-error msg:\n{pformat(msg)}' - ) - - _err: Optional[BaseException] = None - ctx._portal = self - - uid = self.channel.uid - cid = ctx.cid - etype: Optional[Type[BaseException]] = None - - # deliver context instance and .started() msg value in open tuple. - try: - async with trio.open_nursery() as scope_nursery: - ctx._scope_nursery = scope_nursery - - # do we need this? - # await trio.lowlevel.checkpoint() - - yield ctx, first - - except ContextCancelled as err: - _err = err - if not ctx._cancel_called: - # context was cancelled at the far end but was - # not part of this end requesting that cancel - # so raise for the local task to respond and handle. - raise - - # if the context was cancelled by client code - # then we don't need to raise since user code - # is expecting this and the block should exit. - else: - log.debug(f'Context {ctx} cancelled gracefully') - - except ( - BaseException, - - # more specifically, we need to handle these but not - # sure it's worth being pedantic: - # Exception, - # trio.Cancelled, - # KeyboardInterrupt, - - ) as err: - etype = type(err) - # the context cancels itself on any cancel - # causing error. - - if ctx.chan.connected(): - log.cancel( - 'Context cancelled for task, sending cancel request..\n' - f'task:{cid}\n' - f'actor:{uid}' - ) - await ctx.cancel() - else: - log.warning( - 'IPC connection for context is broken?\n' - f'task:{cid}\n' - f'actor:{uid}' - ) - - raise - - 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? - if ctx.chan.connected(): - log.info( - 'Waiting on final context-task result for\n' - f'task: {cid}\n' - f'actor: {uid}' - ) - result = await ctx.result() - log.runtime( - f'Context {fn_name} returned ' - f'value from callee `{result}`' - ) - - # though it should be impossible for any tasks - # operating *in* this scope to have survived - # we tear down the runtime feeder chan last - # to avoid premature stream clobbers. - if ctx._recv_chan is not None: - # should we encapsulate this in the context api? - await ctx._recv_chan.aclose() - - if etype: - if ctx._cancel_called: - log.cancel( - f'Context {fn_name} cancelled by caller with\n{etype}' - ) - elif _err is not None: - log.cancel( - f'Context for task cancelled by callee with {etype}\n' - f'target: `{fn_name}`\n' - f'task:{cid}\n' - f'actor:{uid}' - ) - # XXX: (MEGA IMPORTANT) if this is a root opened process we - # wait for any immediate child in debug before popping the - # context from the runtime msg loop otherwise inside - # ``Actor._push_result()`` the msg will be discarded and in - # the case where that msg is global debugger unlock (via - # a "stop" msg for a stream), this can result in a deadlock - # where the root is waiting on the lock to clear but the - # child has already cleared it and clobbered IPC. - from ._debug import maybe_wait_for_debugger - await maybe_wait_for_debugger() - - # remove the context from runtime tracking - self.actor._contexts.pop( - (self.channel.uid, ctx.cid), - None, - ) + # NOTE: impl is found in `._context`` mod to make + # reading/groking the details simpler code-org-wise. This + # method does not have to be used over that `@acm` module func + # directly, it is for conventience and from the original API + # design. + open_context = open_context_from_portal @dataclass @@ -566,11 +472,11 @@ class LocalPortal: return await func(**kwargs) -@asynccontextmanager +@acm async def open_portal( channel: Channel, - nursery: Optional[trio.Nursery] = None, + nursery: trio.Nursery|None = None, start_msg_loop: bool = True, shield: bool = False, @@ -595,7 +501,7 @@ async def open_portal( if channel.uid is None: await actor._do_handshake(channel) - msg_loop_cs: Optional[trio.CancelScope] = None + msg_loop_cs: trio.CancelScope|None = None if start_msg_loop: from ._runtime import process_messages msg_loop_cs = await nursery.start( diff --git a/tractor/_root.py b/tractor/_root.py index 64652a19..881dc90f 100644 --- a/tractor/_root.py +++ b/tractor/_root.py @@ -29,12 +29,13 @@ import typing import warnings -from exceptiongroup import BaseExceptionGroup import trio from ._runtime import ( Actor, Arbiter, + # TODO: rename and make a non-actor subtype? + # Arbiter as Registry, async_main, ) from . import _debug @@ -237,8 +238,11 @@ async def open_root_actor( entered = await _debug._maybe_enter_pm(err) - if not entered and not is_multi_cancelled(err): - logger.exception("Root actor crashed:") + if ( + not entered + and not is_multi_cancelled(err) + ): + logger.exception('Root actor crashed:\n') # always re-raise raise @@ -253,10 +257,13 @@ async def open_root_actor( # for an in nurseries: # tempn.start_soon(an.exited.wait) - logger.cancel("Shutting down root actor") - await actor.cancel() + logger.info( + 'Closing down root actor' + ) + await actor.cancel(None) # self cancel finally: _state._current_actor = None + _state._last_actor_terminated = actor # restore breakpoint hook state sys.breakpointhook = builtin_bp_handler diff --git a/tractor/_rpc.py b/tractor/_rpc.py new file mode 100644 index 00000000..47548106 --- /dev/null +++ b/tractor/_rpc.py @@ -0,0 +1,1109 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +''' +Remote (task) Procedure Call (scheduling) with SC transitive semantics. + +''' +from __future__ import annotations +from contextlib import ( + asynccontextmanager as acm, + aclosing, +) +from functools import partial +import inspect +from pprint import pformat +from types import ModuleType +from typing import ( + Any, + Callable, + Coroutine, + TYPE_CHECKING, +) +import warnings + +import trio +from trio import ( + CancelScope, + Nursery, + TaskStatus, +) + +from .msg import NamespacePath +from ._ipc import Channel +from ._context import ( + Context, +) +from ._exceptions import ( + ModuleNotExposed, + is_multi_cancelled, + ContextCancelled, + pack_error, + unpack_error, + TransportClosed, +) +from . import _debug +from . import _state +from .log import get_logger + +if TYPE_CHECKING: + from ._runtime import Actor + +log = get_logger('tractor') + + +async def _invoke_non_context( + actor: Actor, + cancel_scope: CancelScope, + ctx: Context, + cid: str, + chan: Channel, + func: Callable, + coro: Coroutine, + kwargs: dict[str, Any], + + treat_as_gen: bool, + is_rpc: bool, + + task_status: TaskStatus[ + Context | BaseException + ] = trio.TASK_STATUS_IGNORED, +): + + # TODO: can we unify this with the `context=True` impl below? + if inspect.isasyncgen(coro): + await chan.send({'functype': 'asyncgen', 'cid': cid}) + # XXX: massive gotcha! If the containing scope + # is cancelled and we execute the below line, + # any ``ActorNursery.__aexit__()`` WON'T be + # triggered in the underlying async gen! So we + # have to properly handle the closing (aclosing) + # of the async gen in order to be sure the cancel + # is propagated! + with cancel_scope as cs: + ctx._scope = cs + task_status.started(ctx) + async with aclosing(coro) as agen: + async for item in agen: + # TODO: can we send values back in here? + # it's gonna require a `while True:` and + # some non-blocking way to retrieve new `asend()` + # values from the channel: + # to_send = await chan.recv_nowait() + # if to_send is not None: + # to_yield = await coro.asend(to_send) + await chan.send({'yield': item, 'cid': cid}) + + log.runtime(f"Finished iterating {coro}") + # TODO: we should really support a proper + # `StopAsyncIteration` system here for returning a final + # value if desired + await chan.send({'stop': True, 'cid': cid}) + + # one way @stream func that gets treated like an async gen + # TODO: can we unify this with the `context=True` impl below? + elif treat_as_gen: + await chan.send({'functype': 'asyncgen', 'cid': cid}) + # XXX: the async-func may spawn further tasks which push + # back values like an async-generator would but must + # manualy construct the response dict-packet-responses as + # above + with cancel_scope as cs: + ctx._scope = cs + task_status.started(ctx) + await coro + + if not cs.cancelled_caught: + # task was not cancelled so we can instruct the + # far end async gen to tear down + await chan.send({'stop': True, 'cid': cid}) + else: + # regular async function/method + # XXX: possibly just a scheduled `Actor._cancel_task()` + # from a remote request to cancel some `Context`. + # ------ - ------ + # TODO: ideally we unify this with the above `context=True` + # block such that for any remote invocation ftype, we + # always invoke the far end RPC task scheduling the same + # way: using the linked IPC context machinery. + failed_resp: bool = False + try: + await chan.send({ + 'functype': 'asyncfunc', + 'cid': cid + }) + except ( + trio.ClosedResourceError, + trio.BrokenResourceError, + BrokenPipeError, + ) as ipc_err: + failed_resp = True + if is_rpc: + raise + else: + # TODO: should this be an `.exception()` call? + log.warning( + f'Failed to respond to non-rpc request: {func}\n' + f'{ipc_err}' + ) + + with cancel_scope as cs: + ctx._scope: CancelScope = cs + task_status.started(ctx) + result = await coro + fname: str = func.__name__ + log.runtime( + 'RPC complete:\n' + f'task: {ctx._task}\n' + f'|_cid={ctx.cid}\n' + f'|_{fname}() -> {pformat(result)}\n' + ) + + # NOTE: only send result if we know IPC isn't down + if ( + not failed_resp + and chan.connected() + ): + try: + await chan.send( + {'return': result, + 'cid': cid} + ) + except ( + BrokenPipeError, + trio.BrokenResourceError, + ): + log.warning( + 'Failed to return result:\n' + f'{func}@{actor.uid}\n' + f'remote chan: {chan.uid}' + ) + +@acm +async def _errors_relayed_via_ipc( + actor: Actor, + chan: Channel, + ctx: Context, + is_rpc: bool, + + hide_tb: bool = False, + debug_kbis: bool = False, + task_status: TaskStatus[ + Context | BaseException + ] = trio.TASK_STATUS_IGNORED, + +) -> None: + __tracebackhide__: bool = hide_tb # TODO: use hide_tb here? + try: + yield # run RPC invoke body + + # box and ship RPC errors for wire-transit via + # the task's requesting parent IPC-channel. + except ( + Exception, + BaseExceptionGroup, + KeyboardInterrupt, + ) as err: + + # always hide this frame from debug REPL if the crash + # originated from an rpc task and we DID NOT fail due to + # an IPC transport error! + if ( + is_rpc + and chan.connected() + ): + __tracebackhide__: bool = hide_tb + + if not is_multi_cancelled(err): + + # TODO: maybe we'll want different "levels" of debugging + # eventualy such as ('app', 'supervisory', 'runtime') ? + + # if not isinstance(err, trio.ClosedResourceError) and ( + # if not is_multi_cancelled(err) and ( + + entered_debug: bool = False + if ( + ( + not isinstance(err, ContextCancelled) + or ( + isinstance(err, ContextCancelled) + and ctx._cancel_called + + # if the root blocks the debugger lock request from a child + # we will get a remote-cancelled condition. + and ctx._enter_debugger_on_cancel + ) + ) + and + ( + not isinstance(err, KeyboardInterrupt) + or ( + isinstance(err, KeyboardInterrupt) + and debug_kbis + ) + ) + ): + # await _debug.pause() + # XXX QUESTION 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:\n') + + # always (try to) ship RPC errors back to caller + if is_rpc: + # + # TODO: tests for this scenario: + # - RPC caller closes connection before getting a response + # should **not** crash this actor.. + await try_ship_error_to_remote( + chan, + err, + cid=ctx.cid, + remote_descr='caller', + hide_tb=hide_tb, + ) + + # error is probably from above coro running code *not from + # the target rpc invocation since a scope was never + # allocated around the coroutine await. + if ctx._scope is None: + # we don't ever raise directly here to allow the + # msg-loop-scheduler to continue running for this + # channel. + task_status.started(err) + + # always reraise KBIs so they propagate at the sys-process + # level. + if isinstance(err, KeyboardInterrupt): + raise + + + # RPC task bookeeping + finally: + try: + ctx, func, is_complete = actor._rpc_tasks.pop( + (chan, ctx.cid) + ) + is_complete.set() + + except KeyError: + if is_rpc: + # If we're cancelled before the task returns then the + # cancel scope will not have been inserted yet + log.warning( + 'RPC task likely errored or cancelled before start?' + f'|_{ctx._task}\n' + f' >> {ctx.repr_rpc}\n' + ) + else: + log.cancel( + 'Failed to de-alloc internal runtime cancel task?\n' + f'|_{ctx._task}\n' + f' >> {ctx.repr_rpc}\n' + ) + + finally: + if not actor._rpc_tasks: + log.runtime("All RPC tasks have completed") + actor._ongoing_rpc_tasks.set() + + +_gb_mod: ModuleType|None|False = None + + +async def maybe_import_gb(): + global _gb_mod + if _gb_mod is False: + return + + try: + import greenback + _gb_mod = greenback + await greenback.ensure_portal() + + except ModuleNotFoundError: + log.debug( + '`greenback` is not installed.\n' + 'No sync debug support!\n' + ) + _gb_mod = False + + +async def _invoke( + + actor: Actor, + cid: str, + chan: Channel, + func: Callable, + kwargs: dict[str, Any], + + is_rpc: bool = True, + hide_tb: bool = True, + + task_status: TaskStatus[ + Context | BaseException + ] = trio.TASK_STATUS_IGNORED, +): + ''' + Schedule a `trio` task-as-func and deliver result(s) over + connected IPC channel. + + This is the core "RPC" `trio.Task` scheduling machinery used to start every + remotely invoked function, normally in `Actor._service_n: Nursery`. + + ''' + __tracebackhide__: bool = hide_tb + treat_as_gen: bool = False + + if _state.debug_mode(): + await maybe_import_gb() + + # TODO: possibly a specially formatted traceback + # (not sure what typing is for this..)? + # tb = None + + cancel_scope = CancelScope() + # activated cancel scope ref + cs: CancelScope|None = None + + ctx = actor.get_context( + chan=chan, + cid=cid, + nsf=NamespacePath.from_ref(func), + + # TODO: if we wanted to get cray and support it? + # side='callee', + + # We shouldn't ever need to pass this through right? + # it's up to the soon-to-be called rpc task to + # open the stream with this option. + # allow_overruns=True, + ) + context: bool = False + + # TODO: deprecate this style.. + if getattr(func, '_tractor_stream_function', False): + # handle decorated ``@tractor.stream`` async functions + sig = inspect.signature(func) + params = sig.parameters + + # compat with old api + kwargs['ctx'] = ctx + treat_as_gen = True + + if 'ctx' in params: + warnings.warn( + "`@tractor.stream decorated funcs should now declare " + "a `stream` arg, `ctx` is now designated for use with " + "@tractor.context", + DeprecationWarning, + stacklevel=2, + ) + + elif 'stream' in params: + assert 'stream' in params + kwargs['stream'] = ctx + + + elif getattr(func, '_tractor_context_function', False): + # handle decorated ``@tractor.context`` async function + kwargs['ctx'] = ctx + context = True + + # errors raised inside this block are propgated back to caller + async with _errors_relayed_via_ipc( + actor, + chan, + ctx, + is_rpc, + hide_tb=hide_tb, + task_status=task_status, + ): + if not ( + inspect.isasyncgenfunction(func) or + inspect.iscoroutinefunction(func) + ): + raise TypeError(f'{func} must be an async function!') + + # init coroutine with `kwargs` to immediately catch any + # type-sig errors. + try: + coro = func(**kwargs) + except TypeError: + raise + + # TODO: implement all these cases in terms of the + # `Context` one! + if not context: + await _invoke_non_context( + actor, + cancel_scope, + ctx, + cid, + chan, + func, + coro, + kwargs, + treat_as_gen, + is_rpc, + task_status, + ) + # below is only for `@context` funcs + return + + # our most general case: a remote SC-transitive, + # IPC-linked, cross-actor-task "context" + # ------ - ------ + # TODO: every other "func type" should be implemented from + # a special case of this impl eventually! + # -[ ] streaming funcs should instead of being async-for + # handled directly here wrapped in + # a async-with-open_stream() closure that does the + # normal thing you'd expect a far end streaming context + # to (if written by the app-dev). + # -[ ] one off async funcs can literally just be called + # here and awaited directly, possibly just with a small + # wrapper that calls `Context.started()` and then does + # the `await coro()`? + + # a "context" endpoint type is the most general and + # "least sugary" type of RPC ep with support for + # bi-dir streaming B) + await chan.send({ + 'functype': 'context', + 'cid': cid + }) + + # TODO: should we also use an `.open_context()` equiv + # for this callee side by factoring the impl from + # `Portal.open_context()` into a common helper? + # + # NOTE: there are many different ctx state details + # in a callee side instance according to current impl: + # - `.cancelled_caught` can never be `True`. + # -> the below scope is never exposed to the + # `@context` marked RPC function. + # - `._portal` is never set. + try: + async with trio.open_nursery() as tn: + ctx._scope_nursery = tn + ctx._scope = tn.cancel_scope + task_status.started(ctx) + + # TODO: should would be nice to have our + # `TaskMngr` nursery here! + res: Any = await coro + ctx._result = res + + # deliver final result to caller side. + await chan.send({ + 'return': res, + 'cid': cid + }) + + # NOTE: this happens IFF `ctx._scope.cancel()` is + # called by any of, + # - *this* callee task manually calling `ctx.cancel()`. + # - the runtime calling `ctx._deliver_msg()` which + # itself calls `ctx._maybe_cancel_and_set_remote_error()` + # which cancels the scope presuming the input error + # is not a `.cancel_acked` pleaser. + # - currently a never-should-happen-fallthrough case + # inside ._context._drain_to_final_msg()`.. + # # TODO: remove this ^ right? + if ctx._scope.cancelled_caught: + our_uid: tuple = actor.uid + + # first check for and raise any remote error + # before raising any context cancelled case + # so that real remote errors don't get masked as + # ``ContextCancelled``s. + if re := ctx._remote_error: + ctx._maybe_raise_remote_err(re) + + cs: CancelScope = ctx._scope + + if cs.cancel_called: + + canceller: tuple = ctx.canceller + msg: str = ( + 'actor was cancelled by ' + ) + + # NOTE / TODO: if we end up having + # ``Actor._cancel_task()`` call + # ``Context.cancel()`` directly, we're going to + # need to change this logic branch since it + # will always enter.. + if ctx._cancel_called: + # TODO: test for this!!!!! + canceller: tuple = our_uid + msg += 'itself ' + + # if the channel which spawned the ctx is the + # one that cancelled it then we report that, vs. + # it being some other random actor that for ex. + # some actor who calls `Portal.cancel_actor()` + # and by side-effect cancels this ctx. + elif canceller == ctx.chan.uid: + msg += 'its caller' + + else: + msg += 'a remote peer' + + div_chars: str = '------ - ------' + div_offset: int = ( + round(len(msg)/2)+1 + + + round(len(div_chars)/2)+1 + ) + div_str: str = ( + '\n' + + + ' '*div_offset + + + f'{div_chars}\n' + ) + msg += ( + div_str + + f'<= canceller: {canceller}\n' + f'=> uid: {our_uid}\n' + f' |_{ctx._task}()' + + # TODO: instead just show the + # ctx.__str__() here? + # -[ ] textwrap.indent() it correctly! + # -[ ] BUT we need to wait until + # the state is filled out before emitting + # this msg right ow its kinda empty? bleh.. + # + # f' |_{ctx}' + ) + + # task-contex was either cancelled by request using + # ``Portal.cancel_actor()`` or ``Context.cancel()`` + # on the far end, or it was cancelled by the local + # (callee) task, so relay this cancel signal to the + # other side. + ctxc = ContextCancelled( + msg, + suberror_type=trio.Cancelled, + canceller=canceller, + ) + # assign local error so that the `.outcome` + # resolves to an error for both reporting and + # state checks. + ctx._local_error = ctxc + raise ctxc + + # XXX: do we ever trigger this block any more? + except ( + BaseExceptionGroup, + trio.Cancelled, + BaseException, + + ) as scope_error: + + # always set this (callee) side's exception as the + # local error on the context + ctx._local_error: BaseException = scope_error + + # if a remote error was set then likely the + # exception group was raised due to that, so + # and we instead raise that error immediately! + ctx.maybe_raise() + + # maybe TODO: pack in come kinda + # `trio.Cancelled.__traceback__` here so they can be + # unwrapped and displayed on the caller side? no se.. + raise + + # `@context` entrypoint task bookeeping. + # i.e. only pop the context tracking if used ;) + finally: + assert chan.uid + + # don't pop the local context until we know the + # associated child isn't in debug any more + await _debug.maybe_wait_for_debugger() + ctx: Context = actor._contexts.pop(( + chan.uid, + cid, + # ctx.side, + )) + + merr: Exception|None = ctx.maybe_error + + ( + res_type_str, + res_str, + ) = ( + ('error', f'{type(merr)}',) + if merr + else ( + 'result', + f'`{repr(ctx.outcome)}`', + ) + ) + log.cancel( + f'IPC context terminated with a final {res_type_str}\n\n' + f'{ctx}\n' + ) + + +async def try_ship_error_to_remote( + channel: Channel, + err: Exception|BaseExceptionGroup, + + cid: str|None = None, + remote_descr: str = 'parent', + hide_tb: bool = True, + +) -> None: + ''' + Box, pack and encode a local runtime(-internal) exception for + an IPC channel `.send()` with transport/network failures and + local cancellation ignored but logged as critical(ly bad). + + ''' + __tracebackhide__: bool = hide_tb + with CancelScope(shield=True): + try: + # NOTE: normally only used for internal runtime errors + # so ship to peer actor without a cid. + msg: dict = pack_error( + err, + cid=cid, + + # TODO: special tb fmting for ctxc cases? + # tb=tb, + ) + # NOTE: the src actor should always be packed into the + # error.. but how should we verify this? + # actor: Actor = _state.current_actor() + # assert err_msg['src_actor_uid'] + # if not err_msg['error'].get('src_actor_uid'): + # import pdbp; pdbp.set_trace() + await channel.send(msg) + + # XXX NOTE XXX in SC terms this is one of the worst things + # that can happen and provides for a 2-general's dilemma.. + except ( + trio.ClosedResourceError, + trio.BrokenResourceError, + BrokenPipeError, + ): + err_msg: dict = msg['error']['tb_str'] + log.critical( + 'IPC transport failure -> ' + f'failed to ship error to {remote_descr}!\n\n' + f'X=> {channel.uid}\n\n' + f'{err_msg}\n' + ) + + +async def process_messages( + actor: Actor, + chan: Channel, + shield: bool = False, + task_status: TaskStatus[CancelScope] = trio.TASK_STATUS_IGNORED, + +) -> bool: + ''' + This is the low-level, per-IPC-channel, RPC task scheduler loop. + + Receive (multiplexed) per-`Channel` RPC requests as msgs from + remote processes; schedule target async funcs as local + `trio.Task`s inside the `Actor._service_n: Nursery`. + + Depending on msg type, non-`cmd` (task spawning/starting) + request payloads (eg. `started`, `yield`, `return`, `error`) + are delivered to locally running, linked-via-`Context`, tasks + with any (boxed) errors and/or final results shipped back to + the remote side. + + All higher level inter-actor comms ops are delivered in some + form by the msg processing here, including: + + - lookup and invocation of any (async) funcs-as-tasks requested + by remote actors presuming the local actor has enabled their + containing module. + + - IPC-session oriented `Context` and `MsgStream` msg payload + delivery such as `started`, `yield` and `return` msgs. + + - cancellation handling for both `Context.cancel()` (which + translate to `Actor._cancel_task()` RPCs server side) + and `Actor.cancel()` process-wide-runtime-shutdown requests + (as utilized inside `Portal.cancel_actor()` ). + + + ''' + # TODO: once `trio` get's an "obvious way" for req/resp we + # should use it? + # https://github.com/python-trio/trio/issues/467 + log.runtime( + 'Entering IPC msg loop:\n' + f'peer: {chan.uid}\n' + f'|_{chan}\n' + ) + nursery_cancelled_before_task: bool = False + msg: dict | None = None + try: + # NOTE: this internal scope allows for keeping this + # message loop running despite the current task having + # been cancelled (eg. `open_portal()` may call this method + # from a locally spawned task) and recieve this scope + # using ``scope = Nursery.start()`` + with CancelScope(shield=shield) as loop_cs: + task_status.started(loop_cs) + async for msg in chan: + + # dedicated loop terminate sentinel + if msg is None: + + tasks: dict[ + tuple[Channel, str], + tuple[Context, Callable, trio.Event] + ] = actor._rpc_tasks.copy() + log.cancel( + f'Peer IPC channel terminated via `None` setinel msg?\n' + f'=> Cancelling all {len(tasks)} local RPC tasks..\n' + f'peer: {chan.uid}\n' + f'|_{chan}\n' + ) + for (channel, cid) in tasks: + if channel is chan: + await actor._cancel_task( + cid, + channel, + requesting_uid=channel.uid, + + ipc_msg=msg, + ) + break + + log.transport( # type: ignore + f'<= IPC msg from peer: {chan.uid}\n\n' + + # TODO: conditionally avoid fmting depending + # on log level (for perf)? + # => specifically `pformat()` sub-call..? + f'{pformat(msg)}\n' + ) + + cid = msg.get('cid') + if cid: + # deliver response to local caller/waiter + # via its per-remote-context memory channel. + await actor._push_result( + chan, + cid, + msg, + ) + + log.runtime( + 'Waiting on next IPC msg from\n' + f'peer: {chan.uid}:\n' + f'|_{chan}\n' + + # f'last msg: {msg}\n' + ) + continue + + # process a 'cmd' request-msg upack + # TODO: impl with native `msgspec.Struct` support !! + # -[ ] implement with ``match:`` syntax? + # -[ ] discard un-authed msgs as per, + # + try: + ( + ns, + funcname, + kwargs, + actorid, + cid, + ) = msg['cmd'] + + except KeyError: + # This is the non-rpc error case, that is, an + # error **not** raised inside a call to ``_invoke()`` + # (i.e. no cid was provided in the msg - see above). + # Push this error to all local channel consumers + # (normally portals) by marking the channel as errored + assert chan.uid + exc = unpack_error(msg, chan=chan) + chan._exc = exc + raise exc + + log.runtime( + 'Handling RPC cmd from\n' + f'peer: {actorid}\n' + '\n' + f'=> {ns}.{funcname}({kwargs})\n' + ) + if ns == 'self': + if funcname == 'cancel': + func: Callable = actor.cancel + kwargs |= { + 'req_chan': chan, + } + + # don't start entire actor runtime cancellation + # if this actor is currently in debug mode! + pdb_complete: trio.Event|None = _debug.Lock.local_pdb_complete + if pdb_complete: + await pdb_complete.wait() + + # Either of `Actor.cancel()`/`.cancel_soon()` + # was called, so terminate this IPC msg + # loop, exit back out into `async_main()`, + # and immediately start the core runtime + # machinery shutdown! + with CancelScope(shield=True): + await _invoke( + actor, + cid, + chan, + func, + kwargs, + is_rpc=False, + ) + + log.runtime( + 'Cancelling IPC transport msg-loop with peer:\n' + f'|_{chan}\n' + ) + loop_cs.cancel() + break + + if funcname == '_cancel_task': + func: Callable = actor._cancel_task + + # we immediately start the runtime machinery + # shutdown + # with CancelScope(shield=True): + target_cid: str = kwargs['cid'] + kwargs |= { + # NOTE: ONLY the rpc-task-owning + # parent IPC channel should be able to + # cancel it! + 'parent_chan': chan, + 'requesting_uid': chan.uid, + 'ipc_msg': msg, + } + # TODO: remove? already have emit in meth. + # log.runtime( + # f'Rx RPC task cancel request\n' + # f'<= canceller: {chan.uid}\n' + # f' |_{chan}\n\n' + # f'=> {actor}\n' + # f' |_cid: {target_cid}\n' + # ) + try: + await _invoke( + actor, + cid, + chan, + func, + kwargs, + is_rpc=False, + ) + except BaseException: + log.exception( + 'Failed to cancel task?\n' + f'<= canceller: {chan.uid}\n' + f' |_{chan}\n\n' + f'=> {actor}\n' + f' |_cid: {target_cid}\n' + ) + continue + else: + # normally registry methods, eg. + # ``.register_actor()`` etc. + func: Callable = getattr(actor, funcname) + + else: + # complain to client about restricted modules + try: + func = actor._get_rpc_func(ns, funcname) + except ( + ModuleNotExposed, + AttributeError, + ) as err: + err_msg: dict[str, dict] = pack_error( + err, + cid=cid, + ) + await chan.send(err_msg) + continue + + # schedule a task for the requested RPC function + # in the actor's main "service nursery". + # TODO: possibly a service-tn per IPC channel for + # supervision isolation? would avoid having to + # manage RPC tasks individually in `._rpc_tasks` + # table? + log.runtime( + f'Spawning task for RPC request\n' + f'<= caller: {chan.uid}\n' + f' |_{chan}\n\n' + # TODO: maddr style repr? + # f' |_@ /ipv4/{chan.raddr}/tcp/{chan.rport}/' + # f'cid="{cid[-16:]} .."\n\n' + + f'=> {actor}\n' + f' |_cid: {cid}\n' + f' |>> {func}()\n' + ) + assert actor._service_n # wait why? do it at top? + try: + ctx: Context = await actor._service_n.start( + partial( + _invoke, + actor, + cid, + chan, + func, + kwargs, + ), + name=funcname, + ) + + except ( + RuntimeError, + BaseExceptionGroup, + ): + # avoid reporting a benign race condition + # during actor runtime teardown. + nursery_cancelled_before_task: bool = True + break + + # in the lone case where a ``Context`` is not + # delivered, it's likely going to be a locally + # scoped exception from ``_invoke()`` itself. + if isinstance(err := ctx, Exception): + log.warning( + 'Task for RPC failed?' + f'|_ {func}()\n\n' + + f'{err}' + ) + continue + + else: + # mark that we have ongoing rpc tasks + actor._ongoing_rpc_tasks = trio.Event() + + # store cancel scope such that the rpc task can be + # cancelled gracefully if requested + actor._rpc_tasks[(chan, cid)] = ( + ctx, + func, + trio.Event(), + ) + + log.runtime( + 'Waiting on next IPC msg from\n' + f'peer: {chan.uid}\n' + f'|_{chan}\n' + ) + + # end of async for, channel disconnect vis + # ``trio.EndOfChannel`` + log.runtime( + f"{chan} for {chan.uid} disconnected, cancelling tasks" + ) + await actor.cancel_rpc_tasks( + req_uid=actor.uid, + # a "self cancel" in terms of the lifetime of the + # IPC connection which is presumed to be the + # source of any requests for spawned tasks. + parent_chan=chan, + ) + + 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. + # TODO: don't show this msg if it's an emphemeral + # discovery ep call? + log.runtime( + f'channel closed abruptly with\n' + f'peer: {chan.uid}\n' + f'|_{chan.raddr}\n' + ) + + # transport **was** disconnected + return True + + except ( + Exception, + BaseExceptionGroup, + ) as err: + + if nursery_cancelled_before_task: + sn: Nursery = actor._service_n + assert sn and sn.cancel_scope.cancel_called # sanity + log.cancel( + f'Service nursery cancelled before it handled {funcname}' + ) + else: + # ship any "internal" exception (i.e. one from internal + # machinery not from an rpc task) to parent + match err: + case ContextCancelled(): + log.cancel( + f'Actor: {actor.uid} was context-cancelled with,\n' + f'str(err)' + ) + case _: + log.exception("Actor errored:") + + if actor._parent_chan: + await try_ship_error_to_remote( + actor._parent_chan, + err, + ) + + # if this is the `MainProcess` we expect the error broadcasting + # above to trigger an error at consuming portal "checkpoints" + raise + + finally: + # msg debugging for when he machinery is brokey + log.runtime( + 'Exiting IPC msg loop with\n' + f'peer: {chan.uid}\n' + f'|_{chan}\n\n' + 'final msg:\n' + f'{pformat(msg)}\n' + ) + + # transport **was not** disconnected + return False diff --git a/tractor/_runtime.py b/tractor/_runtime.py index 707b9dd6..cc8eaf5f 100644 --- a/tractor/_runtime.py +++ b/tractor/_runtime.py @@ -14,52 +14,78 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . -""" -Actor primitives and helpers +''' +The fundamental core machinery implementing every "actor" +including the process-local, or "python-interpreter (aka global) +singleton) `Actor` primitive(s) and its internal `trio` machinery +implementing the low level runtime system supporting the +discovery, communication, spawning, supervision and cancellation +of other actors in a hierarchincal process tree. -""" +The runtime's main entry point: `async_main()` opens the top level +supervision and service `trio.Nursery`s which manage the tasks responsible +for running all lower level spawning, supervision and msging layers: + +- lowlevel transport-protocol init and persistent connectivity on + top of `._ipc` primitives; the transport layer. +- bootstrapping of connection/runtime config from the spawning + parent (actor). +- starting and supervising IPC-channel msg processing loops around + tranport connections from parent/peer actors in order to deliver + SC-transitive RPC via scheduling of `trio` tasks. +- registration of newly spawned actors with the discovery sys. + +''' from __future__ import annotations +from contextlib import ( + ExitStack, +) from collections import defaultdict from functools import partial from itertools import chain import importlib import importlib.util -import inspect +from pprint import pformat import signal import sys from typing import ( - Any, Optional, - Union, TYPE_CHECKING, + Any, Callable, + TYPE_CHECKING, ) import uuid from types import ModuleType import os -from contextlib import ExitStack -import warnings -from async_generator import aclosing -from exceptiongroup import BaseExceptionGroup -import trio # type: ignore -from trio_typing import TaskStatus +import trio +from trio import ( + CancelScope, + Nursery, + TaskStatus, +) +from .msg import NamespacePath from ._ipc import Channel -from ._streaming import Context +from ._context import ( + mk_context, + Context, +) from .log import get_logger from ._exceptions import ( - pack_error, unpack_error, ModuleNotExposed, - is_multi_cancelled, ContextCancelled, TransportClosed, - StreamOverrun, ) -from . import _debug from ._discovery import get_arbiter +from . import _debug from ._portal import Portal from . import _state from . import _mp_fixup_main +from ._rpc import ( + process_messages, + try_ship_error_to_remote, +) if TYPE_CHECKING: @@ -69,307 +95,10 @@ if TYPE_CHECKING: log = get_logger('tractor') -async def _invoke( - - actor: 'Actor', - cid: str, - chan: Channel, - func: Callable, - kwargs: dict[str, Any], - - is_rpc: bool = True, - task_status: TaskStatus[ - Union[trio.CancelScope, BaseException] - ] = trio.TASK_STATUS_IGNORED, -): - ''' - Invoke local func and deliver result(s) over provided channel. - - This is the core "RPC task" starting machinery. - - ''' - __tracebackhide__ = True - treat_as_gen: bool = False - failed_resp: bool = False - - # possibly a traceback (not sure what typing is for this..) - tb = None - - cancel_scope = trio.CancelScope() - # activated cancel scope ref - cs: Optional[trio.CancelScope] = None - - ctx = actor.get_context(chan, cid) - context: bool = False - - if getattr(func, '_tractor_stream_function', False): - # handle decorated ``@tractor.stream`` async functions - sig = inspect.signature(func) - params = sig.parameters - - # compat with old api - kwargs['ctx'] = ctx - - if 'ctx' in params: - warnings.warn( - "`@tractor.stream decorated funcs should now declare " - "a `stream` arg, `ctx` is now designated for use with " - "@tractor.context", - DeprecationWarning, - stacklevel=2, - ) - - elif 'stream' in params: - assert 'stream' in params - kwargs['stream'] = ctx - - treat_as_gen = True - - elif getattr(func, '_tractor_context_function', False): - # handle decorated ``@tractor.context`` async function - kwargs['ctx'] = ctx - context = True - - # errors raised inside this block are propgated back to caller - try: - if not ( - inspect.isasyncgenfunction(func) or - inspect.iscoroutinefunction(func) - ): - raise TypeError(f'{func} must be an async function!') - - coro = func(**kwargs) - - if inspect.isasyncgen(coro): - await chan.send({'functype': 'asyncgen', 'cid': cid}) - # XXX: massive gotcha! If the containing scope - # is cancelled and we execute the below line, - # any ``ActorNursery.__aexit__()`` WON'T be - # triggered in the underlying async gen! So we - # have to properly handle the closing (aclosing) - # of the async gen in order to be sure the cancel - # is propagated! - with cancel_scope as cs: - task_status.started(cs) - async with aclosing(coro) as agen: - async for item in agen: - # TODO: can we send values back in here? - # it's gonna require a `while True:` and - # some non-blocking way to retrieve new `asend()` - # values from the channel: - # to_send = await chan.recv_nowait() - # if to_send is not None: - # to_yield = await coro.asend(to_send) - await chan.send({'yield': item, 'cid': cid}) - - log.runtime(f"Finished iterating {coro}") - # TODO: we should really support a proper - # `StopAsyncIteration` system here for returning a final - # value if desired - await chan.send({'stop': True, 'cid': cid}) - - # one way @stream func that gets treated like an async gen - elif treat_as_gen: - await chan.send({'functype': 'asyncgen', 'cid': cid}) - # XXX: the async-func may spawn further tasks which push - # back values like an async-generator would but must - # manualy construct the response dict-packet-responses as - # above - with cancel_scope as cs: - task_status.started(cs) - await coro - - if not cs.cancelled_caught: - # task was not cancelled so we can instruct the - # far end async gen to tear down - await chan.send({'stop': True, 'cid': cid}) - - elif context: - # context func with support for bi-dir streaming - await chan.send({'functype': 'context', 'cid': cid}) - - try: - async with trio.open_nursery() as scope_nursery: - ctx._scope_nursery = scope_nursery - cs = scope_nursery.cancel_scope - task_status.started(cs) - res = await coro - await chan.send({'return': res, 'cid': cid}) - - except BaseExceptionGroup: - # if a context error was set then likely - # thei multierror was raised due to that - if ctx._error is not None: - raise ctx._error from None - - raise - - finally: - # XXX: only pop the context tracking if - # a ``@tractor.context`` entrypoint was called - assert chan.uid - - # don't pop the local context until we know the - # associated child isn't in debug any more - await _debug.maybe_wait_for_debugger() - ctx = actor._contexts.pop((chan.uid, cid)) - - if ctx: - log.runtime( - f'Context entrypoint {func} was terminated:\n{ctx}' - ) - - assert cs - if cs.cancelled_caught: - - # TODO: pack in ``trio.Cancelled.__traceback__`` here - # so they can be unwrapped and displayed on the caller - # side! - - fname = func.__name__ - if ctx._cancel_called: - msg = f'`{fname}()` cancelled itself' - - elif cs.cancel_called: - msg = ( - f'`{fname}()` was remotely cancelled by its caller ' - f'{ctx.chan.uid}' - ) - - if ctx._cancel_msg: - msg += f' with msg:\n{ctx._cancel_msg}' - - # task-contex was cancelled so relay to the cancel to caller - raise ContextCancelled( - msg, - suberror_type=trio.Cancelled, - ) - - else: - # regular async function - try: - await chan.send({'functype': 'asyncfunc', 'cid': cid}) - except trio.BrokenResourceError: - failed_resp = True - if is_rpc: - raise - else: - log.warning( - f'Failed to respond to non-rpc request: {func}' - ) - - with cancel_scope as cs: - task_status.started(cs) - result = await coro - log.cancel(f'result: {result}') - if not failed_resp: - # only send result if we know IPC isn't down - await chan.send({'return': result, 'cid': cid}) - - except ( - Exception, - BaseExceptionGroup, - ) as err: - - if not is_multi_cancelled(err): - - # TODO: maybe we'll want different "levels" of debugging - # eventualy such as ('app', 'supervisory', 'runtime') ? - - # if not isinstance(err, trio.ClosedResourceError) and ( - # if not is_multi_cancelled(err) and ( - - entered_debug: bool = False - if ( - not isinstance(err, ContextCancelled) - or ( - isinstance(err, ContextCancelled) - and ctx._cancel_called - - # if the root blocks the debugger lock request from a child - # we will get a remote-cancelled condition. - and ctx._enter_debugger_on_cancel - ) - ): - # 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:") - - # always ship errors back to caller - err_msg = pack_error(err, tb=tb) - err_msg['cid'] = cid - try: - await chan.send(err_msg) - - # TODO: tests for this scenario: - # - RPC caller closes connection before getting a response - # should **not** crash this actor.. - except ( - trio.ClosedResourceError, - trio.BrokenResourceError, - BrokenPipeError, - ): - # if we can't propagate the error that's a big boo boo - log.exception( - f"Failed to ship error to caller @ {chan.uid} !?" - ) - - if cs is None: - # error is from above code not from rpc invocation - task_status.started(err) - - finally: - # RPC task bookeeping - try: - scope, func, is_complete = actor._rpc_tasks.pop((chan, cid)) - is_complete.set() - - except KeyError: - if is_rpc: - # If we're cancelled before the task returns then the - # cancel scope will not have been inserted yet - log.warning( - f"Task {func} likely errored or cancelled before start") - finally: - if not actor._rpc_tasks: - log.runtime("All RPC tasks have completed") - actor._ongoing_rpc_tasks.set() - - def _get_mod_abspath(module): return os.path.abspath(module.__file__) -async def try_ship_error_to_parent( - channel: Channel, - err: Union[Exception, BaseExceptionGroup], - -) -> None: - with trio.CancelScope(shield=True): - try: - # internal error so ship to parent without cid - await channel.send(pack_error(err)) - except ( - trio.ClosedResourceError, - trio.BrokenResourceError, - ): - # in SC terms this is one of the worst things that can - # happen and creates the 2-general's dilemma. - log.critical( - f"Failed to ship error to parent " - f"{channel.uid}, channel was closed" - ) - - class Actor: ''' The fundamental "runtime" concurrency primitive. @@ -381,17 +110,18 @@ class Actor: (swappable) network protocols. - Each "actor" is ``trio.run()`` scheduled "runtime" composed of many - concurrent tasks in a single thread. The "runtime" tasks conduct - a slew of low(er) level functions to make it possible for message - passing between actors as well as the ability to create new actors - (aka new "runtimes" in new processes which are supervised via - a nursery construct). Each task which sends messages to a task in - a "peer" (not necessarily a parent-child, depth hierarchy)) is able - to do so via an "address", which maps IPC connections across memory - boundaries, and task request id which allows for per-actor - tasks to send and receive messages to specific peer-actor tasks with - which there is an ongoing RPC/IPC dialog. + Each "actor" is ``trio.run()`` scheduled "runtime" composed of + many concurrent tasks in a single thread. The "runtime" tasks + conduct a slew of low(er) level functions to make it possible + for message passing between actors as well as the ability to + create new actors (aka new "runtimes" in new processes which + are supervised via a nursery construct). Each task which sends + messages to a task in a "peer" (not necessarily a parent-child, + depth hierarchy) is able to do so via an "address", which maps + IPC connections across memory boundaries, and a task request id + which allows for per-actor tasks to send and receive messages + to specific peer-actor tasks with which there is an ongoing + RPC/IPC dialog. ''' # ugh, we need to get rid of this and replace with a "registry" sys @@ -400,20 +130,29 @@ class Actor: msg_buffer_size: int = 2**6 # nursery placeholders filled in by `async_main()` after fork - _root_n: Optional[trio.Nursery] = None - _service_n: Optional[trio.Nursery] = None - _server_n: Optional[trio.Nursery] = None + _root_n: Nursery | None = None + _service_n: Nursery | None = None + _server_n: Nursery | None = None # Information about `__main__` from parent _parent_main_data: dict[str, str] - _parent_chan_cs: Optional[trio.CancelScope] = None + _parent_chan_cs: CancelScope | None = None # syncs for setup/teardown sequences - _server_down: Optional[trio.Event] = None + _server_down: trio.Event | None = None + + # user toggled crash handling (including monkey-patched in + # `trio.open_nursery()` via `.trionics._supervisor` B) + _debug_mode: bool = False # if started on ``asycio`` running ``trio`` in guest mode _infected_aio: bool = False + # _ans: dict[ + # tuple[str, str], + # list[ActorNursery], + # ] = {} + # Process-global stack closed at end on actor runtime teardown. # NOTE: this is currently an undocumented public api. lifetime_stack: ExitStack = ExitStack() @@ -425,8 +164,8 @@ class Actor: enable_modules: list[str] = [], uid: str | None = None, loglevel: str | None = None, - arbiter_addr: Optional[tuple[str, int]] = None, - spawn_method: Optional[str] = None + arbiter_addr: tuple[str, int] | None = None, + spawn_method: str | None = None ) -> None: ''' This constructor is called in the parent actor **before** the spawning @@ -434,9 +173,13 @@ class Actor: ''' self.name = name - self.uid = (name, uid or str(uuid.uuid4())) + self.uid = ( + name, + uid or str(uuid.uuid4()) + ) self._cancel_complete = trio.Event() + self._cancel_called_by_remote: tuple[str, tuple] | None = None self._cancel_called: bool = False # retreive and store parent `__main__` data which @@ -475,19 +218,22 @@ class Actor: # (chan, cid) -> (cancel_scope, func) self._rpc_tasks: dict[ tuple[Channel, str], - tuple[trio.CancelScope, Callable, trio.Event] + tuple[Context, Callable, trio.Event] ] = {} # map {actor uids -> Context} self._contexts: dict[ - tuple[tuple[str, str], str], + tuple[ + tuple[str, str], # .uid + str, # .cid + str, # .side + ], Context ] = {} self._listeners: list[trio.abc.Listener] = [] - self._parent_chan: Optional[Channel] = None - self._forkserver_info: Optional[ - tuple[Any, Any, Any, Any, Any]] = None + self._parent_chan: Channel | None = None + self._forkserver_info: tuple | None = None self._actoruid2nursery: dict[ tuple[str, str], ActorNursery | None, @@ -497,8 +243,8 @@ class Actor: self, uid: tuple[str, str] ) -> tuple[trio.Event, Channel]: ''' - Wait for a connection back from a spawned actor with a given - ``uid``. + Wait for a connection back from a spawned actor with a `uid` + using a `trio.Event` for sync. ''' log.runtime(f"Waiting for peer {uid} to connect") @@ -507,13 +253,16 @@ class Actor: log.runtime(f"{uid} successfully connected back to us") return event, self._peers[uid][-1] - def load_modules(self) -> None: + def load_modules( + self, + debug_mode: bool = False, + ) -> None: ''' - Load allowed RPC modules locally (after fork). + Load enabled RPC py-modules locally (after process fork/spawn). Since this actor may be spawned on a different machine from the original nursery we need to try and load the local module - code (if it exists). + code (presuming it exists). ''' try: @@ -539,10 +288,17 @@ class Actor: except ModuleNotFoundError: # it is expected the corresponding `ModuleNotExposed` error # will be raised later - log.error(f"Failed to import {modpath} in {self.name}") + log.error( + f"Failed to import {modpath} in {self.name}" + ) raise def _get_rpc_func(self, ns, funcname): + ''' + Try to lookup and return a target RPC func from the + post-fork enabled module set. + + ''' try: return getattr(self._mods[ns], funcname) except KeyError as err: @@ -572,19 +328,34 @@ class Actor: stream: trio.SocketStream, ) -> None: - """Entry point for new inbound connections to the channel server. - - """ - self._no_more_peers = trio.Event() # unset + ''' + Entry point for new inbound IPC connections on a specific + transport server. + ''' + self._no_more_peers = trio.Event() # unset by making new chan = Channel.from_stream(stream) - uid: Optional[tuple[str, str]] = chan.uid - log.runtime(f"New connection to us {chan}") + their_uid: tuple[str, str]|None = chan.uid + con_msg: str = '' + if their_uid: + # NOTE: `.uid` is only set after first contact + con_msg = ( + 'IPC Re-connection from already known peer? ' + ) + else: + con_msg = ( + 'New IPC connection to us ' + ) + + con_msg += ( + f'<= @{chan.raddr}\n' + f'|_{chan}\n' + # f' |_@{chan.raddr}\n\n' + ) # send/receive initial handshake response try: - uid = await self._do_handshake(chan) - + uid: tuple|None = await self._do_handshake(chan) except ( # we need this for ``msgspec`` for some reason? # for now, it's been put in the stream backend. @@ -599,59 +370,84 @@ class Actor: # 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( + con_msg + + + ' -> But failed to handshake? Ignoring..\n' + ) return - # channel tracking - event = self._peer_connected.pop(uid, None) + con_msg += ( + f' -> Handshake with actor `{uid[0]}[{uid[1][-6:]}]` complete\n' + ) + # IPC connection tracking for both peers and new children: + # - if this is a new channel to a locally spawned + # sub-actor there will be a spawn wait even registered + # by a call to `.wait_for_peer()`. + # - if a peer is connecting no such event will exit. + event: trio.Event|None = self._peer_connected.pop( + uid, + None, + ) if event: - # Instructing connection: this is likely a new channel to - # a recently spawned actor which we'd like to control via - # async-rpc calls. - log.runtime(f"Waking channel waiters {event.statistics()}") - # Alert any task waiting on this connection to come up + con_msg += ( + ' -> Waking subactor spawn waiters: ' + f'{event.statistics().tasks_waiting}\n' + f' -> Registered IPC chan for child actor {uid}@{chan.raddr}\n' + # f' {event}\n' + # f' |{event.statistics()}\n' + ) + # wake tasks waiting on this IPC-transport "connect-back" event.set() - chans = self._peers[uid] + else: + con_msg += ( + f' -> Registered IPC chan for peer actor {uid}@{chan.raddr}\n' + ) # type: ignore - # TODO: re-use channels for new connections instead - # of always new ones; will require changing all the - # discovery funcs - if chans: - log.runtime( - f"already have channel(s) for {uid}:{chans}?" - ) + chans: list[Channel] = self._peers[uid] + # if chans: + # # TODO: re-use channels for new connections instead + # # of always new ones? + # # => will require changing all the discovery funcs.. - log.runtime(f"Registered {chan} for {uid}") # type: ignore # append new channel - self._peers[uid].append(chan) + # TODO: can we just use list-ref directly? + chans.append(chan) - local_nursery: Optional[ActorNursery] = None # noqa - disconnected: bool = False + log.runtime(con_msg) # Begin channel management - respond to remote requests and # process received reponses. + disconnected: bool = False try: - disconnected = await process_messages(self, chan) - - except ( - trio.Cancelled, - ): - log.cancel(f"Msg loop was cancelled for {chan}") + disconnected: bool = await process_messages( + self, + chan, + ) + except trio.Cancelled: + log.cancel( + 'IPC transport msg loop was cancelled for \n' + f'|_{chan}\n' + ) raise finally: - local_nursery = self._actoruid2nursery.get(uid, local_nursery) + local_nursery: ( + ActorNursery|None + ) = self._actoruid2nursery.get(uid) # This is set in ``Portal.cancel_actor()``. So if # the peer was cancelled we try to wait for them # to tear down their side of the connection before # moving on with closing our own side. - if ( - local_nursery - ): + if local_nursery: + if chan._cancel_called: + log.cancel( + 'Waiting on cancel request to peer\n' + f'`Portal.cancel_actor()` => {chan.uid}\n' + ) - log.cancel(f"Waiting on cancel request to peer {chan.uid}") # XXX: this is a soft wait on the channel (and its # underlying transport protocol) to close from the # remote peer side since we presume that any channel @@ -663,15 +459,13 @@ class Actor: # loop processing. with trio.move_on_after(0.5) as cs: cs.shield = True - # Attempt to wait for the far end to close the channel - # and bail after timeout (2-generals on closure). - assert chan.msgstream - log.runtime( - f'Draining lingering msgs from stream {chan.msgstream}' - ) + # attempt to wait for the far end to close the + # channel and bail after timeout (a 2-generals + # problem on closure). + assert chan.transport + async for msg in chan.transport.drain(): - async for msg in chan.msgstream.drain(): # try to deliver any lingering msgs # before we destroy the channel. # This accomplishes deterministic @@ -679,13 +473,47 @@ class Actor: # making sure any RPC response to that call is # delivered the local calling task. # TODO: factor this into a helper? - log.runtime(f'drained {msg} for {chan.uid}') + log.warning( + 'Draining msg from disconnected peer\n' + f'{chan.uid}\n' + f'|_{chan}\n' + f' |_{chan.transport}\n\n' + + f'{pformat(msg)}\n' + ) cid = msg.get('cid') if cid: # deliver response to local caller/waiter - await self._push_result(chan, cid, msg) + await self._push_result( + chan, + cid, + msg, + ) - await local_nursery.exited.wait() + # NOTE: when no call to `open_root_actor()` was + # made, we implicitly make that call inside + # the first `.open_nursery()`, in this case we + # can assume that we are the root actor and do + # not have to wait for the nursery-enterer to + # exit before shutting down the actor runtime. + # + # see matching note inside `._supervise.open_nursery()` + if not local_nursery._implicit_runtime_started: + log.runtime( + 'Waiting on local actor nursery to exit..\n' + f'|_{local_nursery}\n' + ) + await local_nursery.exited.wait() + + if ( + cs.cancelled_caught + and not local_nursery._implicit_runtime_started + ): + log.warning( + 'Failed to exit local actor nursery?\n' + f'|_{local_nursery}\n' + ) + # await _debug.pause() if disconnected: # if the transport died and this actor is still @@ -694,26 +522,48 @@ class Actor: # the cause of other downstream errors. entry = local_nursery._children.get(uid) if entry: + proc: trio.Process _, proc, _ = entry poll = getattr(proc, 'poll', None) if poll and poll() is None: log.cancel( - f'Actor {uid} IPC broke but proc is alive?' + f'Peer IPC broke but subproc is alive?\n\n' + + f'<=x {chan.uid}@{chan.raddr}\n' + f' |_{proc}\n' ) # ``Channel`` teardown and closure sequence - - # Drop ref to channel so it can be gc-ed and disconnected - log.runtime(f"Releasing channel {chan} from {chan.uid}") - chans = self._peers.get(chan.uid) + # drop ref to channel so it can be gc-ed and disconnected + log.runtime( + f'Disconnected IPC channel:\n' + f'uid: {chan.uid}\n' + f'|_{pformat(chan)}\n' + ) chans.remove(chan) + # TODO: do we need to be this pedantic? if not chans: - log.runtime(f"No more channels for {chan.uid}") + log.runtime( + f'No more channels with {chan.uid}' + ) self._peers.pop(uid, None) - log.runtime(f"Peers is {self._peers}") + peers_str: str = '' + for uid, chans in self._peers.items(): + peers_str += ( + f'|_ uid: {uid}\n' + ) + for i, chan in enumerate(chans): + peers_str += ( + f' |_[{i}] {pformat(chan)}\n' + ) + + log.runtime( + f'Remaining IPC {len(self._peers)} peers:\n' + + peers_str + ) # No more channels to other actors (at all) registered # as connected. @@ -729,15 +579,58 @@ class Actor: if _state.is_root_process(): pdb_lock = _debug.Lock pdb_lock._blocked.add(uid) - log.runtime(f"{uid} blocked from pdb locking") + # TODO: NEEEDS TO BE TESTED! + # actually, no idea if this ever even enters.. XD + pdb_user_uid: tuple = pdb_lock.global_actor_in_debug + if ( + pdb_user_uid + and local_nursery + ): + entry: tuple|None = local_nursery._children.get(pdb_user_uid) + if entry: + proc: trio.Process + _, proc, _ = entry + + if ( + (poll := getattr(proc, 'poll', None)) + and poll() is None + ): + log.cancel( + 'Root actor reports no-more-peers, BUT ' + 'a DISCONNECTED child still has the debug ' + 'lock!\n' + f'root uid: {self.uid}\n' + f'last disconnected child uid: {uid}\n' + f'locking child uid: {pdb_user_uid}\n' + ) + await _debug.maybe_wait_for_debugger( + child_in_debug=True + ) + + # TODO: just bc a child's transport dropped + # doesn't mean it's not still using the pdb + # REPL! so, + # -[ ] ideally we can check out child proc + # tree to ensure that its alive (and + # actually using the REPL) before we cancel + # it's lock acquire by doing the below! + # -[ ] create a way to read the tree of each actor's + # grandchildren such that when an + # intermediary parent is cancelled but their + # child has locked the tty, the grandparent + # will not allow the parent to cancel or + # zombie reap the child! see open issue: + # - https://github.com/goodboy/tractor/issues/320 + # ------ - ------ # if a now stale local task has the TTY lock still # we cancel it to allow servicing other requests for # the lock. - db_cs = pdb_lock._root_local_task_cs_in_debug + db_cs: trio.CancelScope|None = pdb_lock._root_local_task_cs_in_debug if ( db_cs and not db_cs.cancel_called + and uid == pdb_user_uid ): log.warning( f'STALE DEBUG LOCK DETECTED FOR {uid}' @@ -746,128 +639,118 @@ class Actor: db_cs.cancel() # XXX: is this necessary (GC should do it)? - 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. + # XXX WARNING XXX + # Be AWARE OF THE INDENT LEVEL HERE + # -> ONLY ENTER THIS BLOCK WHEN ._peers IS + # EMPTY!!!! + if ( + not self._peers + and 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.runtime( + 'Terminating channel with `None` setinel msg\n' + f'|_{chan}\n' + ) + try: + # send msg loop terminate sentinel which + # triggers cancellation of all remotely + # started tasks. + await chan.send(None) - log.runtime(f"Disconnecting channel {chan}") - try: - # send a msg loop terminate sentinel - await chan.send(None) + # XXX: do we want this? no right? + # causes "[104] connection reset by peer" on other end + # await chan.aclose() - # XXX: do we want this? - # causes "[104] connection reset by peer" on other end - # await chan.aclose() - - except trio.BrokenResourceError: - log.runtime(f"Channel {chan.uid} was already closed") + except trio.BrokenResourceError: + log.runtime(f"Channel {chan.uid} was already closed") + # TODO: rename to `._deliver_payload()` since this handles + # more then just `result` msgs now obvi XD async def _push_result( self, chan: Channel, cid: str, msg: dict[str, Any], - ) -> None: + + ) -> None|bool: ''' - Push an RPC result to the local consumer's queue. + Push an RPC msg-payload to the local consumer peer-task's + queue. ''' - uid = chan.uid + uid: tuple[str, str] = chan.uid assert uid, f"`chan.uid` can't be {uid}" try: - ctx = self._contexts[(uid, cid)] + ctx: Context = self._contexts[( + uid, + cid, + + # TODO: how to determine this tho? + # side, + )] except KeyError: log.warning( - f'Ignoring msg from [no-longer/un]known context {uid}:' - f'\n{msg}') + 'Ignoring invalid IPC ctx msg!\n\n' + f'<= sender: {uid}\n' + f'=> cid: {cid}\n\n' + + f'{msg}\n' + ) return - send_chan = ctx._send_chan - - log.runtime(f"Delivering {msg} from {chan.uid} to caller {cid}") - - # XXX: we do **not** maintain backpressure and instead - # opt to relay stream overrun errors to the sender. - try: - send_chan.send_nowait(msg) - # if an error is deteced we should always - # expect it to be raised by any context (stream) - # consumer task - await ctx._maybe_raise_from_remote_msg(msg) - - except trio.BrokenResourceError: - # TODO: what is the right way to handle the case where the - # local task has already sent a 'stop' / StopAsyncInteration - # to the other side but and possibly has closed the local - # feeder mem chan? Do we wait for some kind of ack or just - # let this fail silently and bubble up (currently)? - - # XXX: local consumer has closed their side - # so cancel the far end streaming task - log.warning(f"{send_chan} consumer is already closed") - return - - except trio.WouldBlock: - # XXX: always push an error even if the local - # receiver is in overrun state. - await ctx._maybe_raise_from_remote_msg(msg) - - uid = chan.uid - lines = [ - 'Task context stream was overrun', - f'local task: {cid} @ {self.uid}', - f'remote sender: {uid}', - ] - if not ctx._stream_opened: - lines.insert( - 1, - f'\n*** No stream open on `{self.uid[0]}` side! ***\n' - ) - text = '\n'.join(lines) - - if ctx._backpressure: - log.warning(text) - try: - await send_chan.send(msg) - except trio.BrokenResourceError: - # XXX: local consumer has closed their side - # so cancel the far end streaming task - log.warning(f"{chan} is already closed") - else: - try: - raise StreamOverrun(text) from None - except StreamOverrun as err: - err_msg = pack_error(err) - err_msg['cid'] = cid - try: - await chan.send(err_msg) - except trio.BrokenResourceError: - # XXX: local consumer has closed their side - # so cancel the far end streaming task - log.warning(f"{chan} is already closed") + return await ctx._deliver_msg(msg) def get_context( self, chan: Channel, cid: str, - msg_buffer_size: Optional[int] = None, + nsf: NamespacePath, + + # TODO: support lookup by `Context.side: str` ? + # -> would allow making a self-context which might have + # certain special use cases where RPC isolation is wanted + # between 2 tasks running in the same process? + # => prolly needs some deeper though on the real use cases + # and whether or not such things should be better + # implemented using a `TaskManager` style nursery.. + # + # side: str|None = None, + + msg_buffer_size: int | None = None, + allow_overruns: bool = False, ) -> Context: ''' - Look up or create a new inter-actor-task-IPC-linked task - "context" which encapsulates the local task's scheduling - enviroment including a ``trio`` cancel scope, a pair of IPC - messaging "feeder" channels, and an RPC id unique to the - task-as-function invocation. + Look-up (existing) or create a new + inter-actor-SC-linked task "context" (a `Context`) which + encapsulates the local RPC task's execution enviroment + around `Channel` relayed msg handling including, + + - a dedicated `trio` cancel scope (`Context._scope`), + - a pair of IPC-msg-relay "feeder" mem-channels + (`Context._recv/send_chan`), + - and a "context id" (cid) unique to the task-pair + msging session's lifetime. ''' - log.runtime(f"Getting result queue for {chan.uid} cid {cid}") actor_uid = chan.uid assert actor_uid try: - ctx = self._contexts[(actor_uid, cid)] + ctx = self._contexts[( + actor_uid, + cid, + # side, + )] + log.runtime( + f'Retreived cached IPC ctx for\n' + f'peer: {chan.uid}\n' + f'cid:{cid}\n' + ) + ctx._allow_overruns = allow_overruns # adjust buffer size if specified state = ctx._send_chan._state # type: ignore @@ -875,53 +758,100 @@ class Actor: state.max_buffer_size = msg_buffer_size except KeyError: - send_chan: trio.MemorySendChannel - recv_chan: trio.MemoryReceiveChannel - send_chan, recv_chan = trio.open_memory_channel( - msg_buffer_size or self.msg_buffer_size) - ctx = Context( + log.runtime( + f'Creating NEW IPC ctx for\n' + f'peer: {chan.uid}\n' + f'cid: {cid}\n' + ) + ctx = mk_context( chan, cid, - _send_chan=send_chan, - _recv_chan=recv_chan, + nsf=nsf, + msg_buffer_size=msg_buffer_size or self.msg_buffer_size, + _allow_overruns=allow_overruns, ) - self._contexts[(actor_uid, cid)] = ctx + self._contexts[( + actor_uid, + cid, + # side, + )] = ctx return ctx async def start_remote_task( self, chan: Channel, - ns: str, - func: str, + nsf: NamespacePath, kwargs: dict, - msg_buffer_size: Optional[int] = None, + + # IPC channel config + msg_buffer_size: int | None = None, + allow_overruns: bool = False, + load_nsf: bool = False, ) -> Context: ''' - Send a ``'cmd'`` message to a remote actor, which starts - a remote task-as-function entrypoint. + Send a `'cmd'` msg to a remote actor, which requests the + start and schedule of a remote task-as-function's + entrypoint. - Synchronously validates the endpoint type and return a caller - side task ``Context`` that can be used to wait for responses - delivered by the local runtime's message processing loop. + Synchronously validates the endpoint type and returns + a (caller side) `Context` that can be used to accept + delivery of msg payloads from the local runtime's + processing loop: `._rpc.process_messages()`. ''' - cid = str(uuid.uuid4()) + cid: str = str(uuid.uuid4()) assert chan.uid - ctx = self.get_context(chan, cid, msg_buffer_size=msg_buffer_size) - log.runtime(f"Sending cmd to {chan.uid}: {ns}.{func}({kwargs})") - await chan.send({'cmd': (ns, func, kwargs, self.uid, cid)}) + ctx = self.get_context( + chan=chan, + cid=cid, + nsf=nsf, + + # side='caller', + msg_buffer_size=msg_buffer_size, + allow_overruns=allow_overruns, + ) + + if ( + 'self' in nsf + or not load_nsf + ): + ns, _, func = nsf.partition(':') + else: + # TODO: pass nsf directly over wire! + # -[ ] but, how to do `self:`?? + ns, func = nsf.to_tuple() + + log.runtime( + 'Sending cmd to\n' + f'peer: {chan.uid} => \n' + '\n' + f'=> {ns}.{func}({kwargs})\n' + ) + await chan.send( + {'cmd': ( + ns, + func, + kwargs, + self.uid, + cid, + )} + ) # Wait on first response msg and validate; this should be # immediate. - first_msg = await ctx._recv_chan.receive() - functype = first_msg.get('functype') + first_msg: dict = await ctx._recv_chan.receive() + functype: str = first_msg.get('functype') if 'error' in first_msg: raise unpack_error(first_msg, chan) - elif functype not in ('asyncfunc', 'asyncgen', 'context'): + elif functype not in ( + 'asyncfunc', + 'asyncgen', + 'context', + ): raise ValueError(f"{first_msg} is an invalid response packet?") ctx._remote_func_type = functype @@ -929,8 +859,18 @@ class Actor: async def _from_parent( self, - parent_addr: Optional[tuple[str, int]], - ) -> tuple[Channel, Optional[tuple[str, int]]]: + parent_addr: tuple[str, int] | None, + + ) -> tuple[ + Channel, + list[tuple[str, int]] | None, + ]: + ''' + Bootstrap this local actor's runtime config from its parent by + connecting back via the IPC transport, handshaking and then + `Channel.recv()`-ing seeded data. + + ''' try: # Connect back to the parent actor and conduct initial # handshake. From this point on if we error, we @@ -943,21 +883,36 @@ class Actor: # Initial handshake: swap names. await self._do_handshake(chan) - accept_addr: Optional[tuple[str, int]] = None + accept_addr: tuple[str, int] | None = None if self._spawn_method == "trio": # Receive runtime state from our parent parent_data: dict[str, Any] parent_data = await chan.recv() log.runtime( - "Received state from parent:\n" - f"{parent_data}" + 'Received state from parent:\n\n' + # TODO: eventually all these msgs as + # `msgspec.Struct` with a special mode that + # pformats them in multi-line mode, BUT only + # if "trace"/"util" mode is enabled? + f'{pformat(parent_data)}\n' ) accept_addr = ( parent_data.pop('bind_host'), parent_data.pop('bind_port'), ) rvs = parent_data.pop('_runtime_vars') + + if rvs['_debug_mode']: + try: + log.info('Enabling `stackscope` traces on SIGUSR1') + from .devx import enable_stack_on_sig + enable_stack_on_sig() + except ImportError: + log.warning( + '`stackscope` not installed for use in debug mode!' + ) + log.runtime(f"Runtime vars are: {rvs}") rvs['_is_root'] = False _state._runtime_vars.update(rvs) @@ -978,14 +933,17 @@ class Actor: except OSError: # failed to connect log.warning( - f"Failed to connect to parent @ {parent_addr}," - " closing server") - await self.cancel() + f'Failed to connect to parent!?\n\n' + 'Closing IPC [TCP] transport server to\n' + f'{parent_addr}\n' + f'|_{self}\n\n' + ) + await self.cancel(chan=None) # self cancel raise async def _serve_forever( self, - handler_nursery: trio.Nursery, + handler_nursery: Nursery, *, # (host, port) to bind for channel server accept_host: tuple[str, int] | None = None, @@ -993,16 +951,17 @@ class Actor: task_status: TaskStatus[trio.Nursery] = trio.TASK_STATUS_IGNORED, ) -> None: ''' - Start the channel server, begin listening for new connections. + Start the IPC transport server, begin listening for new connections. - This will cause an actor to continue living (blocking) until - ``cancel_server()`` is called. + This will cause an actor to continue living (and thus + blocking at the process/OS-thread level) until + `.cancel_server()` is called. ''' self._server_down = trio.Event() try: async with trio.open_nursery() as server_n: - l: list[trio.abc.Listener] = await server_n.start( + listeners: list[trio.abc.Listener] = await server_n.start( partial( trio.serve_tcp, self._stream_handler, @@ -1013,10 +972,15 @@ class Actor: host=accept_host, ) ) + sockets: list[trio.socket] = [ + getattr(listener, 'socket', 'unknown socket') + for listener in listeners + ] log.runtime( - "Started tcp server(s) on" - f" {[getattr(l, 'socket', 'unknown socket') for l in l]}") - self._listeners.extend(l) + 'Started TCP server(s)\n' + f'|_{sockets}\n' + ) + self._listeners.extend(listeners) task_status.started(server_n) finally: # signal the server is down since nursery above terminated @@ -1026,41 +990,89 @@ class Actor: ''' Cancel this actor asap; can be called from a sync context. - Schedules `.cancel()` to be run immediately just like when - cancelled by the parent. + Schedules runtime cancellation via `Actor.cancel()` inside + the RPC service nursery. ''' assert self._service_n - self._service_n.start_soon(self.cancel) + self._service_n.start_soon( + self.cancel, + None, # self cancel all rpc tasks + ) - async def cancel(self) -> bool: + async def cancel( + self, + + # chan whose lifetime limits the lifetime of its remotely + # requested and locally spawned RPC tasks - similar to the + # supervision semantics of a nursery wherein the actual + # implementation does start all such tasks in + # a sub-nursery. + req_chan: Channel|None, + + ) -> bool: ''' - Cancel this actor's runtime. + Cancel this actor's runtime, eventually resulting in + termination of its containing OS process. - The "deterministic" teardown sequence in order is: - - cancel all ongoing rpc tasks by cancel scope - - cancel the channel server to prevent new inbound - connections - - cancel the "service" nursery reponsible for - spawning new rpc tasks - - return control the parent channel message loop + The ideal "deterministic" teardown sequence in order is: + - cancel all ongoing rpc tasks by cancel scope. + - cancel the channel server to prevent new inbound + connections. + - cancel the "service" nursery reponsible for + spawning new rpc tasks. + - return control the parent channel message loop. ''' - log.cancel(f"{self.uid} is trying to cancel") + ( + requesting_uid, + requester_type, + req_chan, + log_meth, + + ) = ( + req_chan.uid, + 'peer', + req_chan, + log.cancel, + + ) if req_chan else ( + + # a self cancel of ALL rpc tasks + self.uid, + 'self', + self, + log.runtime, + ) + # TODO: just use the new `Context.repr_rpc: str` (and + # other) repr fields instead of doing this all manual.. + msg: str = ( + f'Runtime cancel request from {requester_type}:\n\n' + f'<= .cancel(): {requesting_uid}\n' + ) + + # TODO: what happens here when we self-cancel tho? + self._cancel_called_by_remote: tuple = requesting_uid self._cancel_called = True # cancel all ongoing rpc tasks - with trio.CancelScope(shield=True): + with CancelScope(shield=True): # kill any debugger request task to avoid deadlock # with the root actor in this tree dbcs = _debug.Lock._debugger_request_cs if dbcs is not None: - log.cancel("Cancelling active debugger request") + msg += ( + '>> Cancelling active debugger request..\n' + f'|_{_debug.Lock}\n' + ) dbcs.cancel() - # kill all ongoing tasks - await self.cancel_rpc_tasks() + # self-cancel **all** ongoing RPC tasks + await self.cancel_rpc_tasks( + req_uid=requesting_uid, + parent_chan=None, + ) # stop channel server self.cancel_server() @@ -1068,13 +1080,14 @@ class Actor: await self._server_down.wait() else: log.warning( - f'{self.uid} was likely cancelled before it started') + 'Transport[TCP] server was cancelled start?' + ) # cancel all rpc tasks permanently if self._service_n: self._service_n.cancel_scope.cancel() - log.cancel(f"{self.uid} called `Actor.cancel()`") + log_meth(msg) self._cancel_complete.set() return True @@ -1086,79 +1099,219 @@ class Actor: # for n in root.child_nurseries: # n.cancel_scope.cancel() - async def _cancel_task(self, cid, chan): - ''' - Cancel a local task by call-id / channel. + async def _cancel_task( + self, + cid: str, + parent_chan: Channel, + requesting_uid: tuple[str, str]|None, - Note this method will be treated as a streaming function - by remote actor-callers due to the declaration of ``ctx`` - in the signature (for now). + ipc_msg: dict|None|bool = False, + + ) -> bool: + ''' + Cancel a local (RPC) task by context-id/channel by calling + `trio.CancelScope.cancel()` on it's surrounding cancel + scope. ''' - # right now this is only implicitly called by - # streaming IPC but it should be called - # to cancel any remotely spawned task + + # this ctx based lookup ensures the requested task to be + # cancelled was indeed spawned by a request from its + # parent (or some grandparent's) channel + ctx: Context + func: Callable + is_complete: trio.Event try: - # this ctx based lookup ensures the requested task to - # be cancelled was indeed spawned by a request from this channel - scope, func, is_complete = self._rpc_tasks[(chan, cid)] + ( + ctx, + func, + is_complete, + ) = self._rpc_tasks[( + parent_chan, + cid, + )] + scope: CancelScope = ctx._scope + except KeyError: - log.cancel(f"{cid} has already completed/terminated?") - return + # NOTE: during msging race conditions this will often + # emit, some examples: + # - callee returns a result before cancel-msg/ctxc-raised + # - callee self raises ctxc before caller send request, + # - callee errors prior to cancel req. + log.cancel( + 'Cancel request invalid, RPC task already completed?\n\n' + f'<= canceller: {requesting_uid}\n\n' + f'=> {cid}@{parent_chan.uid}\n' + f' |_{parent_chan}\n' + ) + return True log.cancel( - f"Cancelling task:\ncid: {cid}\nfunc: {func}\n" - f"peer: {chan.uid}\n") + 'Cancel request for RPC task\n\n' + f'<= Actor._cancel_task(): {requesting_uid}\n\n' + f'=> {ctx._task}\n' + f' |_ >> {ctx.repr_rpc}\n' + # f' >> Actor._cancel_task() => {ctx._task}\n' + # f' |_ {ctx._task}\n\n' + + # TODO: better ascii repr for "supervisor" like + # a nursery or context scope? + # f'=> {parent_chan}\n' + # f' |_{ctx._task}\n' + # TODO: simplified `Context.__repr__()` fields output + # shows only application state-related stuff like, + # - ._stream + # - .closed + # - .started_called + # - .. etc. + # f' >> {ctx.repr_rpc}\n' + # f' |_ctx: {cid}\n' + # f' >> {ctx._nsf}()\n' + ) + if ( + ctx._canceller is None + and requesting_uid + ): + ctx._canceller: tuple = requesting_uid + + # TODO: pack the RPC `{'cmd': }` msg into a ctxc and + # then raise and pack it here? + if ( + ipc_msg + and ctx._cancel_msg is None + ): + # assign RPC msg directly from the loop which usually + # the case with `ctx.cancel()` on the other side. + ctx._cancel_msg = ipc_msg # don't allow cancelling this function mid-execution # (is this necessary?) if func is self._cancel_task: - return + log.error('Do not cancel a cancel!?') + return True + # TODO: shouldn't we eventually be calling ``Context.cancel()`` + # directly here instead (since that method can handle both + # side's calls into it? + # await ctx.cancel() scope.cancel() # wait for _invoke to mark the task complete + flow_info: str = ( + f'<= canceller: {requesting_uid}\n' + f'=> ipc-parent: {parent_chan}\n' + f' |_{ctx}\n' + ) log.runtime( - f"Waiting on task to cancel:\ncid: {cid}\nfunc: {func}\n" - f"peer: {chan.uid}\n") + 'Waiting on RPC task to cancel\n' + f'{flow_info}' + ) await is_complete.wait() - log.runtime( - f"Sucessfully cancelled task:\ncid: {cid}\nfunc: {func}\n" - f"peer: {chan.uid}\n") + f'Sucessfully cancelled RPC task\n' + f'{flow_info}' + ) + return True async def cancel_rpc_tasks( self, - only_chan: Optional[Channel] = None, + req_uid: tuple[str, str], + + # NOTE: when None is passed we cancel **all** rpc + # tasks running in this actor! + parent_chan: Channel|None, + ) -> None: ''' - Cancel all existing RPC responder tasks using the cancel scope - registered for each. + Cancel all ongoing RPC tasks owned/spawned for a given + `parent_chan: Channel` or simply all tasks (inside + `._service_n`) when `parent_chan=None`. ''' - tasks = self._rpc_tasks + tasks: dict = self._rpc_tasks + if not tasks: + log.runtime( + 'Actor has no cancellable RPC tasks?\n' + f'<= canceller: {req_uid}\n' + ) + return + + # TODO: seriously factor this into some helper funcs XD + tasks_str: str = '' + for (ctx, func, _) in tasks.values(): + + # TODO: std repr of all primitives in + # a hierarchical tree format, since we can!! + # like => repr for funcs/addrs/msg-typing: + # + # -[ ] use a proper utf8 "arm" like + # `stackscope` has! + # -[ ] for typed msging, show the + # py-type-annot style? + # - maybe auto-gen via `inspect` / `typing` type-sig: + # https://stackoverflow.com/a/57110117 + # => see ex. code pasted into `.msg.types` + # + # -[ ] proper .maddr() for IPC primitives? + # - `Channel.maddr() -> str:` obvi! + # - `Context.maddr() -> str:` + tasks_str += ( + f' |_@ /ipv4/tcp/cid="{ctx.cid[-16:]} .."\n' + f' |>> {ctx._nsf}() -> dict:\n' + ) + + descr: str = ( + 'all' if not parent_chan + else + "IPC channel's " + ) + rent_chan_repr: str = ( + f'|_{parent_chan}' + if parent_chan + else '' + ) + log.cancel( + f'Cancelling {descr} {len(tasks)} rpc tasks\n\n' + f'<= `Actor.cancel_rpc_tasks()`: {req_uid}\n' + f' {rent_chan_repr}\n' + # f'{self}\n' + # f'{tasks_str}' + ) + for ( + (task_caller_chan, cid), + (ctx, func, is_complete), + ) in tasks.copy().items(): + + if ( + # maybe filter to specific IPC channel? + (parent_chan + and + task_caller_chan != parent_chan) + + # never "cancel-a-cancel" XD + or (func == self._cancel_task) + ): + continue + + # TODO: this maybe block on the task cancellation + # and so should really done in a nursery batch? + await self._cancel_task( + cid, + task_caller_chan, + requesting_uid=req_uid, + ) + if tasks: - log.cancel(f"Cancelling all {len(tasks)} rpc tasks:\n{tasks} ") - for ( - (chan, cid), - (scope, func, is_complete), - ) in tasks.copy().items(): - if only_chan is not None: - if only_chan != chan: - continue - - # TODO: this should really done in a nursery batch - if func != self._cancel_task: - await self._cancel_task(cid, chan) - log.cancel( - f"Waiting for remaining rpc tasks to complete {tasks}") - await self._ongoing_rpc_tasks.wait() + 'Waiting for remaining rpc tasks to complete\n' + f'|_{tasks}' + ) + await self._ongoing_rpc_tasks.wait() def cancel_server(self) -> None: ''' - Cancel the internal channel server nursery thereby - preventing any new inbound connections from being established. + Cancel the internal IPC transport server nursery thereby + preventing any new inbound IPC connections establishing. ''' if self._server_n: @@ -1166,9 +1319,10 @@ class Actor: self._server_n.cancel_scope.cancel() @property - def accept_addr(self) -> Optional[tuple[str, int]]: + def accept_addr(self) -> tuple[str, int]: ''' - Primary address to which the channel server is bound. + Primary address to which the IPC transport server is + bound. ''' # throws OSError on failure @@ -1176,7 +1330,7 @@ class Actor: def get_parent(self) -> Portal: ''' - Return a portal to our parent actor. + Return a `Portal` to our parent. ''' assert self._parent_chan, "No parent channel for this actor?" @@ -1184,7 +1338,7 @@ class Actor: def get_chans(self, uid: tuple[str, str]) -> list[Channel]: ''' - Return all channels to the actor with provided uid. + Return all IPC channels to the actor with provided `uid`. ''' return self._peers[uid] @@ -1195,30 +1349,37 @@ class Actor: ) -> tuple[str, str]: ''' - Exchange (name, UUIDs) identifiers as the first communication step. + Exchange `(name, UUIDs)` identifiers as the first + communication step with any (peer) remote `Actor`. - These are essentially the "mailbox addresses" found in actor model - parlance. + These are essentially the "mailbox addresses" found in + "actor model" parlance. ''' await chan.send(self.uid) - value = await chan.recv() + value: tuple = await chan.recv() uid: tuple[str, str] = (str(value[0]), str(value[1])) if not isinstance(uid, tuple): raise ValueError(f"{uid} is not a valid uid?!") chan.uid = str(uid[0]), str(uid[1]) - log.runtime(f"Handshake with actor {uid}@{chan.raddr} complete") return uid def is_infected_aio(self) -> bool: + ''' + If `True`, this actor is running `trio` in guest mode on + the `asyncio` event loop and thus can use the APIs in + `.to_asyncio` to coordinate tasks running in each + framework but within the same actor runtime. + + ''' return self._infected_aio async def async_main( actor: Actor, - accept_addr: Optional[tuple[str, int]] = None, + accept_addr: tuple[str, int] | None = None, # XXX: currently ``parent_addr`` is only needed for the # ``multiprocessing`` backend (which pickles state sent to @@ -1227,16 +1388,19 @@ async def async_main( # change this to a simple ``is_subactor: bool`` which will # be False when running as root actor and True when as # a subactor. - parent_addr: Optional[tuple[str, int]] = None, + parent_addr: tuple[str, int] | None = None, task_status: TaskStatus[None] = trio.TASK_STATUS_IGNORED, ) -> None: ''' - Actor runtime entrypoint; start the IPC channel server, maybe connect - back to the parent, and startup all core machinery tasks. + Main `Actor` runtime entrypoint; start the transport-specific + IPC channel server, (maybe) connect back to parent (to receive + additional config), startup all core `trio` machinery for + delivering RPCs, register with the discovery system. - A "root-most" (or "top-level") nursery for this actor is opened here - and when cancelled effectively cancels the actor. + The "root" (or "top-level") and "service" `trio.Nursery`s are + opened here and when cancelled/terminated effectively shutdown + the actor's "runtime" and all thus all ongoing RPC tasks. ''' # attempt to retreive ``trio``'s sigint handler and stash it @@ -1258,12 +1422,6 @@ async def async_main( if accept_addr_rent is not None: accept_addr = accept_addr_rent - # load exposed/allowed RPC modules - # XXX: do this **after** establishing a channel to the parent - # but **before** starting the message loop for that channel - # such that import errors are properly propagated upwards - actor.load_modules() - # The "root" nursery ensures the channel with the immediate # parent is kept alive as a resilient service until # cancellation steps have (mostly) occurred in @@ -1280,7 +1438,25 @@ async def async_main( actor._service_n = service_nursery assert actor._service_n - # Startup up the channel server with, + # load exposed/allowed RPC modules + # XXX: do this **after** establishing a channel to the parent + # but **before** starting the message loop for that channel + # such that import errors are properly propagated upwards + actor.load_modules() + + # XXX TODO XXX: figuring out debugging of this + # would somemwhat guarantee "self-hosted" runtime + # debugging (since it hits all the ede cases?) + # + # `tractor.pause()` right? + # try: + # actor.load_modules() + # except ModuleNotFoundError as err: + # _debug.pause_from_sync() + # import pdbp; pdbp.set_trace() + # raise + + # Startup up the transport(-channel) server with, # - subactor: the bind address is sent by our parent # over our established channel # - root actor: the ``accept_addr`` passed to this method @@ -1329,14 +1505,19 @@ async def async_main( shield=True, ) ) - log.runtime("Waiting on service nursery to complete") - log.runtime("Service nursery complete") - log.runtime("Waiting on root nursery to complete") + log.runtime( + 'Actor runtime is up!' + # 'Blocking on service nursery to exit..\n' + ) + log.runtime( + "Service nursery complete\n" + "Waiting on root nursery to complete" + ) # Blocks here as expected until the root nursery is # killed (i.e. this actor is cancelled or signalled by the parent) except Exception as err: - log.info("Closing all actor lifetime contexts") + log.runtime("Closing all actor lifetime contexts") actor.lifetime_stack.close() if not registered_with_arbiter: @@ -1354,18 +1535,30 @@ async def async_main( ) if actor._parent_chan: - await try_ship_error_to_parent(actor._parent_chan, err) + await try_ship_error_to_remote( + actor._parent_chan, + err, + ) # always! - log.exception("Actor errored:") + match err: + case ContextCancelled(): + log.cancel( + f'Actor: {actor.uid} was task-context-cancelled with,\n' + f'str(err)' + ) + case _: + log.exception("Actor errored:") raise finally: - log.info("Runtime nursery complete") - + log.runtime( + 'Runtime nursery complete' + '-> Closing all actor lifetime contexts..' + ) # tear down all lifetime contexts if not in guest mode # XXX: should this just be in the entrypoint? - log.info("Closing all actor lifetime contexts") + actor.lifetime_stack.close() # TODO: we can't actually do this bc the debugger # uses the _service_n to spawn the lock task, BUT, @@ -1373,12 +1566,10 @@ async def async_main( # block it might be actually possible to debug THIS # machinery in the same way as user task code? # if actor.name == 'brokerd.ib': - # with trio.CancelScope(shield=True): + # with CancelScope(shield=True): # await _debug.breakpoint() - actor.lifetime_stack.close() - - # Unregister actor from the arbiter + # Unregister actor from the registry-sys / registrar. if ( registered_with_arbiter and not actor.is_arbiter @@ -1409,245 +1600,14 @@ async def async_main( ): log.runtime( f"Waiting for remaining peers {actor._peers} to clear") - with trio.CancelScope(shield=True): + with CancelScope(shield=True): await actor._no_more_peers.wait() log.runtime("All peer channels are complete") log.runtime("Runtime completed") -async def process_messages( - actor: Actor, - chan: Channel, - shield: bool = False, - task_status: TaskStatus[trio.CancelScope] = trio.TASK_STATUS_IGNORED, - -) -> bool: - ''' - Process messages for the IPC transport channel async-RPC style. - - Receive multiplexed RPC requests, spawn handler tasks and deliver - responses over or boxed errors back to the "caller" task. - - ''' - # TODO: once https://github.com/python-trio/trio/issues/467 gets - # worked out we'll likely want to use that! - msg = None - nursery_cancelled_before_task: bool = False - - log.runtime(f"Entering msg loop for {chan} from {chan.uid}") - try: - with trio.CancelScope(shield=shield) as loop_cs: - # this internal scope allows for keeping this message - # loop running despite the current task having been - # cancelled (eg. `open_portal()` may call this method from - # a locally spawned task) and recieve this scope using - # ``scope = Nursery.start()`` - task_status.started(loop_cs) - async for msg in chan: - - if msg is None: # loop terminate sentinel - - log.cancel( - f"Channel to {chan.uid} terminated?\n" - "Cancelling all associated tasks..") - - for (channel, cid) in actor._rpc_tasks.copy(): - if channel is chan: - await actor._cancel_task(cid, channel) - - log.runtime( - f"Msg loop signalled to terminate for" - f" {chan} from {chan.uid}") - - break - - log.transport( # type: ignore - f"Received msg {msg} from {chan.uid}") - - cid = msg.get('cid') - if cid: - # deliver response to local caller/waiter - await actor._push_result(chan, cid, msg) - - log.runtime( - f"Waiting on next msg for {chan} from {chan.uid}") - continue - - # process command request - try: - ns, funcname, kwargs, actorid, cid = msg['cmd'] - except KeyError: - # This is the non-rpc error case, that is, an - # error **not** raised inside a call to ``_invoke()`` - # (i.e. no cid was provided in the msg - see above). - # Push this error to all local channel consumers - # (normally portals) by marking the channel as errored - assert chan.uid - exc = unpack_error(msg, chan=chan) - chan._exc = exc - raise exc - - log.runtime( - f"Processing request from {actorid}\n" - f"{ns}.{funcname}({kwargs})") - - if ns == 'self': - func = getattr(actor, funcname) - - if funcname == 'cancel': - - # don't start entire actor runtime - # cancellation if this actor is in debug - # mode - pdb_complete = _debug.Lock.local_pdb_complete - if pdb_complete: - await pdb_complete.wait() - - # we immediately start the runtime machinery - # shutdown - with trio.CancelScope(shield=True): - # actor.cancel() was called so kill this - # msg loop and break out into - # ``async_main()`` - log.cancel( - f"Actor {actor.uid} was remotely cancelled " - f"by {chan.uid}" - ) - await _invoke( - actor, cid, chan, func, kwargs, is_rpc=False - ) - - loop_cs.cancel() - break - - if funcname == '_cancel_task': - - # we immediately start the runtime machinery - # shutdown - with trio.CancelScope(shield=True): - # actor.cancel() was called so kill this - # msg loop and break out into - # ``async_main()`` - kwargs['chan'] = chan - log.cancel( - f'Remote request to cancel task\n' - f'remote actor: {chan.uid}\n' - f'task: {cid}' - ) - try: - await _invoke( - actor, - cid, - chan, - func, - kwargs, - is_rpc=False, - ) - except BaseException: - log.exception("failed to cancel task?") - - continue - else: - # complain to client about restricted modules - try: - func = actor._get_rpc_func(ns, funcname) - except (ModuleNotExposed, AttributeError) as err: - err_msg = pack_error(err) - err_msg['cid'] = cid - await chan.send(err_msg) - continue - - # spin up a task for the requested function - log.runtime(f"Spawning task for {func}") - assert actor._service_n - try: - cs = await actor._service_n.start( - partial(_invoke, actor, cid, chan, func, kwargs), - name=funcname, - ) - except ( - RuntimeError, - BaseExceptionGroup, - ): - # avoid reporting a benign race condition - # during actor runtime teardown. - nursery_cancelled_before_task = True - break - - # never allow cancelling cancel requests (results in - # deadlock and other weird behaviour) - # if func != actor.cancel: - if isinstance(cs, Exception): - log.warning( - f"Task for RPC func {func} failed with" - f"{cs}") - else: - # mark that we have ongoing rpc tasks - actor._ongoing_rpc_tasks = trio.Event() - log.runtime(f"RPC func is {func}") - # store cancel scope such that the rpc task can be - # cancelled gracefully if requested - actor._rpc_tasks[(chan, cid)] = ( - cs, func, trio.Event()) - - log.runtime( - f"Waiting on next msg for {chan} from {chan.uid}") - - # end of async for, channel disconnect vis - # ``trio.EndOfChannel`` - log.runtime( - f"{chan} for {chan.uid} disconnected, cancelling tasks" - ) - await actor.cancel_rpc_tasks(chan) - - 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.runtime( - f'channel from {chan.uid} closed abruptly:\n' - f'-> {chan.raddr}\n' - ) - - # transport **was** disconnected - return True - - except ( - Exception, - BaseExceptionGroup, - ) as err: - if nursery_cancelled_before_task: - sn = actor._service_n - assert sn and sn.cancel_scope.cancel_called - log.cancel( - f'Service nursery cancelled before it handled {funcname}' - ) - else: - # ship any "internal" exception (i.e. one from internal - # machinery not from an rpc task) to parent - log.exception("Actor errored:") - if actor._parent_chan: - await try_ship_error_to_parent(actor._parent_chan, err) - - # if this is the `MainProcess` we expect the error broadcasting - # above to trigger an error at consuming portal "checkpoints" - raise - - finally: - # msg debugging for when he machinery is brokey - log.runtime( - f"Exiting msg loop for {chan} from {chan.uid} " - f"with last msg:\n{msg}") - - # transport **was not** disconnected - return False - - +# TODO: rename to `Registry` and move to `._discovery`! class Arbiter(Actor): ''' A special actor who knows all the other actors and always has @@ -1721,6 +1681,11 @@ class Arbiter(Actor): sockaddr: tuple[str, int] for (aname, _), sockaddr in self._registry.items(): + log.runtime( + f'Actor mailbox info:\n' + f'aname: {aname}\n' + f'sockaddr: {sockaddr}\n' + ) if name == aname: sockaddrs.append(sockaddr) diff --git a/tractor/_spawn.py b/tractor/_spawn.py index e0472a98..e91638bc 100644 --- a/tractor/_spawn.py +++ b/tractor/_spawn.py @@ -19,6 +19,7 @@ Machinery for actor process spawning using multiple backends. """ from __future__ import annotations +import multiprocessing as mp import sys import platform from typing import ( @@ -30,30 +31,28 @@ from typing import ( TYPE_CHECKING, ) -from exceptiongroup import BaseExceptionGroup import trio -from trio_typing import TaskStatus +from trio import TaskStatus from ._debug import ( maybe_wait_for_debugger, acquire_debug_lock, ) -from ._state import ( +from tractor._state import ( current_actor, is_main_process, is_root_process, debug_mode, ) -from .log import get_logger -from ._portal import Portal -from ._runtime import Actor -from ._entry import _mp_main -from ._exceptions import ActorFailure +from tractor.log import get_logger +from tractor._portal import Portal +from tractor._runtime import Actor +from tractor._entry import _mp_main +from tractor._exceptions import ActorFailure if TYPE_CHECKING: from ._supervise import ActorNursery - import multiprocessing as mp ProcessType = TypeVar('ProcessType', mp.Process, trio.Process) log = get_logger('tractor') @@ -70,7 +69,6 @@ _spawn_method: SpawnMethodKey = 'trio' if platform.system() == 'Windows': - import multiprocessing as mp _ctx = mp.get_context("spawn") async def proc_waiter(proc: mp.Process) -> None: @@ -145,7 +143,7 @@ async def exhaust_portal( # XXX: streams should never be reaped here since they should # always be established and shutdown using a context manager api - final = await portal.result() + final: Any = await portal.result() except ( Exception, @@ -153,13 +151,23 @@ async def exhaust_portal( ) as err: # we reraise in the parent task via a ``BaseExceptionGroup`` return err + except trio.Cancelled as err: # lol, of course we need this too ;P # TODO: merge with above? - log.warning(f"Cancelled result waiter for {portal.actor.uid}") + log.warning( + 'Cancelled portal result waiter task:\n' + f'uid: {portal.channel.uid}\n' + f'error: {err}\n' + ) return err + else: - log.debug(f"Returning final result: {final}") + log.debug( + f'Returning final result from portal:\n' + f'uid: {portal.channel.uid}\n' + f'result: {final}\n' + ) return final @@ -171,41 +179,71 @@ async def cancel_on_completion( ) -> None: ''' - Cancel actor gracefully once it's "main" portal's + Cancel actor gracefully once its "main" portal's result arrives. - Should only be called for actors spawned with `run_in_actor()`. + Should only be called for actors spawned via the + `Portal.run_in_actor()` API. + + => and really this API will be deprecated and should be + re-implemented as a `.hilevel.one_shot_task_nursery()`..) ''' # if this call errors we store the exception for later # in ``errors`` which will be reraised inside # an exception group and we still send out a cancel request - result = await exhaust_portal(portal, actor) + result: Any|Exception = await exhaust_portal(portal, actor) if isinstance(result, Exception): - errors[actor.uid] = result - log.warning( - f"Cancelling {portal.channel.uid} after error {result}" + errors[actor.uid]: Exception = result + log.cancel( + 'Cancelling subactor runtime due to error:\n\n' + f'Portal.cancel_actor() => {portal.channel.uid}\n\n' + f'error: {result}\n' ) else: log.runtime( - f"Cancelling {portal.channel.uid} gracefully " - f"after result {result}") + 'Cancelling subactor gracefully:\n\n' + f'Portal.cancel_actor() => {portal.channel.uid}\n\n' + f'result: {result}\n' + ) # cancel the process now that we have a final result await portal.cancel_actor() -async def do_hard_kill( +async def hard_kill( proc: trio.Process, - terminate_after: int = 3, + terminate_after: int = 1.6, + + # NOTE: for mucking with `.pause()`-ing inside the runtime + # whilst also hacking on it XD + # terminate_after: int = 99999, ) -> None: + ''' + Un-gracefully terminate an OS level `trio.Process` after timeout. + + Used in 2 main cases: + + - "unknown remote runtime state": a hanging/stalled actor that + isn't responding after sending a (graceful) runtime cancel + request via an IPC msg. + - "cancelled during spawn": a process who's actor runtime was + cancelled before full startup completed (such that + cancel-request-handling machinery was never fully + initialized) and thus a "cancel request msg" is never going + to be handled. + + ''' + log.cancel( + 'Terminating sub-proc:\n' + f'|_{proc}\n' + ) # NOTE: this timeout used to do nothing since we were shielding # the ``.wait()`` inside ``new_proc()`` which will pretty much # never release until the process exits, now it acts as # a hard-kill time ultimatum. - log.debug(f"Terminating {proc}") with trio.move_on_after(terminate_after) as cs: # NOTE: code below was copied verbatim from the now deprecated @@ -216,6 +254,9 @@ async def do_hard_kill( # and wait for it to exit. If cancelled, kills the process and # waits for it to finish exiting before propagating the # cancellation. + # + # This code was originally triggred by ``proc.__aexit__()`` + # but now must be called manually. with trio.CancelScope(shield=True): if proc.stdin is not None: await proc.stdin.aclose() @@ -231,15 +272,25 @@ async def do_hard_kill( with trio.CancelScope(shield=True): await proc.wait() + # XXX NOTE XXX: zombie squad dispatch: + # (should ideally never, but) If we do get here it means + # graceful termination of a process failed and we need to + # resort to OS level signalling to interrupt and cancel the + # (presumably stalled or hung) actor. Since we never allow + # zombies (as a feature) we ask the OS to do send in the + # removal swad as the last resort. if cs.cancelled_caught: - # XXX: should pretty much never get here unless we have - # to move the bits from ``proc.__aexit__()`` out and - # into here. - log.critical(f"#ZOMBIE_LORD_IS_HERE: {proc}") + # TODO: toss in the skynet-logo face as ascii art? + log.critical( + # 'Well, the #ZOMBIE_LORD_IS_HERE# to collect\n' + '#T-800 deployed to collect zombie B0\n' + f'|\n' + f'|_{proc}\n' + ) proc.kill() -async def soft_wait( +async def soft_kill( proc: ProcessType, wait_func: Callable[ @@ -249,14 +300,26 @@ async def soft_wait( portal: Portal, ) -> None: - # Wait for proc termination but **dont' yet** call - # ``trio.Process.__aexit__()`` (it tears down stdio - # which will kill any waiting remote pdb trace). - # This is a "soft" (cancellable) join/reap. - uid = portal.channel.uid + ''' + Wait for proc termination but **don't yet** teardown + std-streams since it will clobber any ongoing pdb REPL + session. + + This is our "soft"/graceful, and thus itself also cancellable, + join/reap on an actor-runtime-in-process shutdown; it is + **not** the same as a "hard kill" via an OS signal (for that + see `.hard_kill()`). + + ''' + uid: tuple[str, str] = portal.channel.uid try: - log.cancel(f'Soft waiting on actor:\n{uid}') + log.cancel( + 'Soft killing sub-actor via `Portal.cancel_actor()`\n' + f'|_{proc}\n' + ) + # wait on sub-proc to signal termination await wait_func(proc) + except trio.Cancelled: # if cancelled during a soft wait, cancel the child # actor before entering the hard reap sequence @@ -268,22 +331,29 @@ async def soft_wait( async def cancel_on_proc_deth(): ''' - Cancel the actor cancel request if we detect that - that the process terminated. + "Cancel-the-cancel" request: if we detect that the + underlying sub-process exited prior to + a `Portal.cancel_actor()` call completing . ''' await wait_func(proc) n.cancel_scope.cancel() + # start a task to wait on the termination of the + # process by itself waiting on a (caller provided) wait + # function which should unblock when the target process + # has terminated. n.start_soon(cancel_on_proc_deth) + + # send the actor-runtime a cancel request. await portal.cancel_actor() if proc.poll() is None: # type: ignore log.warning( - 'Actor still alive after cancel request:\n' - f'{uid}' + 'Subactor still alive after cancel request?\n\n' + f'uid: {uid}\n' + f'|_{proc}\n' ) - n.cancel_scope.cancel() raise @@ -307,7 +377,7 @@ async def new_proc( ) -> None: # lookup backend spawning target - target = _methods[_spawn_method] + target: Callable = _methods[_spawn_method] # mark the new actor with the global spawn method subactor._spawn_method = _spawn_method @@ -375,19 +445,22 @@ async def trio_proc( spawn_cmd.append("--asyncio") cancelled_during_spawn: bool = False - proc: trio.Process | None = None + proc: trio.Process|None = None try: try: # TODO: needs ``trio_typing`` patch? proc = await trio.lowlevel.open_process(spawn_cmd) - - log.runtime(f"Started {proc}") + log.runtime( + 'Started new sub-proc\n' + f'|_{proc}\n' + ) # wait for actor to spawn and connect back to us # channel should have handshake completed by the # local actor by the time we get a ref to it event, chan = await actor_nursery._actor.wait_for_peer( - subactor.uid) + subactor.uid + ) except trio.Cancelled: cancelled_during_spawn = True @@ -449,7 +522,7 @@ async def trio_proc( # This is a "soft" (cancellable) join/reap which # will remote cancel the actor on a ``trio.Cancelled`` # condition. - await soft_wait( + await soft_kill( proc, trio.Process.wait, portal @@ -457,9 +530,10 @@ async def trio_proc( # cancel result waiter that may have been spawned in # tandem if not done already - log.warning( - "Cancelling existing result waiter task for " - f"{subactor.uid}") + log.cancel( + 'Cancelling existing result waiter task for ' + f'{subactor.uid}' + ) nursery.cancel_scope.cancel() finally: @@ -477,22 +551,39 @@ async def trio_proc( with trio.move_on_after(0.5): await proc.wait() - if is_root_process(): - # TODO: solve the following issue where we need - # to do a similar wait like this but in an - # "intermediary" parent actor that itself isn't - # in debug but has a child that is, and we need - # to hold off on relaying SIGINT until that child - # is complete. - # https://github.com/goodboy/tractor/issues/320 - await maybe_wait_for_debugger( - child_in_debug=_runtime_vars.get( - '_debug_mode', False), - ) + log.pdb( + 'Delaying subproc reaper while debugger locked..' + ) + await maybe_wait_for_debugger( + child_in_debug=_runtime_vars.get( + '_debug_mode', False + ), + # TODO: need a diff value then default? + # poll_steps=9999999, + ) + # TODO: solve the following issue where we need + # to do a similar wait like this but in an + # "intermediary" parent actor that itself isn't + # in debug but has a child that is, and we need + # to hold off on relaying SIGINT until that child + # is complete. + # https://github.com/goodboy/tractor/issues/320 + # -[ ] we need to handle non-root parent-actors specially + # by somehow determining if a child is in debug and then + # avoiding cancel/kill of said child by this + # (intermediary) parent until such a time as the root says + # the pdb lock is released and we are good to tear down + # (our children).. + # + # -[ ] so maybe something like this where we try to + # acquire the lock and get notified of who has it, + # check that uid against our known children? + # this_uid: tuple[str, str] = current_actor().uid + # await acquire_debug_lock(this_uid) if proc.poll() is None: log.cancel(f"Attempting to hard kill {proc}") - await do_hard_kill(proc) + await hard_kill(proc) log.debug(f"Joined {proc}") else: @@ -636,7 +727,7 @@ async def mp_proc( # This is a "soft" (cancellable) join/reap which # will remote cancel the actor on a ``trio.Cancelled`` # condition. - await soft_wait( + await soft_kill( proc, proc_waiter, portal diff --git a/tractor/_state.py b/tractor/_state.py index 28fa16e7..f3917436 100644 --- a/tractor/_state.py +++ b/tractor/_state.py @@ -18,17 +18,18 @@ Per process state """ +from __future__ import annotations from typing import ( - Optional, Any, + TYPE_CHECKING, ) -import trio - -from ._exceptions import NoRuntime +if TYPE_CHECKING: + from ._runtime import Actor -_current_actor: Optional['Actor'] = None # type: ignore # noqa +_current_actor: Actor|None = None # type: ignore # noqa +_last_actor_terminated: Actor|None = None _runtime_vars: dict[str, Any] = { '_debug_mode': False, '_is_root': False, @@ -36,26 +37,68 @@ _runtime_vars: dict[str, Any] = { } -def current_actor(err_on_no_runtime: bool = True) -> 'Actor': # type: ignore # noqa - """Get the process-local actor instance. - """ - if _current_actor is None and err_on_no_runtime: - raise NoRuntime("No local actor has been initialized yet") +def last_actor() -> Actor|None: + ''' + Try to return last active `Actor` singleton + for this process. + + For case where runtime already exited but someone is asking + about the "last" actor probably to get its `.uid: tuple`. + + ''' + return _last_actor_terminated + + +def current_actor( + err_on_no_runtime: bool = True, +) -> Actor: + ''' + Get the process-local actor instance. + + ''' + if ( + err_on_no_runtime + and _current_actor is None + ): + msg: str = 'No local actor has been initialized yet' + from ._exceptions import NoRuntime + + if last := last_actor(): + msg += ( + f'Apparently the lact active actor was\n' + f'|_{last}\n' + f'|_{last.uid}\n' + ) + # no actor runtime has (as of yet) ever been started for + # this process. + else: + msg += ( + 'No last actor found?\n' + 'Did you forget to open one of:\n\n' + '- `tractor.open_root_actor()`\n' + '- `tractor.open_nursery()`\n' + ) + + raise NoRuntime(msg) return _current_actor def is_main_process() -> bool: - """Bool determining if this actor is running in the top-most process. - """ + ''' + Bool determining if this actor is running in the top-most process. + + ''' import multiprocessing as mp return mp.current_process().name == 'MainProcess' def debug_mode() -> bool: - """Bool determining if "debug mode" is on which enables + ''' + Bool determining if "debug mode" is on which enables remote subactor pdb entry on crashes. - """ + + ''' return bool(_runtime_vars['_debug_mode']) diff --git a/tractor/_streaming.py b/tractor/_streaming.py index b1129567..e0015fe4 100644 --- a/tractor/_streaming.py +++ b/tractor/_streaming.py @@ -14,31 +14,38 @@ # You should have received a copy of the GNU Affero General Public License # along with this program. If not, see . -""" +''' Message stream types and APIs. -""" +The machinery and types behind ``Context.open_stream()`` + +''' from __future__ import annotations +from contextlib import asynccontextmanager as acm import inspect -from contextlib import asynccontextmanager -from dataclasses import dataclass +from pprint import pformat from typing import ( Any, - Optional, Callable, - AsyncGenerator, - AsyncIterator + AsyncIterator, + TYPE_CHECKING, ) - import warnings import trio -from ._ipc import Channel -from ._exceptions import unpack_error, ContextCancelled -from ._state import current_actor +from ._exceptions import ( + _raise_from_no_key_in_msg, + ContextCancelled, +) from .log import get_logger -from .trionics import broadcast_receiver, BroadcastReceiver +from .trionics import ( + broadcast_receiver, + BroadcastReceiver, +) + +if TYPE_CHECKING: + from ._context import Context log = get_logger(__name__) @@ -49,7 +56,6 @@ log = get_logger(__name__) # messages? class ReceiveChannel(AsyncResource, Generic[ReceiveType]): # - use __slots__ on ``Context``? - class MsgStream(trio.abc.Channel): ''' A bidirectional message stream for receiving logically sequenced @@ -70,9 +76,9 @@ class MsgStream(trio.abc.Channel): ''' def __init__( self, - ctx: 'Context', # typing: ignore # noqa + ctx: Context, # typing: ignore # noqa rx_chan: trio.MemoryReceiveChannel, - _broadcaster: Optional[BroadcastReceiver] = None, + _broadcaster: BroadcastReceiver | None = None, ) -> None: self._ctx = ctx @@ -80,122 +86,248 @@ class MsgStream(trio.abc.Channel): self._broadcaster = _broadcaster # flag to denote end of stream - self._eoc: bool = False - self._closed: bool = False + self._eoc: bool|trio.EndOfChannel = False + self._closed: bool|trio.ClosedResourceError = False # delegate directly to underlying mem channel - def receive_nowait(self): - msg = self._rx_chan.receive_nowait() - return msg['yield'] + def receive_nowait( + self, + allow_msg_keys: list[str] = ['yield'], + ): + msg: dict = self._rx_chan.receive_nowait() + for ( + i, + key, + ) in enumerate(allow_msg_keys): + try: + return msg[key] + except KeyError as kerr: + if i < (len(allow_msg_keys) - 1): + continue - async def receive(self): - '''Async receive a single msg from the IPC transport, the next - in sequence for this stream. + _raise_from_no_key_in_msg( + ctx=self._ctx, + msg=msg, + src_err=kerr, + log=log, + expect_key=key, + stream=self, + ) + + async def receive( + self, + + hide_tb: bool = True, + ): + ''' + Receive a single msg from the IPC transport, the next in + sequence sent by the far end task (possibly in order as + determined by the underlying protocol). ''' + __tracebackhide__: bool = hide_tb + + # NOTE: `trio.ReceiveChannel` implements + # EOC handling as follows (aka uses it + # to gracefully exit async for loops): + # + # async def __anext__(self) -> ReceiveType: + # try: + # return await self.receive() + # except trio.EndOfChannel: + # raise StopAsyncIteration + # # see ``.aclose()`` for notes on the old behaviour prior to # introducing this if self._eoc: - raise trio.EndOfChannel + raise self._eoc if self._closed: - raise trio.ClosedResourceError('This stream was closed') + raise self._closed + src_err: Exception|None = None # orig tb try: - msg = await self._rx_chan.receive() - return msg['yield'] + try: + msg = await self._rx_chan.receive() + return msg['yield'] - except KeyError as err: - # internal error should never get here - assert msg.get('cid'), ("Received internal error at portal?") + except KeyError as kerr: + src_err = kerr - # TODO: handle 2 cases with 3.10 match syntax - # - 'stop' - # - 'error' - # possibly just handle msg['stop'] here! - - if self._closed: - raise trio.ClosedResourceError('This stream was closed') - - if msg.get('stop') or self._eoc: - log.debug(f"{self} was stopped at remote end") - - # XXX: important to set so that a new ``.receive()`` - # call (likely by another task using a broadcast receiver) - # doesn't accidentally pull the ``return`` message - # value out of the underlying feed mem chan! - self._eoc = True - - # # when the send is closed we assume the stream has - # # terminated and signal this local iterator to stop - # await self.aclose() - - # XXX: this causes ``ReceiveChannel.__anext__()`` to - # raise a ``StopAsyncIteration`` **and** in our catch - # block below it will trigger ``.aclose()``. - raise trio.EndOfChannel from err - - # TODO: test that shows stream raising an expected error!!! - elif msg.get('error'): - # raise the error message - raise unpack_error(msg, self._ctx.chan) - - else: - raise + # NOTE: may raise any of the below error types + # includg EoC when a 'stop' msg is found. + _raise_from_no_key_in_msg( + ctx=self._ctx, + msg=msg, + src_err=kerr, + log=log, + expect_key='yield', + stream=self, + ) + # XXX: the stream terminates on either of: + # - via `self._rx_chan.receive()` raising after manual closure + # by the rpc-runtime OR, + # - via a received `{'stop': ...}` msg from remote side. + # |_ NOTE: previously this was triggered by calling + # ``._rx_chan.aclose()`` on the send side of the channel inside + # `Actor._push_result()`, but now the 'stop' message handling + # has been put just above inside `_raise_from_no_key_in_msg()`. except ( - trio.ClosedResourceError, # by self._rx_chan - trio.EndOfChannel, # by self._rx_chan or `stop` msg from far end - ): - # XXX: we close the stream on any of these error conditions: - - # a ``ClosedResourceError`` indicates that the internal - # feeder memory receive channel was closed likely by the - # runtime after the associated transport-channel - # disconnected or broke. - - # an ``EndOfChannel`` indicates either the internal recv - # memchan exhausted **or** we raisesd it just above after - # receiving a `stop` message from the far end of the stream. - - # Previously this was triggered by calling ``.aclose()`` on - # the send side of the channel inside - # ``Actor._push_result()`` (should still be commented code - # there - which should eventually get removed), but now the - # 'stop' message handling has been put just above. + trio.EndOfChannel, + ) as eoc: + src_err = eoc + self._eoc = eoc # TODO: Locally, we want to close this stream gracefully, by # terminating any local consumers tasks deterministically. - # One we have broadcast support, we **don't** want to be + # Once we have broadcast support, we **don't** want to be # closing this stream and not flushing a final value to # remaining (clone) consumers who may not have been # scheduled to receive it yet. + # try: + # maybe_err_msg_or_res: dict = self._rx_chan.receive_nowait() + # if maybe_err_msg_or_res: + # log.warning( + # 'Discarding un-processed msg:\n' + # f'{maybe_err_msg_or_res}' + # ) + # except trio.WouldBlock: + # # no queued msgs that might be another remote + # # error, so just raise the original EoC + # pass - # when the send is closed we assume the stream has - # terminated and signal this local iterator to stop - await self.aclose() + # raise eoc - raise # propagate + # a ``ClosedResourceError`` indicates that the internal + # feeder memory receive channel was closed likely by the + # runtime after the associated transport-channel + # disconnected or broke. + except trio.ClosedResourceError as cre: # by self._rx_chan.receive() + src_err = cre + log.warning( + '`Context._rx_chan` was already closed?' + ) + self._closed = cre - async def aclose(self): + # when the send is closed we assume the stream has + # terminated and signal this local iterator to stop + drained: list[Exception|dict] = await self.aclose() + if drained: + # from .devx import pause + # await pause() + log.warning( + 'Drained context msgs during closure:\n' + f'{drained}' + ) + # TODO: pass these to the `._ctx._drained_msgs: deque` + # and then iterate them as part of any `.result()` call? + + # NOTE XXX: if the context was cancelled or remote-errored + # but we received the stream close msg first, we + # probably want to instead raise the remote error + # over the end-of-stream connection error since likely + # the remote error was the source cause? + ctx: Context = self._ctx + ctx.maybe_raise( + raise_ctxc_from_self_call=True, + ) + + # propagate any error but hide low-level frame details + # from the caller by default for debug noise reduction. + if ( + hide_tb + + # XXX NOTE XXX don't reraise on certain + # stream-specific internal error types like, + # + # - `trio.EoC` since we want to use the exact instance + # to ensure that it is the error that bubbles upward + # for silent absorption by `Context.open_stream()`. + and not self._eoc + + # - `RemoteActorError` (or `ContextCancelled`) if it gets + # raised from `_raise_from_no_key_in_msg()` since we + # want the same (as the above bullet) for any + # `.open_context()` block bubbled error raised by + # any nearby ctx API remote-failures. + # and not isinstance(src_err, RemoteActorError) + ): + raise type(src_err)(*src_err.args) from src_err + else: + raise src_err + + async def aclose(self) -> list[Exception|dict]: ''' Cancel associated remote actor task and local memory channel on close. + Notes: + - REMEMBER that this is also called by `.__aexit__()` so + careful consideration must be made to handle whatever + internal stsate is mutated, particuarly in terms of + draining IPC msgs! + + - more or less we try to maintain adherance to trio's `.aclose()` semantics: + https://trio.readthedocs.io/en/stable/reference-io.html#trio.abc.AsyncResource.aclose ''' - # XXX: keep proper adherance to trio's `.aclose()` semantics: - # https://trio.readthedocs.io/en/stable/reference-io.html#trio.abc.AsyncResource.aclose - rx_chan = self._rx_chan - if rx_chan._closed: - log.cancel(f"{self} is already closed") + # rx_chan = self._rx_chan + # XXX NOTE XXX + # it's SUPER IMPORTANT that we ensure we don't DOUBLE + # DRAIN msgs on closure so avoid getting stuck handing on + # the `._rx_chan` since we call this method on + # `.__aexit__()` as well!!! + # => SO ENSURE WE CATCH ALL TERMINATION STATES in this + # block including the EoC.. + if self.closed: # this stream has already been closed so silently succeed as # per ``trio.AsyncResource`` semantics. # https://trio.readthedocs.io/en/stable/reference-io.html#trio.abc.AsyncResource.aclose - return + return [] - self._eoc = True + ctx: Context = self._ctx + drained: list[Exception|dict] = [] + while not drained: + try: + maybe_final_msg = self.receive_nowait( + allow_msg_keys=['yield', 'return'], + ) + if maybe_final_msg: + log.debug( + 'Drained un-processed stream msg:\n' + f'{pformat(maybe_final_msg)}' + ) + # TODO: inject into parent `Context` buf? + drained.append(maybe_final_msg) + + # NOTE: we only need these handlers due to the + # `.receive_nowait()` call above which may re-raise + # one of these errors on a msg key error! + + except trio.WouldBlock as be: + drained.append(be) + break + + except trio.EndOfChannel as eoc: + self._eoc: Exception = eoc + drained.append(eoc) + break + + except trio.ClosedResourceError as cre: + self._closed = cre + drained.append(cre) + break + + except ContextCancelled as ctxc: + # log.exception('GOT CTXC') + log.cancel( + 'Context was cancelled during stream closure:\n' + f'canceller: {ctxc.canceller}\n' + f'{pformat(ctxc.msgdata)}' + ) + break # NOTE: this is super subtle IPC messaging stuff: # Relay stop iteration to far end **iff** we're @@ -226,26 +358,40 @@ class MsgStream(trio.abc.Channel): except ( trio.BrokenResourceError, trio.ClosedResourceError - ): + ) as re: # the underlying channel may already have been pulled # in which case our stop message is meaningless since # it can't traverse the transport. - ctx = self._ctx log.warning( f'Stream was already destroyed?\n' f'actor: {ctx.chan.uid}\n' f'ctx id: {ctx.cid}' ) + drained.append(re) + self._closed = re - self._closed = True + # if caught_eoc: + # # from .devx import _debug + # # await _debug.pause() + # with trio.CancelScope(shield=True): + # await rx_chan.aclose() - # Do we close the local mem chan ``self._rx_chan`` ??!? - - # NO, DEFINITELY NOT if we're a bi-dir ``MsgStream``! - # BECAUSE this same core-msg-loop mem recv-chan is used to deliver - # the potential final result from the surrounding inter-actor - # `Context` so we don't want to close it until that context has - # run to completion. + if not self._eoc: + log.cancel( + 'Stream closed before it received an EoC?\n' + 'Setting eoc manually..\n..' + ) + self._eoc: bool = trio.EndOfChannel( + f'Context stream closed by {self._ctx.side}\n' + f'|_{self}\n' + ) + # ?XXX WAIT, why do we not close the local mem chan `._rx_chan` XXX? + # => NO, DEFINITELY NOT! <= + # if we're a bi-dir ``MsgStream`` BECAUSE this same + # core-msg-loop mem recv-chan is used to deliver the + # potential final result from the surrounding inter-actor + # `Context` so we don't want to close it until that + # context has run to completion. # XXX: Notes on old behaviour: # await rx_chan.aclose() @@ -274,8 +420,28 @@ class MsgStream(trio.abc.Channel): # runtime's closure of ``rx_chan`` in the case where we may # still need to consume msgs that are "in transit" from the far # end (eg. for ``Context.result()``). + # self._closed = True + return drained - @asynccontextmanager + @property + def closed(self) -> bool: + + rxc: bool = self._rx_chan._closed + _closed: bool|Exception = self._closed + _eoc: bool|trio.EndOfChannel = self._eoc + if rxc or _closed or _eoc: + log.runtime( + f'`MsgStream` is already closed\n' + f'{self}\n' + f' |_cid: {self._ctx.cid}\n' + f' |_rx_chan._closed: {type(rxc)} = {rxc}\n' + f' |_closed: {type(_closed)} = {_closed}\n' + f' |_eoc: {type(_eoc)} = {_eoc}' + ) + return True + return False + + @acm async def subscribe( self, @@ -329,386 +495,50 @@ class MsgStream(trio.abc.Channel): async def send( self, - data: Any + data: Any, + + hide_tb: bool = True, ) -> None: ''' Send a message over this stream to the far end. ''' - if self._ctx._error: - raise self._ctx._error # from None + __tracebackhide__: bool = hide_tb + + # raise any alreay known error immediately + self._ctx.maybe_raise() + if self._eoc: + raise self._eoc if self._closed: - raise trio.ClosedResourceError('This stream was already closed') + raise self._closed - await self._ctx.chan.send({'yield': data, 'cid': self._ctx.cid}) - - -@dataclass -class Context: - ''' - An inter-actor, ``trio`` task communication context. - - NB: This class should never be instatiated directly, it is delivered - by either runtime machinery to a remotely started task or by entering - ``Portal.open_context()``. - - Allows maintaining task or protocol specific state between - 2 communicating actor tasks. A unique context is created on the - callee side/end for every request to a remote actor from a portal. - - A context can be cancelled and (possibly eventually restarted) from - either side of the underlying IPC channel, open task oriented - message streams and acts as an IPC aware inter-actor-task cancel - scope. - - ''' - chan: Channel - cid: str - - # these are the "feeder" channels for delivering - # message values to the local task from the runtime - # msg processing loop. - _recv_chan: trio.MemoryReceiveChannel - _send_chan: trio.MemorySendChannel - - _remote_func_type: Optional[str] = None - - # only set on the caller side - _portal: Optional['Portal'] = None # type: ignore # noqa - _result: Optional[Any] = False - _error: Optional[BaseException] = None - - # status flags - _cancel_called: bool = False - _cancel_msg: Optional[str] = None - _enter_debugger_on_cancel: bool = True - _started_called: bool = False - _started_received: bool = False - _stream_opened: bool = False - - # only set on the callee side - _scope_nursery: Optional[trio.Nursery] = None - - _backpressure: bool = False - - async def send_yield(self, data: Any) -> None: - - warnings.warn( - "`Context.send_yield()` is now deprecated. " - "Use ``MessageStream.send()``. ", - DeprecationWarning, - stacklevel=2, - ) - await self.chan.send({'yield': data, 'cid': self.cid}) - - async def send_stop(self) -> None: - await self.chan.send({'stop': True, 'cid': self.cid}) - - async def _maybe_raise_from_remote_msg( - self, - msg: dict[str, Any], - - ) -> None: - ''' - (Maybe) unpack and raise a msg error into the local scope - nursery for this context. - - Acts as a form of "relay" for a remote error raised - in the corresponding remote callee task. - - ''' - error = msg.get('error') - if error: - # If this is an error message from a context opened by - # ``Portal.open_context()`` we want to interrupt any ongoing - # (child) tasks within that context to be notified of the remote - # error relayed here. - # - # The reason we may want to raise the remote error immediately - # is that there is no guarantee the associated local task(s) - # will attempt to read from any locally opened stream any time - # soon. - # - # NOTE: this only applies when - # ``Portal.open_context()`` has been called since it is assumed - # (currently) that other portal APIs (``Portal.run()``, - # ``.run_in_actor()``) do their own error checking at the point - # of the call and result processing. - log.error( - f'Remote context error for {self.chan.uid}:{self.cid}:\n' - f'{msg["error"]["tb_str"]}' + try: + await self._ctx.chan.send( + payload={ + 'yield': data, + 'cid': self._ctx.cid, + }, + # hide_tb=hide_tb, ) - error = unpack_error(msg, self.chan) - if ( - isinstance(error, ContextCancelled) and - self._cancel_called - ): - # this is an expected cancel request response message - # and we don't need to raise it in scope since it will - # potentially override a real error - return - - self._error = error - - # TODO: tempted to **not** do this by-reraising in a - # nursery and instead cancel a surrounding scope, detect - # the cancellation, then lookup the error that was set? - if self._scope_nursery: - - async def raiser(): - raise self._error from None - - # from trio.testing import wait_all_tasks_blocked - # await wait_all_tasks_blocked() - if not self._scope_nursery._closed: # type: ignore - self._scope_nursery.start_soon(raiser) - - async def cancel( - self, - msg: Optional[str] = None, - - ) -> None: - ''' - Cancel this inter-actor-task context. - - Request that the far side cancel it's current linked context, - Timeout quickly in an attempt to sidestep 2-generals... - - ''' - side = 'caller' if self._portal else 'callee' - if msg: - assert side == 'callee', 'Only callee side can provide cancel msg' - - log.cancel(f'Cancelling {side} side of context to {self.chan.uid}') - - self._cancel_called = True - - if side == 'caller': - if not self._portal: - raise RuntimeError( - "No portal found, this is likely a callee side context" - ) - - cid = self.cid - with trio.move_on_after(0.5) as cs: - cs.shield = True - log.cancel( - f"Cancelling stream {cid} to " - f"{self._portal.channel.uid}") - - # NOTE: we're telling the far end actor to cancel a task - # corresponding to *this actor*. The far end local channel - # instance is passed to `Actor._cancel_task()` implicitly. - await self._portal.run_from_ns('self', '_cancel_task', cid=cid) - - if cs.cancelled_caught: - # XXX: there's no way to know if the remote task was indeed - # cancelled in the case where the connection is broken or - # some other network error occurred. - # if not self._portal.channel.connected(): - if not self.chan.connected(): - log.cancel( - "May have failed to cancel remote task " - f"{cid} for {self._portal.channel.uid}") - else: - log.cancel( - "Timed out on cancelling remote task " - f"{cid} for {self._portal.channel.uid}") - - # callee side remote task - else: - self._cancel_msg = msg - - # TODO: should we have an explicit cancel message - # or is relaying the local `trio.Cancelled` as an - # {'error': trio.Cancelled, cid: "blah"} enough? - # This probably gets into the discussion in - # https://github.com/goodboy/tractor/issues/36 - assert self._scope_nursery - self._scope_nursery.cancel_scope.cancel() - - if self._recv_chan: - await self._recv_chan.aclose() - - @asynccontextmanager - async def open_stream( - - self, - backpressure: Optional[bool] = True, - msg_buffer_size: Optional[int] = None, - - ) -> AsyncGenerator[MsgStream, None]: - ''' - Open a ``MsgStream``, a bi-directional stream connected to the - cross-actor (far end) task for this ``Context``. - - This context manager must be entered on both the caller and - callee for the stream to logically be considered "connected". - - A ``MsgStream`` is currently "one-shot" use, meaning if you - close it you can not "re-open" it for streaming and instead you - must re-establish a new surrounding ``Context`` using - ``Portal.open_context()``. In the future this may change but - currently there seems to be no obvious reason to support - "re-opening": - - pausing a stream can be done with a message. - - task errors will normally require a restart of the entire - scope of the inter-actor task context due to the nature of - ``trio``'s cancellation system. - - ''' - actor = current_actor() - - # here we create a mem chan that corresponds to the - # far end caller / callee. - - # Likewise if the surrounding context has been cancelled we error here - # since it likely means the surrounding block was exited or - # killed - - if self._cancel_called: - task = trio.lowlevel.current_task().name - raise ContextCancelled( - f'Context around {actor.uid[0]}:{task} was already cancelled!' - ) - - if not self._portal and not self._started_called: - raise RuntimeError( - 'Context.started()` must be called before opening a stream' - ) - - # NOTE: in one way streaming this only happens on the - # caller side inside `Actor.start_remote_task()` 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. - ctx = actor.get_context( - self.chan, - self.cid, - msg_buffer_size=msg_buffer_size, - ) - ctx._backpressure = backpressure - assert ctx is self - - # XXX: If the underlying channel feeder receive mem chan has - # been closed then likely client code has already exited - # a ``.open_stream()`` block prior or there was some other - # unanticipated error or cancellation from ``trio``. - - if ctx._recv_chan._closed: - raise trio.ClosedResourceError( - 'The underlying channel for this stream was already closed!?') - - async with MsgStream( - ctx=self, - rx_chan=ctx._recv_chan, - ) as stream: - - if self._portal: - self._portal._streams.add(stream) - - try: - self._stream_opened = True - - # XXX: do we need this? - # ensure we aren't cancelled before yielding the stream - # await trio.lowlevel.checkpoint() - yield stream - - # NOTE: Make the stream "one-shot use". On exit, signal - # ``trio.EndOfChannel``/``StopAsyncIteration`` to the - # far end. - await stream.aclose() - - finally: - if self._portal: - try: - self._portal._streams.remove(stream) - except KeyError: - log.warning( - f'Stream was already destroyed?\n' - f'actor: {self.chan.uid}\n' - f'ctx id: {self.cid}' - ) - - async def result(self) -> Any: - ''' - From a caller side, wait for and return the final result from - the callee side task. - - ''' - assert self._portal, "Context.result() can not be called from callee!" - assert self._recv_chan - - if self._result is False: - - if not self._recv_chan._closed: # type: ignore - - # wait for a final context result consuming - # and discarding any bi dir stream msgs still - # in transit from the far end. - while True: - - msg = await self._recv_chan.receive() - try: - self._result = msg['return'] - break - except KeyError as msgerr: - - if 'yield' in msg: - # far end task is still streaming to us so discard - log.warning(f'Discarding stream delivered {msg}') - continue - - elif 'stop' in msg: - log.debug('Remote stream terminated') - continue - - # internal error should never get here - assert msg.get('cid'), ( - "Received internal error at portal?") - - raise unpack_error( - msg, self._portal.channel - ) from msgerr - - return self._result - - async def started( - self, - value: Optional[Any] = None - - ) -> None: - ''' - Indicate to calling actor's task that this linked context - has started and send ``value`` to the other side. - - On the calling side ``value`` is the second item delivered - in the tuple returned by ``Portal.open_context()``. - - ''' - if self._portal: - raise RuntimeError( - f"Caller side context {self} can not call started!") - - elif self._started_called: - raise RuntimeError( - f"called 'started' twice on context with {self.chan.uid}") - - await self.chan.send({'started': value, 'cid': self.cid}) - self._started_called = True - - # TODO: do we need a restart api? - # async def restart(self) -> None: - # pass + except ( + trio.ClosedResourceError, + trio.BrokenResourceError, + BrokenPipeError, + ) as trans_err: + if hide_tb: + raise type(trans_err)( + *trans_err.args + ) from trans_err + else: + raise def stream(func: Callable) -> Callable: - """Mark an async function as a streaming routine with ``@stream``. + ''' + Mark an async function as a streaming routine with ``@stream``. - """ - # annotate + ''' # TODO: apply whatever solution ``mypy`` ends up picking for this: # https://github.com/python/mypy/issues/2087#issuecomment-769266912 func._tractor_stream_function = True # type: ignore @@ -734,22 +564,3 @@ def stream(func: Callable) -> Callable: "(Or ``to_trio`` if using ``asyncio`` in guest mode)." ) return func - - -def context(func: Callable) -> Callable: - """Mark an async function as a streaming routine with ``@context``. - - """ - # annotate - # TODO: apply whatever solution ``mypy`` ends up picking for this: - # https://github.com/python/mypy/issues/2087#issuecomment-769266912 - func._tractor_context_function = True # type: ignore - - sig = inspect.signature(func) - params = sig.parameters - if 'ctx' not in params: - raise TypeError( - "The first argument to the context function " - f"{func.__name__} must be `ctx: tractor.Context`" - ) - return func diff --git a/tractor/_supervise.py b/tractor/_supervise.py index 7f77784b..1e5ea387 100644 --- a/tractor/_supervise.py +++ b/tractor/_supervise.py @@ -21,6 +21,7 @@ from contextlib import asynccontextmanager as acm from functools import partial import inspect +from pprint import pformat from typing import ( Optional, TYPE_CHECKING, @@ -28,7 +29,6 @@ from typing import ( import typing import warnings -from exceptiongroup import BaseExceptionGroup import trio from ._debug import maybe_wait_for_debugger @@ -36,7 +36,10 @@ from ._state import current_actor, is_main_process from .log import get_logger, get_loglevel from ._runtime import Actor from ._portal import Portal -from ._exceptions import is_multi_cancelled +from ._exceptions import ( + is_multi_cancelled, + ContextCancelled, +) from ._root import open_root_actor from . import _state from . import _spawn @@ -106,6 +109,14 @@ class ActorNursery: self.errors = errors self.exited = trio.Event() + # NOTE: when no explicit call is made to + # `.open_root_actor()` by application code, + # `.open_nursery()` will implicitly call it to start the + # actor-tree runtime. In this case we mark ourselves as + # such so that runtime components can be aware for logging + # and syncing purposes to any actor opened nurseries. + self._implicit_runtime_started: bool = False + async def start_actor( self, name: str, @@ -157,7 +168,7 @@ class ActorNursery: # start a task to spawn a process # blocks until process has been started and a portal setup - nursery = nursery or self._da_nursery + nursery: trio.Nursery = nursery or self._da_nursery # XXX: the type ignore is actually due to a `mypy` bug return await nursery.start( # type: ignore @@ -190,14 +201,16 @@ class ActorNursery: **kwargs, # explicit args to ``fn`` ) -> Portal: - """Spawn a new actor, run a lone task, then terminate the actor and + ''' + Spawn a new actor, run a lone task, then terminate the actor and return its result. Actors spawned using this method are kept alive at nursery teardown until the task spawned by executing ``fn`` completes at which point the actor is terminated. - """ - mod_path = fn.__module__ + + ''' + mod_path: str = fn.__module__ if name is None: # use the explicit function name if not provided @@ -232,21 +245,37 @@ class ActorNursery: ) return portal - async def cancel(self, hard_kill: bool = False) -> None: - """Cancel this nursery by instructing each subactor to cancel + async def cancel( + self, + hard_kill: bool = False, + + ) -> None: + ''' + Cancel this nursery by instructing each subactor to cancel itself and wait for all subactors to terminate. If ``hard_killl`` is set to ``True`` then kill the processes directly without any far end graceful ``trio`` cancellation. - """ + + ''' self.cancelled = True - log.cancel(f"Cancelling nursery in {self._actor.uid}") + # TODO: impl a repr for spawn more compact + # then `._children`.. + children: dict = self._children + child_count: int = len(children) + msg: str = f'Cancelling actor nursery with {child_count} children\n' with trio.move_on_after(3) as cs: + async with trio.open_nursery() as tn: - async with trio.open_nursery() as nursery: - - for subactor, proc, portal in self._children.values(): + subactor: Actor + proc: trio.Process + portal: Portal + for ( + subactor, + proc, + portal, + ) in children.values(): # TODO: are we ever even going to use this or # is the spawning backend responsible for such @@ -258,12 +287,13 @@ class ActorNursery: if portal is None: # actor hasn't fully spawned yet event = self._actor._peer_connected[subactor.uid] log.warning( - f"{subactor.uid} wasn't finished spawning?") + f"{subactor.uid} never 't finished spawning?" + ) await event.wait() # channel/portal should now be up - _, _, portal = self._children[subactor.uid] + _, _, portal = children[subactor.uid] # XXX should be impossible to get here # unless method was called from within @@ -280,14 +310,24 @@ class ActorNursery: # spawn cancel tasks for each sub-actor assert portal if portal.channel.connected(): - nursery.start_soon(portal.cancel_actor) + tn.start_soon(portal.cancel_actor) + log.cancel(msg) # if we cancelled the cancel (we hung cancelling remote actors) # then hard kill all sub-processes if cs.cancelled_caught: log.error( - f"Failed to cancel {self}\nHard killing process tree!") - for subactor, proc, portal in self._children.values(): + f'Failed to cancel {self}?\n' + 'Hard killing underlying subprocess tree!\n' + ) + subactor: Actor + proc: trio.Process + portal: Portal + for ( + subactor, + proc, + portal, + ) in children.values(): log.warning(f"Hard killing process {proc}") proc.terminate() @@ -327,7 +367,7 @@ async def _open_and_supervise_one_cancels_all_nursery( # the above "daemon actor" nursery will be notified. async with trio.open_nursery() as ria_nursery: - anursery = ActorNursery( + an = ActorNursery( actor, ria_nursery, da_nursery, @@ -336,16 +376,16 @@ async def _open_and_supervise_one_cancels_all_nursery( try: # spawning of actors happens in the caller's scope # after we yield upwards - yield anursery + yield an # When we didn't error in the caller's scope, # signal all process-monitor-tasks to conduct # the "hard join phase". log.runtime( - f"Waiting on subactors {anursery._children} " - "to complete" + 'Waiting on subactors to complete:\n' + f'{pformat(an._children)}\n' ) - anursery._join_procs.set() + an._join_procs.set() except BaseException as inner_err: errors[actor.uid] = inner_err @@ -357,37 +397,60 @@ async def _open_and_supervise_one_cancels_all_nursery( # Instead try to wait for pdb to be released before # tearing down. await maybe_wait_for_debugger( - child_in_debug=anursery._at_least_one_child_in_debug + child_in_debug=an._at_least_one_child_in_debug ) # if the caller's scope errored then we activate our # one-cancels-all supervisor strategy (don't # worry more are coming). - anursery._join_procs.set() + an._join_procs.set() - # XXX: hypothetically an error could be - # raised and then a cancel signal shows up + # XXX NOTE XXX: hypothetically an error could + # be raised and then a cancel signal shows up # slightly after in which case the `else:` # block here might not complete? For now, # shield both. with trio.CancelScope(shield=True): - etype = type(inner_err) + etype: type = type(inner_err) if etype in ( trio.Cancelled, - KeyboardInterrupt + KeyboardInterrupt, ) or ( is_multi_cancelled(inner_err) ): log.cancel( - f"Nursery for {current_actor().uid} " - f"was cancelled with {etype}") + f'Actor-nursery cancelled by {etype}\n\n' + + f'{current_actor().uid}\n' + f' |_{an}\n\n' + + # TODO: show tb str? + # f'{tb_str}' + ) + elif etype in { + ContextCancelled, + }: + log.cancel( + 'Actor-nursery caught remote cancellation\n\n' + + f'{inner_err.tb_str}' + ) else: log.exception( - f"Nursery for {current_actor().uid} " - f"errored with") + 'Nursery errored with:\n' + + # TODO: same thing as in + # `._invoke()` to compute how to + # place this div-line in the + # middle of the above msg + # content.. + # -[ ] prolly helper-func it too + # in our `.log` module.. + # '------ - ------' + ) # cancel all subactors - await anursery.cancel() + await an.cancel() # ria_nursery scope end @@ -408,18 +471,22 @@ async def _open_and_supervise_one_cancels_all_nursery( # XXX: yet another guard before allowing the cancel # sequence in case a (single) child is in debug. await maybe_wait_for_debugger( - child_in_debug=anursery._at_least_one_child_in_debug + child_in_debug=an._at_least_one_child_in_debug ) # If actor-local error was raised while waiting on # ".run_in_actor()" actors then we also want to cancel all # remaining sub-actors (due to our lone strategy: # one-cancels-all). - log.cancel(f"Nursery cancelling due to {err}") - if anursery._children: + if an._children: + log.cancel( + 'Actor-nursery cancelling due error type:\n' + f'{err}\n' + ) with trio.CancelScope(shield=True): - await anursery.cancel() + await an.cancel() raise + finally: # No errors were raised while awaiting ".run_in_actor()" # actors but those actors may have returned remote errors as @@ -428,9 +495,9 @@ async def _open_and_supervise_one_cancels_all_nursery( # collected in ``errors`` so cancel all actors, summarize # all errors and re-raise. if errors: - if anursery._children: + if an._children: with trio.CancelScope(shield=True): - await anursery.cancel() + await an.cancel() # use `BaseExceptionGroup` as needed if len(errors) > 1: @@ -465,19 +532,20 @@ async def open_nursery( which cancellation scopes correspond to each spawned subactor set. ''' - implicit_runtime = False - - actor = current_actor(err_on_no_runtime=False) - + implicit_runtime: bool = False + actor: Actor = current_actor(err_on_no_runtime=False) + an: ActorNursery|None = None try: - if actor is None and is_main_process(): - + if ( + actor is None + and is_main_process() + ): # if we are the parent process start the # actor runtime implicitly log.info("Starting actor runtime!") # mark us for teardown on exit - implicit_runtime = True + implicit_runtime: bool = True async with open_root_actor(**kwargs) as actor: assert actor is current_actor() @@ -485,24 +553,42 @@ async def open_nursery( try: async with _open_and_supervise_one_cancels_all_nursery( actor - ) as anursery: - yield anursery + ) as an: + + # NOTE: mark this nursery as having + # implicitly started the root actor so + # that `._runtime` machinery can avoid + # certain teardown synchronization + # blocking/waits and any associated (warn) + # logging when it's known that this + # nursery shouldn't be exited before the + # root actor is. + an._implicit_runtime_started = True + yield an finally: - anursery.exited.set() + # XXX: this event will be set after the root actor + # runtime is already torn down, so we want to + # avoid any blocking on it. + an.exited.set() else: # sub-nursery case try: async with _open_and_supervise_one_cancels_all_nursery( actor - ) as anursery: - yield anursery + ) as an: + yield an finally: - anursery.exited.set() + an.exited.set() finally: - log.debug("Nursery teardown complete") + msg: str = ( + 'Actor-nursery exited\n' + f'|_{an}\n\n' + ) # shutdown runtime if it was started if implicit_runtime: - log.info("Shutting down actor tree") + msg += '=> Shutting down actor runtime <=\n' + + log.info(msg) diff --git a/tractor/_testing/__init__.py b/tractor/_testing/__init__.py new file mode 100644 index 00000000..876c87e8 --- /dev/null +++ b/tractor/_testing/__init__.py @@ -0,0 +1,74 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +''' +Various helpers/utils for auditing your `tractor` app and/or the +core runtime. + +''' +from contextlib import asynccontextmanager as acm +import pathlib + +import tractor +from .pytest import ( + tractor_test as tractor_test +) + + +def repodir() -> pathlib.Path: + ''' + Return the abspath to the repo directory. + + ''' + # 2 parents up to step up through tests/ + return pathlib.Path( + __file__ + + # 3 .parents bc: + # <._testing-pkg>.. + # /$HOME/..//tractor/_testing/__init__.py + ).parent.parent.parent.absolute() + + +def examples_dir() -> pathlib.Path: + ''' + Return the abspath to the examples directory as `pathlib.Path`. + + ''' + return repodir() / 'examples' + + +@acm +async def expect_ctxc( + yay: bool, + reraise: bool = False, +) -> None: + ''' + Small acm to catch `ContextCancelled` errors when expected + below it in a `async with ()` block. + + ''' + if yay: + try: + yield + raise RuntimeError('Never raised ctxc?') + except tractor.ContextCancelled: + if reraise: + raise + else: + return + else: + yield diff --git a/tractor/_testing/pytest.py b/tractor/_testing/pytest.py new file mode 100644 index 00000000..93eeaf72 --- /dev/null +++ b/tractor/_testing/pytest.py @@ -0,0 +1,113 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +''' +`pytest` utils helpers and plugins for testing `tractor`'s runtime +and applications. + +''' +from functools import ( + partial, + wraps, +) +import inspect +import platform + +import tractor +import trio + + +def tractor_test(fn): + ''' + Decorator for async test funcs to present them as "native" + looking sync funcs runnable by `pytest` using `trio.run()`. + + Use: + + @tractor_test + async def test_whatever(): + await ... + + If fixtures: + + - ``reg_addr`` (a socket addr tuple where arbiter is listening) + - ``loglevel`` (logging level passed to tractor internals) + - ``start_method`` (subprocess spawning backend) + + are defined in the `pytest` fixture space they will be automatically + injected to tests declaring these funcargs. + ''' + @wraps(fn) + def wrapper( + *args, + loglevel=None, + reg_addr=None, + start_method: str|None = None, + debug_mode: bool = False, + **kwargs + ): + # __tracebackhide__ = True + + # NOTE: inject ant test func declared fixture + # names by manually checking! + if 'reg_addr' in inspect.signature(fn).parameters: + # injects test suite fixture value to test as well + # as `run()` + kwargs['reg_addr'] = reg_addr + + if 'loglevel' in inspect.signature(fn).parameters: + # allows test suites to define a 'loglevel' fixture + # that activates the internal logging + kwargs['loglevel'] = loglevel + + if start_method is None: + if platform.system() == "Windows": + start_method = 'trio' + + if 'start_method' in inspect.signature(fn).parameters: + # set of subprocess spawning backends + kwargs['start_method'] = start_method + + if 'debug_mode' in inspect.signature(fn).parameters: + # set of subprocess spawning backends + kwargs['debug_mode'] = debug_mode + + + if kwargs: + + # use explicit root actor start + async def _main(): + async with tractor.open_root_actor( + # **kwargs, + registry_addrs=[reg_addr] if reg_addr else None, + loglevel=loglevel, + start_method=start_method, + + # TODO: only enable when pytest is passed --pdb + debug_mode=debug_mode, + + ): + await fn(*args, **kwargs) + + main = _main + + else: + # use implicit root actor start + main = partial(fn, *args, **kwargs) + + return trio.run(main) + + return wrapper diff --git a/tractor/experimental/_pubsub.py b/tractor/experimental/_pubsub.py index 99117b01..b894ed49 100644 --- a/tractor/experimental/_pubsub.py +++ b/tractor/experimental/_pubsub.py @@ -31,13 +31,13 @@ from typing import ( Callable, ) from functools import partial -from async_generator import aclosing +from contextlib import aclosing import trio import wrapt from ..log import get_logger -from .._streaming import Context +from .._context import Context __all__ = ['pub'] @@ -148,7 +148,8 @@ def pub( *, tasks: set[str] = set(), ): - """Publisher async generator decorator. + ''' + Publisher async generator decorator. A publisher can be called multiple times from different actors but will only spawn a finite set of internal tasks to stream values to @@ -227,7 +228,8 @@ def pub( running in a single actor to stream data to an arbitrary number of subscribers. If you are ok to have a new task running for every call to ``pub_service()`` then probably don't need this. - """ + + ''' global _pubtask2lock # handle the decorator not called with () case diff --git a/tractor/log.py b/tractor/log.py index 342257fa..6c040209 100644 --- a/tractor/log.py +++ b/tractor/log.py @@ -48,12 +48,15 @@ LOG_FORMAT = ( DATE_FORMAT = '%b %d %H:%M:%S' -LEVELS = { +LEVELS: dict[str, int] = { 'TRANSPORT': 5, 'RUNTIME': 15, 'CANCEL': 16, 'PDB': 500, } +# _custom_levels: set[str] = { +# lvlname.lower for lvlname in LEVELS.keys() +# } STD_PALETTE = { 'CRITICAL': 'red', @@ -82,6 +85,10 @@ class StackLevelAdapter(logging.LoggerAdapter): msg: str, ) -> None: + ''' + IPC level msg-ing. + + ''' return self.log(5, msg) def runtime( @@ -94,22 +101,57 @@ class StackLevelAdapter(logging.LoggerAdapter): self, msg: str, ) -> None: - return self.log(16, msg) + ''' + Cancellation logging, mostly for runtime reporting. + + ''' + return self.log( + level=16, + msg=msg, + # stacklevel=4, + ) def pdb( self, msg: str, ) -> None: + ''' + Debugger logging. + + ''' return self.log(500, msg) - def log(self, level, msg, *args, **kwargs): - """ + def log( + self, + level, + msg, + *args, + **kwargs, + ): + ''' Delegate a log call to the underlying logger, after adding contextual information from this adapter instance. - """ + + ''' if self.isEnabledFor(level): + stacklevel: int = 3 + if ( + level in LEVELS.values() + # or level in _custom_levels + ): + stacklevel: int = 4 + # msg, kwargs = self.process(msg, kwargs) - self._log(level, msg, args, **kwargs) + self._log( + level=level, + msg=msg, + args=args, + # NOTE: not sure how this worked before but, it + # seems with our custom level methods defined above + # we do indeed (now) require another stack level?? + stacklevel=stacklevel, + **kwargs, + ) # LOL, the stdlib doesn't allow passing through ``stacklevel``.. def _log( @@ -122,12 +164,15 @@ class StackLevelAdapter(logging.LoggerAdapter): stack_info=False, # XXX: bit we added to show fileinfo from actual caller. - # this level then ``.log()`` then finally the caller's level.. - stacklevel=3, + # - this level + # - then ``.log()`` + # - then finally the caller's level.. + stacklevel=4, ): - """ + ''' Low-level log implementation, proxied to allow nested logger adapters. - """ + + ''' return self.logger._log( level, msg, @@ -181,15 +226,39 @@ def get_logger( ''' log = rlog = logging.getLogger(_root_name) - if name and name != _proj_name: + if ( + name + and name != _proj_name + ): - # handling for modules that use ``get_logger(__name__)`` to - # avoid duplicate project-package token in msg output - rname, _, tail = name.partition('.') - if rname == _root_name: - name = tail + # NOTE: for handling for modules that use ``get_logger(__name__)`` + # we make the following stylistic choice: + # - always avoid duplicate project-package token + # in msg output: i.e. tractor.tractor _ipc.py in header + # looks ridiculous XD + # - never show the leaf module name in the {name} part + # since in python the {filename} is always this same + # module-file. + + sub_name: None | str = None + rname, _, sub_name = name.partition('.') + pkgpath, _, modfilename = sub_name.rpartition('.') + + # NOTE: for tractor itself never include the last level + # module key in the name such that something like: eg. + # 'tractor.trionics._broadcast` only includes the first + # 2 tokens in the (coloured) name part. + if rname == 'tractor': + sub_name = pkgpath + + if _root_name in sub_name: + duplicate, _, sub_name = sub_name.partition('.') + + if not sub_name: + log = rlog + else: + log = rlog.getChild(sub_name) - log = rlog.getChild(name) log.level = rlog.level # add our actor-task aware adapter which will dynamically look up @@ -220,11 +289,19 @@ def get_console_log( if not level: return log - log.setLevel(level.upper() if not isinstance(level, int) else level) + log.setLevel( + level.upper() + if not isinstance(level, int) + else level + ) if not any( handler.stream == sys.stderr # type: ignore - for handler in logger.handlers if getattr(handler, 'stream', None) + for handler in logger.handlers if getattr( + handler, + 'stream', + None, + ) ): handler = logging.StreamHandler() formatter = colorlog.ColoredFormatter( @@ -242,3 +319,7 @@ def get_console_log( def get_loglevel() -> str: return _default_loglevel + + +# global module logger for tractor itself +log = get_logger('tractor') diff --git a/tractor/msg/__init__.py b/tractor/msg/__init__.py new file mode 100644 index 00000000..906627cf --- /dev/null +++ b/tractor/msg/__init__.py @@ -0,0 +1,26 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +''' +Built-in messaging patterns, types, APIs and helpers. + +''' +from .ptr import ( + NamespacePath as NamespacePath, +) +from .types import ( + Struct as Struct, +) diff --git a/tractor/msg.py b/tractor/msg/ptr.py similarity index 50% rename from tractor/msg.py rename to tractor/msg/ptr.py index 9af3ccd7..4d089c3e 100644 --- a/tractor/msg.py +++ b/tractor/msg/ptr.py @@ -15,7 +15,7 @@ # along with this program. If not, see . ''' -Built-in messaging patterns, types, APIs and helpers. +IPC-compat cross-mem-boundary object pointer. ''' @@ -43,38 +43,92 @@ Built-in messaging patterns, types, APIs and helpers. # - https://github.com/msgpack/msgpack-python#packingunpacking-of-custom-data-type from __future__ import annotations +from inspect import ( + isfunction, + ismethod, +) from pkgutil import resolve_name class NamespacePath(str): ''' - A serializeable description of a (function) Python object location - described by the target's module path and namespace key meant as - a message-native "packet" to allows actors to point-and-load objects - by absolute reference. + A serializeable `str`-subtype implementing a "namespace + pointer" to any Python object reference (like a function) + using the same format as the built-in `pkgutil.resolve_name()` + system. + + A value describes a target's module-path and namespace-key + separated by a ':' and thus can be easily used as + a IPC-message-native reference-type allowing memory isolated + actors to point-and-load objects via a minimal `str` value. ''' - _ref: object = None + _ref: object | type | None = None - def load_ref(self) -> object: + # TODO: support providing the ns instance in + # order to support 'self.` style to make + # `Portal.run_from_ns()` work! + # _ns: ModuleType|type|None = None + + def load_ref(self) -> object | type: if self._ref is None: self._ref = resolve_name(self) return self._ref - def to_tuple( - self, + @staticmethod + def _mk_fqnp(ref: type | object) -> tuple[str, str]: + ''' + Generate a minial ``str`` pair which describes a python + object's namespace path and object/type name. - ) -> tuple[str, str]: - ref = self.load_ref() - return ref.__module__, getattr(ref, '__name__', '') + In more precise terms something like: + - 'py.namespace.path:object_name', + - eg.'tractor.msg:NamespacePath' will be the ``str`` form + of THIS type XD + + ''' + if ( + isfunction(ref) + ): + name: str = getattr(ref, '__name__') + + elif ismethod(ref): + # build out the path manually i guess..? + # TODO: better way? + name: str = '.'.join([ + type(ref.__self__).__name__, + ref.__func__.__name__, + ]) + + else: # object or other? + # isinstance(ref, object) + # and not isfunction(ref) + name: str = type(ref).__name__ + + # fully qualified namespace path, tuple. + fqnp: tuple[str, str] = ( + ref.__module__, + name, + ) + return fqnp @classmethod def from_ref( cls, - ref, + ref: type | object, ) -> NamespacePath: - return cls(':'.join( - (ref.__module__, - getattr(ref, '__name__', '')) - )) + + fqnp: tuple[str, str] = cls._mk_fqnp(ref) + return cls(':'.join(fqnp)) + + def to_tuple( + self, + + # TODO: could this work re `self:` case from above? + # load_ref: bool = True, + + ) -> tuple[str, str]: + return self._mk_fqnp( + self.load_ref() + ) diff --git a/tractor/msg/types.py b/tractor/msg/types.py new file mode 100644 index 00000000..3ceff845 --- /dev/null +++ b/tractor/msg/types.py @@ -0,0 +1,270 @@ +# tractor: structured concurrent "actors". +# Copyright 2018-eternity Tyler Goodlet. + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +''' +Extensions to built-in or (heavily used but 3rd party) friend-lib +types. + +''' +from __future__ import annotations +from collections import UserList +from pprint import ( + saferepr, +) +from typing import ( + Any, + Iterator, +) + +from msgspec import ( + msgpack, + Struct as _Struct, + structs, +) + +# TODO: auto-gen type sig for input func both for +# type-msgs and logging of RPC tasks? +# taken and modified from: +# https://stackoverflow.com/a/57110117 +# import inspect +# from typing import List + +# def my_function(input_1: str, input_2: int) -> list[int]: +# pass + +# def types_of(func): +# specs = inspect.getfullargspec(func) +# return_type = specs.annotations['return'] +# input_types = [t.__name__ for s, t in specs.annotations.items() if s != 'return'] +# return f'{func.__name__}({": ".join(input_types)}) -> {return_type}' + +# types_of(my_function) + + +class DiffDump(UserList): + ''' + Very simple list delegator that repr() dumps (presumed) tuple + elements of the form `tuple[str, Any, Any]` in a nice + multi-line readable form for analyzing `Struct` diffs. + + ''' + def __repr__(self) -> str: + if not len(self): + return super().__repr__() + + # format by displaying item pair's ``repr()`` on multiple, + # indented lines such that they are more easily visually + # comparable when printed to console when printed to + # console. + repstr: str = '[\n' + for k, left, right in self: + repstr += ( + f'({k},\n' + f'\t{repr(left)},\n' + f'\t{repr(right)},\n' + ')\n' + ) + repstr += ']\n' + return repstr + + +class Struct( + _Struct, + + # https://jcristharif.com/msgspec/structs.html#tagged-unions + # tag='pikerstruct', + # tag=True, +): + ''' + A "human friendlier" (aka repl buddy) struct subtype. + + ''' + def _sin_props(self) -> Iterator[ + tuple[ + structs.FieldIinfo, + str, + Any, + ] + ]: + ''' + Iterate over all non-@property fields of this struct. + + ''' + fi: structs.FieldInfo + for fi in structs.fields(self): + key: str = fi.name + val: Any = getattr(self, key) + yield fi, key, val + + def to_dict( + self, + include_non_members: bool = True, + + ) -> dict: + ''' + Like it sounds.. direct delegation to: + https://jcristharif.com/msgspec/api.html#msgspec.structs.asdict + + BUT, by default we pop all non-member (aka not defined as + struct fields) fields by default. + + ''' + asdict: dict = structs.asdict(self) + if include_non_members: + return asdict + + # only return a dict of the struct members + # which were provided as input, NOT anything + # added as type-defined `@property` methods! + sin_props: dict = {} + fi: structs.FieldInfo + for fi, k, v in self._sin_props(): + sin_props[k] = asdict[k] + + return sin_props + + def pformat( + self, + field_indent: int = 2, + indent: int = 0, + + ) -> str: + ''' + Recursion-safe `pprint.pformat()` style formatting of + a `msgspec.Struct` for sane reading by a human using a REPL. + + ''' + # global whitespace indent + ws: str = ' '*indent + + # field whitespace indent + field_ws: str = ' '*(field_indent + indent) + + # qtn: str = ws + self.__class__.__qualname__ + qtn: str = self.__class__.__qualname__ + + obj_str: str = '' # accumulator + fi: structs.FieldInfo + k: str + v: Any + for fi, k, v in self._sin_props(): + + # TODO: how can we prefer `Literal['option1', 'option2, + # ..]` over .__name__ == `Literal` but still get only the + # latter for simple types like `str | int | None` etc..? + ft: type = fi.type + typ_name: str = getattr(ft, '__name__', str(ft)) + + # recurse to get sub-struct's `.pformat()` output Bo + if isinstance(v, Struct): + val_str: str = v.pformat( + indent=field_indent + indent, + field_indent=indent + field_indent, + ) + + else: # the `pprint` recursion-safe format: + # https://docs.python.org/3.11/library/pprint.html#pprint.saferepr + val_str: str = saferepr(v) + + # TODO: LOLOL use `textwrap.indent()` instead dawwwwwg! + obj_str += (field_ws + f'{k}: {typ_name} = {val_str},\n') + + return ( + f'{qtn}(\n' + f'{obj_str}' + f'{ws})' + ) + + # TODO: use a pprint.PrettyPrinter instance around ONLY rendering + # inside a known tty? + # def __repr__(self) -> str: + # ... + + # __str__ = __repr__ = pformat + __repr__ = pformat + + def copy( + self, + update: dict | None = None, + + ) -> Struct: + ''' + Validate-typecast all self defined fields, return a copy of + us with all such fields. + + NOTE: This is kinda like the default behaviour in + `pydantic.BaseModel` except a copy of the object is + returned making it compat with `frozen=True`. + + ''' + if update: + for k, v in update.items(): + setattr(self, k, v) + + # NOTE: roundtrip serialize to validate + # - enode to msgpack binary format, + # - decode that back to a struct. + return msgpack.Decoder(type=type(self)).decode( + msgpack.Encoder().encode(self) + ) + + def typecast( + self, + + # TODO: allow only casting a named subset? + # fields: set[str] | None = None, + + ) -> None: + ''' + Cast all fields using their declared type annotations + (kinda like what `pydantic` does by default). + + NOTE: this of course won't work on frozen types, use + ``.copy()`` above in such cases. + + ''' + # https://jcristharif.com/msgspec/api.html#msgspec.structs.fields + fi: structs.FieldInfo + for fi in structs.fields(self): + setattr( + self, + fi.name, + fi.type(getattr(self, fi.name)), + ) + + def __sub__( + self, + other: Struct, + + ) -> DiffDump[tuple[str, Any, Any]]: + ''' + Compare fields/items key-wise and return a ``DiffDump`` + for easy visual REPL comparison B) + + ''' + diffs: DiffDump[tuple[str, Any, Any]] = DiffDump() + for fi in structs.fields(self): + attr_name: str = fi.name + ours: Any = getattr(self, attr_name) + theirs: Any = getattr(other, attr_name) + if ours != theirs: + diffs.append(( + attr_name, + ours, + theirs, + )) + + return diffs diff --git a/tractor/to_asyncio.py b/tractor/to_asyncio.py index be3ac8d3..174a99d3 100644 --- a/tractor/to_asyncio.py +++ b/tractor/to_asyncio.py @@ -217,7 +217,14 @@ def _run_asyncio_task( try: result = await coro except BaseException as aio_err: - log.exception('asyncio task errored') + if isinstance(aio_err, CancelledError): + log.runtime( + '`asyncio` task was cancelled..\n' + ) + else: + log.exception( + '`asyncio` task errored\n' + ) chan._aio_err = aio_err raise @@ -272,12 +279,22 @@ def _run_asyncio_task( except BaseException as terr: task_err = terr + msg: str = ( + 'Infected `asyncio` task {etype_str}\n' + f'|_{task}\n' + ) if isinstance(terr, CancelledError): - log.cancel(f'`asyncio` task cancelled: {task.get_name()}') + log.cancel( + msg.format(etype_str='cancelled') + ) else: - log.exception(f'`asyncio` task: {task.get_name()} errored') + log.exception( + msg.format(etype_str='cancelled') + ) - assert type(terr) is type(aio_err), 'Asyncio task error mismatch?' + assert type(terr) is type(aio_err), ( + '`asyncio` task error mismatch?!?' + ) if aio_err is not None: # XXX: uhh is this true? @@ -290,18 +307,22 @@ def _run_asyncio_task( # We might want to change this in the future though. from_aio.close() - if type(aio_err) is CancelledError: - log.cancel("infected task was cancelled") - - # TODO: show that the cancellation originated - # from the ``trio`` side? right? - # if cancel_scope.cancelled: - # raise aio_err from err - - elif task_err is None: + if task_err is None: assert aio_err aio_err.with_traceback(aio_err.__traceback__) - log.error('infected task errorred') + # log.error( + # 'infected task errorred' + # ) + + # TODO: show that the cancellation originated + # from the ``trio`` side? right? + # elif type(aio_err) is CancelledError: + # log.cancel( + # 'infected task was cancelled' + # ) + + # if cancel_scope.cancelled: + # raise aio_err from err # XXX: alway cancel the scope on error # in case the trio task is blocking diff --git a/tractor/trionics/__init__.py b/tractor/trionics/__init__.py index 31e49a9a..c51b7c51 100644 --- a/tractor/trionics/__init__.py +++ b/tractor/trionics/__init__.py @@ -19,22 +19,13 @@ Sugary patterns for trio + tractor designs. ''' from ._mngrs import ( - gather_contexts, - maybe_open_context, - maybe_open_nursery, + gather_contexts as gather_contexts, + maybe_open_context as maybe_open_context, + maybe_open_nursery as maybe_open_nursery, ) from ._broadcast import ( - broadcast_receiver, - BroadcastReceiver, - Lagged, + AsyncReceiver as AsyncReceiver, + broadcast_receiver as broadcast_receiver, + BroadcastReceiver as BroadcastReceiver, + Lagged as Lagged, ) - - -__all__ = [ - 'gather_contexts', - 'broadcast_receiver', - 'BroadcastReceiver', - 'Lagged', - 'maybe_open_context', - 'maybe_open_nursery', -] diff --git a/tractor/trionics/_broadcast.py b/tractor/trionics/_broadcast.py index 42b1704b..a5d31871 100644 --- a/tractor/trionics/_broadcast.py +++ b/tractor/trionics/_broadcast.py @@ -25,8 +25,15 @@ from collections import deque from contextlib import asynccontextmanager from functools import partial from operator import ne -from typing import Optional, Callable, Awaitable, Any, AsyncIterator, Protocol -from typing import Generic, TypeVar +from typing import ( + Callable, + Awaitable, + Any, + AsyncIterator, + Protocol, + Generic, + TypeVar, +) import trio from trio._core._run import Task @@ -37,6 +44,11 @@ from tractor.log import get_logger log = get_logger(__name__) +# TODO: use new type-vars syntax from 3.12 +# https://realpython.com/python312-new-features/#dedicated-type-variable-syntax +# https://docs.python.org/3/whatsnew/3.12.html#whatsnew312-pep695 +# https://docs.python.org/3/reference/simple_stmts.html#type +# # A regular invariant generic type T = TypeVar("T") @@ -102,7 +114,7 @@ class BroadcastState(Struct): # broadcast event to wake up all sleeping consumer tasks # on a newly produced value from the sender. - recv_ready: Optional[tuple[int, trio.Event]] = None + recv_ready: tuple[int, trio.Event]|None = None # if a ``trio.EndOfChannel`` is received on any # consumer all consumers should be placed in this state @@ -156,7 +168,7 @@ class BroadcastReceiver(ReceiveChannel): rx_chan: AsyncReceiver, state: BroadcastState, - receive_afunc: Optional[Callable[[], Awaitable[Any]]] = None, + receive_afunc: Callable[[], Awaitable[Any]]|None = None, raise_on_lag: bool = True, ) -> None: @@ -444,7 +456,7 @@ def broadcast_receiver( recv_chan: AsyncReceiver, max_buffer_size: int, - receive_afunc: Optional[Callable[[], Awaitable[Any]]] = None, + receive_afunc: Callable[[], Awaitable[Any]]|None = None, raise_on_lag: bool = True, ) -> BroadcastReceiver: diff --git a/tractor/trionics/_mngrs.py b/tractor/trionics/_mngrs.py index 5621f79d..f57be0a7 100644 --- a/tractor/trionics/_mngrs.py +++ b/tractor/trionics/_mngrs.py @@ -33,10 +33,9 @@ from typing import ( ) import trio -from trio_typing import TaskStatus -from .._state import current_actor -from ..log import get_logger +from tractor._state import current_actor +from tractor.log import get_logger log = get_logger(__name__) @@ -70,6 +69,7 @@ async def _enter_and_wait( unwrapped: dict[int, T], all_entered: trio.Event, parent_exit: trio.Event, + seed: int, ) -> None: ''' @@ -80,7 +80,10 @@ async def _enter_and_wait( async with mngr as value: unwrapped[id(mngr)] = value - if all(unwrapped.values()): + if all( + val != seed + for val in unwrapped.values() + ): all_entered.set() await parent_exit.wait() @@ -91,7 +94,13 @@ async def gather_contexts( mngrs: Sequence[AsyncContextManager[T]], -) -> AsyncGenerator[tuple[Optional[T], ...], None]: +) -> AsyncGenerator[ + tuple[ + T | None, + ... + ], + None, +]: ''' Concurrently enter a sequence of async context managers, each in a separate ``trio`` task and deliver the unwrapped values in the @@ -104,7 +113,11 @@ async def gather_contexts( entered and exited, and cancellation just works. ''' - unwrapped: dict[int, Optional[T]] = {}.fromkeys(id(mngr) for mngr in mngrs) + seed: int = id(mngrs) + unwrapped: dict[int, T | None] = {}.fromkeys( + (id(mngr) for mngr in mngrs), + seed, + ) all_entered = trio.Event() parent_exit = trio.Event() @@ -116,8 +129,9 @@ async def gather_contexts( if not mngrs: raise ValueError( - 'input mngrs is empty?\n' - 'Did try to use inline generator syntax?' + '`.trionics.gather_contexts()` input mngrs is empty?\n' + 'Did try to use inline generator syntax?\n' + 'Use a non-lazy iterator or sequence type intead!' ) async with trio.open_nursery() as n: @@ -128,6 +142,7 @@ async def gather_contexts( unwrapped, all_entered, parent_exit, + seed, ) # deliver control once all managers have started up @@ -168,7 +183,7 @@ class _Cache: cls, mng, ctx_key: tuple, - task_status: TaskStatus[T] = trio.TASK_STATUS_IGNORED, + task_status: trio.TaskStatus[T] = trio.TASK_STATUS_IGNORED, ) -> None: async with mng as value: @@ -209,6 +224,7 @@ async def maybe_open_context( # yielded output yielded: Any = None + lock_registered: bool = False # Lock resource acquisition around task racing / ``trio``'s # scheduler protocol. @@ -216,6 +232,7 @@ async def maybe_open_context( # to allow re-entrant use cases where one `maybe_open_context()` # wrapped factor may want to call into another. lock = _Cache.locks.setdefault(fid, trio.Lock()) + lock_registered: bool = True await lock.acquire() # XXX: one singleton nursery per actor and we want to @@ -237,7 +254,7 @@ async def maybe_open_context( yielded = _Cache.values[ctx_key] except KeyError: - log.info(f'Allocating new {acm_func} for {ctx_key}') + log.debug(f'Allocating new {acm_func} for {ctx_key}') mngr = acm_func(**kwargs) resources = _Cache.resources assert not resources.get(ctx_key), f'Resource exists? {ctx_key}' @@ -265,7 +282,7 @@ async def maybe_open_context( if yielded is not None: # if no more consumers, teardown the client if _Cache.users <= 0: - log.info(f'De-allocating resource for {ctx_key}') + log.debug(f'De-allocating resource for {ctx_key}') # XXX: if we're cancelled we the entry may have never # been entered since the nursery task was killed. @@ -275,4 +292,9 @@ async def maybe_open_context( _, no_more_users = entry no_more_users.set() - _Cache.locks.pop(fid) + if lock_registered: + maybe_lock = _Cache.locks.pop(fid, None) + if maybe_lock is None: + log.error( + f'Resource lock for {fid} ALREADY POPPED?' + )