forked from goodboy/tractor
				
			
						commit
						49a02e6700
					
				| 
						 | 
				
			
			@ -17,10 +17,13 @@ async def name_error():
 | 
			
		|||
async def main():
 | 
			
		||||
    """Test breakpoint in a streaming actor.
 | 
			
		||||
    """
 | 
			
		||||
    async with tractor.open_nursery() as n:
 | 
			
		||||
    async with tractor.open_nursery(
 | 
			
		||||
        debug_mode=True,
 | 
			
		||||
        loglevel='error',
 | 
			
		||||
    ) as n:
 | 
			
		||||
 | 
			
		||||
        p0 = await n.start_actor('bp_forever', rpc_module_paths=[__name__])
 | 
			
		||||
        p1 = await n.start_actor('name_error', rpc_module_paths=[__name__])
 | 
			
		||||
        p0 = await n.start_actor('bp_forever', enable_modules=[__name__])
 | 
			
		||||
        p1 = await n.start_actor('name_error', enable_modules=[__name__])
 | 
			
		||||
 | 
			
		||||
        # retreive results
 | 
			
		||||
        stream = await p0.run(breakpoint_forever)
 | 
			
		||||
| 
						 | 
				
			
			@ -28,4 +31,4 @@ async def main():
 | 
			
		|||
 | 
			
		||||
 | 
			
		||||
if __name__ == '__main__':
 | 
			
		||||
    tractor.run(main, debug_mode=True, loglevel='error')
 | 
			
		||||
    trio.run(main)
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -1,3 +1,4 @@
 | 
			
		|||
import trio
 | 
			
		||||
import tractor
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -50,7 +51,9 @@ async def main():
 | 
			
		|||
             └─ python -m tractor._child --uid ('spawn_until_0', 'de918e6d ...)
 | 
			
		||||
 | 
			
		||||
    """
 | 
			
		||||
    async with tractor.open_nursery() as n:
 | 
			
		||||
    async with tractor.open_nursery(
 | 
			
		||||
        debug_mode=True,
 | 
			
		||||
    ) as n:
 | 
			
		||||
 | 
			
		||||
        # spawn both actors
 | 
			
		||||
        portal = await n.run_in_actor(
 | 
			
		||||
| 
						 | 
				
			
			@ -70,4 +73,4 @@ async def main():
 | 
			
		|||
 | 
			
		||||
 | 
			
		||||
if __name__ == '__main__':
 | 
			
		||||
    tractor.run(main, debug_mode=True)
 | 
			
		||||
    trio.run(main)
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -82,7 +82,7 @@ def test_root_actor_error(spawn, user_in_out):
 | 
			
		|||
    before = str(child.before.decode())
 | 
			
		||||
 | 
			
		||||
    # make sure expected logging and error arrives
 | 
			
		||||
    assert "Attaching to pdb in crashed actor: ('arbiter'" in before
 | 
			
		||||
    assert "Attaching to pdb in crashed actor: ('root'" in before
 | 
			
		||||
    assert 'AssertionError' in before
 | 
			
		||||
 | 
			
		||||
    # send user command
 | 
			
		||||
| 
						 | 
				
			
			@ -170,7 +170,7 @@ def test_subactor_error(spawn):
 | 
			
		|||
    before = str(child.before.decode())
 | 
			
		||||
 | 
			
		||||
    # root actor gets debugger engaged
 | 
			
		||||
    assert "Attaching to pdb in crashed actor: ('arbiter'" in before
 | 
			
		||||
    assert "Attaching to pdb in crashed actor: ('root'" in before
 | 
			
		||||
 | 
			
		||||
    # error is a remote error propagated from the subactor
 | 
			
		||||
    assert "RemoteActorError: ('name_error'" in before
 | 
			
		||||
| 
						 | 
				
			
			@ -276,7 +276,7 @@ def test_multi_subactors(spawn):
 | 
			
		|||
    child.sendline('q')
 | 
			
		||||
    child.expect(r"\(Pdb\+\+\)")
 | 
			
		||||
    before = str(child.before.decode())
 | 
			
		||||
    assert "Attaching to pdb in crashed actor: ('arbiter'" in before
 | 
			
		||||
    assert "Attaching to pdb in crashed actor: ('root'" in before
 | 
			
		||||
    assert "RemoteActorError: ('breakpoint_forever'" in before
 | 
			
		||||
    assert 'bdb.BdbQuit' in before
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -323,7 +323,6 @@ def test_multi_daemon_subactors(spawn, loglevel):
 | 
			
		|||
        child.expect(pexpect.EOF)
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
def test_multi_subactors_root_errors(spawn):
 | 
			
		||||
    """Multiple subactors, both erroring and breakpointing as well as
 | 
			
		||||
    a nested subactor erroring.
 | 
			
		||||
| 
						 | 
				
			
			@ -345,7 +344,7 @@ def test_multi_subactors_root_errors(spawn):
 | 
			
		|||
    before = str(child.before.decode())
 | 
			
		||||
 | 
			
		||||
    # should have come just after priot prompt
 | 
			
		||||
    assert "Attaching to pdb in crashed actor: ('arbiter'" in before
 | 
			
		||||
    assert "Attaching to pdb in crashed actor: ('root'" in before
 | 
			
		||||
    assert "AssertionError" in before
 | 
			
		||||
 | 
			
		||||
    # warnings assert we probably don't need
 | 
			
		||||
| 
						 | 
				
			
			@ -402,7 +401,10 @@ def test_multi_nested_subactors_error_through_nurseries(spawn):
 | 
			
		|||
        assert "NameError" in before
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
def test_root_nursery_cancels_before_child_releases_tty_lock(spawn, start_method):
 | 
			
		||||
def test_root_nursery_cancels_before_child_releases_tty_lock(
 | 
			
		||||
    spawn,
 | 
			
		||||
    start_method
 | 
			
		||||
):
 | 
			
		||||
    """Test that when the root sends a cancel message before a nested
 | 
			
		||||
    child has unblocked (which can happen when it has the tty lock and
 | 
			
		||||
    is engaged in pdb) it is indeed cancelled after exiting the debugger.
 | 
			
		||||
| 
						 | 
				
			
			@ -420,7 +422,6 @@ def test_root_nursery_cancels_before_child_releases_tty_lock(spawn, start_method
 | 
			
		|||
 | 
			
		||||
    child.sendline('c')
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
    for i in range(4):
 | 
			
		||||
        time.sleep(0.5)
 | 
			
		||||
        try:
 | 
			
		||||
| 
						 | 
				
			
			@ -440,7 +441,6 @@ def test_root_nursery_cancels_before_child_releases_tty_lock(spawn, start_method
 | 
			
		|||
            # race conditions on how fast the continue is sent?
 | 
			
		||||
            break
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
        before = str(child.before.decode())
 | 
			
		||||
        assert "NameError: name 'doggypants' is not defined" in before
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -161,7 +161,7 @@ def test_multi_actor_subs_arbiter_pub(
 | 
			
		|||
 | 
			
		||||
        async with tractor.open_nursery() as n:
 | 
			
		||||
 | 
			
		||||
            name = 'arbiter'
 | 
			
		||||
            name = 'root'
 | 
			
		||||
 | 
			
		||||
            if pub_actor == 'streamer':
 | 
			
		||||
                # start the publisher as a daemon
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -53,7 +53,7 @@ def test_rpc_errors(arb_addr, to_call, testdir):
 | 
			
		|||
    exposed_mods, funcname, inside_err = to_call
 | 
			
		||||
    subactor_exposed_mods = []
 | 
			
		||||
    func_defined = globals().get(funcname, False)
 | 
			
		||||
    subactor_requests_to = 'arbiter'
 | 
			
		||||
    subactor_requests_to = 'root'
 | 
			
		||||
    remote_err = tractor.RemoteActorError
 | 
			
		||||
 | 
			
		||||
    # remote module that fails at import time
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -164,7 +164,7 @@ async def open_root_actor(
 | 
			
		|||
            )
 | 
			
		||||
            try:
 | 
			
		||||
                yield actor
 | 
			
		||||
                # result = await main()
 | 
			
		||||
 | 
			
		||||
            except (Exception, trio.MultiError) as err:
 | 
			
		||||
                logger.exception("Actor crashed:")
 | 
			
		||||
                await _debug._maybe_enter_pm(err)
 | 
			
		||||
| 
						 | 
				
			
			@ -185,7 +185,7 @@ def run(
 | 
			
		|||
    *args,
 | 
			
		||||
 | 
			
		||||
    # runtime kwargs
 | 
			
		||||
    name: Optional[str] = None,
 | 
			
		||||
    name: Optional[str] = 'root',
 | 
			
		||||
    arbiter_addr: Tuple[str, int] = (
 | 
			
		||||
        _default_arbiter_host,
 | 
			
		||||
        _default_arbiter_port,
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -5,7 +5,6 @@ from functools import partial
 | 
			
		|||
import multiprocessing as mp
 | 
			
		||||
from typing import Tuple, List, Dict, Optional
 | 
			
		||||
import typing
 | 
			
		||||
from contextlib import AsyncExitStack
 | 
			
		||||
import warnings
 | 
			
		||||
 | 
			
		||||
import trio
 | 
			
		||||
| 
						 | 
				
			
			@ -201,38 +200,9 @@ class ActorNursery:
 | 
			
		|||
 | 
			
		||||
 | 
			
		||||
@asynccontextmanager
 | 
			
		||||
async def open_nursery(
 | 
			
		||||
    **kwargs,
 | 
			
		||||
async def _open_and_supervise_one_cancels_all_nursery(
 | 
			
		||||
    actor: Actor,
 | 
			
		||||
) -> typing.AsyncGenerator[ActorNursery, None]:
 | 
			
		||||
    """Create and yield a new ``ActorNursery`` to be used for spawning
 | 
			
		||||
    structured concurrent subactors.
 | 
			
		||||
 | 
			
		||||
    When an actor is spawned a new trio task is started which
 | 
			
		||||
    invokes one of the process spawning backends to create and start
 | 
			
		||||
    a new subprocess. These tasks are started by one of two nurseries
 | 
			
		||||
    detailed below. The reason for spawning processes from within
 | 
			
		||||
    a new task is because ``trio_run_in_process`` itself creates a new
 | 
			
		||||
    internal nursery and the same task that opens a nursery **must**
 | 
			
		||||
    close it. It turns out this approach is probably more correct
 | 
			
		||||
    anyway since it is more clear from the following nested nurseries
 | 
			
		||||
    which cancellation scopes correspond to each spawned subactor set.
 | 
			
		||||
    """
 | 
			
		||||
    implicit_runtime = False
 | 
			
		||||
 | 
			
		||||
    actor = current_actor(err_on_no_runtime=False)
 | 
			
		||||
 | 
			
		||||
    if actor is None and is_main_process():
 | 
			
		||||
 | 
			
		||||
        # if we are the parent process start the actor runtime implicitly
 | 
			
		||||
        log.info("Starting actor runtime!")
 | 
			
		||||
        root_runtime_stack = AsyncExitStack()
 | 
			
		||||
        actor = await root_runtime_stack.enter_async_context(
 | 
			
		||||
            open_root_actor(**kwargs)
 | 
			
		||||
        )
 | 
			
		||||
        assert actor is current_actor()
 | 
			
		||||
 | 
			
		||||
        # mark us for teardown on exit
 | 
			
		||||
        implicit_runtime = True
 | 
			
		||||
 | 
			
		||||
    # the collection of errors retreived from spawned sub-actors
 | 
			
		||||
    errors: Dict[Tuple[str, str], Exception] = {}
 | 
			
		||||
| 
						 | 
				
			
			@ -243,7 +213,8 @@ async def open_nursery(
 | 
			
		|||
    # a supervisor strategy **before** blocking indefinitely to wait for
 | 
			
		||||
    # actors spawned in "daemon mode" (aka started using
 | 
			
		||||
    # ``ActorNursery.start_actor()``).
 | 
			
		||||
    try:
 | 
			
		||||
 | 
			
		||||
    # errors from this daemon actor nursery bubble up to caller
 | 
			
		||||
    async with trio.open_nursery() as da_nursery:
 | 
			
		||||
        try:
 | 
			
		||||
            # This is the inner level "run in actor" nursery. It is
 | 
			
		||||
| 
						 | 
				
			
			@ -348,10 +319,58 @@ async def open_nursery(
 | 
			
		|||
        # ria_nursery scope end - nursery checkpoint
 | 
			
		||||
 | 
			
		||||
    # after nursery exit
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
@asynccontextmanager
 | 
			
		||||
async def open_nursery(
 | 
			
		||||
    **kwargs,
 | 
			
		||||
) -> typing.AsyncGenerator[ActorNursery, None]:
 | 
			
		||||
    """Create and yield a new ``ActorNursery`` to be used for spawning
 | 
			
		||||
    structured concurrent subactors.
 | 
			
		||||
 | 
			
		||||
    When an actor is spawned a new trio task is started which
 | 
			
		||||
    invokes one of the process spawning backends to create and start
 | 
			
		||||
    a new subprocess. These tasks are started by one of two nurseries
 | 
			
		||||
    detailed below. The reason for spawning processes from within
 | 
			
		||||
    a new task is because ``trio_run_in_process`` itself creates a new
 | 
			
		||||
    internal nursery and the same task that opens a nursery **must**
 | 
			
		||||
    close it. It turns out this approach is probably more correct
 | 
			
		||||
    anyway since it is more clear from the following nested nurseries
 | 
			
		||||
    which cancellation scopes correspond to each spawned subactor set.
 | 
			
		||||
    """
 | 
			
		||||
    implicit_runtime = False
 | 
			
		||||
 | 
			
		||||
    actor = current_actor(err_on_no_runtime=False)
 | 
			
		||||
 | 
			
		||||
    try:
 | 
			
		||||
        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
 | 
			
		||||
 | 
			
		||||
            async with open_root_actor(**kwargs) as actor:
 | 
			
		||||
                assert actor is current_actor()
 | 
			
		||||
 | 
			
		||||
                async with _open_and_supervise_one_cancels_all_nursery(
 | 
			
		||||
                    actor
 | 
			
		||||
                ) as anursery:
 | 
			
		||||
 | 
			
		||||
                    yield anursery
 | 
			
		||||
 | 
			
		||||
        else:  # sub-nursery case
 | 
			
		||||
 | 
			
		||||
            async with _open_and_supervise_one_cancels_all_nursery(
 | 
			
		||||
                actor
 | 
			
		||||
            ) as anursery:
 | 
			
		||||
 | 
			
		||||
                yield anursery
 | 
			
		||||
 | 
			
		||||
    finally:
 | 
			
		||||
        log.debug("Nursery teardown complete")
 | 
			
		||||
 | 
			
		||||
        # shutdown runtime if it was started
 | 
			
		||||
        if implicit_runtime:
 | 
			
		||||
            log.info("Shutting down actor tree")
 | 
			
		||||
            await root_runtime_stack.aclose()
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
		Loading…
	
		Reference in New Issue