commit
49a02e6700
|
@ -17,10 +17,13 @@ async def name_error():
|
||||||
async def main():
|
async def main():
|
||||||
"""Test breakpoint in a streaming actor.
|
"""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__])
|
p0 = await n.start_actor('bp_forever', enable_modules=[__name__])
|
||||||
p1 = await n.start_actor('name_error', rpc_module_paths=[__name__])
|
p1 = await n.start_actor('name_error', enable_modules=[__name__])
|
||||||
|
|
||||||
# retreive results
|
# retreive results
|
||||||
stream = await p0.run(breakpoint_forever)
|
stream = await p0.run(breakpoint_forever)
|
||||||
|
@ -28,4 +31,4 @@ async def main():
|
||||||
|
|
||||||
|
|
||||||
if __name__ == '__main__':
|
if __name__ == '__main__':
|
||||||
tractor.run(main, debug_mode=True, loglevel='error')
|
trio.run(main)
|
||||||
|
|
|
@ -1,3 +1,4 @@
|
||||||
|
import trio
|
||||||
import tractor
|
import tractor
|
||||||
|
|
||||||
|
|
||||||
|
@ -50,7 +51,9 @@ async def main():
|
||||||
└─ python -m tractor._child --uid ('spawn_until_0', 'de918e6d ...)
|
└─ 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
|
# spawn both actors
|
||||||
portal = await n.run_in_actor(
|
portal = await n.run_in_actor(
|
||||||
|
@ -70,4 +73,4 @@ async def main():
|
||||||
|
|
||||||
|
|
||||||
if __name__ == '__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())
|
before = str(child.before.decode())
|
||||||
|
|
||||||
# make sure expected logging and error arrives
|
# 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
|
assert 'AssertionError' in before
|
||||||
|
|
||||||
# send user command
|
# send user command
|
||||||
|
@ -170,7 +170,7 @@ def test_subactor_error(spawn):
|
||||||
before = str(child.before.decode())
|
before = str(child.before.decode())
|
||||||
|
|
||||||
# root actor gets debugger engaged
|
# 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
|
# error is a remote error propagated from the subactor
|
||||||
assert "RemoteActorError: ('name_error'" in before
|
assert "RemoteActorError: ('name_error'" in before
|
||||||
|
@ -276,7 +276,7 @@ def test_multi_subactors(spawn):
|
||||||
child.sendline('q')
|
child.sendline('q')
|
||||||
child.expect(r"\(Pdb\+\+\)")
|
child.expect(r"\(Pdb\+\+\)")
|
||||||
before = str(child.before.decode())
|
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 "RemoteActorError: ('breakpoint_forever'" in before
|
||||||
assert 'bdb.BdbQuit' in before
|
assert 'bdb.BdbQuit' in before
|
||||||
|
|
||||||
|
@ -323,7 +323,6 @@ def test_multi_daemon_subactors(spawn, loglevel):
|
||||||
child.expect(pexpect.EOF)
|
child.expect(pexpect.EOF)
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
def test_multi_subactors_root_errors(spawn):
|
def test_multi_subactors_root_errors(spawn):
|
||||||
"""Multiple subactors, both erroring and breakpointing as well as
|
"""Multiple subactors, both erroring and breakpointing as well as
|
||||||
a nested subactor erroring.
|
a nested subactor erroring.
|
||||||
|
@ -345,7 +344,7 @@ def test_multi_subactors_root_errors(spawn):
|
||||||
before = str(child.before.decode())
|
before = str(child.before.decode())
|
||||||
|
|
||||||
# should have come just after priot prompt
|
# 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
|
assert "AssertionError" in before
|
||||||
|
|
||||||
# warnings assert we probably don't need
|
# warnings assert we probably don't need
|
||||||
|
@ -402,7 +401,10 @@ def test_multi_nested_subactors_error_through_nurseries(spawn):
|
||||||
assert "NameError" in before
|
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
|
"""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
|
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.
|
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')
|
child.sendline('c')
|
||||||
|
|
||||||
|
|
||||||
for i in range(4):
|
for i in range(4):
|
||||||
time.sleep(0.5)
|
time.sleep(0.5)
|
||||||
try:
|
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?
|
# race conditions on how fast the continue is sent?
|
||||||
break
|
break
|
||||||
|
|
||||||
|
|
||||||
before = str(child.before.decode())
|
before = str(child.before.decode())
|
||||||
assert "NameError: name 'doggypants' is not defined" in before
|
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:
|
async with tractor.open_nursery() as n:
|
||||||
|
|
||||||
name = 'arbiter'
|
name = 'root'
|
||||||
|
|
||||||
if pub_actor == 'streamer':
|
if pub_actor == 'streamer':
|
||||||
# start the publisher as a daemon
|
# 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
|
exposed_mods, funcname, inside_err = to_call
|
||||||
subactor_exposed_mods = []
|
subactor_exposed_mods = []
|
||||||
func_defined = globals().get(funcname, False)
|
func_defined = globals().get(funcname, False)
|
||||||
subactor_requests_to = 'arbiter'
|
subactor_requests_to = 'root'
|
||||||
remote_err = tractor.RemoteActorError
|
remote_err = tractor.RemoteActorError
|
||||||
|
|
||||||
# remote module that fails at import time
|
# remote module that fails at import time
|
||||||
|
|
|
@ -164,7 +164,7 @@ async def open_root_actor(
|
||||||
)
|
)
|
||||||
try:
|
try:
|
||||||
yield actor
|
yield actor
|
||||||
# result = await main()
|
|
||||||
except (Exception, trio.MultiError) as err:
|
except (Exception, trio.MultiError) as err:
|
||||||
logger.exception("Actor crashed:")
|
logger.exception("Actor crashed:")
|
||||||
await _debug._maybe_enter_pm(err)
|
await _debug._maybe_enter_pm(err)
|
||||||
|
@ -185,7 +185,7 @@ def run(
|
||||||
*args,
|
*args,
|
||||||
|
|
||||||
# runtime kwargs
|
# runtime kwargs
|
||||||
name: Optional[str] = None,
|
name: Optional[str] = 'root',
|
||||||
arbiter_addr: Tuple[str, int] = (
|
arbiter_addr: Tuple[str, int] = (
|
||||||
_default_arbiter_host,
|
_default_arbiter_host,
|
||||||
_default_arbiter_port,
|
_default_arbiter_port,
|
||||||
|
|
|
@ -5,7 +5,6 @@ from functools import partial
|
||||||
import multiprocessing as mp
|
import multiprocessing as mp
|
||||||
from typing import Tuple, List, Dict, Optional
|
from typing import Tuple, List, Dict, Optional
|
||||||
import typing
|
import typing
|
||||||
from contextlib import AsyncExitStack
|
|
||||||
import warnings
|
import warnings
|
||||||
|
|
||||||
import trio
|
import trio
|
||||||
|
@ -201,38 +200,9 @@ class ActorNursery:
|
||||||
|
|
||||||
|
|
||||||
@asynccontextmanager
|
@asynccontextmanager
|
||||||
async def open_nursery(
|
async def _open_and_supervise_one_cancels_all_nursery(
|
||||||
**kwargs,
|
actor: Actor,
|
||||||
) -> typing.AsyncGenerator[ActorNursery, None]:
|
) -> 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
|
# the collection of errors retreived from spawned sub-actors
|
||||||
errors: Dict[Tuple[str, str], Exception] = {}
|
errors: Dict[Tuple[str, str], Exception] = {}
|
||||||
|
@ -243,7 +213,8 @@ async def open_nursery(
|
||||||
# a supervisor strategy **before** blocking indefinitely to wait for
|
# a supervisor strategy **before** blocking indefinitely to wait for
|
||||||
# actors spawned in "daemon mode" (aka started using
|
# actors spawned in "daemon mode" (aka started using
|
||||||
# ``ActorNursery.start_actor()``).
|
# ``ActorNursery.start_actor()``).
|
||||||
try:
|
|
||||||
|
# errors from this daemon actor nursery bubble up to caller
|
||||||
async with trio.open_nursery() as da_nursery:
|
async with trio.open_nursery() as da_nursery:
|
||||||
try:
|
try:
|
||||||
# This is the inner level "run in actor" nursery. It is
|
# 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
|
# ria_nursery scope end - nursery checkpoint
|
||||||
|
|
||||||
# after nursery exit
|
# 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:
|
finally:
|
||||||
log.debug("Nursery teardown complete")
|
log.debug("Nursery teardown complete")
|
||||||
|
|
||||||
# shutdown runtime if it was started
|
# shutdown runtime if it was started
|
||||||
if implicit_runtime:
|
if implicit_runtime:
|
||||||
log.info("Shutting down actor tree")
|
log.info("Shutting down actor tree")
|
||||||
await root_runtime_stack.aclose()
|
|
||||||
|
|
Loading…
Reference in New Issue