Compare commits
No commits in common. "e8bd834b5b6ba49ef24c98208ac69318c07750b9" and "3ccbfd7e54e8d60f61b69afd367bfaf7f94da8a6" have entirely different histories.
e8bd834b5b
...
3ccbfd7e54
|
@ -77,9 +77,7 @@ async def main(
|
|||
|
||||
) -> None:
|
||||
|
||||
async with tractor.open_nursery(
|
||||
# debug_mode=True,
|
||||
) as n:
|
||||
async with tractor.open_nursery() as n:
|
||||
|
||||
p = await n.start_actor(
|
||||
'aio_daemon',
|
||||
|
|
|
@ -4,15 +4,9 @@ import trio
|
|||
|
||||
async def breakpoint_forever():
|
||||
"Indefinitely re-enter debugger in child actor."
|
||||
try:
|
||||
while True:
|
||||
yield 'yo'
|
||||
await tractor.breakpoint()
|
||||
except BaseException:
|
||||
tractor.log.get_console_log().exception(
|
||||
'Cancelled while trying to enter pause point!'
|
||||
)
|
||||
raise
|
||||
|
||||
|
||||
async def name_error():
|
||||
|
@ -25,7 +19,7 @@ async def main():
|
|||
"""
|
||||
async with tractor.open_nursery(
|
||||
debug_mode=True,
|
||||
loglevel='cancel',
|
||||
loglevel='error',
|
||||
) as n:
|
||||
|
||||
p0 = await n.start_actor('bp_forever', enable_modules=[__name__])
|
||||
|
@ -38,7 +32,7 @@ async def main():
|
|||
try:
|
||||
await p1.run(name_error)
|
||||
except tractor.RemoteActorError as rae:
|
||||
assert rae.boxed_type is NameError
|
||||
assert rae.type is NameError
|
||||
|
||||
async for i in stream:
|
||||
|
||||
|
|
|
@ -45,7 +45,6 @@ async def spawn_until(depth=0):
|
|||
)
|
||||
|
||||
|
||||
# TODO: notes on the new boxed-relayed errors through proxy actors
|
||||
async def main():
|
||||
"""The main ``tractor`` routine.
|
||||
|
||||
|
|
|
@ -38,7 +38,6 @@ async def main():
|
|||
"""
|
||||
async with tractor.open_nursery(
|
||||
debug_mode=True,
|
||||
# loglevel='runtime',
|
||||
) as n:
|
||||
|
||||
# Spawn both actors, don't bother with collecting results
|
||||
|
|
|
@ -23,6 +23,5 @@ async def main():
|
|||
n.start_soon(debug_actor.run, die)
|
||||
n.start_soon(crash_boi.run, die)
|
||||
|
||||
|
||||
if __name__ == '__main__':
|
||||
trio.run(main)
|
||||
|
|
|
@ -2,13 +2,10 @@ import trio
|
|||
import tractor
|
||||
|
||||
|
||||
async def main(
|
||||
registry_addrs: tuple[str, int]|None = None
|
||||
):
|
||||
async def main():
|
||||
|
||||
async with tractor.open_root_actor(
|
||||
debug_mode=True,
|
||||
# loglevel='runtime',
|
||||
):
|
||||
while True:
|
||||
await tractor.breakpoint()
|
||||
|
|
|
@ -3,20 +3,17 @@ import tractor
|
|||
|
||||
|
||||
async def breakpoint_forever():
|
||||
'''
|
||||
Indefinitely re-enter debugger in child actor.
|
||||
|
||||
'''
|
||||
"""Indefinitely re-enter debugger in child actor.
|
||||
"""
|
||||
while True:
|
||||
await trio.sleep(0.1)
|
||||
await tractor.pause()
|
||||
await tractor.breakpoint()
|
||||
|
||||
|
||||
async def main():
|
||||
|
||||
async with tractor.open_nursery(
|
||||
debug_mode=True,
|
||||
loglevel='cancel',
|
||||
) as n:
|
||||
|
||||
portal = await n.run_in_actor(
|
||||
|
|
|
@ -3,26 +3,16 @@ import tractor
|
|||
|
||||
|
||||
async def name_error():
|
||||
getattr(doggypants) # noqa (on purpose)
|
||||
getattr(doggypants)
|
||||
|
||||
|
||||
async def main():
|
||||
async with tractor.open_nursery(
|
||||
debug_mode=True,
|
||||
# loglevel='transport',
|
||||
) as an:
|
||||
) as n:
|
||||
|
||||
# TODO: ideally the REPL arrives at this frame in the parent,
|
||||
# ABOVE the @api_frame of `Portal.run_in_actor()` (which
|
||||
# should eventually not even be a portal method ... XD)
|
||||
# await tractor.pause()
|
||||
p: tractor.Portal = await an.run_in_actor(name_error)
|
||||
|
||||
# with this style, should raise on this line
|
||||
await p.result()
|
||||
|
||||
# with this alt style should raise at `open_nusery()`
|
||||
# return await p.result()
|
||||
portal = await n.run_in_actor(name_error)
|
||||
await portal.result()
|
||||
|
||||
|
||||
if __name__ == '__main__':
|
||||
|
|
|
@ -1,75 +0,0 @@
|
|||
import trio
|
||||
import tractor
|
||||
|
||||
|
||||
def sync_pause(
|
||||
use_builtin: bool = True,
|
||||
error: bool = False,
|
||||
):
|
||||
if use_builtin:
|
||||
breakpoint(hide_tb=False)
|
||||
|
||||
else:
|
||||
tractor.pause_from_sync()
|
||||
|
||||
if error:
|
||||
raise RuntimeError('yoyo sync code error')
|
||||
|
||||
|
||||
@tractor.context
|
||||
async def start_n_sync_pause(
|
||||
ctx: tractor.Context,
|
||||
):
|
||||
actor: tractor.Actor = tractor.current_actor()
|
||||
|
||||
# sync to parent-side task
|
||||
await ctx.started()
|
||||
|
||||
print(f'entering SYNC PAUSE in {actor.uid}')
|
||||
sync_pause()
|
||||
print(f'back from SYNC PAUSE in {actor.uid}')
|
||||
|
||||
|
||||
async def main() -> None:
|
||||
async with tractor.open_nursery(
|
||||
# NOTE: required for pausing from sync funcs
|
||||
maybe_enable_greenback=True,
|
||||
debug_mode=True,
|
||||
) as an:
|
||||
|
||||
p: tractor.Portal = await an.start_actor(
|
||||
'subactor',
|
||||
enable_modules=[__name__],
|
||||
# infect_asyncio=True,
|
||||
debug_mode=True,
|
||||
loglevel='cancel',
|
||||
)
|
||||
|
||||
# TODO: 3 sub-actor usage cases:
|
||||
# -[ ] via a `.run_in_actor()` call
|
||||
# -[ ] via a `.run()`
|
||||
# -[ ] via a `.open_context()`
|
||||
#
|
||||
async with p.open_context(
|
||||
start_n_sync_pause,
|
||||
) as (ctx, first):
|
||||
assert first is None
|
||||
|
||||
await tractor.pause()
|
||||
sync_pause()
|
||||
|
||||
# TODO: make this work!!
|
||||
await trio.to_thread.run_sync(
|
||||
sync_pause,
|
||||
abandon_on_cancel=False,
|
||||
)
|
||||
|
||||
await ctx.cancel()
|
||||
|
||||
# TODO: case where we cancel from trio-side while asyncio task
|
||||
# has debugger lock?
|
||||
await p.cancel_actor()
|
||||
|
||||
|
||||
if __name__ == '__main__':
|
||||
trio.run(main)
|
|
@ -1,8 +0,0 @@
|
|||
# vim: ft=ini
|
||||
# pytest.ini for tractor
|
||||
|
||||
[pytest]
|
||||
# don't show frickin captured logs AGAIN in the report..
|
||||
addopts = --show-capture='no'
|
||||
log_cli = false
|
||||
; minversion = 6.0
|
|
@ -114,18 +114,12 @@ _reg_addr: tuple[str, int] = (
|
|||
'127.0.0.1',
|
||||
random.randint(1000, 9999),
|
||||
)
|
||||
_arb_addr = _reg_addr
|
||||
|
||||
|
||||
@pytest.fixture(scope='session')
|
||||
def reg_addr() -> tuple[str, int]:
|
||||
|
||||
# globally override the runtime to the per-test-session-dynamic
|
||||
# addr so that all tests never conflict with any other actor
|
||||
# tree using the default.
|
||||
from tractor import _root
|
||||
_root._default_lo_addrs = [_reg_addr]
|
||||
|
||||
return _reg_addr
|
||||
def arb_addr():
|
||||
return _arb_addr
|
||||
|
||||
|
||||
def pytest_generate_tests(metafunc):
|
||||
|
@ -167,35 +161,30 @@ def sig_prog(proc, sig):
|
|||
def daemon(
|
||||
loglevel: str,
|
||||
testdir,
|
||||
reg_addr: tuple[str, int],
|
||||
arb_addr: tuple[str, int],
|
||||
):
|
||||
'''
|
||||
Run a daemon root actor as a separate actor-process tree and
|
||||
"remote registrar" for discovery-protocol related tests.
|
||||
Run a daemon actor as a "remote arbiter".
|
||||
|
||||
'''
|
||||
if loglevel in ('trace', 'debug'):
|
||||
# XXX: too much logging will lock up the subproc (smh)
|
||||
loglevel: str = 'info'
|
||||
# too much logging will lock up the subproc (smh)
|
||||
loglevel = 'info'
|
||||
|
||||
code: str = (
|
||||
"import tractor; "
|
||||
"tractor.run_daemon([], registry_addrs={reg_addrs}, loglevel={ll})"
|
||||
).format(
|
||||
reg_addrs=str([reg_addr]),
|
||||
ll="'{}'".format(loglevel) if loglevel else None,
|
||||
)
|
||||
cmd: list[str] = [
|
||||
sys.executable,
|
||||
'-c', code,
|
||||
cmdargs = [
|
||||
sys.executable, '-c',
|
||||
"import tractor; tractor.run_daemon([], registry_addr={}, loglevel={})"
|
||||
.format(
|
||||
arb_addr,
|
||||
"'{}'".format(loglevel) if loglevel else None)
|
||||
]
|
||||
kwargs = {}
|
||||
kwargs = dict()
|
||||
if platform.system() == 'Windows':
|
||||
# without this, tests hang on windows forever
|
||||
kwargs['creationflags'] = subprocess.CREATE_NEW_PROCESS_GROUP
|
||||
|
||||
proc = testdir.popen(
|
||||
cmd,
|
||||
cmdargs,
|
||||
stdout=subprocess.PIPE,
|
||||
stderr=subprocess.PIPE,
|
||||
**kwargs,
|
||||
|
|
|
@ -95,7 +95,6 @@ def test_ipc_channel_break_during_stream(
|
|||
mod: ModuleType = import_path(
|
||||
examples_dir() / 'advanced_faults' / 'ipc_failure_during_stream.py',
|
||||
root=examples_dir(),
|
||||
consider_namespace_packages=False,
|
||||
)
|
||||
|
||||
# by def we expect KBI from user after a simulated "hang
|
||||
|
|
|
@ -45,7 +45,7 @@ async def do_nuthin():
|
|||
],
|
||||
ids=['no_args', 'unexpected_args'],
|
||||
)
|
||||
def test_remote_error(reg_addr, args_err):
|
||||
def test_remote_error(arb_addr, args_err):
|
||||
'''
|
||||
Verify an error raised in a subactor that is propagated
|
||||
to the parent nursery, contains the underlying boxed builtin
|
||||
|
@ -57,7 +57,7 @@ def test_remote_error(reg_addr, args_err):
|
|||
|
||||
async def main():
|
||||
async with tractor.open_nursery(
|
||||
registry_addrs=[reg_addr],
|
||||
arbiter_addr=arb_addr,
|
||||
) as nursery:
|
||||
|
||||
# on a remote type error caused by bad input args
|
||||
|
@ -77,7 +77,7 @@ def test_remote_error(reg_addr, args_err):
|
|||
# of this actor nursery.
|
||||
await portal.result()
|
||||
except tractor.RemoteActorError as err:
|
||||
assert err.boxed_type == errtype
|
||||
assert err.type == errtype
|
||||
print("Look Maa that actor failed hard, hehh")
|
||||
raise
|
||||
|
||||
|
@ -86,7 +86,7 @@ def test_remote_error(reg_addr, args_err):
|
|||
with pytest.raises(tractor.RemoteActorError) as excinfo:
|
||||
trio.run(main)
|
||||
|
||||
assert excinfo.value.boxed_type == errtype
|
||||
assert excinfo.value.type == errtype
|
||||
|
||||
else:
|
||||
# the root task will also error on the `.result()` call
|
||||
|
@ -96,10 +96,10 @@ def test_remote_error(reg_addr, args_err):
|
|||
|
||||
# ensure boxed errors
|
||||
for exc in excinfo.value.exceptions:
|
||||
assert exc.boxed_type == errtype
|
||||
assert exc.type == errtype
|
||||
|
||||
|
||||
def test_multierror(reg_addr):
|
||||
def test_multierror(arb_addr):
|
||||
'''
|
||||
Verify we raise a ``BaseExceptionGroup`` out of a nursery where
|
||||
more then one actor errors.
|
||||
|
@ -107,7 +107,7 @@ def test_multierror(reg_addr):
|
|||
'''
|
||||
async def main():
|
||||
async with tractor.open_nursery(
|
||||
registry_addrs=[reg_addr],
|
||||
arbiter_addr=arb_addr,
|
||||
) as nursery:
|
||||
|
||||
await nursery.run_in_actor(assert_err, name='errorer1')
|
||||
|
@ -117,7 +117,7 @@ def test_multierror(reg_addr):
|
|||
try:
|
||||
await portal2.result()
|
||||
except tractor.RemoteActorError as err:
|
||||
assert err.boxed_type == AssertionError
|
||||
assert err.type == AssertionError
|
||||
print("Look Maa that first actor failed hard, hehh")
|
||||
raise
|
||||
|
||||
|
@ -132,14 +132,14 @@ def test_multierror(reg_addr):
|
|||
@pytest.mark.parametrize(
|
||||
'num_subactors', range(25, 26),
|
||||
)
|
||||
def test_multierror_fast_nursery(reg_addr, start_method, num_subactors, delay):
|
||||
def test_multierror_fast_nursery(arb_addr, start_method, num_subactors, delay):
|
||||
"""Verify we raise a ``BaseExceptionGroup`` out of a nursery where
|
||||
more then one actor errors and also with a delay before failure
|
||||
to test failure during an ongoing spawning.
|
||||
"""
|
||||
async def main():
|
||||
async with tractor.open_nursery(
|
||||
registry_addrs=[reg_addr],
|
||||
arbiter_addr=arb_addr,
|
||||
) as nursery:
|
||||
|
||||
for i in range(num_subactors):
|
||||
|
@ -169,7 +169,7 @@ def test_multierror_fast_nursery(reg_addr, start_method, num_subactors, delay):
|
|||
|
||||
for exc in exceptions:
|
||||
assert isinstance(exc, tractor.RemoteActorError)
|
||||
assert exc.boxed_type == AssertionError
|
||||
assert exc.type == AssertionError
|
||||
|
||||
|
||||
async def do_nothing():
|
||||
|
@ -177,20 +177,15 @@ async def do_nothing():
|
|||
|
||||
|
||||
@pytest.mark.parametrize('mechanism', ['nursery_cancel', KeyboardInterrupt])
|
||||
def test_cancel_single_subactor(reg_addr, mechanism):
|
||||
'''
|
||||
Ensure a ``ActorNursery.start_actor()`` spawned subactor
|
||||
def test_cancel_single_subactor(arb_addr, mechanism):
|
||||
"""Ensure a ``ActorNursery.start_actor()`` spawned subactor
|
||||
cancels when the nursery is cancelled.
|
||||
|
||||
'''
|
||||
"""
|
||||
async def spawn_actor():
|
||||
'''
|
||||
Spawn an actor that blocks indefinitely then cancel via
|
||||
either `ActorNursery.cancel()` or an exception raise.
|
||||
|
||||
'''
|
||||
"""Spawn an actor that blocks indefinitely.
|
||||
"""
|
||||
async with tractor.open_nursery(
|
||||
registry_addrs=[reg_addr],
|
||||
arbiter_addr=arb_addr,
|
||||
) as nursery:
|
||||
|
||||
portal = await nursery.start_actor(
|
||||
|
@ -310,7 +305,7 @@ async def test_some_cancels_all(num_actors_and_errs, start_method, loglevel):
|
|||
await portal.run(func, **kwargs)
|
||||
|
||||
except tractor.RemoteActorError as err:
|
||||
assert err.boxed_type == err_type
|
||||
assert err.type == err_type
|
||||
# we only expect this first error to propogate
|
||||
# (all other daemons are cancelled before they
|
||||
# can be scheduled)
|
||||
|
@ -329,11 +324,11 @@ async def test_some_cancels_all(num_actors_and_errs, start_method, loglevel):
|
|||
assert len(err.exceptions) == num_actors
|
||||
for exc in err.exceptions:
|
||||
if isinstance(exc, tractor.RemoteActorError):
|
||||
assert exc.boxed_type == err_type
|
||||
assert exc.type == err_type
|
||||
else:
|
||||
assert isinstance(exc, trio.Cancelled)
|
||||
elif isinstance(err, tractor.RemoteActorError):
|
||||
assert err.boxed_type == err_type
|
||||
assert err.type == err_type
|
||||
|
||||
assert n.cancelled is True
|
||||
assert not n._children
|
||||
|
@ -412,7 +407,7 @@ async def test_nested_multierrors(loglevel, start_method):
|
|||
elif isinstance(subexc, tractor.RemoteActorError):
|
||||
# on windows it seems we can't exactly be sure wtf
|
||||
# will happen..
|
||||
assert subexc.boxed_type in (
|
||||
assert subexc.type in (
|
||||
tractor.RemoteActorError,
|
||||
trio.Cancelled,
|
||||
BaseExceptionGroup,
|
||||
|
@ -422,7 +417,7 @@ async def test_nested_multierrors(loglevel, start_method):
|
|||
for subsub in subexc.exceptions:
|
||||
|
||||
if subsub in (tractor.RemoteActorError,):
|
||||
subsub = subsub.boxed_type
|
||||
subsub = subsub.type
|
||||
|
||||
assert type(subsub) in (
|
||||
trio.Cancelled,
|
||||
|
@ -437,16 +432,16 @@ async def test_nested_multierrors(loglevel, start_method):
|
|||
# we get back the (sent) cancel signal instead
|
||||
if is_win():
|
||||
if isinstance(subexc, tractor.RemoteActorError):
|
||||
assert subexc.boxed_type in (
|
||||
assert subexc.type in (
|
||||
BaseExceptionGroup,
|
||||
tractor.RemoteActorError
|
||||
)
|
||||
else:
|
||||
assert isinstance(subexc, BaseExceptionGroup)
|
||||
else:
|
||||
assert subexc.boxed_type is ExceptionGroup
|
||||
assert subexc.type is ExceptionGroup
|
||||
else:
|
||||
assert subexc.boxed_type in (
|
||||
assert subexc.type in (
|
||||
tractor.RemoteActorError,
|
||||
trio.Cancelled
|
||||
)
|
||||
|
|
|
@ -142,7 +142,7 @@ async def open_actor_local_nursery(
|
|||
)
|
||||
def test_actor_managed_trio_nursery_task_error_cancels_aio(
|
||||
asyncio_mode: bool,
|
||||
reg_addr: tuple,
|
||||
arb_addr
|
||||
):
|
||||
'''
|
||||
Verify that a ``trio`` nursery created managed in a child actor
|
||||
|
@ -171,4 +171,4 @@ def test_actor_managed_trio_nursery_task_error_cancels_aio(
|
|||
|
||||
# verify boxed error
|
||||
err = excinfo.value
|
||||
assert err.boxed_type is NameError
|
||||
assert isinstance(err.type(), NameError)
|
||||
|
|
|
@ -795,7 +795,7 @@ async def test_callee_cancels_before_started(
|
|||
|
||||
# raises a special cancel signal
|
||||
except tractor.ContextCancelled as ce:
|
||||
ce.boxed_type == trio.Cancelled
|
||||
ce.type == trio.Cancelled
|
||||
|
||||
# the traceback should be informative
|
||||
assert 'itself' in ce.msgdata['tb_str']
|
||||
|
@ -903,7 +903,7 @@ def test_one_end_stream_not_opened(
|
|||
with pytest.raises(tractor.RemoteActorError) as excinfo:
|
||||
trio.run(main)
|
||||
|
||||
assert excinfo.value.boxed_type == StreamOverrun
|
||||
assert excinfo.value.type == StreamOverrun
|
||||
|
||||
elif overrunner == 'callee':
|
||||
with pytest.raises(tractor.RemoteActorError) as excinfo:
|
||||
|
@ -912,7 +912,7 @@ def test_one_end_stream_not_opened(
|
|||
# TODO: embedded remote errors so that we can verify the source
|
||||
# error? the callee delivers an error which is an overrun
|
||||
# wrapped in a remote actor error.
|
||||
assert excinfo.value.boxed_type == tractor.RemoteActorError
|
||||
assert excinfo.value.type == tractor.RemoteActorError
|
||||
|
||||
else:
|
||||
trio.run(main)
|
||||
|
@ -1131,7 +1131,7 @@ def test_maybe_allow_overruns_stream(
|
|||
# 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.boxed_type in (
|
||||
assert err.type in (
|
||||
tractor.RemoteActorError,
|
||||
StreamOverrun,
|
||||
)
|
||||
|
@ -1139,10 +1139,10 @@ def test_maybe_allow_overruns_stream(
|
|||
elif (
|
||||
slow_side == 'child'
|
||||
):
|
||||
assert err.boxed_type == StreamOverrun
|
||||
assert err.type == StreamOverrun
|
||||
|
||||
elif slow_side == 'parent':
|
||||
assert err.boxed_type == tractor.RemoteActorError
|
||||
assert err.type == tractor.RemoteActorError
|
||||
assert 'StreamOverrun' in err.msgdata['tb_str']
|
||||
|
||||
else:
|
||||
|
|
|
@ -83,7 +83,7 @@ has_nested_actors = pytest.mark.has_nested_actors
|
|||
def spawn(
|
||||
start_method,
|
||||
testdir,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
) -> 'pexpect.spawn':
|
||||
|
||||
if start_method != 'trio':
|
||||
|
@ -203,7 +203,7 @@ def ctlc(
|
|||
# XXX: disable pygments highlighting for auto-tests
|
||||
# since some envs (like actions CI) will struggle
|
||||
# the the added color-char encoding..
|
||||
from tractor.devx._debug import TractorConfig
|
||||
from tractor._debug import TractorConfig
|
||||
TractorConfig.use_pygements = False
|
||||
|
||||
yield use_ctlc
|
||||
|
@ -685,7 +685,7 @@ def test_multi_daemon_subactors(
|
|||
# now the root actor won't clobber the bp_forever child
|
||||
# during it's first access to the debug lock, but will instead
|
||||
# wait for the lock to release, by the edge triggered
|
||||
# ``devx._debug.Lock.no_remote_has_tty`` event before sending cancel messages
|
||||
# ``_debug.Lock.no_remote_has_tty`` event before sending cancel messages
|
||||
# (via portals) to its underlings B)
|
||||
|
||||
# at some point here there should have been some warning msg from
|
||||
|
@ -1025,67 +1025,3 @@ def test_different_debug_mode_per_actor(
|
|||
# instead crashed completely
|
||||
assert "tractor._exceptions.RemoteActorError: ('crash_boi'" in before
|
||||
assert "RuntimeError" in before
|
||||
|
||||
|
||||
|
||||
def test_pause_from_sync(
|
||||
spawn,
|
||||
ctlc: bool
|
||||
):
|
||||
'''
|
||||
Verify we can use the `pdbp` REPL from sync functions AND from
|
||||
any thread spawned with `trio.to_thread.run_sync()`.
|
||||
|
||||
`examples/debugging/sync_bp.py`
|
||||
|
||||
'''
|
||||
child = spawn('sync_bp')
|
||||
child.expect(PROMPT)
|
||||
assert_before(
|
||||
child,
|
||||
[
|
||||
'`greenback` portal opened!',
|
||||
# pre-prompt line
|
||||
_pause_msg, "('root'",
|
||||
]
|
||||
)
|
||||
if ctlc:
|
||||
do_ctlc(child)
|
||||
child.sendline('c')
|
||||
child.expect(PROMPT)
|
||||
|
||||
# XXX shouldn't see gb loaded again
|
||||
before = str(child.before.decode())
|
||||
assert not in_prompt_msg(
|
||||
before,
|
||||
['`greenback` portal opened!'],
|
||||
)
|
||||
assert_before(
|
||||
child,
|
||||
[_pause_msg, "('root'",],
|
||||
)
|
||||
|
||||
if ctlc:
|
||||
do_ctlc(child)
|
||||
child.sendline('c')
|
||||
child.expect(PROMPT)
|
||||
assert_before(
|
||||
child,
|
||||
[_pause_msg, "('subactor'",],
|
||||
)
|
||||
|
||||
if ctlc:
|
||||
do_ctlc(child)
|
||||
child.sendline('c')
|
||||
child.expect(PROMPT)
|
||||
# non-main thread case
|
||||
# TODO: should we agument the pre-prompt msg in this case?
|
||||
assert_before(
|
||||
child,
|
||||
[_pause_msg, "('root'",],
|
||||
)
|
||||
|
||||
if ctlc:
|
||||
do_ctlc(child)
|
||||
child.sendline('c')
|
||||
child.expect(pexpect.EOF)
|
||||
|
|
|
@ -14,19 +14,19 @@ import trio
|
|||
|
||||
|
||||
@tractor_test
|
||||
async def test_reg_then_unreg(reg_addr):
|
||||
async def test_reg_then_unreg(arb_addr):
|
||||
actor = tractor.current_actor()
|
||||
assert actor.is_arbiter
|
||||
assert len(actor._registry) == 1 # only self is registered
|
||||
|
||||
async with tractor.open_nursery(
|
||||
registry_addrs=[reg_addr],
|
||||
arbiter_addr=arb_addr,
|
||||
) as n:
|
||||
|
||||
portal = await n.start_actor('actor', enable_modules=[__name__])
|
||||
uid = portal.channel.uid
|
||||
|
||||
async with tractor.get_arbiter(*reg_addr) as aportal:
|
||||
async with tractor.get_arbiter(*arb_addr) as aportal:
|
||||
# this local actor should be the arbiter
|
||||
assert actor is aportal.actor
|
||||
|
||||
|
@ -52,27 +52,15 @@ async def hi():
|
|||
return the_line.format(tractor.current_actor().name)
|
||||
|
||||
|
||||
async def say_hello(
|
||||
other_actor: str,
|
||||
reg_addr: tuple[str, int],
|
||||
):
|
||||
async def say_hello(other_actor):
|
||||
await trio.sleep(1) # wait for other actor to spawn
|
||||
async with tractor.find_actor(
|
||||
other_actor,
|
||||
registry_addrs=[reg_addr],
|
||||
) as portal:
|
||||
async with tractor.find_actor(other_actor) as portal:
|
||||
assert portal is not None
|
||||
return await portal.run(__name__, 'hi')
|
||||
|
||||
|
||||
async def say_hello_use_wait(
|
||||
other_actor: str,
|
||||
reg_addr: tuple[str, int],
|
||||
):
|
||||
async with tractor.wait_for_actor(
|
||||
other_actor,
|
||||
registry_addr=reg_addr,
|
||||
) as portal:
|
||||
async def say_hello_use_wait(other_actor):
|
||||
async with tractor.wait_for_actor(other_actor) as portal:
|
||||
assert portal is not None
|
||||
result = await portal.run(__name__, 'hi')
|
||||
return result
|
||||
|
@ -80,29 +68,21 @@ async def say_hello_use_wait(
|
|||
|
||||
@tractor_test
|
||||
@pytest.mark.parametrize('func', [say_hello, say_hello_use_wait])
|
||||
async def test_trynamic_trio(
|
||||
func,
|
||||
start_method,
|
||||
reg_addr,
|
||||
):
|
||||
'''
|
||||
Root actor acting as the "director" and running one-shot-task-actors
|
||||
for the directed subs.
|
||||
|
||||
'''
|
||||
async def test_trynamic_trio(func, start_method, arb_addr):
|
||||
"""Main tractor entry point, the "master" process (for now
|
||||
acts as the "director").
|
||||
"""
|
||||
async with tractor.open_nursery() as n:
|
||||
print("Alright... Action!")
|
||||
|
||||
donny = await n.run_in_actor(
|
||||
func,
|
||||
other_actor='gretchen',
|
||||
reg_addr=reg_addr,
|
||||
name='donny',
|
||||
)
|
||||
gretchen = await n.run_in_actor(
|
||||
func,
|
||||
other_actor='donny',
|
||||
reg_addr=reg_addr,
|
||||
name='gretchen',
|
||||
)
|
||||
print(await gretchen.result())
|
||||
|
@ -150,7 +130,7 @@ async def unpack_reg(actor_or_portal):
|
|||
|
||||
|
||||
async def spawn_and_check_registry(
|
||||
reg_addr: tuple,
|
||||
arb_addr: tuple,
|
||||
use_signal: bool,
|
||||
remote_arbiter: bool = False,
|
||||
with_streaming: bool = False,
|
||||
|
@ -158,9 +138,9 @@ async def spawn_and_check_registry(
|
|||
) -> None:
|
||||
|
||||
async with tractor.open_root_actor(
|
||||
registry_addrs=[reg_addr],
|
||||
arbiter_addr=arb_addr,
|
||||
):
|
||||
async with tractor.get_arbiter(*reg_addr) as portal:
|
||||
async with tractor.get_arbiter(*arb_addr) as portal:
|
||||
# runtime needs to be up to call this
|
||||
actor = tractor.current_actor()
|
||||
|
||||
|
@ -232,19 +212,17 @@ async def spawn_and_check_registry(
|
|||
def test_subactors_unregister_on_cancel(
|
||||
start_method,
|
||||
use_signal,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
with_streaming,
|
||||
):
|
||||
'''
|
||||
Verify that cancelling a nursery results in all subactors
|
||||
"""Verify that cancelling a nursery results in all subactors
|
||||
deregistering themselves with the arbiter.
|
||||
|
||||
'''
|
||||
"""
|
||||
with pytest.raises(KeyboardInterrupt):
|
||||
trio.run(
|
||||
partial(
|
||||
spawn_and_check_registry,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
use_signal,
|
||||
remote_arbiter=False,
|
||||
with_streaming=with_streaming,
|
||||
|
@ -258,7 +236,7 @@ def test_subactors_unregister_on_cancel_remote_daemon(
|
|||
daemon,
|
||||
start_method,
|
||||
use_signal,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
with_streaming,
|
||||
):
|
||||
"""Verify that cancelling a nursery results in all subactors
|
||||
|
@ -269,7 +247,7 @@ def test_subactors_unregister_on_cancel_remote_daemon(
|
|||
trio.run(
|
||||
partial(
|
||||
spawn_and_check_registry,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
use_signal,
|
||||
remote_arbiter=True,
|
||||
with_streaming=with_streaming,
|
||||
|
@ -283,7 +261,7 @@ async def streamer(agen):
|
|||
|
||||
|
||||
async def close_chans_before_nursery(
|
||||
reg_addr: tuple,
|
||||
arb_addr: tuple,
|
||||
use_signal: bool,
|
||||
remote_arbiter: bool = False,
|
||||
) -> None:
|
||||
|
@ -296,9 +274,9 @@ async def close_chans_before_nursery(
|
|||
entries_at_end = 1
|
||||
|
||||
async with tractor.open_root_actor(
|
||||
registry_addrs=[reg_addr],
|
||||
arbiter_addr=arb_addr,
|
||||
):
|
||||
async with tractor.get_arbiter(*reg_addr) as aportal:
|
||||
async with tractor.get_arbiter(*arb_addr) as aportal:
|
||||
try:
|
||||
get_reg = partial(unpack_reg, aportal)
|
||||
|
||||
|
@ -350,7 +328,7 @@ async def close_chans_before_nursery(
|
|||
def test_close_channel_explicit(
|
||||
start_method,
|
||||
use_signal,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
):
|
||||
"""Verify that closing a stream explicitly and killing the actor's
|
||||
"root nursery" **before** the containing nursery tears down also
|
||||
|
@ -360,7 +338,7 @@ def test_close_channel_explicit(
|
|||
trio.run(
|
||||
partial(
|
||||
close_chans_before_nursery,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
use_signal,
|
||||
remote_arbiter=False,
|
||||
),
|
||||
|
@ -372,7 +350,7 @@ def test_close_channel_explicit_remote_arbiter(
|
|||
daemon,
|
||||
start_method,
|
||||
use_signal,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
):
|
||||
"""Verify that closing a stream explicitly and killing the actor's
|
||||
"root nursery" **before** the containing nursery tears down also
|
||||
|
@ -382,7 +360,7 @@ def test_close_channel_explicit_remote_arbiter(
|
|||
trio.run(
|
||||
partial(
|
||||
close_chans_before_nursery,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
use_signal,
|
||||
remote_arbiter=True,
|
||||
),
|
||||
|
|
|
@ -20,7 +20,7 @@ from tractor._testing import (
|
|||
def run_example_in_subproc(
|
||||
loglevel: str,
|
||||
testdir,
|
||||
reg_addr: tuple[str, int],
|
||||
arb_addr: tuple[str, int],
|
||||
):
|
||||
|
||||
@contextmanager
|
||||
|
|
|
@ -47,7 +47,7 @@ async def trio_cancels_single_aio_task():
|
|||
await tractor.to_asyncio.run_task(sleep_forever)
|
||||
|
||||
|
||||
def test_trio_cancels_aio_on_actor_side(reg_addr):
|
||||
def test_trio_cancels_aio_on_actor_side(arb_addr):
|
||||
'''
|
||||
Spawn an infected actor that is cancelled by the ``trio`` side
|
||||
task using std cancel scope apis.
|
||||
|
@ -55,7 +55,7 @@ def test_trio_cancels_aio_on_actor_side(reg_addr):
|
|||
'''
|
||||
async def main():
|
||||
async with tractor.open_nursery(
|
||||
registry_addrs=[reg_addr]
|
||||
arbiter_addr=arb_addr
|
||||
) as n:
|
||||
await n.run_in_actor(
|
||||
trio_cancels_single_aio_task,
|
||||
|
@ -94,7 +94,7 @@ async def asyncio_actor(
|
|||
raise
|
||||
|
||||
|
||||
def test_aio_simple_error(reg_addr):
|
||||
def test_aio_simple_error(arb_addr):
|
||||
'''
|
||||
Verify a simple remote asyncio error propagates back through trio
|
||||
to the parent actor.
|
||||
|
@ -103,7 +103,7 @@ def test_aio_simple_error(reg_addr):
|
|||
'''
|
||||
async def main():
|
||||
async with tractor.open_nursery(
|
||||
registry_addrs=[reg_addr]
|
||||
arbiter_addr=arb_addr
|
||||
) as n:
|
||||
await n.run_in_actor(
|
||||
asyncio_actor,
|
||||
|
@ -128,10 +128,10 @@ def test_aio_simple_error(reg_addr):
|
|||
assert err
|
||||
|
||||
assert isinstance(err, RemoteActorError)
|
||||
assert err.boxed_type == AssertionError
|
||||
assert err.type == AssertionError
|
||||
|
||||
|
||||
def test_tractor_cancels_aio(reg_addr):
|
||||
def test_tractor_cancels_aio(arb_addr):
|
||||
'''
|
||||
Verify we can cancel a spawned asyncio task gracefully.
|
||||
|
||||
|
@ -150,7 +150,7 @@ def test_tractor_cancels_aio(reg_addr):
|
|||
trio.run(main)
|
||||
|
||||
|
||||
def test_trio_cancels_aio(reg_addr):
|
||||
def test_trio_cancels_aio(arb_addr):
|
||||
'''
|
||||
Much like the above test with ``tractor.Portal.cancel_actor()``
|
||||
except we just use a standard ``trio`` cancellation api.
|
||||
|
@ -206,7 +206,7 @@ async def trio_ctx(
|
|||
ids='parent_actor_cancels_child={}'.format
|
||||
)
|
||||
def test_context_spawns_aio_task_that_errors(
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
parent_cancels: bool,
|
||||
):
|
||||
'''
|
||||
|
@ -272,7 +272,7 @@ def test_context_spawns_aio_task_that_errors(
|
|||
|
||||
err = excinfo.value
|
||||
assert isinstance(err, expect)
|
||||
assert err.boxed_type == AssertionError
|
||||
assert err.type == AssertionError
|
||||
|
||||
|
||||
async def aio_cancel():
|
||||
|
@ -288,7 +288,7 @@ async def aio_cancel():
|
|||
await sleep_forever()
|
||||
|
||||
|
||||
def test_aio_cancelled_from_aio_causes_trio_cancelled(reg_addr):
|
||||
def test_aio_cancelled_from_aio_causes_trio_cancelled(arb_addr):
|
||||
|
||||
async def main():
|
||||
async with tractor.open_nursery() as n:
|
||||
|
@ -314,7 +314,7 @@ def test_aio_cancelled_from_aio_causes_trio_cancelled(reg_addr):
|
|||
assert err
|
||||
|
||||
# ensure boxed error is correct
|
||||
assert err.boxed_type == to_asyncio.AsyncioCancelled
|
||||
assert err.type == to_asyncio.AsyncioCancelled
|
||||
|
||||
|
||||
# TODO: verify open_channel_from will fail on this..
|
||||
|
@ -436,7 +436,7 @@ async def stream_from_aio(
|
|||
'fan_out', [False, True],
|
||||
ids='fan_out_w_chan_subscribe={}'.format
|
||||
)
|
||||
def test_basic_interloop_channel_stream(reg_addr, fan_out):
|
||||
def test_basic_interloop_channel_stream(arb_addr, fan_out):
|
||||
async def main():
|
||||
async with tractor.open_nursery() as n:
|
||||
portal = await n.run_in_actor(
|
||||
|
@ -450,7 +450,7 @@ def test_basic_interloop_channel_stream(reg_addr, fan_out):
|
|||
|
||||
|
||||
# TODO: parametrize the above test and avoid the duplication here?
|
||||
def test_trio_error_cancels_intertask_chan(reg_addr):
|
||||
def test_trio_error_cancels_intertask_chan(arb_addr):
|
||||
async def main():
|
||||
async with tractor.open_nursery() as n:
|
||||
portal = await n.run_in_actor(
|
||||
|
@ -466,10 +466,10 @@ def test_trio_error_cancels_intertask_chan(reg_addr):
|
|||
|
||||
# ensure boxed errors
|
||||
for exc in excinfo.value.exceptions:
|
||||
assert exc.boxed_type == Exception
|
||||
assert exc.type == Exception
|
||||
|
||||
|
||||
def test_trio_closes_early_and_channel_exits(reg_addr):
|
||||
def test_trio_closes_early_and_channel_exits(arb_addr):
|
||||
async def main():
|
||||
async with tractor.open_nursery() as n:
|
||||
portal = await n.run_in_actor(
|
||||
|
@ -484,7 +484,7 @@ def test_trio_closes_early_and_channel_exits(reg_addr):
|
|||
trio.run(main)
|
||||
|
||||
|
||||
def test_aio_errors_and_channel_propagates_and_closes(reg_addr):
|
||||
def test_aio_errors_and_channel_propagates_and_closes(arb_addr):
|
||||
async def main():
|
||||
async with tractor.open_nursery() as n:
|
||||
portal = await n.run_in_actor(
|
||||
|
@ -500,7 +500,7 @@ def test_aio_errors_and_channel_propagates_and_closes(reg_addr):
|
|||
|
||||
# ensure boxed errors
|
||||
for exc in excinfo.value.exceptions:
|
||||
assert exc.boxed_type == Exception
|
||||
assert exc.type == Exception
|
||||
|
||||
|
||||
@tractor.context
|
||||
|
@ -561,7 +561,7 @@ async def trio_to_aio_echo_server(
|
|||
ids='raise_error={}'.format,
|
||||
)
|
||||
def test_echoserver_detailed_mechanics(
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
raise_error_mid_stream,
|
||||
):
|
||||
|
||||
|
@ -601,8 +601,7 @@ def test_echoserver_detailed_mechanics(
|
|||
pass
|
||||
else:
|
||||
pytest.fail(
|
||||
'stream not stopped after sentinel ?!'
|
||||
)
|
||||
"stream wasn't stopped after sentinel?!")
|
||||
|
||||
# TODO: the case where this blocks and
|
||||
# is cancelled by kbi or out of task cancellation
|
||||
|
@ -614,37 +613,3 @@ def test_echoserver_detailed_mechanics(
|
|||
|
||||
else:
|
||||
trio.run(main)
|
||||
|
||||
|
||||
# TODO: debug_mode tests once we get support for `asyncio`!
|
||||
#
|
||||
# -[ ] need tests to wrap both scripts:
|
||||
# - [ ] infected_asyncio_echo_server.py
|
||||
# - [ ] debugging/asyncio_bp.py
|
||||
# -[ ] consider moving ^ (some of) these ^ to `test_debugger`?
|
||||
#
|
||||
# -[ ] missing impl outstanding includes:
|
||||
# - [x] for sync pauses we need to ensure we open yet another
|
||||
# `greenback` portal in the asyncio task
|
||||
# => completed using `.bestow_portal(task)` inside
|
||||
# `.to_asyncio._run_asyncio_task()` right?
|
||||
# -[ ] translation func to get from `asyncio` task calling to
|
||||
# `._debug.wait_for_parent_stdin_hijack()` which does root
|
||||
# call to do TTY locking.
|
||||
#
|
||||
def test_sync_breakpoint():
|
||||
'''
|
||||
Verify we can do sync-func/code breakpointing using the
|
||||
`breakpoint()` builtin inside infected mode actors.
|
||||
|
||||
'''
|
||||
pytest.xfail('This support is not implemented yet!')
|
||||
|
||||
|
||||
def test_debug_mode_crash_handling():
|
||||
'''
|
||||
Verify mult-actor crash handling works with a combo of infected-`asyncio`-mode
|
||||
and normal `trio` actors despite nested process trees.
|
||||
|
||||
'''
|
||||
pytest.xfail('This support is not implemented yet!')
|
||||
|
|
|
@ -16,11 +16,6 @@ from tractor import ( # typing
|
|||
Portal,
|
||||
Context,
|
||||
ContextCancelled,
|
||||
RemoteActorError,
|
||||
)
|
||||
from tractor._testing import (
|
||||
# tractor_test,
|
||||
expect_ctxc,
|
||||
)
|
||||
|
||||
# XXX TODO cases:
|
||||
|
@ -161,11 +156,10 @@ def test_do_not_swallow_error_before_started_by_remote_contextcancelled(
|
|||
):
|
||||
await trio.sleep_forever()
|
||||
|
||||
with pytest.raises(RemoteActorError) as excinfo:
|
||||
with pytest.raises(tractor.RemoteActorError) as excinfo:
|
||||
trio.run(main)
|
||||
|
||||
rae = excinfo.value
|
||||
assert rae.boxed_type == TypeError
|
||||
assert excinfo.value.type == TypeError
|
||||
|
||||
|
||||
@tractor.context
|
||||
|
@ -745,16 +739,14 @@ def test_peer_canceller(
|
|||
with pytest.raises(ContextCancelled) as excinfo:
|
||||
trio.run(main)
|
||||
|
||||
assert excinfo.value.boxed_type == ContextCancelled
|
||||
assert excinfo.value.type == ContextCancelled
|
||||
assert excinfo.value.canceller[0] == 'canceller'
|
||||
|
||||
|
||||
@tractor.context
|
||||
async def basic_echo_server(
|
||||
ctx: Context,
|
||||
peer_name: str = 'wittle_bruv',
|
||||
|
||||
err_after: int|None = None,
|
||||
peer_name: str = 'stepbro',
|
||||
|
||||
) -> None:
|
||||
'''
|
||||
|
@ -782,31 +774,17 @@ async def basic_echo_server(
|
|||
# assert 0
|
||||
await ipc.send(resp)
|
||||
|
||||
if (
|
||||
err_after
|
||||
and i > err_after
|
||||
):
|
||||
raise RuntimeError(
|
||||
f'Simulated error in `{peer_name}`'
|
||||
)
|
||||
|
||||
|
||||
@tractor.context
|
||||
async def serve_subactors(
|
||||
ctx: Context,
|
||||
peer_name: str,
|
||||
debug_mode: bool,
|
||||
|
||||
) -> None:
|
||||
async with open_nursery() as an:
|
||||
|
||||
# sanity
|
||||
if debug_mode:
|
||||
assert tractor._state.debug_mode()
|
||||
|
||||
await ctx.started(peer_name)
|
||||
async with ctx.open_stream() as ipc:
|
||||
async for msg in ipc:
|
||||
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,
|
||||
|
@ -817,7 +795,7 @@ async def serve_subactors(
|
|||
f'{peer_name}\n'
|
||||
f'|_{peer}\n'
|
||||
)
|
||||
await ipc.send((
|
||||
await reqs.send((
|
||||
peer.chan.uid,
|
||||
peer.chan.raddr,
|
||||
))
|
||||
|
@ -829,20 +807,14 @@ async def serve_subactors(
|
|||
async def client_req_subactor(
|
||||
ctx: Context,
|
||||
peer_name: str,
|
||||
debug_mode: bool,
|
||||
|
||||
# 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,
|
||||
sub_err_after: int|None = None,
|
||||
|
||||
) -> None:
|
||||
# sanity
|
||||
if debug_mode:
|
||||
assert tractor._state.debug_mode()
|
||||
|
||||
# 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.
|
||||
|
@ -864,7 +836,6 @@ async def client_req_subactor(
|
|||
spawner.open_context(
|
||||
serve_subactors,
|
||||
peer_name=peer_name,
|
||||
debug_mode=debug_mode,
|
||||
) as (spawner_ctx, first),
|
||||
):
|
||||
assert first == peer_name
|
||||
|
@ -886,7 +857,6 @@ async def client_req_subactor(
|
|||
await tell_little_bro(
|
||||
actor_name=sub_uid[0],
|
||||
caller='client',
|
||||
err_after=sub_err_after,
|
||||
)
|
||||
|
||||
# TODO: test different scope-layers of
|
||||
|
@ -898,7 +868,9 @@ async def client_req_subactor(
|
|||
# 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:
|
||||
except (
|
||||
trio.Cancelled
|
||||
) as err:
|
||||
_err = err
|
||||
if reraise_on_cancel:
|
||||
errtype = globals()['__builtins__'][reraise_on_cancel]
|
||||
|
@ -925,9 +897,7 @@ async def client_req_subactor(
|
|||
|
||||
async def tell_little_bro(
|
||||
actor_name: str,
|
||||
|
||||
caller: str = '',
|
||||
err_after: int|None = None,
|
||||
caller: str = ''
|
||||
):
|
||||
# contact target actor, do a stream dialog.
|
||||
async with (
|
||||
|
@ -936,12 +906,10 @@ async def tell_little_bro(
|
|||
) as lb,
|
||||
lb.open_context(
|
||||
basic_echo_server,
|
||||
|
||||
# XXX proxy any delayed err condition
|
||||
err_after=err_after,
|
||||
) as (sub_ctx, first),
|
||||
|
||||
sub_ctx.open_stream() as echo_ipc,
|
||||
sub_ctx.open_stream(
|
||||
basic_echo_server,
|
||||
) as echo_ipc,
|
||||
):
|
||||
actor: Actor = current_actor()
|
||||
uid: tuple = actor.uid
|
||||
|
@ -968,15 +936,9 @@ async def tell_little_bro(
|
|||
'raise_client_error',
|
||||
[None, 'KeyboardInterrupt'],
|
||||
)
|
||||
@pytest.mark.parametrize(
|
||||
'raise_sub_spawn_error_after',
|
||||
[None, 50],
|
||||
)
|
||||
def test_peer_spawns_and_cancels_service_subactor(
|
||||
debug_mode: bool,
|
||||
raise_client_error: str,
|
||||
reg_addr: tuple[str, int],
|
||||
raise_sub_spawn_error_after: int|None,
|
||||
):
|
||||
# NOTE: this tests for the modden `mod wks open piker` bug
|
||||
# discovered as part of implementing workspace ctx
|
||||
|
@ -990,21 +952,10 @@ def test_peer_spawns_and_cancels_service_subactor(
|
|||
# and the server's spawned child should cancel and terminate!
|
||||
peer_name: str = 'little_bro'
|
||||
|
||||
def check_inner_rte(rae: RemoteActorError):
|
||||
'''
|
||||
Validate the little_bro's relayed inception!
|
||||
|
||||
'''
|
||||
assert rae.boxed_type is RemoteActorError
|
||||
assert rae.src_type is RuntimeError
|
||||
assert 'client' in rae.relay_uid
|
||||
assert peer_name in rae.src_uid
|
||||
|
||||
async def main():
|
||||
async with tractor.open_nursery(
|
||||
# NOTE: to halt the peer tasks on ctxc, uncomment this.
|
||||
debug_mode=debug_mode,
|
||||
registry_addrs=[reg_addr],
|
||||
) as an:
|
||||
server: Portal = await an.start_actor(
|
||||
(server_name := 'spawn_server'),
|
||||
|
@ -1023,24 +974,14 @@ def test_peer_spawns_and_cancels_service_subactor(
|
|||
server.open_context(
|
||||
serve_subactors,
|
||||
peer_name=peer_name,
|
||||
debug_mode=debug_mode,
|
||||
|
||||
) as (spawn_ctx, first),
|
||||
|
||||
client.open_context(
|
||||
client_req_subactor,
|
||||
peer_name=peer_name,
|
||||
debug_mode=debug_mode,
|
||||
reraise_on_cancel=raise_client_error,
|
||||
|
||||
# trigger for error condition in sub
|
||||
# during streaming.
|
||||
sub_err_after=raise_sub_spawn_error_after,
|
||||
|
||||
) as (client_ctx, client_says),
|
||||
):
|
||||
root: Actor = current_actor()
|
||||
spawner_uid: tuple = spawn_ctx.chan.uid
|
||||
print(
|
||||
f'Server says: {first}\n'
|
||||
f'Client says: {client_says}\n'
|
||||
|
@ -1050,7 +991,6 @@ def test_peer_spawns_and_cancels_service_subactor(
|
|||
# (grandchild of this root actor) "little_bro"
|
||||
# and ensure we can also use it as an echo
|
||||
# server.
|
||||
sub: Portal
|
||||
async with tractor.wait_for_actor(
|
||||
name=peer_name,
|
||||
) as sub:
|
||||
|
@ -1062,116 +1002,41 @@ def test_peer_spawns_and_cancels_service_subactor(
|
|||
f'.uid: {sub.actor.uid}\n'
|
||||
f'chan.raddr: {sub.chan.raddr}\n'
|
||||
)
|
||||
|
||||
async with expect_ctxc(
|
||||
yay=raise_sub_spawn_error_after,
|
||||
reraise=False,
|
||||
):
|
||||
await tell_little_bro(
|
||||
actor_name=peer_name,
|
||||
caller='root',
|
||||
)
|
||||
|
||||
if not raise_sub_spawn_error_after:
|
||||
|
||||
# signal client to cancel and maybe raise a KBI
|
||||
# signal client to raise a KBI
|
||||
await client_ctx.cancel()
|
||||
print(
|
||||
'-> root cancelling client,\n'
|
||||
'-> root checking `client_ctx.result()`,\n'
|
||||
f'-> checking that sub-spawn {peer_name} is down\n'
|
||||
)
|
||||
# else:
|
||||
print('root cancelled client, checking that sub-spawn is down')
|
||||
|
||||
try:
|
||||
res = await client_ctx.result(hide_tb=False)
|
||||
|
||||
# in remote (relayed inception) error
|
||||
# case, we should error on the line above!
|
||||
if raise_sub_spawn_error_after:
|
||||
pytest.fail(
|
||||
'Never rxed proxied `RemoteActorError[RuntimeError]` !?'
|
||||
)
|
||||
|
||||
assert isinstance(res, ContextCancelled)
|
||||
assert client_ctx.cancel_acked
|
||||
assert res.canceller == root.uid
|
||||
|
||||
except RemoteActorError as rae:
|
||||
_err = rae
|
||||
assert raise_sub_spawn_error_after
|
||||
|
||||
# since this is a "relayed error" via the client
|
||||
# sub-actor, it is expected to be
|
||||
# a `RemoteActorError` boxing another
|
||||
# `RemoteActorError` otherwise known as
|
||||
# an "inception" (from `trio`'s parlance)
|
||||
# ((or maybe a "Matryoshka" and/or "matron"
|
||||
# in our own working parlance)) which
|
||||
# contains the source error from the
|
||||
# little_bro: a `RuntimeError`.
|
||||
#
|
||||
check_inner_rte(rae)
|
||||
assert rae.relay_uid == client.chan.uid
|
||||
assert rae.src_uid == sub.chan.uid
|
||||
|
||||
assert not client_ctx.cancel_acked
|
||||
assert (
|
||||
client_ctx.maybe_error
|
||||
is client_ctx.outcome
|
||||
is rae
|
||||
)
|
||||
raise
|
||||
# await tractor.pause()
|
||||
|
||||
else:
|
||||
assert not raise_sub_spawn_error_after
|
||||
|
||||
# cancelling the spawner sub should
|
||||
# transitively cancel it's sub, the little
|
||||
# bruv.
|
||||
print('root cancelling server/client sub-actors')
|
||||
await spawn_ctx.cancel()
|
||||
async with tractor.find_actor(
|
||||
name=peer_name,
|
||||
) as sub:
|
||||
assert not sub
|
||||
|
||||
# await server.cancel_actor()
|
||||
print('root cancelling server/client sub-actors')
|
||||
|
||||
except RemoteActorError as rae:
|
||||
# XXX more-or-less same as above handler
|
||||
# this is just making sure the error bubbles out
|
||||
# of the
|
||||
_err = rae
|
||||
assert raise_sub_spawn_error_after
|
||||
raise
|
||||
# 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:
|
||||
_ctxc = ctxc
|
||||
print(
|
||||
f'{root.uid} caught ctxc from ctx with {client_ctx.chan.uid}\n'
|
||||
f'{repr(ctxc)}\n'
|
||||
)
|
||||
|
||||
if not raise_sub_spawn_error_after:
|
||||
assert ctxc.canceller == root.uid
|
||||
else:
|
||||
assert ctxc.canceller == spawner_uid
|
||||
|
||||
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
|
||||
|
||||
if raise_sub_spawn_error_after:
|
||||
pytest.fail(
|
||||
'context block(s) in PARENT never raised?!?'
|
||||
)
|
||||
|
||||
if not raise_sub_spawn_error_after:
|
||||
# assert spawn_ctx.cancel_acked
|
||||
assert spawn_ctx.cancel_acked
|
||||
assert client_ctx.cancel_acked
|
||||
|
@ -1189,12 +1054,4 @@ def test_peer_spawns_and_cancels_service_subactor(
|
|||
|
||||
# assert spawn_ctx.cancelled_caught
|
||||
|
||||
if raise_sub_spawn_error_after:
|
||||
with pytest.raises(RemoteActorError) as excinfo:
|
||||
trio.run(main)
|
||||
|
||||
rae: RemoteActorError = excinfo.value
|
||||
check_inner_rte(rae)
|
||||
|
||||
else:
|
||||
trio.run(main)
|
||||
|
|
|
@ -58,7 +58,7 @@ async def context_stream(
|
|||
|
||||
|
||||
async def stream_from_single_subactor(
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
start_method,
|
||||
stream_func,
|
||||
):
|
||||
|
@ -67,7 +67,7 @@ async def stream_from_single_subactor(
|
|||
# only one per host address, spawns an actor if None
|
||||
|
||||
async with tractor.open_nursery(
|
||||
registry_addrs=[reg_addr],
|
||||
arbiter_addr=arb_addr,
|
||||
start_method=start_method,
|
||||
) as nursery:
|
||||
|
||||
|
@ -118,13 +118,13 @@ async def stream_from_single_subactor(
|
|||
@pytest.mark.parametrize(
|
||||
'stream_func', [async_gen_stream, context_stream]
|
||||
)
|
||||
def test_stream_from_single_subactor(reg_addr, start_method, stream_func):
|
||||
def test_stream_from_single_subactor(arb_addr, start_method, stream_func):
|
||||
"""Verify streaming from a spawned async generator.
|
||||
"""
|
||||
trio.run(
|
||||
partial(
|
||||
stream_from_single_subactor,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
start_method,
|
||||
stream_func=stream_func,
|
||||
),
|
||||
|
@ -228,14 +228,14 @@ async def a_quadruple_example():
|
|||
return result_stream
|
||||
|
||||
|
||||
async def cancel_after(wait, reg_addr):
|
||||
async with tractor.open_root_actor(registry_addrs=[reg_addr]):
|
||||
async def cancel_after(wait, arb_addr):
|
||||
async with tractor.open_root_actor(arbiter_addr=arb_addr):
|
||||
with trio.move_on_after(wait):
|
||||
return await a_quadruple_example()
|
||||
|
||||
|
||||
@pytest.fixture(scope='module')
|
||||
def time_quad_ex(reg_addr, ci_env, spawn_backend):
|
||||
def time_quad_ex(arb_addr, ci_env, spawn_backend):
|
||||
if spawn_backend == 'mp':
|
||||
"""no idea but the mp *nix runs are flaking out here often...
|
||||
"""
|
||||
|
@ -243,7 +243,7 @@ def time_quad_ex(reg_addr, ci_env, spawn_backend):
|
|||
|
||||
timeout = 7 if platform.system() in ('Windows', 'Darwin') else 4
|
||||
start = time.time()
|
||||
results = trio.run(cancel_after, timeout, reg_addr)
|
||||
results = trio.run(cancel_after, timeout, arb_addr)
|
||||
diff = time.time() - start
|
||||
assert results
|
||||
return results, diff
|
||||
|
@ -263,14 +263,14 @@ def test_a_quadruple_example(time_quad_ex, ci_env, spawn_backend):
|
|||
list(map(lambda i: i/10, range(3, 9)))
|
||||
)
|
||||
def test_not_fast_enough_quad(
|
||||
reg_addr, time_quad_ex, cancel_delay, ci_env, spawn_backend
|
||||
arb_addr, time_quad_ex, cancel_delay, ci_env, spawn_backend
|
||||
):
|
||||
"""Verify we can cancel midway through the quad example and all actors
|
||||
cancel gracefully.
|
||||
"""
|
||||
results, diff = time_quad_ex
|
||||
delay = max(diff - cancel_delay, 0)
|
||||
results = trio.run(cancel_after, delay, reg_addr)
|
||||
results = trio.run(cancel_after, delay, arb_addr)
|
||||
system = platform.system()
|
||||
if system in ('Windows', 'Darwin') and results is not None:
|
||||
# In CI envoirments it seems later runs are quicker then the first
|
||||
|
@ -283,7 +283,7 @@ def test_not_fast_enough_quad(
|
|||
|
||||
@tractor_test
|
||||
async def test_respawn_consumer_task(
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
spawn_backend,
|
||||
loglevel,
|
||||
):
|
||||
|
|
|
@ -24,7 +24,7 @@ async def test_no_runtime():
|
|||
|
||||
|
||||
@tractor_test
|
||||
async def test_self_is_registered(reg_addr):
|
||||
async def test_self_is_registered(arb_addr):
|
||||
"Verify waiting on the arbiter to register itself using the standard api."
|
||||
actor = tractor.current_actor()
|
||||
assert actor.is_arbiter
|
||||
|
@ -34,20 +34,20 @@ async def test_self_is_registered(reg_addr):
|
|||
|
||||
|
||||
@tractor_test
|
||||
async def test_self_is_registered_localportal(reg_addr):
|
||||
async def test_self_is_registered_localportal(arb_addr):
|
||||
"Verify waiting on the arbiter to register itself using a local portal."
|
||||
actor = tractor.current_actor()
|
||||
assert actor.is_arbiter
|
||||
async with tractor.get_arbiter(*reg_addr) as portal:
|
||||
async with tractor.get_arbiter(*arb_addr) as portal:
|
||||
assert isinstance(portal, tractor._portal.LocalPortal)
|
||||
|
||||
with trio.fail_after(0.2):
|
||||
sockaddr = await portal.run_from_ns(
|
||||
'self', 'wait_for_actor', name='root')
|
||||
assert sockaddr[0] == reg_addr
|
||||
assert sockaddr[0] == arb_addr
|
||||
|
||||
|
||||
def test_local_actor_async_func(reg_addr):
|
||||
def test_local_actor_async_func(arb_addr):
|
||||
"""Verify a simple async function in-process.
|
||||
"""
|
||||
nums = []
|
||||
|
@ -55,7 +55,7 @@ def test_local_actor_async_func(reg_addr):
|
|||
async def print_loop():
|
||||
|
||||
async with tractor.open_root_actor(
|
||||
registry_addrs=[reg_addr],
|
||||
arbiter_addr=arb_addr,
|
||||
):
|
||||
# arbiter is started in-proc if dne
|
||||
assert tractor.current_actor().is_arbiter
|
||||
|
|
|
@ -30,9 +30,9 @@ def test_abort_on_sigint(daemon):
|
|||
|
||||
|
||||
@tractor_test
|
||||
async def test_cancel_remote_arbiter(daemon, reg_addr):
|
||||
async def test_cancel_remote_arbiter(daemon, arb_addr):
|
||||
assert not tractor.current_actor().is_arbiter
|
||||
async with tractor.get_arbiter(*reg_addr) as portal:
|
||||
async with tractor.get_arbiter(*arb_addr) as portal:
|
||||
await portal.cancel_actor()
|
||||
|
||||
time.sleep(0.1)
|
||||
|
@ -41,16 +41,16 @@ async def test_cancel_remote_arbiter(daemon, reg_addr):
|
|||
|
||||
# no arbiter socket should exist
|
||||
with pytest.raises(OSError):
|
||||
async with tractor.get_arbiter(*reg_addr) as portal:
|
||||
async with tractor.get_arbiter(*arb_addr) as portal:
|
||||
pass
|
||||
|
||||
|
||||
def test_register_duplicate_name(daemon, reg_addr):
|
||||
def test_register_duplicate_name(daemon, arb_addr):
|
||||
|
||||
async def main():
|
||||
|
||||
async with tractor.open_nursery(
|
||||
registry_addrs=[reg_addr],
|
||||
arbiter_addr=arb_addr,
|
||||
) as n:
|
||||
|
||||
assert not tractor.current_actor().is_arbiter
|
||||
|
|
|
@ -159,7 +159,7 @@ async def test_required_args(callwith_expecterror):
|
|||
)
|
||||
def test_multi_actor_subs_arbiter_pub(
|
||||
loglevel,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
pub_actor,
|
||||
):
|
||||
"""Try out the neato @pub decorator system.
|
||||
|
@ -169,7 +169,7 @@ def test_multi_actor_subs_arbiter_pub(
|
|||
async def main():
|
||||
|
||||
async with tractor.open_nursery(
|
||||
registry_addrs=[reg_addr],
|
||||
arbiter_addr=arb_addr,
|
||||
enable_modules=[__name__],
|
||||
) as n:
|
||||
|
||||
|
@ -254,12 +254,12 @@ def test_multi_actor_subs_arbiter_pub(
|
|||
|
||||
def test_single_subactor_pub_multitask_subs(
|
||||
loglevel,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
):
|
||||
async def main():
|
||||
|
||||
async with tractor.open_nursery(
|
||||
registry_addrs=[reg_addr],
|
||||
arbiter_addr=arb_addr,
|
||||
enable_modules=[__name__],
|
||||
) as n:
|
||||
|
||||
|
|
|
@ -15,19 +15,9 @@ async def sleep_back_actor(
|
|||
func_name,
|
||||
func_defined,
|
||||
exposed_mods,
|
||||
*,
|
||||
reg_addr: tuple,
|
||||
):
|
||||
if actor_name:
|
||||
async with tractor.find_actor(
|
||||
actor_name,
|
||||
# NOTE: must be set manually since
|
||||
# the subactor doesn't have the reg_addr
|
||||
# fixture code run in it!
|
||||
# TODO: maybe we should just set this once in the
|
||||
# _state mod and derive to all children?
|
||||
registry_addrs=[reg_addr],
|
||||
) as portal:
|
||||
async with tractor.find_actor(actor_name) as portal:
|
||||
try:
|
||||
await portal.run(__name__, func_name)
|
||||
except tractor.RemoteActorError as err:
|
||||
|
@ -36,7 +26,7 @@ async def sleep_back_actor(
|
|||
if not exposed_mods:
|
||||
expect = tractor.ModuleNotExposed
|
||||
|
||||
assert err.boxed_type is expect
|
||||
assert err.type is expect
|
||||
raise
|
||||
else:
|
||||
await trio.sleep(float('inf'))
|
||||
|
@ -62,17 +52,11 @@ async def short_sleep():
|
|||
'fail_on_syntax',
|
||||
],
|
||||
)
|
||||
def test_rpc_errors(
|
||||
reg_addr,
|
||||
to_call,
|
||||
testdir,
|
||||
):
|
||||
'''
|
||||
Test errors when making various RPC requests to an actor
|
||||
def test_rpc_errors(arb_addr, to_call, testdir):
|
||||
"""Test errors when making various RPC requests to an actor
|
||||
that either doesn't have the requested module exposed or doesn't define
|
||||
the named function.
|
||||
|
||||
'''
|
||||
"""
|
||||
exposed_mods, funcname, inside_err = to_call
|
||||
subactor_exposed_mods = []
|
||||
func_defined = globals().get(funcname, False)
|
||||
|
@ -100,13 +84,8 @@ def test_rpc_errors(
|
|||
|
||||
# spawn a subactor which calls us back
|
||||
async with tractor.open_nursery(
|
||||
registry_addrs=[reg_addr],
|
||||
arbiter_addr=arb_addr,
|
||||
enable_modules=exposed_mods.copy(),
|
||||
|
||||
# NOTE: will halt test in REPL if uncommented, so only
|
||||
# do that if actually debugging subactor but keep it
|
||||
# disabled for the test.
|
||||
# debug_mode=True,
|
||||
) as n:
|
||||
|
||||
actor = tractor.current_actor()
|
||||
|
@ -123,7 +102,6 @@ def test_rpc_errors(
|
|||
exposed_mods=exposed_mods,
|
||||
func_defined=True if func_defined else False,
|
||||
enable_modules=subactor_exposed_mods,
|
||||
reg_addr=reg_addr,
|
||||
)
|
||||
|
||||
def run():
|
||||
|
@ -150,4 +128,4 @@ def test_rpc_errors(
|
|||
))
|
||||
|
||||
if getattr(value, 'type', None):
|
||||
assert value.boxed_type is inside_err
|
||||
assert value.type is inside_err
|
||||
|
|
|
@ -16,14 +16,14 @@ data_to_pass_down = {'doggy': 10, 'kitty': 4}
|
|||
async def spawn(
|
||||
is_arbiter: bool,
|
||||
data: dict,
|
||||
reg_addr: tuple[str, int],
|
||||
arb_addr: tuple[str, int],
|
||||
):
|
||||
namespaces = [__name__]
|
||||
|
||||
await trio.sleep(0.1)
|
||||
|
||||
async with tractor.open_root_actor(
|
||||
arbiter_addr=reg_addr,
|
||||
arbiter_addr=arb_addr,
|
||||
):
|
||||
|
||||
actor = tractor.current_actor()
|
||||
|
@ -32,7 +32,8 @@ async def spawn(
|
|||
|
||||
if actor.is_arbiter:
|
||||
|
||||
async with tractor.open_nursery() as nursery:
|
||||
async with tractor.open_nursery(
|
||||
) as nursery:
|
||||
|
||||
# forks here
|
||||
portal = await nursery.run_in_actor(
|
||||
|
@ -40,7 +41,7 @@ async def spawn(
|
|||
is_arbiter=False,
|
||||
name='sub-actor',
|
||||
data=data,
|
||||
reg_addr=reg_addr,
|
||||
arb_addr=arb_addr,
|
||||
enable_modules=namespaces,
|
||||
)
|
||||
|
||||
|
@ -54,14 +55,12 @@ async def spawn(
|
|||
return 10
|
||||
|
||||
|
||||
def test_local_arbiter_subactor_global_state(
|
||||
reg_addr,
|
||||
):
|
||||
def test_local_arbiter_subactor_global_state(arb_addr):
|
||||
result = trio.run(
|
||||
spawn,
|
||||
True,
|
||||
data_to_pass_down,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
)
|
||||
assert result == 10
|
||||
|
||||
|
@ -141,7 +140,7 @@ async def check_loglevel(level):
|
|||
def test_loglevel_propagated_to_subactor(
|
||||
start_method,
|
||||
capfd,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
):
|
||||
if start_method == 'mp_forkserver':
|
||||
pytest.skip(
|
||||
|
@ -153,7 +152,7 @@ def test_loglevel_propagated_to_subactor(
|
|||
async with tractor.open_nursery(
|
||||
name='arbiter',
|
||||
start_method=start_method,
|
||||
arbiter_addr=reg_addr,
|
||||
arbiter_addr=arb_addr,
|
||||
|
||||
) as tn:
|
||||
await tn.run_in_actor(
|
||||
|
|
|
@ -66,13 +66,13 @@ async def ensure_sequence(
|
|||
async def open_sequence_streamer(
|
||||
|
||||
sequence: list[int],
|
||||
reg_addr: tuple[str, int],
|
||||
arb_addr: tuple[str, int],
|
||||
start_method: str,
|
||||
|
||||
) -> tractor.MsgStream:
|
||||
|
||||
async with tractor.open_nursery(
|
||||
arbiter_addr=reg_addr,
|
||||
arbiter_addr=arb_addr,
|
||||
start_method=start_method,
|
||||
) as tn:
|
||||
|
||||
|
@ -93,7 +93,7 @@ async def open_sequence_streamer(
|
|||
|
||||
|
||||
def test_stream_fan_out_to_local_subscriptions(
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
start_method,
|
||||
):
|
||||
|
||||
|
@ -103,7 +103,7 @@ def test_stream_fan_out_to_local_subscriptions(
|
|||
|
||||
async with open_sequence_streamer(
|
||||
sequence,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
start_method,
|
||||
) as stream:
|
||||
|
||||
|
@ -138,7 +138,7 @@ def test_stream_fan_out_to_local_subscriptions(
|
|||
]
|
||||
)
|
||||
def test_consumer_and_parent_maybe_lag(
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
start_method,
|
||||
task_delays,
|
||||
):
|
||||
|
@ -150,7 +150,7 @@ def test_consumer_and_parent_maybe_lag(
|
|||
|
||||
async with open_sequence_streamer(
|
||||
sequence,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
start_method,
|
||||
) as stream:
|
||||
|
||||
|
@ -211,7 +211,7 @@ def test_consumer_and_parent_maybe_lag(
|
|||
|
||||
|
||||
def test_faster_task_to_recv_is_cancelled_by_slower(
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
start_method,
|
||||
):
|
||||
'''
|
||||
|
@ -225,7 +225,7 @@ def test_faster_task_to_recv_is_cancelled_by_slower(
|
|||
|
||||
async with open_sequence_streamer(
|
||||
sequence,
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
start_method,
|
||||
|
||||
) as stream:
|
||||
|
@ -302,7 +302,7 @@ def test_subscribe_errors_after_close():
|
|||
|
||||
|
||||
def test_ensure_slow_consumers_lag_out(
|
||||
reg_addr,
|
||||
arb_addr,
|
||||
start_method,
|
||||
):
|
||||
'''This is a pure local task test; no tractor
|
||||
|
|
|
@ -18,48 +18,74 @@
|
|||
tractor: structured concurrent ``trio``-"actors".
|
||||
|
||||
"""
|
||||
|
||||
from ._clustering import (
|
||||
open_actor_cluster as open_actor_cluster,
|
||||
)
|
||||
from ._clustering import open_actor_cluster
|
||||
from ._context import (
|
||||
Context as Context, # the type
|
||||
context as context, # a func-decorator
|
||||
Context, # the type
|
||||
context, # a func-decorator
|
||||
)
|
||||
from ._streaming import (
|
||||
MsgStream as MsgStream,
|
||||
stream as stream,
|
||||
MsgStream,
|
||||
stream,
|
||||
)
|
||||
from ._discovery import (
|
||||
get_arbiter as get_arbiter,
|
||||
find_actor as find_actor,
|
||||
wait_for_actor as wait_for_actor,
|
||||
query_actor as query_actor,
|
||||
)
|
||||
from ._supervise import (
|
||||
open_nursery as open_nursery,
|
||||
ActorNursery as ActorNursery,
|
||||
get_arbiter,
|
||||
find_actor,
|
||||
wait_for_actor,
|
||||
query_actor,
|
||||
)
|
||||
from ._supervise import open_nursery
|
||||
from ._state import (
|
||||
current_actor as current_actor,
|
||||
is_root_process as is_root_process,
|
||||
current_actor,
|
||||
is_root_process,
|
||||
)
|
||||
from ._exceptions import (
|
||||
RemoteActorError as RemoteActorError,
|
||||
ModuleNotExposed as ModuleNotExposed,
|
||||
ContextCancelled as ContextCancelled,
|
||||
RemoteActorError,
|
||||
ModuleNotExposed,
|
||||
ContextCancelled,
|
||||
)
|
||||
from .devx import (
|
||||
breakpoint as breakpoint,
|
||||
pause as pause,
|
||||
pause_from_sync as pause_from_sync,
|
||||
post_mortem as post_mortem,
|
||||
breakpoint,
|
||||
pause,
|
||||
pause_from_sync,
|
||||
post_mortem,
|
||||
)
|
||||
from . import msg as msg
|
||||
from . import msg
|
||||
from ._root import (
|
||||
run_daemon as run_daemon,
|
||||
open_root_actor as open_root_actor,
|
||||
run_daemon,
|
||||
open_root_actor,
|
||||
)
|
||||
from ._ipc import Channel as Channel
|
||||
from ._portal import Portal as Portal
|
||||
from ._runtime import Actor as Actor
|
||||
from ._ipc import Channel
|
||||
from ._portal import Portal
|
||||
from ._runtime import Actor
|
||||
|
||||
|
||||
__all__ = [
|
||||
'Actor',
|
||||
'BaseExceptionGroup',
|
||||
'Channel',
|
||||
'Context',
|
||||
'ContextCancelled',
|
||||
'ModuleNotExposed',
|
||||
'MsgStream',
|
||||
'Portal',
|
||||
'RemoteActorError',
|
||||
'breakpoint',
|
||||
'context',
|
||||
'current_actor',
|
||||
'find_actor',
|
||||
'query_actor',
|
||||
'get_arbiter',
|
||||
'is_root_process',
|
||||
'msg',
|
||||
'open_actor_cluster',
|
||||
'open_nursery',
|
||||
'open_root_actor',
|
||||
'pause',
|
||||
'post_mortem',
|
||||
'pause_from_sync',
|
||||
'query_actor',
|
||||
'run_daemon',
|
||||
'stream',
|
||||
'to_asyncio',
|
||||
'wait_for_actor',
|
||||
]
|
||||
|
|
|
@ -36,7 +36,6 @@ def parse_ipaddr(arg):
|
|||
|
||||
|
||||
if __name__ == "__main__":
|
||||
__tracebackhide__: bool = True
|
||||
|
||||
parser = argparse.ArgumentParser()
|
||||
parser.add_argument("--uid", type=parse_uid)
|
||||
|
|
|
@ -351,7 +351,7 @@ class Context:
|
|||
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 `._rpc._invoke()` as a `ctx` arg
|
||||
- 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()``.
|
||||
|
@ -361,10 +361,10 @@ class Context:
|
|||
`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 ``._rpc._invoke()``.
|
||||
always allocated inside ``._runtime._invoke()``.
|
||||
|
||||
TODO: more detailed writeup on cancellation, error and
|
||||
streaming semantics..
|
||||
# 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
|
||||
|
@ -1209,9 +1209,7 @@ class Context:
|
|||
# await pause()
|
||||
log.warning(
|
||||
'Stream was terminated by EoC\n\n'
|
||||
# NOTE: won't show the error <Type> but
|
||||
# does show txt followed by IPC msg.
|
||||
f'{str(eoc)}\n'
|
||||
f'{repr(eoc)}\n'
|
||||
)
|
||||
|
||||
finally:
|
||||
|
@ -1308,7 +1306,7 @@ class Context:
|
|||
# `._cancel_called == True`.
|
||||
not raise_overrun_from_self
|
||||
and isinstance(remote_error, RemoteActorError)
|
||||
and remote_error.msgdata['boxed_type_str'] == 'StreamOverrun'
|
||||
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
|
||||
|
@ -1885,19 +1883,6 @@ class Context:
|
|||
return False
|
||||
|
||||
|
||||
# TODO: exception tb masking by using a manual
|
||||
# `.__aexit__()`/.__aenter__()` pair on a type?
|
||||
# => currently this is one of the few places we can't easily
|
||||
# mask errors - on the exit side of a `Portal.open_context()`..
|
||||
# there's # => currently this is one of the few places we can't
|
||||
# there's 2 ways to approach it:
|
||||
# - manually write an @acm type as per above
|
||||
# - use `contextlib.AsyncContextDecorator` to override the default
|
||||
# impl to suppress traceback frames:
|
||||
# * https://docs.python.org/3/library/contextlib.html#contextlib.AsyncContextDecorator
|
||||
# * https://docs.python.org/3/library/contextlib.html#contextlib.ContextDecorator
|
||||
# - also we could just override directly the underlying
|
||||
# `contextlib._AsyncGeneratorContextManager`?
|
||||
@acm
|
||||
async def open_context_from_portal(
|
||||
portal: Portal,
|
||||
|
|
|
@ -15,45 +15,32 @@
|
|||
# along with this program. If not, see <https://www.gnu.org/licenses/>.
|
||||
|
||||
"""
|
||||
Discovery (protocols) API for automatic addressing and location
|
||||
management of (service) actors.
|
||||
Actor discovery API.
|
||||
|
||||
"""
|
||||
from __future__ import annotations
|
||||
from typing import (
|
||||
Optional,
|
||||
Union,
|
||||
AsyncGenerator,
|
||||
AsyncContextManager,
|
||||
TYPE_CHECKING,
|
||||
)
|
||||
from contextlib import asynccontextmanager as acm
|
||||
import warnings
|
||||
|
||||
from .trionics import gather_contexts
|
||||
from ._ipc import _connect_chan, Channel
|
||||
from ._portal import (
|
||||
Portal,
|
||||
open_portal,
|
||||
LocalPortal,
|
||||
)
|
||||
from ._state import (
|
||||
current_actor,
|
||||
_runtime_vars,
|
||||
)
|
||||
|
||||
|
||||
if TYPE_CHECKING:
|
||||
from ._runtime import Actor
|
||||
from ._state import current_actor, _runtime_vars
|
||||
|
||||
|
||||
@acm
|
||||
async def get_registry(
|
||||
async def get_arbiter(
|
||||
|
||||
host: str,
|
||||
port: int,
|
||||
|
||||
) -> AsyncGenerator[
|
||||
Portal | LocalPortal | None,
|
||||
None,
|
||||
]:
|
||||
) -> AsyncGenerator[Union[Portal, LocalPortal], None]:
|
||||
'''
|
||||
Return a portal instance connected to a local or remote
|
||||
arbiter.
|
||||
|
@ -64,33 +51,16 @@ async def get_registry(
|
|||
if not actor:
|
||||
raise RuntimeError("No actor instance has been defined yet?")
|
||||
|
||||
if actor.is_registrar:
|
||||
if actor.is_arbiter:
|
||||
# we're already the arbiter
|
||||
# (likely a re-entrant call from the arbiter actor)
|
||||
yield LocalPortal(
|
||||
actor,
|
||||
Channel((host, port))
|
||||
)
|
||||
yield LocalPortal(actor, Channel((host, port)))
|
||||
else:
|
||||
async with (
|
||||
_connect_chan(host, port) as chan,
|
||||
open_portal(chan) as regstr_ptl,
|
||||
):
|
||||
yield regstr_ptl
|
||||
async with _connect_chan(host, port) as chan:
|
||||
|
||||
async with open_portal(chan) as arb_portal:
|
||||
|
||||
|
||||
# TODO: deprecate and this remove _arbiter form!
|
||||
@acm
|
||||
async def get_arbiter(*args, **kwargs):
|
||||
warnings.warn(
|
||||
'`tractor.get_arbiter()` is now deprecated!\n'
|
||||
'Use `.get_registry()` instead!',
|
||||
DeprecationWarning,
|
||||
stacklevel=2,
|
||||
)
|
||||
async with get_registry(*args, **kwargs) as to_yield:
|
||||
yield to_yield
|
||||
yield arb_portal
|
||||
|
||||
|
||||
@acm
|
||||
|
@ -98,80 +68,51 @@ async def get_root(
|
|||
**kwargs,
|
||||
) -> AsyncGenerator[Portal, None]:
|
||||
|
||||
# TODO: rename mailbox to `_root_maddr` when we finally
|
||||
# add and impl libp2p multi-addrs?
|
||||
host, port = _runtime_vars['_root_mailbox']
|
||||
assert host is not None
|
||||
|
||||
async with (
|
||||
_connect_chan(host, port) as chan,
|
||||
open_portal(chan, **kwargs) as portal,
|
||||
):
|
||||
async with _connect_chan(host, port) as chan:
|
||||
async with open_portal(chan, **kwargs) as portal:
|
||||
yield portal
|
||||
|
||||
|
||||
@acm
|
||||
async def query_actor(
|
||||
name: str,
|
||||
arbiter_sockaddr: tuple[str, int] | None = None,
|
||||
regaddr: tuple[str, int] | None = None,
|
||||
arbiter_sockaddr: Optional[tuple[str, int]] = None,
|
||||
|
||||
) -> AsyncGenerator[
|
||||
tuple[str, int] | None,
|
||||
None,
|
||||
]:
|
||||
) -> AsyncGenerator[tuple[str, int], None]:
|
||||
'''
|
||||
Make a transport address lookup for an actor name to a specific
|
||||
registrar.
|
||||
Simple address lookup for a given actor name.
|
||||
|
||||
Returns the (socket) address or ``None`` if no entry under that
|
||||
name exists for the given registrar listening @ `regaddr`.
|
||||
Returns the (socket) address or ``None``.
|
||||
|
||||
'''
|
||||
actor: Actor = current_actor()
|
||||
if (
|
||||
name == 'registrar'
|
||||
and actor.is_registrar
|
||||
):
|
||||
raise RuntimeError(
|
||||
'The current actor IS the registry!?'
|
||||
)
|
||||
actor = current_actor()
|
||||
async with get_arbiter(
|
||||
*arbiter_sockaddr or actor._arb_addr
|
||||
) as arb_portal:
|
||||
|
||||
if arbiter_sockaddr is not None:
|
||||
warnings.warn(
|
||||
'`tractor.query_actor(regaddr=<blah>)` is deprecated.\n'
|
||||
'Use `registry_addrs: list[tuple]` instead!',
|
||||
DeprecationWarning,
|
||||
stacklevel=2,
|
||||
)
|
||||
regaddr: list[tuple[str, int]] = arbiter_sockaddr
|
||||
|
||||
reg_portal: Portal
|
||||
regaddr: tuple[str, int] = regaddr or actor.reg_addrs[0]
|
||||
async with get_registry(*regaddr) as reg_portal:
|
||||
# TODO: return portals to all available actors - for now
|
||||
# just the last one that registered
|
||||
sockaddr: tuple[str, int] = await reg_portal.run_from_ns(
|
||||
sockaddr = await arb_portal.run_from_ns(
|
||||
'self',
|
||||
'find_actor',
|
||||
name=name,
|
||||
)
|
||||
yield sockaddr
|
||||
|
||||
# TODO: return portals to all available actors - for now just
|
||||
# the last one that registered
|
||||
if name == 'arbiter' and actor.is_arbiter:
|
||||
raise RuntimeError("The current actor is the arbiter")
|
||||
|
||||
yield sockaddr if sockaddr else None
|
||||
|
||||
|
||||
@acm
|
||||
async def find_actor(
|
||||
name: str,
|
||||
arbiter_sockaddr: tuple[str, int]|None = None,
|
||||
registry_addrs: list[tuple[str, int]]|None = None,
|
||||
arbiter_sockaddr: tuple[str, int] | None = None
|
||||
|
||||
only_first: bool = True,
|
||||
raise_on_none: bool = False,
|
||||
|
||||
) -> AsyncGenerator[
|
||||
Portal | list[Portal] | None,
|
||||
None,
|
||||
]:
|
||||
) -> AsyncGenerator[Optional[Portal], None]:
|
||||
'''
|
||||
Ask the arbiter to find actor(s) by name.
|
||||
|
||||
|
@ -179,23 +120,11 @@ async def find_actor(
|
|||
known to the arbiter.
|
||||
|
||||
'''
|
||||
if arbiter_sockaddr is not None:
|
||||
warnings.warn(
|
||||
'`tractor.find_actor(arbiter_sockaddr=<blah>)` is deprecated.\n'
|
||||
'Use `registry_addrs: list[tuple]` instead!',
|
||||
DeprecationWarning,
|
||||
stacklevel=2,
|
||||
)
|
||||
registry_addrs: list[tuple[str, int]] = [arbiter_sockaddr]
|
||||
|
||||
@acm
|
||||
async def maybe_open_portal_from_reg_addr(
|
||||
addr: tuple[str, int],
|
||||
):
|
||||
async with query_actor(
|
||||
name=name,
|
||||
regaddr=addr,
|
||||
arbiter_sockaddr=arbiter_sockaddr,
|
||||
) as sockaddr:
|
||||
|
||||
if sockaddr:
|
||||
async with _connect_chan(*sockaddr) as chan:
|
||||
async with open_portal(chan) as portal:
|
||||
|
@ -203,59 +132,12 @@ async def find_actor(
|
|||
else:
|
||||
yield None
|
||||
|
||||
if not registry_addrs:
|
||||
# XXX NOTE: make sure to dynamically read the value on
|
||||
# every call since something may change it globally (eg.
|
||||
# like in our discovery test suite)!
|
||||
from . import _root
|
||||
registry_addrs = (
|
||||
_runtime_vars['_registry_addrs']
|
||||
or
|
||||
_root._default_lo_addrs
|
||||
)
|
||||
|
||||
maybe_portals: list[
|
||||
AsyncContextManager[tuple[str, int]]
|
||||
] = list(
|
||||
maybe_open_portal_from_reg_addr(addr)
|
||||
for addr in registry_addrs
|
||||
)
|
||||
|
||||
async with gather_contexts(
|
||||
mngrs=maybe_portals,
|
||||
) as portals:
|
||||
# log.runtime(
|
||||
# 'Gathered portals:\n'
|
||||
# f'{portals}'
|
||||
# )
|
||||
# NOTE: `gather_contexts()` will return a
|
||||
# `tuple[None, None, ..., None]` if no contact
|
||||
# can be made with any regstrar at any of the
|
||||
# N provided addrs!
|
||||
if not any(portals):
|
||||
if raise_on_none:
|
||||
raise RuntimeError(
|
||||
f'No actor "{name}" found registered @ {registry_addrs}'
|
||||
)
|
||||
yield None
|
||||
return
|
||||
|
||||
portals: list[Portal] = list(portals)
|
||||
if only_first:
|
||||
yield portals[0]
|
||||
|
||||
else:
|
||||
# TODO: currently this may return multiple portals
|
||||
# given there are multi-homed or multiple registrars..
|
||||
# SO, we probably need de-duplication logic?
|
||||
yield portals
|
||||
|
||||
|
||||
@acm
|
||||
async def wait_for_actor(
|
||||
name: str,
|
||||
arbiter_sockaddr: tuple[str, int] | None = None,
|
||||
registry_addr: tuple[str, int] | None = None,
|
||||
# registry_addr: tuple[str, int] | None = None,
|
||||
|
||||
) -> AsyncGenerator[Portal, None]:
|
||||
'''
|
||||
|
@ -264,31 +146,17 @@ async def wait_for_actor(
|
|||
A portal to the first registered actor is returned.
|
||||
|
||||
'''
|
||||
actor: Actor = current_actor()
|
||||
actor = current_actor()
|
||||
|
||||
if arbiter_sockaddr is not None:
|
||||
warnings.warn(
|
||||
'`tractor.wait_for_actor(arbiter_sockaddr=<foo>)` is deprecated.\n'
|
||||
'Use `registry_addr: tuple` instead!',
|
||||
DeprecationWarning,
|
||||
stacklevel=2,
|
||||
)
|
||||
registry_addr: tuple[str, int] = arbiter_sockaddr
|
||||
|
||||
# TODO: use `.trionics.gather_contexts()` like
|
||||
# above in `find_actor()` as well?
|
||||
reg_portal: Portal
|
||||
regaddr: tuple[str, int] = registry_addr or actor.reg_addrs[0]
|
||||
async with get_registry(*regaddr) as reg_portal:
|
||||
sockaddrs = await reg_portal.run_from_ns(
|
||||
async with get_arbiter(
|
||||
*arbiter_sockaddr or actor._arb_addr,
|
||||
) as arb_portal:
|
||||
sockaddrs = await arb_portal.run_from_ns(
|
||||
'self',
|
||||
'wait_for_actor',
|
||||
name=name,
|
||||
)
|
||||
|
||||
# get latest registered addr by default?
|
||||
# TODO: offer multi-portal yields in multi-homed case?
|
||||
sockaddr: tuple[str, int] = sockaddrs[-1]
|
||||
sockaddr = sockaddrs[-1]
|
||||
|
||||
async with _connect_chan(*sockaddr) as chan:
|
||||
async with open_portal(chan) as portal:
|
||||
|
|
|
@ -47,8 +47,8 @@ log = get_logger(__name__)
|
|||
|
||||
def _mp_main(
|
||||
|
||||
actor: Actor,
|
||||
accept_addrs: list[tuple[str, int]],
|
||||
actor: Actor, # type: ignore
|
||||
accept_addr: tuple[str, int],
|
||||
forkserver_info: tuple[Any, Any, Any, Any, Any],
|
||||
start_method: SpawnMethodKey,
|
||||
parent_addr: tuple[str, int] | None = None,
|
||||
|
@ -77,8 +77,8 @@ def _mp_main(
|
|||
log.debug(f"parent_addr is {parent_addr}")
|
||||
trio_main = partial(
|
||||
async_main,
|
||||
actor=actor,
|
||||
accept_addrs=accept_addrs,
|
||||
actor,
|
||||
accept_addr,
|
||||
parent_addr=parent_addr
|
||||
)
|
||||
try:
|
||||
|
@ -96,7 +96,7 @@ def _mp_main(
|
|||
|
||||
def _trio_main(
|
||||
|
||||
actor: Actor,
|
||||
actor: Actor, # type: ignore
|
||||
*,
|
||||
parent_addr: tuple[str, int] | None = None,
|
||||
infect_asyncio: bool = False,
|
||||
|
@ -106,7 +106,6 @@ def _trio_main(
|
|||
Entry point for a `trio_run_in_process` subactor.
|
||||
|
||||
'''
|
||||
__tracebackhide__: bool = True
|
||||
_state._current_actor = actor
|
||||
trio_main = partial(
|
||||
async_main,
|
||||
|
|
|
@ -58,44 +58,16 @@ class InternalError(RuntimeError):
|
|||
'''
|
||||
|
||||
_body_fields: list[str] = [
|
||||
'boxed_type',
|
||||
'src_type',
|
||||
# TODO: format this better if we're going to include it.
|
||||
# 'relay_path',
|
||||
'src_uid',
|
||||
|
||||
# only in sub-types
|
||||
'src_actor_uid',
|
||||
'canceller',
|
||||
'sender',
|
||||
]
|
||||
|
||||
_msgdata_keys: list[str] = [
|
||||
'boxed_type_str',
|
||||
'type_str',
|
||||
] + _body_fields
|
||||
|
||||
|
||||
def get_err_type(type_name: str) -> BaseException|None:
|
||||
'''
|
||||
Look up an exception type by name from the set of locally
|
||||
known namespaces:
|
||||
|
||||
- `builtins`
|
||||
- `tractor._exceptions`
|
||||
- `trio`
|
||||
|
||||
'''
|
||||
for ns in [
|
||||
builtins,
|
||||
_this_mod,
|
||||
trio,
|
||||
]:
|
||||
if type_ref := getattr(
|
||||
ns,
|
||||
type_name,
|
||||
False,
|
||||
):
|
||||
return type_ref
|
||||
|
||||
|
||||
# TODO: rename to just `RemoteError`?
|
||||
class RemoteActorError(Exception):
|
||||
|
@ -109,14 +81,13 @@ class RemoteActorError(Exception):
|
|||
|
||||
'''
|
||||
reprol_fields: list[str] = [
|
||||
'src_uid',
|
||||
'relay_path',
|
||||
'src_actor_uid',
|
||||
]
|
||||
|
||||
def __init__(
|
||||
self,
|
||||
message: str,
|
||||
boxed_type: Type[BaseException]|None = None,
|
||||
suberror_type: Type[BaseException] | None = None,
|
||||
**msgdata
|
||||
|
||||
) -> None:
|
||||
|
@ -130,112 +101,20 @@ class RemoteActorError(Exception):
|
|||
# - .remote_type
|
||||
# also pertains to our long long oustanding issue XD
|
||||
# https://github.com/goodboy/tractor/issues/5
|
||||
#
|
||||
# TODO: always set ._boxed_type` as `None` by default
|
||||
# and instead render if from `.boxed_type_str`?
|
||||
self._boxed_type: BaseException = boxed_type
|
||||
self._src_type: BaseException|None = None
|
||||
self.boxed_type: str = suberror_type
|
||||
self.msgdata: dict[str, Any] = msgdata
|
||||
|
||||
# TODO: mask out eventually or place in `pack_error()`
|
||||
# pre-`return` lines?
|
||||
# sanity on inceptions
|
||||
if boxed_type is RemoteActorError:
|
||||
assert self.src_type_str != 'RemoteActorError'
|
||||
assert self.src_uid not in self.relay_path
|
||||
|
||||
# ensure type-str matches and round-tripping from that
|
||||
# str results in same error type.
|
||||
#
|
||||
# TODO NOTE: this is currently exclusively for the
|
||||
# `ContextCancelled(boxed_type=trio.Cancelled)` case as is
|
||||
# used inside `._rpc._invoke()` atm though probably we
|
||||
# should better emphasize that special (one off?) case
|
||||
# either by customizing `ContextCancelled.__init__()` or
|
||||
# through a special factor func?
|
||||
elif boxed_type:
|
||||
if not self.msgdata.get('boxed_type_str'):
|
||||
self.msgdata['boxed_type_str'] = str(
|
||||
type(boxed_type).__name__
|
||||
)
|
||||
|
||||
assert self.boxed_type_str == self.msgdata['boxed_type_str']
|
||||
assert self.boxed_type is boxed_type
|
||||
@property
|
||||
def type(self) -> str:
|
||||
return self.boxed_type
|
||||
|
||||
@property
|
||||
def src_type_str(self) -> str:
|
||||
'''
|
||||
String-name of the source error's type.
|
||||
|
||||
This should be the same as `.boxed_type_str` when unpacked
|
||||
at the first relay/hop's receiving actor.
|
||||
|
||||
'''
|
||||
return self.msgdata['src_type_str']
|
||||
def type_str(self) -> str:
|
||||
return str(type(self.boxed_type).__name__)
|
||||
|
||||
@property
|
||||
def src_type(self) -> str:
|
||||
'''
|
||||
Error type raised by original remote faulting actor.
|
||||
|
||||
'''
|
||||
if self._src_type is None:
|
||||
self._src_type = get_err_type(
|
||||
self.msgdata['src_type_str']
|
||||
)
|
||||
|
||||
return self._src_type
|
||||
|
||||
@property
|
||||
def boxed_type_str(self) -> str:
|
||||
'''
|
||||
String-name of the (last hop's) boxed error type.
|
||||
|
||||
'''
|
||||
return self.msgdata['boxed_type_str']
|
||||
|
||||
@property
|
||||
def boxed_type(self) -> str:
|
||||
'''
|
||||
Error type boxed by last actor IPC hop.
|
||||
|
||||
'''
|
||||
if self._boxed_type is None:
|
||||
self._boxed_type = get_err_type(
|
||||
self.msgdata['boxed_type_str']
|
||||
)
|
||||
|
||||
return self._boxed_type
|
||||
|
||||
@property
|
||||
def relay_path(self) -> list[tuple]:
|
||||
'''
|
||||
Return the list of actors which consecutively relayed
|
||||
a boxed `RemoteActorError` the src error up until THIS
|
||||
actor's hop.
|
||||
|
||||
NOTE: a `list` field with the same name is expected to be
|
||||
passed/updated in `.msgdata`.
|
||||
|
||||
'''
|
||||
return self.msgdata['relay_path']
|
||||
|
||||
@property
|
||||
def relay_uid(self) -> tuple[str, str]|None:
|
||||
return tuple(
|
||||
self.msgdata['relay_path'][-1]
|
||||
)
|
||||
|
||||
@property
|
||||
def src_uid(self) -> tuple[str, str]|None:
|
||||
if src_uid := (
|
||||
self.msgdata.get('src_uid')
|
||||
):
|
||||
return tuple(src_uid)
|
||||
# TODO: use path lookup instead?
|
||||
# return tuple(
|
||||
# self.msgdata['relay_path'][0]
|
||||
# )
|
||||
def src_actor_uid(self) -> tuple[str, str]|None:
|
||||
return self.msgdata.get('src_actor_uid')
|
||||
|
||||
@property
|
||||
def tb_str(
|
||||
|
@ -250,56 +129,28 @@ class RemoteActorError(Exception):
|
|||
|
||||
return ''
|
||||
|
||||
def _mk_fields_str(
|
||||
self,
|
||||
fields: list[str],
|
||||
end_char: str = '\n',
|
||||
) -> str:
|
||||
_repr: str = ''
|
||||
for key in fields:
|
||||
val: Any|None = (
|
||||
getattr(self, key, None)
|
||||
or
|
||||
self.msgdata.get(key)
|
||||
)
|
||||
# TODO: for `.relay_path` on multiline?
|
||||
# if not isinstance(val, str):
|
||||
# val_str = pformat(val)
|
||||
# else:
|
||||
val_str: str = repr(val)
|
||||
|
||||
if val:
|
||||
_repr += f'{key}={val_str}{end_char}'
|
||||
|
||||
return _repr
|
||||
|
||||
def reprol(self) -> str:
|
||||
'''
|
||||
Represent this error for "one line" display, like in
|
||||
a field of our `Context.__repr__()` output.
|
||||
|
||||
'''
|
||||
# TODO: use this matryoshka emjoi XD
|
||||
# => 🪆
|
||||
reprol_str: str = f'{type(self).__name__}('
|
||||
_repr: str = self._mk_fields_str(
|
||||
self.reprol_fields,
|
||||
end_char=' ',
|
||||
)
|
||||
return (
|
||||
reprol_str
|
||||
+
|
||||
_repr
|
||||
)
|
||||
_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:
|
||||
'''
|
||||
Nicely formatted boxed error meta data + traceback.
|
||||
|
||||
'''
|
||||
fields: str = self._mk_fields_str(
|
||||
_body_fields,
|
||||
)
|
||||
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,
|
||||
|
@ -314,6 +165,8 @@ class RemoteActorError(Exception):
|
|||
f' ------ - ------\n'
|
||||
f' _|\n'
|
||||
)
|
||||
# f'|\n'
|
||||
# f' |\n'
|
||||
if indent:
|
||||
body: str = textwrap.indent(
|
||||
body,
|
||||
|
@ -325,47 +178,9 @@ class RemoteActorError(Exception):
|
|||
')>'
|
||||
)
|
||||
|
||||
def unwrap(
|
||||
self,
|
||||
) -> BaseException:
|
||||
'''
|
||||
Unpack the inner-most source error from it's original IPC msg data.
|
||||
|
||||
We attempt to reconstruct (as best as we can) the original
|
||||
`Exception` from as it would have been raised in the
|
||||
failing actor's remote env.
|
||||
|
||||
'''
|
||||
src_type_ref: Type[BaseException] = self.src_type
|
||||
if not src_type_ref:
|
||||
raise TypeError(
|
||||
'Failed to lookup src error type:\n'
|
||||
f'{self.src_type_str}'
|
||||
)
|
||||
|
||||
# TODO: better tb insertion and all the fancier dunder
|
||||
# metadata stuff as per `.__context__` etc. and friends:
|
||||
# https://github.com/python-trio/trio/issues/611
|
||||
return src_type_ref(self.tb_str)
|
||||
|
||||
# TODO: local recontruction of nested inception for a given
|
||||
# "hop" / relay-node in this error's relay_path?
|
||||
# => so would render a `RAE[RAE[RAE[Exception]]]` instance
|
||||
# with all inner errors unpacked?
|
||||
# -[ ] if this is useful shouldn't be too hard to impl right?
|
||||
# TODO: local recontruction of remote exception deats
|
||||
# def unbox(self) -> BaseException:
|
||||
# '''
|
||||
# Unbox to the prior relays (aka last boxing actor's)
|
||||
# inner error.
|
||||
|
||||
# '''
|
||||
# if not self.relay_path:
|
||||
# return self.unwrap()
|
||||
|
||||
# # TODO..
|
||||
# # return self.boxed_type(
|
||||
# # boxed_type=get_type_ref(..
|
||||
# raise NotImplementedError
|
||||
# ...
|
||||
|
||||
|
||||
class InternalActorError(RemoteActorError):
|
||||
|
@ -417,7 +232,7 @@ class ContextCancelled(RemoteActorError):
|
|||
f'{self}'
|
||||
)
|
||||
|
||||
# TODO: to make `.__repr__()` work uniformly?
|
||||
# to make `.__repr__()` work uniformly
|
||||
# src_actor_uid = canceller
|
||||
|
||||
|
||||
|
@ -468,8 +283,7 @@ class MessagingError(Exception):
|
|||
|
||||
|
||||
def pack_error(
|
||||
exc: BaseException|RemoteActorError,
|
||||
|
||||
exc: BaseException,
|
||||
tb: str|None = None,
|
||||
cid: str|None = None,
|
||||
|
||||
|
@ -486,56 +300,27 @@ def pack_error(
|
|||
else:
|
||||
tb_str = traceback.format_exc()
|
||||
|
||||
error_msg: dict[ # for IPC
|
||||
error_msg: dict[
|
||||
str,
|
||||
str | tuple[str, str]
|
||||
] = {}
|
||||
our_uid: tuple = current_actor().uid
|
||||
] = {
|
||||
'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, RemoteActorError)
|
||||
isinstance(exc, ContextCancelled)
|
||||
or isinstance(exc, StreamOverrun)
|
||||
):
|
||||
error_msg.update(exc.msgdata)
|
||||
|
||||
# an onion/inception we need to pack
|
||||
if (
|
||||
type(exc) is RemoteActorError
|
||||
and (boxed := exc.boxed_type)
|
||||
and boxed != RemoteActorError
|
||||
):
|
||||
# sanity on source error (if needed when tweaking this)
|
||||
assert (src_type := exc.src_type) != RemoteActorError
|
||||
assert error_msg['src_type_str'] != 'RemoteActorError'
|
||||
assert error_msg['src_type_str'] == src_type.__name__
|
||||
assert error_msg['src_uid'] != our_uid
|
||||
|
||||
# set the boxed type to be another boxed type thus
|
||||
# creating an "inception" when unpacked by
|
||||
# `unpack_error()` in another actor who gets "relayed"
|
||||
# this error Bo
|
||||
#
|
||||
# NOTE on WHY: since we are re-boxing and already
|
||||
# boxed src error, we want to overwrite the original
|
||||
# `boxed_type_str` and instead set it to the type of
|
||||
# the input `exc` type.
|
||||
error_msg['boxed_type_str'] = 'RemoteActorError'
|
||||
|
||||
else:
|
||||
error_msg['src_uid'] = our_uid
|
||||
error_msg['src_type_str'] = type(exc).__name__
|
||||
error_msg['boxed_type_str'] = type(exc).__name__
|
||||
|
||||
# XXX alawys append us the last relay in error propagation path
|
||||
error_msg.setdefault(
|
||||
'relay_path',
|
||||
[],
|
||||
).append(our_uid)
|
||||
|
||||
# XXX NOTE: always ensure the traceback-str is from the
|
||||
# locally raised error (**not** the prior relay's boxed
|
||||
# content's `.msgdata`).
|
||||
error_msg['tb_str'] = tb_str
|
||||
|
||||
pkt: dict = {'error': error_msg}
|
||||
if cid:
|
||||
pkt['cid'] = cid
|
||||
|
@ -544,6 +329,7 @@ def pack_error(
|
|||
|
||||
|
||||
def unpack_error(
|
||||
|
||||
msg: dict[str, Any],
|
||||
|
||||
chan: Channel|None = None,
|
||||
|
@ -571,32 +357,35 @@ def unpack_error(
|
|||
|
||||
# retrieve the remote error's msg encoded details
|
||||
tb_str: str = error_dict.get('tb_str', '')
|
||||
message: str = (
|
||||
f'{chan.uid}\n'
|
||||
+
|
||||
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
|
||||
|
||||
# try to lookup a suitable error type from the local runtime
|
||||
# env then use it to construct a local instance.
|
||||
boxed_type_str: str = error_dict['boxed_type_str']
|
||||
boxed_type: Type[BaseException] = get_err_type(boxed_type_str)
|
||||
|
||||
if boxed_type_str == 'ContextCancelled':
|
||||
if type_name == 'ContextCancelled':
|
||||
box_type = ContextCancelled
|
||||
assert boxed_type is box_type
|
||||
suberror_type = box_type
|
||||
|
||||
# TODO: already included by `_this_mod` in else loop right?
|
||||
#
|
||||
# we have an inception/onion-error so ensure
|
||||
# we include the relay_path info and the
|
||||
# original source error.
|
||||
elif boxed_type_str == 'RemoteActorError':
|
||||
assert boxed_type is RemoteActorError
|
||||
assert len(error_dict['relay_path']) >= 1
|
||||
else: # try to lookup a suitable local error type
|
||||
for ns in [
|
||||
builtins,
|
||||
_this_mod,
|
||||
trio,
|
||||
]:
|
||||
if suberror_type := getattr(
|
||||
ns,
|
||||
type_name,
|
||||
False,
|
||||
):
|
||||
break
|
||||
|
||||
exc = box_type(
|
||||
message,
|
||||
suberror_type=suberror_type,
|
||||
|
||||
# unpack other fields into error type init
|
||||
**error_dict,
|
||||
)
|
||||
|
||||
|
@ -712,11 +501,6 @@ def _raise_from_no_key_in_msg(
|
|||
# destined for the `Context.result()` call during ctx-exit!
|
||||
stream._eoc: Exception = eoc
|
||||
|
||||
# in case there already is some underlying remote error
|
||||
# that arrived which is probably the source of this stream
|
||||
# closure
|
||||
ctx.maybe_raise()
|
||||
|
||||
raise eoc from src_err
|
||||
|
||||
if (
|
||||
|
|
|
@ -517,9 +517,7 @@ class Channel:
|
|||
|
||||
@acm
|
||||
async def _connect_chan(
|
||||
host: str,
|
||||
port: int
|
||||
|
||||
host: str, port: int
|
||||
) -> typing.AsyncGenerator[Channel, None]:
|
||||
'''
|
||||
Create and connect a channel with disconnect on context manager
|
||||
|
|
|
@ -1,151 +0,0 @@
|
|||
# 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 <https://www.gnu.org/licenses/>.
|
||||
|
||||
'''
|
||||
Multiaddress parser and utils according the spec(s) defined by
|
||||
`libp2p` and used in dependent project such as `ipfs`:
|
||||
|
||||
- https://docs.libp2p.io/concepts/fundamentals/addressing/
|
||||
- https://github.com/libp2p/specs/blob/master/addressing/README.md
|
||||
|
||||
'''
|
||||
from typing import Iterator
|
||||
|
||||
from bidict import bidict
|
||||
|
||||
# TODO: see if we can leverage libp2p ecosys projects instead of
|
||||
# rolling our own (parser) impls of the above addressing specs:
|
||||
# - https://github.com/libp2p/py-libp2p
|
||||
# - https://docs.libp2p.io/concepts/nat/circuit-relay/#relay-addresses
|
||||
# prots: bidict[int, str] = bidict({
|
||||
prots: bidict[int, str] = {
|
||||
'ipv4': 3,
|
||||
'ipv6': 3,
|
||||
'wg': 3,
|
||||
|
||||
'tcp': 4,
|
||||
'udp': 4,
|
||||
|
||||
# TODO: support the next-gen shite Bo
|
||||
# 'quic': 4,
|
||||
# 'ssh': 7, # via rsyscall bootstrapping
|
||||
}
|
||||
|
||||
prot_params: dict[str, tuple[str]] = {
|
||||
'ipv4': ('addr',),
|
||||
'ipv6': ('addr',),
|
||||
'wg': ('addr', 'port', 'pubkey'),
|
||||
|
||||
'tcp': ('port',),
|
||||
'udp': ('port',),
|
||||
|
||||
# 'quic': ('port',),
|
||||
# 'ssh': ('port',),
|
||||
}
|
||||
|
||||
|
||||
def iter_prot_layers(
|
||||
multiaddr: str,
|
||||
) -> Iterator[
|
||||
tuple[
|
||||
int,
|
||||
list[str]
|
||||
]
|
||||
]:
|
||||
'''
|
||||
Unpack a libp2p style "multiaddress" into multiple "segments"
|
||||
for each "layer" of the protocoll stack (in OSI terms).
|
||||
|
||||
'''
|
||||
tokens: list[str] = multiaddr.split('/')
|
||||
root, tokens = tokens[0], tokens[1:]
|
||||
assert not root # there is a root '/' on LHS
|
||||
itokens = iter(tokens)
|
||||
|
||||
prot: str | None = None
|
||||
params: list[str] = []
|
||||
for token in itokens:
|
||||
# every prot path should start with a known
|
||||
# key-str.
|
||||
if token in prots:
|
||||
if prot is None:
|
||||
prot: str = token
|
||||
else:
|
||||
yield prot, params
|
||||
prot = token
|
||||
|
||||
params = []
|
||||
|
||||
elif token not in prots:
|
||||
params.append(token)
|
||||
|
||||
else:
|
||||
yield prot, params
|
||||
|
||||
|
||||
def parse_maddr(
|
||||
multiaddr: str,
|
||||
) -> dict[str, str | int | dict]:
|
||||
'''
|
||||
Parse a libp2p style "multiaddress" into its distinct protocol
|
||||
segments where each segment is of the form:
|
||||
|
||||
`../<protocol>/<param0>/<param1>/../<paramN>`
|
||||
|
||||
and is loaded into a (order preserving) `layers: dict[str,
|
||||
dict[str, Any]` which holds each protocol-layer-segment of the
|
||||
original `str` path as a separate entry according to its approx
|
||||
OSI "layer number".
|
||||
|
||||
Any `paramN` in the path must be distinctly defined by a str-token in the
|
||||
(module global) `prot_params` table.
|
||||
|
||||
For eg. for wireguard which requires an address, port number and publickey
|
||||
the protocol params are specified as the entry:
|
||||
|
||||
'wg': ('addr', 'port', 'pubkey'),
|
||||
|
||||
and are thus parsed from a maddr in that order:
|
||||
`'/wg/1.1.1.1/51820/<pubkey>'`
|
||||
|
||||
'''
|
||||
layers: dict[str, str | int | dict] = {}
|
||||
for (
|
||||
prot_key,
|
||||
params,
|
||||
) in iter_prot_layers(multiaddr):
|
||||
|
||||
layer: int = prots[prot_key] # OSI layer used for sorting
|
||||
ep: dict[str, int | str] = {'layer': layer}
|
||||
layers[prot_key] = ep
|
||||
|
||||
# TODO; validation and resolving of names:
|
||||
# - each param via a validator provided as part of the
|
||||
# prot_params def? (also see `"port"` case below..)
|
||||
# - do a resolv step that will check addrs against
|
||||
# any loaded network.resolv: dict[str, str]
|
||||
rparams: list = list(reversed(params))
|
||||
for key in prot_params[prot_key]:
|
||||
val: str | int = rparams.pop()
|
||||
|
||||
# TODO: UGHH, dunno what we should do for validation
|
||||
# here, put it in the params spec somehow?
|
||||
if key == 'port':
|
||||
val = int(val)
|
||||
|
||||
ep[key] = val
|
||||
|
||||
return layers
|
|
@ -461,12 +461,7 @@ class LocalPortal:
|
|||
actor: 'Actor' # type: ignore # noqa
|
||||
channel: Channel
|
||||
|
||||
async def run_from_ns(
|
||||
self,
|
||||
ns: str,
|
||||
func_name: str,
|
||||
**kwargs,
|
||||
) -> Any:
|
||||
async def run_from_ns(self, ns: str, func_name: str, **kwargs) -> Any:
|
||||
'''
|
||||
Run a requested local function from a namespace path and
|
||||
return it's result.
|
||||
|
|
238
tractor/_root.py
238
tractor/_root.py
|
@ -22,10 +22,10 @@ from contextlib import asynccontextmanager
|
|||
from functools import partial
|
||||
import importlib
|
||||
import logging
|
||||
import os
|
||||
import signal
|
||||
import sys
|
||||
from typing import Callable
|
||||
import os
|
||||
import typing
|
||||
import warnings
|
||||
|
||||
|
||||
|
@ -47,14 +47,8 @@ from ._exceptions import is_multi_cancelled
|
|||
|
||||
|
||||
# set at startup and after forks
|
||||
_default_host: str = '127.0.0.1'
|
||||
_default_port: int = 1616
|
||||
|
||||
# default registry always on localhost
|
||||
_default_lo_addrs: list[tuple[str, int]] = [(
|
||||
_default_host,
|
||||
_default_port,
|
||||
)]
|
||||
_default_arbiter_host: str = '127.0.0.1'
|
||||
_default_arbiter_port: int = 1616
|
||||
|
||||
|
||||
logger = log.get_logger('tractor')
|
||||
|
@ -65,73 +59,38 @@ async def open_root_actor(
|
|||
|
||||
*,
|
||||
# defaults are above
|
||||
registry_addrs: list[tuple[str, int]]|None = None,
|
||||
arbiter_addr: tuple[str, int] | None = None,
|
||||
|
||||
# defaults are above
|
||||
arbiter_addr: tuple[str, int]|None = None,
|
||||
registry_addr: tuple[str, int] | None = None,
|
||||
|
||||
name: str|None = 'root',
|
||||
name: str | None = 'root',
|
||||
|
||||
# either the `multiprocessing` start method:
|
||||
# https://docs.python.org/3/library/multiprocessing.html#contexts-and-start-methods
|
||||
# OR `trio` (the new default).
|
||||
start_method: _spawn.SpawnMethodKey|None = None,
|
||||
start_method: _spawn.SpawnMethodKey | None = None,
|
||||
|
||||
# enables the multi-process debugger support
|
||||
debug_mode: bool = False,
|
||||
maybe_enable_greenback: bool = False, # `.pause_from_sync()/breakpoint()` support
|
||||
enable_stack_on_sig: bool = False,
|
||||
|
||||
# internal logging
|
||||
loglevel: str|None = None,
|
||||
loglevel: str | None = None,
|
||||
|
||||
enable_modules: list|None = None,
|
||||
rpc_module_paths: list|None = None,
|
||||
enable_modules: list | None = None,
|
||||
rpc_module_paths: list | None = None,
|
||||
|
||||
# NOTE: allow caller to ensure that only one registry exists
|
||||
# and that this call creates it.
|
||||
ensure_registry: bool = False,
|
||||
|
||||
) -> Actor:
|
||||
) -> typing.Any:
|
||||
'''
|
||||
Runtime init entry point for ``tractor``.
|
||||
|
||||
'''
|
||||
# TODO: stick this in a `@cm` defined in `devx._debug`?
|
||||
#
|
||||
# Override the global debugger hook to make it play nice with
|
||||
# ``trio``, see much discussion in:
|
||||
# https://github.com/python-trio/trio/issues/1155#issuecomment-742964018
|
||||
builtin_bp_handler: Callable = sys.breakpointhook
|
||||
orig_bp_path: str|None = os.environ.get(
|
||||
'PYTHONBREAKPOINT',
|
||||
None,
|
||||
)
|
||||
if (
|
||||
debug_mode
|
||||
and maybe_enable_greenback
|
||||
and await _debug.maybe_init_greenback(
|
||||
raise_not_found=False,
|
||||
)
|
||||
):
|
||||
os.environ['PYTHONBREAKPOINT'] = (
|
||||
'tractor.devx._debug.pause_from_sync'
|
||||
)
|
||||
else:
|
||||
# TODO: disable `breakpoint()` by default (without
|
||||
# `greenback`) since it will break any multi-actor
|
||||
# usage by a clobbered TTY's stdstreams!
|
||||
def block_bps(*args, **kwargs):
|
||||
raise RuntimeError(
|
||||
'Trying to use `breakpoint()` eh?\n'
|
||||
'Welp, `tractor` blocks `breakpoint()` built-in calls by default!\n'
|
||||
'If you need to use it please install `greenback` and set '
|
||||
'`debug_mode=True` when opening the runtime '
|
||||
'(either via `.open_nursery()` or `open_root_actor()`)\n'
|
||||
)
|
||||
|
||||
sys.breakpointhook = block_bps
|
||||
# os.environ['PYTHONBREAKPOINT'] = None
|
||||
builtin_bp_handler = sys.breakpointhook
|
||||
orig_bp_path: str | None = os.environ.get('PYTHONBREAKPOINT', None)
|
||||
os.environ['PYTHONBREAKPOINT'] = 'tractor.devx._debug.pause_from_sync'
|
||||
|
||||
# attempt to retreive ``trio``'s sigint handler and stash it
|
||||
# on our debugger lock state.
|
||||
|
@ -141,11 +100,7 @@ async def open_root_actor(
|
|||
_state._runtime_vars['_is_root'] = True
|
||||
|
||||
# caps based rpc list
|
||||
enable_modules = (
|
||||
enable_modules
|
||||
or
|
||||
[]
|
||||
)
|
||||
enable_modules = enable_modules or []
|
||||
|
||||
if rpc_module_paths:
|
||||
warnings.warn(
|
||||
|
@ -161,19 +116,20 @@ async def open_root_actor(
|
|||
|
||||
if arbiter_addr is not None:
|
||||
warnings.warn(
|
||||
'`arbiter_addr` is now deprecated\n'
|
||||
'Use `registry_addrs: list[tuple]` instead..',
|
||||
'`arbiter_addr` is now deprecated and has been renamed to'
|
||||
'`registry_addr`.\nUse that instead..',
|
||||
DeprecationWarning,
|
||||
stacklevel=2,
|
||||
)
|
||||
registry_addrs = [arbiter_addr]
|
||||
|
||||
registry_addrs: list[tuple[str, int]] = (
|
||||
registry_addrs
|
||||
or
|
||||
_default_lo_addrs
|
||||
registry_addr = (host, port) = (
|
||||
registry_addr
|
||||
or arbiter_addr
|
||||
or (
|
||||
_default_arbiter_host,
|
||||
_default_arbiter_port,
|
||||
)
|
||||
)
|
||||
assert registry_addrs
|
||||
|
||||
loglevel = (
|
||||
loglevel
|
||||
|
@ -211,11 +167,7 @@ async def open_root_actor(
|
|||
assert _log
|
||||
|
||||
# TODO: factor this into `.devx._stackscope`!!
|
||||
if (
|
||||
debug_mode
|
||||
and
|
||||
enable_stack_on_sig
|
||||
):
|
||||
if debug_mode:
|
||||
try:
|
||||
logger.info('Enabling `stackscope` traces on SIGUSR1')
|
||||
from .devx import enable_stack_on_sig
|
||||
|
@ -225,131 +177,73 @@ async def open_root_actor(
|
|||
'`stackscope` not installed for use in debug mode!'
|
||||
)
|
||||
|
||||
# closed into below ping task-func
|
||||
ponged_addrs: list[tuple[str, int]] = []
|
||||
|
||||
async def ping_tpt_socket(
|
||||
addr: tuple[str, int],
|
||||
timeout: float = 1,
|
||||
) -> None:
|
||||
'''
|
||||
Attempt temporary connection to see if a registry is
|
||||
listening at the requested address by a tranport layer
|
||||
ping.
|
||||
|
||||
If a connection can't be made quickly we assume none no
|
||||
server is listening at that addr.
|
||||
|
||||
'''
|
||||
try:
|
||||
# TODO: this connect-and-bail forces us to have to
|
||||
# carefully rewrap TCP 104-connection-reset errors as
|
||||
# EOF so as to avoid propagating cancel-causing errors
|
||||
# to the channel-msg loop machinery. Likely it would
|
||||
# be better to eventually have a "discovery" protocol
|
||||
# with basic handshake instead?
|
||||
with trio.move_on_after(timeout):
|
||||
async with _connect_chan(*addr):
|
||||
ponged_addrs.append(addr)
|
||||
# make a temporary connection to see if an arbiter exists,
|
||||
# if one can't be made quickly we assume none exists.
|
||||
arbiter_found = False
|
||||
|
||||
# TODO: this connect-and-bail forces us to have to carefully
|
||||
# rewrap TCP 104-connection-reset errors as EOF so as to avoid
|
||||
# propagating cancel-causing errors to the channel-msg loop
|
||||
# machinery. Likely it would be better to eventually have
|
||||
# a "discovery" protocol with basic handshake instead.
|
||||
with trio.move_on_after(1):
|
||||
async with _connect_chan(host, port):
|
||||
arbiter_found = True
|
||||
|
||||
except OSError:
|
||||
# TODO: make this a "discovery" log level?
|
||||
logger.warning(f'No actor registry found @ {addr}')
|
||||
logger.warning(f"No actor registry found @ {host}:{port}")
|
||||
|
||||
async with trio.open_nursery() as tn:
|
||||
for addr in registry_addrs:
|
||||
tn.start_soon(
|
||||
ping_tpt_socket,
|
||||
tuple(addr), # TODO: just drop this requirement?
|
||||
)
|
||||
|
||||
trans_bind_addrs: list[tuple[str, int]] = []
|
||||
|
||||
# Create a new local root-actor instance which IS NOT THE
|
||||
# REGISTRAR
|
||||
if ponged_addrs:
|
||||
|
||||
if ensure_registry:
|
||||
raise RuntimeError(
|
||||
f'Failed to open `{name}`@{ponged_addrs}: '
|
||||
'registry socket(s) already bound'
|
||||
)
|
||||
# create a local actor and start up its main routine/task
|
||||
if arbiter_found:
|
||||
|
||||
# we were able to connect to an arbiter
|
||||
logger.info(
|
||||
f'Registry(s) seem(s) to exist @ {ponged_addrs}'
|
||||
)
|
||||
logger.info(f"Arbiter seems to exist @ {host}:{port}")
|
||||
|
||||
actor = Actor(
|
||||
name=name or 'anonymous',
|
||||
registry_addrs=ponged_addrs,
|
||||
name or 'anonymous',
|
||||
arbiter_addr=registry_addr,
|
||||
loglevel=loglevel,
|
||||
enable_modules=enable_modules,
|
||||
)
|
||||
# DO NOT use the registry_addrs as the transport server
|
||||
# addrs for this new non-registar, root-actor.
|
||||
for host, port in ponged_addrs:
|
||||
# NOTE: zero triggers dynamic OS port allocation
|
||||
trans_bind_addrs.append((host, 0))
|
||||
host, port = (host, 0)
|
||||
|
||||
# Start this local actor as the "registrar", aka a regular
|
||||
# actor who manages the local registry of "mailboxes" of
|
||||
# other process-tree-local sub-actors.
|
||||
else:
|
||||
# start this local actor as the arbiter (aka a regular actor who
|
||||
# manages the local registry of "mailboxes")
|
||||
|
||||
# NOTE that if the current actor IS THE REGISTAR, the
|
||||
# following init steps are taken:
|
||||
# - the tranport layer server is bound to each (host, port)
|
||||
# pair defined in provided registry_addrs, or the default.
|
||||
trans_bind_addrs = registry_addrs
|
||||
|
||||
# - it is normally desirable for any registrar to stay up
|
||||
# indefinitely until either all registered (child/sub)
|
||||
# actors are terminated (via SC supervision) or,
|
||||
# a re-election process has taken place.
|
||||
# NOTE: all of ^ which is not implemented yet - see:
|
||||
# https://github.com/goodboy/tractor/issues/216
|
||||
# https://github.com/goodboy/tractor/pull/348
|
||||
# https://github.com/goodboy/tractor/issues/296
|
||||
# Note that if the current actor is the arbiter it is desirable
|
||||
# for it to stay up indefinitely until a re-election process has
|
||||
# taken place - which is not implemented yet FYI).
|
||||
|
||||
actor = Arbiter(
|
||||
name or 'registrar',
|
||||
registry_addrs=registry_addrs,
|
||||
name or 'arbiter',
|
||||
arbiter_addr=registry_addr,
|
||||
loglevel=loglevel,
|
||||
enable_modules=enable_modules,
|
||||
)
|
||||
|
||||
# Start up main task set via core actor-runtime nurseries.
|
||||
try:
|
||||
# assign process-local actor
|
||||
_state._current_actor = actor
|
||||
|
||||
# start local channel-server and fake the portal API
|
||||
# NOTE: this won't block since we provide the nursery
|
||||
ml_addrs_str: str = '\n'.join(
|
||||
f'@{addr}' for addr in trans_bind_addrs
|
||||
)
|
||||
logger.info(
|
||||
f'Starting local {actor.uid} on the following transport addrs:\n'
|
||||
f'{ml_addrs_str}'
|
||||
)
|
||||
logger.info(f"Starting local {actor} @ {host}:{port}")
|
||||
|
||||
# start the actor runtime in a new task
|
||||
async with trio.open_nursery() as nursery:
|
||||
|
||||
# ``_runtime.async_main()`` creates an internal nursery
|
||||
# and blocks here until any underlying actor(-process)
|
||||
# tree has terminated thereby conducting so called
|
||||
# "end-to-end" structured concurrency throughout an
|
||||
# entire hierarchical python sub-process set; all
|
||||
# "actor runtime" primitives are SC-compat and thus all
|
||||
# transitively spawned actors/processes must be as
|
||||
# well.
|
||||
# ``_runtime.async_main()`` creates an internal nursery and
|
||||
# thus blocks here until the entire underlying actor tree has
|
||||
# terminated thereby conducting structured concurrency.
|
||||
|
||||
await nursery.start(
|
||||
partial(
|
||||
async_main,
|
||||
actor,
|
||||
accept_addrs=trans_bind_addrs,
|
||||
accept_addr=(host, port),
|
||||
parent_addr=None
|
||||
)
|
||||
)
|
||||
|
@ -361,7 +255,7 @@ async def open_root_actor(
|
|||
BaseExceptionGroup,
|
||||
) as err:
|
||||
|
||||
entered: bool = await _debug._maybe_enter_pm(err)
|
||||
entered = await _debug._maybe_enter_pm(err)
|
||||
if (
|
||||
not entered
|
||||
and
|
||||
|
@ -369,8 +263,7 @@ async def open_root_actor(
|
|||
):
|
||||
logger.exception('Root actor crashed:\n')
|
||||
|
||||
# ALWAYS re-raise any error bubbled up from the
|
||||
# runtime!
|
||||
# always re-raise
|
||||
raise
|
||||
|
||||
finally:
|
||||
|
@ -391,9 +284,7 @@ async def open_root_actor(
|
|||
_state._current_actor = None
|
||||
_state._last_actor_terminated = actor
|
||||
|
||||
# restore built-in `breakpoint()` hook state
|
||||
if debug_mode:
|
||||
if builtin_bp_handler is not None:
|
||||
# restore breakpoint hook state
|
||||
sys.breakpointhook = builtin_bp_handler
|
||||
if orig_bp_path is not None:
|
||||
os.environ['PYTHONBREAKPOINT'] = orig_bp_path
|
||||
|
@ -409,7 +300,10 @@ def run_daemon(
|
|||
|
||||
# runtime kwargs
|
||||
name: str | None = 'root',
|
||||
registry_addrs: list[tuple[str, int]] = _default_lo_addrs,
|
||||
registry_addr: tuple[str, int] = (
|
||||
_default_arbiter_host,
|
||||
_default_arbiter_port,
|
||||
),
|
||||
|
||||
start_method: str | None = None,
|
||||
debug_mode: bool = False,
|
||||
|
@ -433,7 +327,7 @@ def run_daemon(
|
|||
async def _main():
|
||||
|
||||
async with open_root_actor(
|
||||
registry_addrs=registry_addrs,
|
||||
registry_addr=registry_addr,
|
||||
name=name,
|
||||
start_method=start_method,
|
||||
debug_mode=debug_mode,
|
||||
|
|
|
@ -26,6 +26,7 @@ from contextlib import (
|
|||
from functools import partial
|
||||
import inspect
|
||||
from pprint import pformat
|
||||
from types import ModuleType
|
||||
from typing import (
|
||||
Any,
|
||||
Callable,
|
||||
|
@ -267,10 +268,7 @@ async def _errors_relayed_via_ipc(
|
|||
entered_debug = await _debug._maybe_enter_pm(err)
|
||||
|
||||
if not entered_debug:
|
||||
log.exception(
|
||||
'RPC task crashed\n'
|
||||
f'|_{ctx}'
|
||||
)
|
||||
log.exception('Actor crashed:\n')
|
||||
|
||||
# always (try to) ship RPC errors back to caller
|
||||
if is_rpc:
|
||||
|
@ -331,6 +329,27 @@ async def _errors_relayed_via_ipc(
|
|||
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,
|
||||
|
@ -358,9 +377,7 @@ async def _invoke(
|
|||
treat_as_gen: bool = False
|
||||
|
||||
if _state.debug_mode():
|
||||
# XXX for .pause_from_sync()` usage we need to make sure
|
||||
# `greenback` is boostrapped in the subactor!
|
||||
await _debug.maybe_init_greenback()
|
||||
await maybe_import_gb()
|
||||
|
||||
# TODO: possibly a specially formatted traceback
|
||||
# (not sure what typing is for this..)?
|
||||
|
@ -591,8 +608,7 @@ async def _invoke(
|
|||
# other side.
|
||||
ctxc = ContextCancelled(
|
||||
msg,
|
||||
boxed_type=trio.Cancelled,
|
||||
# boxed_type_str='Cancelled',
|
||||
suberror_type=trio.Cancelled,
|
||||
canceller=canceller,
|
||||
)
|
||||
# assign local error so that the `.outcome`
|
||||
|
@ -650,7 +666,7 @@ async def _invoke(
|
|||
f'`{repr(ctx.outcome)}`',
|
||||
)
|
||||
)
|
||||
log.runtime(
|
||||
log.cancel(
|
||||
f'IPC context terminated with a final {res_type_str}\n\n'
|
||||
f'{ctx}\n'
|
||||
)
|
||||
|
@ -683,6 +699,12 @@ async def try_ship_error_to_remote(
|
|||
# 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
|
||||
|
|
|
@ -45,7 +45,6 @@ from functools import partial
|
|||
from itertools import chain
|
||||
import importlib
|
||||
import importlib.util
|
||||
import os
|
||||
from pprint import pformat
|
||||
import signal
|
||||
import sys
|
||||
|
@ -56,7 +55,7 @@ from typing import (
|
|||
)
|
||||
import uuid
|
||||
from types import ModuleType
|
||||
import warnings
|
||||
import os
|
||||
|
||||
import trio
|
||||
from trio import (
|
||||
|
@ -78,8 +77,8 @@ from ._exceptions import (
|
|||
ContextCancelled,
|
||||
TransportClosed,
|
||||
)
|
||||
from ._discovery import get_arbiter
|
||||
from .devx import _debug
|
||||
from ._discovery import get_registry
|
||||
from ._portal import Portal
|
||||
from . import _state
|
||||
from . import _mp_fixup_main
|
||||
|
@ -128,24 +127,19 @@ class Actor:
|
|||
# ugh, we need to get rid of this and replace with a "registry" sys
|
||||
# https://github.com/goodboy/tractor/issues/216
|
||||
is_arbiter: bool = False
|
||||
|
||||
@property
|
||||
def is_registrar(self) -> bool:
|
||||
return self.is_arbiter
|
||||
|
||||
msg_buffer_size: int = 2**6
|
||||
|
||||
# nursery placeholders filled in by `async_main()` after fork
|
||||
_root_n: Nursery|None = None
|
||||
_service_n: Nursery|None = None
|
||||
_server_n: Nursery|None = 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: CancelScope|None = None
|
||||
_parent_chan_cs: CancelScope | None = None
|
||||
|
||||
# syncs for setup/teardown sequences
|
||||
_server_down: trio.Event|None = None
|
||||
_server_down: trio.Event | None = None
|
||||
|
||||
# user toggled crash handling (including monkey-patched in
|
||||
# `trio.open_nursery()` via `.trionics._supervisor` B)
|
||||
|
@ -168,14 +162,10 @@ class Actor:
|
|||
name: str,
|
||||
*,
|
||||
enable_modules: list[str] = [],
|
||||
uid: str|None = None,
|
||||
loglevel: str|None = None,
|
||||
registry_addrs: list[tuple[str, int]]|None = None,
|
||||
spawn_method: str|None = None,
|
||||
|
||||
# TODO: remove!
|
||||
arbiter_addr: tuple[str, int]|None = None,
|
||||
|
||||
uid: str | None = None,
|
||||
loglevel: str | None = 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
|
||||
|
@ -189,7 +179,7 @@ class Actor:
|
|||
)
|
||||
|
||||
self._cancel_complete = trio.Event()
|
||||
self._cancel_called_by_remote: tuple[str, tuple]|None = None
|
||||
self._cancel_called_by_remote: tuple[str, tuple] | None = None
|
||||
self._cancel_called: bool = False
|
||||
|
||||
# retreive and store parent `__main__` data which
|
||||
|
@ -199,30 +189,27 @@ class Actor:
|
|||
# always include debugging tools module
|
||||
enable_modules.append('tractor.devx._debug')
|
||||
|
||||
self.enable_modules: dict[str, str] = {}
|
||||
mods = {}
|
||||
for name in enable_modules:
|
||||
mod: ModuleType = importlib.import_module(name)
|
||||
self.enable_modules[name] = _get_mod_abspath(mod)
|
||||
mod = importlib.import_module(name)
|
||||
mods[name] = _get_mod_abspath(mod)
|
||||
|
||||
self.enable_modules = mods
|
||||
self._mods: dict[str, ModuleType] = {}
|
||||
self.loglevel: str = loglevel
|
||||
self.loglevel = loglevel
|
||||
|
||||
if arbiter_addr is not None:
|
||||
warnings.warn(
|
||||
'`Actor(arbiter_addr=<blah>)` is now deprecated.\n'
|
||||
'Use `registry_addrs: list[tuple]` instead.',
|
||||
DeprecationWarning,
|
||||
stacklevel=2,
|
||||
)
|
||||
registry_addrs: list[tuple[str, int]] = [arbiter_addr]
|
||||
self._arb_addr: tuple[str, int] | None = (
|
||||
str(arbiter_addr[0]),
|
||||
int(arbiter_addr[1])
|
||||
) if arbiter_addr else None
|
||||
|
||||
# marked by the process spawning backend at startup
|
||||
# will be None for the parent most process started manually
|
||||
# by the user (currently called the "arbiter")
|
||||
self._spawn_method: str = spawn_method
|
||||
self._spawn_method = spawn_method
|
||||
|
||||
self._peers: defaultdict = defaultdict(list)
|
||||
self._peer_connected: dict[tuple[str, str], trio.Event] = {}
|
||||
self._peer_connected: dict = {}
|
||||
self._no_more_peers = trio.Event()
|
||||
self._no_more_peers.set()
|
||||
self._ongoing_rpc_tasks = trio.Event()
|
||||
|
@ -245,52 +232,13 @@ class Actor:
|
|||
] = {}
|
||||
|
||||
self._listeners: list[trio.abc.Listener] = []
|
||||
self._parent_chan: Channel|None = None
|
||||
self._forkserver_info: tuple|None = None
|
||||
self._parent_chan: Channel | None = None
|
||||
self._forkserver_info: tuple | None = None
|
||||
self._actoruid2nursery: dict[
|
||||
tuple[str, str],
|
||||
ActorNursery|None,
|
||||
ActorNursery | None,
|
||||
] = {} # type: ignore # noqa
|
||||
|
||||
# when provided, init the registry addresses property from
|
||||
# input via the validator.
|
||||
self._reg_addrs: list[tuple[str, int]] = []
|
||||
if registry_addrs:
|
||||
self.reg_addrs: list[tuple[str, int]] = registry_addrs
|
||||
_state._runtime_vars['_registry_addrs'] = registry_addrs
|
||||
|
||||
@property
|
||||
def reg_addrs(self) -> list[tuple[str, int]]:
|
||||
'''
|
||||
List of (socket) addresses for all known (and contactable)
|
||||
registry actors.
|
||||
|
||||
'''
|
||||
return self._reg_addrs
|
||||
|
||||
@reg_addrs.setter
|
||||
def reg_addrs(
|
||||
self,
|
||||
addrs: list[tuple[str, int]],
|
||||
) -> None:
|
||||
if not addrs:
|
||||
log.warning(
|
||||
'Empty registry address list is invalid:\n'
|
||||
f'{addrs}'
|
||||
)
|
||||
return
|
||||
|
||||
# always sanity check the input list since it's critical
|
||||
# that addrs are correct for discovery sys operation.
|
||||
for addr in addrs:
|
||||
if not isinstance(addr, tuple):
|
||||
raise ValueError(
|
||||
'Expected `Actor.reg_addrs: list[tuple[str, int]]`\n'
|
||||
f'Got {addrs}'
|
||||
)
|
||||
|
||||
self._reg_addrs = addrs
|
||||
|
||||
async def wait_for_peer(
|
||||
self, uid: tuple[str, str]
|
||||
) -> tuple[trio.Event, Channel]:
|
||||
|
@ -388,12 +336,6 @@ class Actor:
|
|||
self._no_more_peers = trio.Event() # unset by making new
|
||||
chan = Channel.from_stream(stream)
|
||||
their_uid: tuple[str, str]|None = chan.uid
|
||||
if their_uid:
|
||||
log.warning(
|
||||
f'Re-connection from already known {their_uid}'
|
||||
)
|
||||
else:
|
||||
log.runtime(f'New connection to us @{chan.raddr}')
|
||||
|
||||
con_msg: str = ''
|
||||
if their_uid:
|
||||
|
@ -575,19 +517,16 @@ class Actor:
|
|||
|
||||
if disconnected:
|
||||
# if the transport died and this actor is still
|
||||
# registered within a local nursery, we report
|
||||
# that the IPC layer may have failed
|
||||
# unexpectedly since it may be the cause of
|
||||
# other downstream errors.
|
||||
# registered within a local nursery, we report that the
|
||||
# IPC layer may have failed unexpectedly since it may be
|
||||
# the cause of other downstream errors.
|
||||
entry = local_nursery._children.get(uid)
|
||||
if entry:
|
||||
proc: trio.Process
|
||||
_, proc, _ = entry
|
||||
|
||||
if (
|
||||
(poll := getattr(proc, 'poll', None))
|
||||
and poll() is None
|
||||
):
|
||||
poll = getattr(proc, 'poll', None)
|
||||
if poll and poll() is None:
|
||||
log.cancel(
|
||||
f'Peer IPC broke but subproc is alive?\n\n'
|
||||
|
||||
|
@ -781,7 +720,7 @@ class Actor:
|
|||
#
|
||||
# side: str|None = None,
|
||||
|
||||
msg_buffer_size: int|None = None,
|
||||
msg_buffer_size: int | None = None,
|
||||
allow_overruns: bool = False,
|
||||
|
||||
) -> Context:
|
||||
|
@ -846,7 +785,7 @@ class Actor:
|
|||
kwargs: dict,
|
||||
|
||||
# IPC channel config
|
||||
msg_buffer_size: int|None = None,
|
||||
msg_buffer_size: int | None = None,
|
||||
allow_overruns: bool = False,
|
||||
load_nsf: bool = False,
|
||||
|
||||
|
@ -920,11 +859,11 @@ class Actor:
|
|||
|
||||
async def _from_parent(
|
||||
self,
|
||||
parent_addr: tuple[str, int]|None,
|
||||
parent_addr: tuple[str, int] | None,
|
||||
|
||||
) -> tuple[
|
||||
Channel,
|
||||
list[tuple[str, int]]|None,
|
||||
list[tuple[str, int]] | None,
|
||||
]:
|
||||
'''
|
||||
Bootstrap this local actor's runtime config from its parent by
|
||||
|
@ -941,11 +880,11 @@ class Actor:
|
|||
)
|
||||
await chan.connect()
|
||||
|
||||
# TODO: move this into a `Channel.handshake()`?
|
||||
# Initial handshake: swap names.
|
||||
await self._do_handshake(chan)
|
||||
|
||||
accept_addrs: list[tuple[str, int]]|None = None
|
||||
accept_addr: tuple[str, int] | None = None
|
||||
|
||||
if self._spawn_method == "trio":
|
||||
# Receive runtime state from our parent
|
||||
parent_data: dict[str, Any]
|
||||
|
@ -958,7 +897,10 @@ class Actor:
|
|||
# if "trace"/"util" mode is enabled?
|
||||
f'{pformat(parent_data)}\n'
|
||||
)
|
||||
accept_addrs: list[tuple[str, int]] = parent_data.pop('bind_addrs')
|
||||
accept_addr = (
|
||||
parent_data.pop('bind_host'),
|
||||
parent_data.pop('bind_port'),
|
||||
)
|
||||
rvs = parent_data.pop('_runtime_vars')
|
||||
|
||||
if rvs['_debug_mode']:
|
||||
|
@ -976,23 +918,18 @@ class Actor:
|
|||
_state._runtime_vars.update(rvs)
|
||||
|
||||
for attr, value in parent_data.items():
|
||||
if (
|
||||
attr == 'reg_addrs'
|
||||
and value
|
||||
):
|
||||
|
||||
if attr == '_arb_addr':
|
||||
# XXX: ``msgspec`` doesn't support serializing tuples
|
||||
# so just cash manually here since it's what our
|
||||
# internals expect.
|
||||
# TODO: we don't really NEED these as
|
||||
# tuples so we can probably drop this
|
||||
# casting since apparently in python lists
|
||||
# are "more efficient"?
|
||||
self.reg_addrs = [tuple(val) for val in value]
|
||||
value = tuple(value) if value else None
|
||||
self._arb_addr = value
|
||||
|
||||
else:
|
||||
setattr(self, attr, value)
|
||||
|
||||
return chan, accept_addrs
|
||||
return chan, accept_addr
|
||||
|
||||
except OSError: # failed to connect
|
||||
log.warning(
|
||||
|
@ -1009,9 +946,9 @@ class Actor:
|
|||
handler_nursery: Nursery,
|
||||
*,
|
||||
# (host, port) to bind for channel server
|
||||
listen_sockaddrs: list[tuple[str, int]]|None = None,
|
||||
|
||||
task_status: TaskStatus[Nursery] = trio.TASK_STATUS_IGNORED,
|
||||
accept_host: tuple[str, int] | None = None,
|
||||
accept_port: int = 0,
|
||||
task_status: TaskStatus[trio.Nursery] = trio.TASK_STATUS_IGNORED,
|
||||
) -> None:
|
||||
'''
|
||||
Start the IPC transport server, begin listening for new connections.
|
||||
|
@ -1021,26 +958,18 @@ class Actor:
|
|||
`.cancel_server()` is called.
|
||||
|
||||
'''
|
||||
if listen_sockaddrs is None:
|
||||
listen_sockaddrs = [(None, 0)]
|
||||
|
||||
self._server_down = trio.Event()
|
||||
try:
|
||||
async with trio.open_nursery() as server_n:
|
||||
|
||||
for host, port in listen_sockaddrs:
|
||||
listeners: list[trio.abc.Listener] = await server_n.start(
|
||||
partial(
|
||||
trio.serve_tcp,
|
||||
|
||||
handler=self._stream_handler,
|
||||
port=port,
|
||||
host=host,
|
||||
|
||||
# NOTE: configured such that new
|
||||
# connections will stay alive even if
|
||||
# this server is cancelled!
|
||||
self._stream_handler,
|
||||
# new connections will stay alive even if this server
|
||||
# is cancelled
|
||||
handler_nursery=handler_nursery,
|
||||
port=accept_port,
|
||||
host=accept_host,
|
||||
)
|
||||
)
|
||||
sockets: list[trio.socket] = [
|
||||
|
@ -1052,9 +981,7 @@ class Actor:
|
|||
f'|_{sockets}\n'
|
||||
)
|
||||
self._listeners.extend(listeners)
|
||||
|
||||
task_status.started(server_n)
|
||||
|
||||
finally:
|
||||
# signal the server is down since nursery above terminated
|
||||
self._server_down.set()
|
||||
|
@ -1391,19 +1318,6 @@ class Actor:
|
|||
log.runtime("Shutting down channel server")
|
||||
self._server_n.cancel_scope.cancel()
|
||||
|
||||
@property
|
||||
def accept_addrs(self) -> list[tuple[str, int]]:
|
||||
'''
|
||||
All addresses to which the transport-channel server binds
|
||||
and listens for new connections.
|
||||
|
||||
'''
|
||||
# throws OSError on failure
|
||||
return [
|
||||
listener.socket.getsockname()
|
||||
for listener in self._listeners
|
||||
] # type: ignore
|
||||
|
||||
@property
|
||||
def accept_addr(self) -> tuple[str, int]:
|
||||
'''
|
||||
|
@ -1412,7 +1326,7 @@ class Actor:
|
|||
|
||||
'''
|
||||
# throws OSError on failure
|
||||
return self.accept_addrs[0]
|
||||
return self._listeners[0].socket.getsockname() # type: ignore
|
||||
|
||||
def get_parent(self) -> Portal:
|
||||
'''
|
||||
|
@ -1429,7 +1343,6 @@ class Actor:
|
|||
'''
|
||||
return self._peers[uid]
|
||||
|
||||
# TODO: move to `Channel.handshake(uid)`
|
||||
async def _do_handshake(
|
||||
self,
|
||||
chan: Channel
|
||||
|
@ -1466,7 +1379,7 @@ class Actor:
|
|||
|
||||
async def async_main(
|
||||
actor: Actor,
|
||||
accept_addrs: tuple[str, int]|None = None,
|
||||
accept_addr: tuple[str, int] | None = None,
|
||||
|
||||
# XXX: currently ``parent_addr`` is only needed for the
|
||||
# ``multiprocessing`` backend (which pickles state sent to
|
||||
|
@ -1475,7 +1388,7 @@ 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: tuple[str, int]|None = None,
|
||||
parent_addr: tuple[str, int] | None = None,
|
||||
task_status: TaskStatus[None] = trio.TASK_STATUS_IGNORED,
|
||||
|
||||
) -> None:
|
||||
|
@ -1494,25 +1407,20 @@ async def async_main(
|
|||
# on our debugger lock state.
|
||||
_debug.Lock._trio_handler = signal.getsignal(signal.SIGINT)
|
||||
|
||||
is_registered: bool = False
|
||||
registered_with_arbiter = False
|
||||
try:
|
||||
|
||||
# establish primary connection with immediate parent
|
||||
actor._parent_chan: Channel|None = None
|
||||
actor._parent_chan = None
|
||||
if parent_addr is not None:
|
||||
|
||||
(
|
||||
actor._parent_chan,
|
||||
set_accept_addr_says_rent,
|
||||
) = await actor._from_parent(parent_addr)
|
||||
actor._parent_chan, accept_addr_rent = await actor._from_parent(
|
||||
parent_addr)
|
||||
|
||||
# either it's passed in because we're not a child or
|
||||
# because we're running in mp mode
|
||||
if (
|
||||
set_accept_addr_says_rent
|
||||
and set_accept_addr_says_rent is not None
|
||||
):
|
||||
accept_addrs = set_accept_addr_says_rent
|
||||
# either it's passed in because we're not a child
|
||||
# or because we're running in mp mode
|
||||
if accept_addr_rent is not None:
|
||||
accept_addr = accept_addr_rent
|
||||
|
||||
# The "root" nursery ensures the channel with the immediate
|
||||
# parent is kept alive as a resilient service until
|
||||
|
@ -1552,72 +1460,34 @@ async def async_main(
|
|||
# - subactor: the bind address is sent by our parent
|
||||
# over our established channel
|
||||
# - root actor: the ``accept_addr`` passed to this method
|
||||
assert accept_addrs
|
||||
assert accept_addr
|
||||
host, port = accept_addr
|
||||
|
||||
try:
|
||||
actor._server_n = await service_nursery.start(
|
||||
partial(
|
||||
actor._serve_forever,
|
||||
service_nursery,
|
||||
listen_sockaddrs=accept_addrs,
|
||||
accept_host=host,
|
||||
accept_port=port
|
||||
)
|
||||
)
|
||||
except OSError as oserr:
|
||||
# NOTE: always allow runtime hackers to debug
|
||||
# tranport address bind errors - normally it's
|
||||
# something silly like the wrong socket-address
|
||||
# passed via a config or CLI Bo
|
||||
entered_debug: bool = await _debug._maybe_enter_pm(oserr)
|
||||
if not entered_debug:
|
||||
log.exception('Failed to init IPC channel server !?\n')
|
||||
raise
|
||||
|
||||
accept_addrs: list[tuple[str, int]] = actor.accept_addrs
|
||||
|
||||
# NOTE: only set the loopback addr for the
|
||||
# process-tree-global "root" mailbox since
|
||||
# all sub-actors should be able to speak to
|
||||
# their root actor over that channel.
|
||||
accept_addr = actor.accept_addr
|
||||
if _state._runtime_vars['_is_root']:
|
||||
for addr in accept_addrs:
|
||||
host, _ = addr
|
||||
# TODO: generic 'lo' detector predicate
|
||||
if '127.0.0.1' in host:
|
||||
_state._runtime_vars['_root_mailbox'] = addr
|
||||
_state._runtime_vars['_root_mailbox'] = accept_addr
|
||||
|
||||
# Register with the arbiter if we're told its addr
|
||||
log.runtime(
|
||||
f'Registering `{actor.name}` ->\n'
|
||||
f'{pformat(accept_addrs)}'
|
||||
)
|
||||
log.runtime(f"Registering {actor} for role `{actor.name}`")
|
||||
assert isinstance(actor._arb_addr, tuple)
|
||||
|
||||
# TODO: ideally we don't fan out to all registrars
|
||||
# if addresses point to the same actor..
|
||||
# So we need a way to detect that? maybe iterate
|
||||
# only on unique actor uids?
|
||||
for addr in actor.reg_addrs:
|
||||
try:
|
||||
assert isinstance(addr, tuple)
|
||||
assert addr[1] # non-zero after bind
|
||||
except AssertionError:
|
||||
await _debug.pause()
|
||||
|
||||
async with get_registry(*addr) as reg_portal:
|
||||
for accept_addr in accept_addrs:
|
||||
|
||||
if not accept_addr[1]:
|
||||
await _debug.pause()
|
||||
|
||||
assert accept_addr[1]
|
||||
|
||||
await reg_portal.run_from_ns(
|
||||
async with get_arbiter(*actor._arb_addr) as arb_portal:
|
||||
await arb_portal.run_from_ns(
|
||||
'self',
|
||||
'register_actor',
|
||||
uid=actor.uid,
|
||||
sockaddr=accept_addr,
|
||||
)
|
||||
|
||||
is_registered: bool = True
|
||||
registered_with_arbiter = True
|
||||
|
||||
# init steps complete
|
||||
task_status.started()
|
||||
|
@ -1650,20 +1520,18 @@ async def async_main(
|
|||
log.runtime("Closing all actor lifetime contexts")
|
||||
actor.lifetime_stack.close()
|
||||
|
||||
if not is_registered:
|
||||
if not registered_with_arbiter:
|
||||
# TODO: I guess we could try to connect back
|
||||
# to the parent through a channel and engage a debugger
|
||||
# once we have that all working with std streams locking?
|
||||
log.exception(
|
||||
f"Actor errored and failed to register with arbiter "
|
||||
f"@ {actor.reg_addrs[0]}?")
|
||||
f"@ {actor._arb_addr}?")
|
||||
log.error(
|
||||
"\n\n\t^^^ THIS IS PROBABLY AN INTERNAL `tractor` BUG! ^^^\n\n"
|
||||
"\t>> CALMLY CALL THE AUTHORITIES AND HIDE YOUR CHILDREN <<\n\n"
|
||||
"\tIf this is a sub-actor hopefully its parent will keep running "
|
||||
"correctly presuming this error was safely ignored..\n\n"
|
||||
"\tPLEASE REPORT THIS TRACEBACK IN A BUG REPORT: "
|
||||
"https://github.com/goodboy/tractor/issues\n"
|
||||
"\n\n\t^^^ THIS IS PROBABLY A TRACTOR BUGGGGG!!! ^^^\n"
|
||||
"\tCALMLY CALL THE AUTHORITIES AND HIDE YOUR CHILDREN.\n\n"
|
||||
"\tYOUR PARENT CODE IS GOING TO KEEP WORKING FINE!!!\n"
|
||||
"\tTHIS IS HOW RELIABlE SYSTEMS ARE SUPPOSED TO WORK!?!?\n"
|
||||
)
|
||||
|
||||
if actor._parent_chan:
|
||||
|
@ -1703,19 +1571,16 @@ async def async_main(
|
|||
|
||||
# Unregister actor from the registry-sys / registrar.
|
||||
if (
|
||||
is_registered
|
||||
and not actor.is_registrar
|
||||
registered_with_arbiter
|
||||
and not actor.is_arbiter
|
||||
):
|
||||
failed: bool = False
|
||||
for addr in actor.reg_addrs:
|
||||
assert isinstance(addr, tuple)
|
||||
failed = False
|
||||
assert isinstance(actor._arb_addr, tuple)
|
||||
with trio.move_on_after(0.5) as cs:
|
||||
cs.shield = True
|
||||
try:
|
||||
async with get_registry(
|
||||
*addr,
|
||||
) as reg_portal:
|
||||
await reg_portal.run_from_ns(
|
||||
async with get_arbiter(*actor._arb_addr) as arb_portal:
|
||||
await arb_portal.run_from_ns(
|
||||
'self',
|
||||
'unregister_actor',
|
||||
uid=actor.uid
|
||||
|
@ -1724,12 +1589,9 @@ async def async_main(
|
|||
failed = True
|
||||
if cs.cancelled_caught:
|
||||
failed = True
|
||||
|
||||
if failed:
|
||||
log.warning(
|
||||
f'Failed to unregister {actor.name} from '
|
||||
f'registar @ {addr}'
|
||||
)
|
||||
f"Failed to unregister {actor.name} from arbiter")
|
||||
|
||||
# Ensure all peers (actors connected to us as clients) are finished
|
||||
if not actor._no_more_peers.is_set():
|
||||
|
@ -1748,36 +1610,18 @@ async def async_main(
|
|||
# TODO: rename to `Registry` and move to `._discovery`!
|
||||
class Arbiter(Actor):
|
||||
'''
|
||||
A special registrar actor who can contact all other actors
|
||||
within its immediate process tree and possibly keeps a registry
|
||||
of others meant to be discoverable in a distributed
|
||||
application. Normally the registrar is also the "root actor"
|
||||
and thus always has access to the top-most-level actor
|
||||
(process) nursery.
|
||||
A special actor who knows all the other actors and always has
|
||||
access to a top level nursery.
|
||||
|
||||
By default, the registrar is always initialized when and if no
|
||||
other registrar socket addrs have been specified to runtime
|
||||
init entry-points (such as `open_root_actor()` or
|
||||
`open_nursery()`). Any time a new main process is launched (and
|
||||
thus thus a new root actor created) and, no existing registrar
|
||||
can be contacted at the provided `registry_addr`, then a new
|
||||
one is always created; however, if one can be reached it is
|
||||
used.
|
||||
|
||||
Normally a distributed app requires at least registrar per
|
||||
logical host where for that given "host space" (aka localhost
|
||||
IPC domain of addresses) it is responsible for making all other
|
||||
host (local address) bound actors *discoverable* to external
|
||||
actor trees running on remote hosts.
|
||||
The arbiter is by default the first actor spawned on each host
|
||||
and is responsible for keeping track of all other actors for
|
||||
coordination purposes. If a new main process is launched and an
|
||||
arbiter is already running that arbiter will be used.
|
||||
|
||||
'''
|
||||
is_arbiter = True
|
||||
|
||||
def __init__(
|
||||
self,
|
||||
*args,
|
||||
**kwargs,
|
||||
) -> None:
|
||||
def __init__(self, *args, **kwargs) -> None:
|
||||
|
||||
self._registry: dict[
|
||||
tuple[str, str],
|
||||
|
@ -1797,7 +1641,7 @@ class Arbiter(Actor):
|
|||
self,
|
||||
name: str,
|
||||
|
||||
) -> tuple[str, int]|None:
|
||||
) -> tuple[str, int] | None:
|
||||
|
||||
for uid, sockaddr in self._registry.items():
|
||||
if name in uid:
|
||||
|
@ -1819,10 +1663,7 @@ class Arbiter(Actor):
|
|||
# unpacker since we have tuples as keys (not this makes the
|
||||
# arbiter suscetible to hashdos):
|
||||
# https://github.com/msgpack/msgpack-python#major-breaking-changes-in-msgpack-10
|
||||
return {
|
||||
'.'.join(key): val
|
||||
for key, val in self._registry.items()
|
||||
}
|
||||
return {'.'.join(key): val for key, val in self._registry.items()}
|
||||
|
||||
async def wait_for_actor(
|
||||
self,
|
||||
|
@ -1865,15 +1706,8 @@ class Arbiter(Actor):
|
|||
sockaddr: tuple[str, int]
|
||||
|
||||
) -> None:
|
||||
uid = name, hash = (str(uid[0]), str(uid[1]))
|
||||
addr = (host, port) = (
|
||||
str(sockaddr[0]),
|
||||
int(sockaddr[1]),
|
||||
)
|
||||
if port == 0:
|
||||
await _debug.pause()
|
||||
assert port # should never be 0-dynamic-os-alloc
|
||||
self._registry[uid] = addr
|
||||
uid = name, _ = (str(uid[0]), str(uid[1]))
|
||||
self._registry[uid] = (str(sockaddr[0]), int(sockaddr[1]))
|
||||
|
||||
# pop and signal all waiter events
|
||||
events = self._waiters.pop(name, [])
|
||||
|
|
|
@ -220,10 +220,6 @@ async def hard_kill(
|
|||
# whilst also hacking on it XD
|
||||
# terminate_after: int = 99999,
|
||||
|
||||
# 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.
|
||||
|
@ -369,7 +365,7 @@ async def new_proc(
|
|||
errors: dict[tuple[str, str], Exception],
|
||||
|
||||
# passed through to actor main
|
||||
bind_addrs: list[tuple[str, int]],
|
||||
bind_addr: tuple[str, int],
|
||||
parent_addr: tuple[str, int],
|
||||
_runtime_vars: dict[str, Any], # serialized and sent to _child
|
||||
|
||||
|
@ -391,7 +387,7 @@ async def new_proc(
|
|||
actor_nursery,
|
||||
subactor,
|
||||
errors,
|
||||
bind_addrs,
|
||||
bind_addr,
|
||||
parent_addr,
|
||||
_runtime_vars, # run time vars
|
||||
infect_asyncio=infect_asyncio,
|
||||
|
@ -406,7 +402,7 @@ async def trio_proc(
|
|||
errors: dict[tuple[str, str], Exception],
|
||||
|
||||
# passed through to actor main
|
||||
bind_addrs: list[tuple[str, int]],
|
||||
bind_addr: tuple[str, int],
|
||||
parent_addr: tuple[str, int],
|
||||
_runtime_vars: dict[str, Any], # serialized and sent to _child
|
||||
*,
|
||||
|
@ -495,15 +491,16 @@ async def trio_proc(
|
|||
|
||||
# send additional init params
|
||||
await chan.send({
|
||||
'_parent_main_data': subactor._parent_main_data,
|
||||
'enable_modules': subactor.enable_modules,
|
||||
'reg_addrs': subactor.reg_addrs,
|
||||
'bind_addrs': bind_addrs,
|
||||
'_runtime_vars': _runtime_vars,
|
||||
"_parent_main_data": subactor._parent_main_data,
|
||||
"enable_modules": subactor.enable_modules,
|
||||
"_arb_addr": subactor._arb_addr,
|
||||
"bind_host": bind_addr[0],
|
||||
"bind_port": bind_addr[1],
|
||||
"_runtime_vars": _runtime_vars,
|
||||
})
|
||||
|
||||
# track subactor in current nursery
|
||||
curr_actor: Actor = current_actor()
|
||||
curr_actor = current_actor()
|
||||
curr_actor._actoruid2nursery[subactor.uid] = actor_nursery
|
||||
|
||||
# resume caller at next checkpoint now that child is up
|
||||
|
@ -605,7 +602,7 @@ async def mp_proc(
|
|||
subactor: Actor,
|
||||
errors: dict[tuple[str, str], Exception],
|
||||
# passed through to actor main
|
||||
bind_addrs: list[tuple[str, int]],
|
||||
bind_addr: tuple[str, int],
|
||||
parent_addr: tuple[str, int],
|
||||
_runtime_vars: dict[str, Any], # serialized and sent to _child
|
||||
*,
|
||||
|
@ -663,7 +660,7 @@ async def mp_proc(
|
|||
target=_mp_main,
|
||||
args=(
|
||||
subactor,
|
||||
bind_addrs,
|
||||
bind_addr,
|
||||
fs_info,
|
||||
_spawn_method,
|
||||
parent_addr,
|
||||
|
|
|
@ -30,16 +30,10 @@ if TYPE_CHECKING:
|
|||
|
||||
_current_actor: Actor|None = None # type: ignore # noqa
|
||||
_last_actor_terminated: Actor|None = None
|
||||
|
||||
# TODO: mk this a `msgspec.Struct`!
|
||||
_runtime_vars: dict[str, Any] = {
|
||||
'_debug_mode': False,
|
||||
'_is_root': False,
|
||||
'_root_mailbox': (None, None),
|
||||
'_registry_addrs': [],
|
||||
|
||||
# for `breakpoint()` support
|
||||
'use_greenback': False,
|
||||
'_root_mailbox': (None, None)
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -22,7 +22,10 @@ from contextlib import asynccontextmanager as acm
|
|||
from functools import partial
|
||||
import inspect
|
||||
from pprint import pformat
|
||||
from typing import TYPE_CHECKING
|
||||
from typing import (
|
||||
Optional,
|
||||
TYPE_CHECKING,
|
||||
)
|
||||
import typing
|
||||
import warnings
|
||||
|
||||
|
@ -94,7 +97,7 @@ class ActorNursery:
|
|||
tuple[
|
||||
Actor,
|
||||
trio.Process | mp.Process,
|
||||
Portal | None,
|
||||
Optional[Portal],
|
||||
]
|
||||
] = {}
|
||||
# portals spawned with ``run_in_actor()`` are
|
||||
|
@ -118,12 +121,12 @@ class ActorNursery:
|
|||
self,
|
||||
name: str,
|
||||
*,
|
||||
bind_addrs: list[tuple[str, int]] = [_default_bind_addr],
|
||||
rpc_module_paths: list[str]|None = None,
|
||||
enable_modules: list[str]|None = None,
|
||||
loglevel: str|None = None, # set log level per subactor
|
||||
nursery: trio.Nursery|None = None,
|
||||
debug_mode: bool|None = None,
|
||||
bind_addr: tuple[str, int] = _default_bind_addr,
|
||||
rpc_module_paths: list[str] | None = None,
|
||||
enable_modules: list[str] | None = None,
|
||||
loglevel: str | None = None, # set log level per subactor
|
||||
nursery: trio.Nursery | None = None,
|
||||
debug_mode: Optional[bool] | None = None,
|
||||
infect_asyncio: bool = False,
|
||||
) -> Portal:
|
||||
'''
|
||||
|
@ -158,9 +161,7 @@ class ActorNursery:
|
|||
# modules allowed to invoked funcs from
|
||||
enable_modules=enable_modules,
|
||||
loglevel=loglevel,
|
||||
|
||||
# verbatim relay this actor's registrar addresses
|
||||
registry_addrs=current_actor().reg_addrs,
|
||||
arbiter_addr=current_actor()._arb_addr,
|
||||
)
|
||||
parent_addr = self._actor.accept_addr
|
||||
assert parent_addr
|
||||
|
@ -177,7 +178,7 @@ class ActorNursery:
|
|||
self,
|
||||
subactor,
|
||||
self.errors,
|
||||
bind_addrs,
|
||||
bind_addr,
|
||||
parent_addr,
|
||||
_rtv, # run time vars
|
||||
infect_asyncio=infect_asyncio,
|
||||
|
@ -190,8 +191,8 @@ class ActorNursery:
|
|||
fn: typing.Callable,
|
||||
*,
|
||||
|
||||
name: str | None = None,
|
||||
bind_addrs: tuple[str, int] = [_default_bind_addr],
|
||||
name: Optional[str] = None,
|
||||
bind_addr: tuple[str, int] = _default_bind_addr,
|
||||
rpc_module_paths: list[str] | None = None,
|
||||
enable_modules: list[str] | None = None,
|
||||
loglevel: str | None = None, # set log level per subactor
|
||||
|
@ -220,7 +221,7 @@ class ActorNursery:
|
|||
enable_modules=[mod_path] + (
|
||||
enable_modules or rpc_module_paths or []
|
||||
),
|
||||
bind_addrs=bind_addrs,
|
||||
bind_addr=bind_addr,
|
||||
loglevel=loglevel,
|
||||
# use the run_in_actor nursery
|
||||
nursery=self._ria_nursery,
|
||||
|
@ -583,7 +584,7 @@ async def open_nursery(
|
|||
finally:
|
||||
msg: str = (
|
||||
'Actor-nursery exited\n'
|
||||
f'|_{an}\n'
|
||||
f'|_{an}\n\n'
|
||||
)
|
||||
|
||||
# shutdown runtime if it was started
|
||||
|
|
|
@ -1,177 +0,0 @@
|
|||
# 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 <https://www.gnu.org/licenses/>.
|
||||
|
||||
'''
|
||||
Tools for code-object annotation, introspection and mutation
|
||||
as it pertains to improving the grok-ability of our runtime!
|
||||
|
||||
'''
|
||||
from __future__ import annotations
|
||||
import inspect
|
||||
# import msgspec
|
||||
# from pprint import pformat
|
||||
from types import (
|
||||
FrameType,
|
||||
FunctionType,
|
||||
MethodType,
|
||||
# CodeType,
|
||||
)
|
||||
from typing import (
|
||||
# Any,
|
||||
Callable,
|
||||
# TYPE_CHECKING,
|
||||
Type,
|
||||
)
|
||||
|
||||
from tractor.msg import (
|
||||
pretty_struct,
|
||||
NamespacePath,
|
||||
)
|
||||
|
||||
|
||||
# TODO: yeah, i don't love this and we should prolly just
|
||||
# write a decorator that actually keeps a stupid ref to the func
|
||||
# obj..
|
||||
def get_class_from_frame(fr: FrameType) -> (
|
||||
FunctionType
|
||||
|MethodType
|
||||
):
|
||||
'''
|
||||
Attempt to get the function (or method) reference
|
||||
from a given `FrameType`.
|
||||
|
||||
Verbatim from an SO:
|
||||
https://stackoverflow.com/a/2220759
|
||||
|
||||
'''
|
||||
args, _, _, value_dict = inspect.getargvalues(fr)
|
||||
|
||||
# we check the first parameter for the frame function is
|
||||
# named 'self'
|
||||
if (
|
||||
len(args)
|
||||
and
|
||||
# TODO: other cases for `@classmethod` etc..?)
|
||||
args[0] == 'self'
|
||||
):
|
||||
# in that case, 'self' will be referenced in value_dict
|
||||
instance: object = value_dict.get('self')
|
||||
if instance:
|
||||
# return its class
|
||||
return getattr(
|
||||
instance,
|
||||
'__class__',
|
||||
None,
|
||||
)
|
||||
|
||||
# return None otherwise
|
||||
return None
|
||||
|
||||
|
||||
def func_ref_from_frame(
|
||||
frame: FrameType,
|
||||
) -> Callable:
|
||||
func_name: str = frame.f_code.co_name
|
||||
try:
|
||||
return frame.f_globals[func_name]
|
||||
except KeyError:
|
||||
cls: Type|None = get_class_from_frame(frame)
|
||||
if cls:
|
||||
return getattr(
|
||||
cls,
|
||||
func_name,
|
||||
)
|
||||
|
||||
|
||||
# TODO: move all this into new `.devx._code`!
|
||||
# -[ ] prolly create a `@runtime_api` dec?
|
||||
# -[ ] ^- make it capture and/or accept buncha optional
|
||||
# meta-data like a fancier version of `@pdbp.hideframe`.
|
||||
#
|
||||
class CallerInfo(pretty_struct.Struct):
|
||||
rt_fi: inspect.FrameInfo
|
||||
call_frame: FrameType
|
||||
|
||||
@property
|
||||
def api_func_ref(self) -> Callable|None:
|
||||
return func_ref_from_frame(self.rt_fi.frame)
|
||||
|
||||
@property
|
||||
def api_nsp(self) -> NamespacePath|None:
|
||||
func: FunctionType = self.api_func_ref
|
||||
if func:
|
||||
return NamespacePath.from_ref(func)
|
||||
|
||||
return '<unknown>'
|
||||
|
||||
@property
|
||||
def caller_func_ref(self) -> Callable|None:
|
||||
return func_ref_from_frame(self.call_frame)
|
||||
|
||||
@property
|
||||
def caller_nsp(self) -> NamespacePath|None:
|
||||
func: FunctionType = self.caller_func_ref
|
||||
if func:
|
||||
return NamespacePath.from_ref(func)
|
||||
|
||||
return '<unknown>'
|
||||
|
||||
|
||||
def find_caller_info(
|
||||
dunder_var: str = '__runtimeframe__',
|
||||
iframes:int = 1,
|
||||
check_frame_depth: bool = True,
|
||||
|
||||
) -> CallerInfo|None:
|
||||
'''
|
||||
Scan up the callstack for a frame with a `dunder_var: str` variable
|
||||
and return the `iframes` frames above it.
|
||||
|
||||
By default we scan for a `__runtimeframe__` scope var which
|
||||
denotes a `tractor` API above which (one frame up) is "user
|
||||
app code" which "called into" the `tractor` method or func.
|
||||
|
||||
TODO: ex with `Portal.open_context()`
|
||||
|
||||
'''
|
||||
# TODO: use this instead?
|
||||
# https://docs.python.org/3/library/inspect.html#inspect.getouterframes
|
||||
frames: list[inspect.FrameInfo] = inspect.stack()
|
||||
for fi in frames:
|
||||
assert (
|
||||
fi.function
|
||||
==
|
||||
fi.frame.f_code.co_name
|
||||
)
|
||||
this_frame: FrameType = fi.frame
|
||||
dunder_val: int|None = this_frame.f_locals.get(dunder_var)
|
||||
if dunder_val:
|
||||
go_up_iframes: int = (
|
||||
dunder_val # could be 0 or `True` i guess?
|
||||
or
|
||||
iframes
|
||||
)
|
||||
rt_frame: FrameType = fi.frame
|
||||
call_frame = rt_frame
|
||||
for i in range(go_up_iframes):
|
||||
call_frame = call_frame.f_back
|
||||
|
||||
return CallerInfo(
|
||||
rt_fi=fi,
|
||||
call_frame=call_frame,
|
||||
)
|
||||
|
||||
return None
|
|
@ -33,46 +33,35 @@ from functools import (
|
|||
import os
|
||||
import signal
|
||||
import sys
|
||||
import threading
|
||||
import traceback
|
||||
from typing import (
|
||||
Any,
|
||||
Callable,
|
||||
AsyncIterator,
|
||||
AsyncGenerator,
|
||||
TYPE_CHECKING,
|
||||
)
|
||||
from types import (
|
||||
FrameType,
|
||||
ModuleType,
|
||||
)
|
||||
from types import FrameType
|
||||
|
||||
import pdbp
|
||||
import sniffio
|
||||
import tractor
|
||||
import trio
|
||||
from trio.lowlevel import current_task
|
||||
from trio import (
|
||||
from trio_typing import (
|
||||
TaskStatus,
|
||||
# Task,
|
||||
)
|
||||
|
||||
from tractor.log import get_logger
|
||||
from tractor._state import (
|
||||
from ..log import get_logger
|
||||
from .._state import (
|
||||
current_actor,
|
||||
is_root_process,
|
||||
debug_mode,
|
||||
)
|
||||
from tractor._exceptions import (
|
||||
from .._exceptions import (
|
||||
is_multi_cancelled,
|
||||
ContextCancelled,
|
||||
)
|
||||
from tractor._ipc import Channel
|
||||
|
||||
if TYPE_CHECKING:
|
||||
from tractor._runtime import (
|
||||
Actor,
|
||||
)
|
||||
from .._ipc import Channel
|
||||
|
||||
log = get_logger(__name__)
|
||||
|
||||
|
@ -127,32 +116,6 @@ class Lock:
|
|||
|
||||
@classmethod
|
||||
def shield_sigint(cls):
|
||||
'''
|
||||
Shield out SIGINT handling (which by default triggers
|
||||
`trio.Task` cancellation) in subactors when the `pdb` REPL
|
||||
is active.
|
||||
|
||||
Avoids cancellation of the current actor (task) when the
|
||||
user mistakenly sends ctl-c or a signal is received from
|
||||
an external request; explicit runtime cancel requests are
|
||||
allowed until the use exits the REPL session using
|
||||
'continue' or 'quit', at which point the orig SIGINT
|
||||
handler is restored.
|
||||
|
||||
'''
|
||||
#
|
||||
# XXX detect whether we're running from a non-main thread
|
||||
# in which case schedule the SIGINT shielding override
|
||||
# to in the main thread.
|
||||
# https://docs.python.org/3/library/signal.html#signals-and-threads
|
||||
if not cls.is_main_trio_thread():
|
||||
cls._orig_sigint_handler: Callable = trio.from_thread.run_sync(
|
||||
signal.signal,
|
||||
signal.SIGINT,
|
||||
shield_sigint_handler,
|
||||
)
|
||||
|
||||
else:
|
||||
cls._orig_sigint_handler = signal.signal(
|
||||
signal.SIGINT,
|
||||
shield_sigint_handler,
|
||||
|
@ -164,60 +127,13 @@ class Lock:
|
|||
# always restore ``trio``'s sigint handler. see notes below in
|
||||
# the pdb factory about the nightmare that is that code swapping
|
||||
# out the handler when the repl activates...
|
||||
if not cls.is_main_trio_thread():
|
||||
trio.from_thread.run_sync(
|
||||
signal.signal,
|
||||
signal.SIGINT,
|
||||
cls._trio_handler,
|
||||
)
|
||||
else:
|
||||
signal.signal(
|
||||
signal.SIGINT,
|
||||
cls._trio_handler,
|
||||
)
|
||||
|
||||
signal.signal(signal.SIGINT, cls._trio_handler)
|
||||
cls._orig_sigint_handler = None
|
||||
|
||||
@classmethod
|
||||
def is_main_trio_thread(cls) -> bool:
|
||||
'''
|
||||
Check if we're the "main" thread (as in the first one
|
||||
started by cpython) AND that it is ALSO the thread that
|
||||
called `trio.run()` and not some thread spawned with
|
||||
`trio.to_thread.run_sync()`.
|
||||
|
||||
'''
|
||||
is_trio_main = (
|
||||
# TODO: since this is private, @oremanj says
|
||||
# we should just copy the impl for now..
|
||||
(is_main_thread := trio._util.is_main_thread())
|
||||
and
|
||||
(async_lib := sniffio.current_async_library()) == 'trio'
|
||||
)
|
||||
if (
|
||||
not is_trio_main
|
||||
and is_main_thread
|
||||
):
|
||||
log.warning(
|
||||
f'Current async-lib detected by `sniffio`: {async_lib}\n'
|
||||
)
|
||||
return is_trio_main
|
||||
# XXX apparently unreliable..see ^
|
||||
# (
|
||||
# threading.current_thread()
|
||||
# is not threading.main_thread()
|
||||
# )
|
||||
|
||||
@classmethod
|
||||
def release(cls):
|
||||
try:
|
||||
if not cls.is_main_trio_thread():
|
||||
trio.from_thread.run_sync(
|
||||
cls._debug_lock.release
|
||||
)
|
||||
else:
|
||||
cls._debug_lock.release()
|
||||
|
||||
except RuntimeError:
|
||||
# uhhh makes no sense but been seeing the non-owner
|
||||
# release error even though this is definitely the task
|
||||
|
@ -484,6 +400,7 @@ async def wait_for_parent_stdin_hijack(
|
|||
|
||||
# this syncs to child's ``Context.started()`` call.
|
||||
async with portal.open_context(
|
||||
|
||||
lock_tty_for_child,
|
||||
subactor_uid=actor_uid,
|
||||
|
||||
|
@ -521,31 +438,11 @@ async def wait_for_parent_stdin_hijack(
|
|||
log.debug('Exiting debugger from child')
|
||||
|
||||
|
||||
def mk_mpdb() -> MultiActorPdb:
|
||||
'''
|
||||
Deliver a new `MultiActorPdb`: a multi-process safe `pdbp`
|
||||
REPL using the magic of SC!
|
||||
def mk_mpdb() -> tuple[MultiActorPdb, Callable]:
|
||||
|
||||
Our `pdb.Pdb` subtype accomplishes multi-process safe debugging
|
||||
by:
|
||||
|
||||
- mutexing access to the root process' TTY & stdstreams
|
||||
via an IPC managed `Lock` singleton per process tree.
|
||||
|
||||
- temporarily overriding any subactor's SIGINT handler to shield during
|
||||
live REPL sessions in sub-actors such that cancellation is
|
||||
never (mistakenly) triggered by a ctrl-c and instead only
|
||||
by either explicit requests in the runtime or
|
||||
|
||||
'''
|
||||
pdb = MultiActorPdb()
|
||||
# signal.signal = pdbp.hideframe(signal.signal)
|
||||
|
||||
# Always shield out SIGINTs for subactors when REPL is active.
|
||||
#
|
||||
# XXX detect whether we're running from a non-main thread
|
||||
# in which case schedule the SIGINT shielding override
|
||||
# to in the main thread.
|
||||
# https://docs.python.org/3/library/signal.html#signals-and-threads
|
||||
Lock.shield_sigint()
|
||||
|
||||
# XXX: These are the important flags mentioned in
|
||||
|
@ -554,7 +451,7 @@ def mk_mpdb() -> MultiActorPdb:
|
|||
pdb.allow_kbdint = True
|
||||
pdb.nosigint = True
|
||||
|
||||
return pdb
|
||||
return pdb, Lock.unshield_sigint
|
||||
|
||||
|
||||
def shield_sigint_handler(
|
||||
|
@ -567,16 +464,17 @@ def shield_sigint_handler(
|
|||
'''
|
||||
Specialized, debugger-aware SIGINT handler.
|
||||
|
||||
In childred we always ignore/shield for SIGINT to avoid
|
||||
deadlocks since cancellation should always be managed by the
|
||||
supervising parent actor. The root actor-proces is always
|
||||
cancelled on ctrl-c.
|
||||
In childred we always ignore to avoid deadlocks since cancellation
|
||||
should always be managed by the parent supervising actor. The root
|
||||
is always cancelled on ctrl-c.
|
||||
|
||||
'''
|
||||
__tracebackhide__: bool = True
|
||||
uid_in_debug: tuple[str, str]|None = Lock.global_actor_in_debug
|
||||
__tracebackhide__ = True
|
||||
|
||||
actor: Actor = current_actor()
|
||||
uid_in_debug: tuple[str, str] | None = Lock.global_actor_in_debug
|
||||
|
||||
actor = current_actor()
|
||||
# print(f'{actor.uid} in HANDLER with ')
|
||||
|
||||
def do_cancel():
|
||||
# If we haven't tried to cancel the runtime then do that instead
|
||||
|
@ -611,7 +509,7 @@ def shield_sigint_handler(
|
|||
return do_cancel()
|
||||
|
||||
# only set in the actor actually running the REPL
|
||||
pdb_obj: MultiActorPdb|None = Lock.repl
|
||||
pdb_obj: MultiActorPdb | None = Lock.repl
|
||||
|
||||
# root actor branch that reports whether or not a child
|
||||
# has locked debugger.
|
||||
|
@ -718,20 +616,14 @@ _pause_msg: str = 'Attaching to pdb REPL in actor'
|
|||
|
||||
|
||||
def _set_trace(
|
||||
actor: tractor.Actor|None = None,
|
||||
pdb: MultiActorPdb|None = None,
|
||||
actor: tractor.Actor | None = None,
|
||||
pdb: MultiActorPdb | None = None,
|
||||
shield: bool = False,
|
||||
|
||||
extra_frames_up_when_async: int = 1,
|
||||
hide_tb: bool = True,
|
||||
):
|
||||
__tracebackhide__: bool = hide_tb
|
||||
|
||||
actor: tractor.Actor = (
|
||||
actor
|
||||
or
|
||||
current_actor()
|
||||
)
|
||||
__tracebackhide__: bool = True
|
||||
actor: tractor.Actor = actor or current_actor()
|
||||
|
||||
# always start 1 level up from THIS in user code.
|
||||
frame: FrameType|None
|
||||
|
@ -777,17 +669,20 @@ def _set_trace(
|
|||
f'Going up frame {i} -> {frame}\n'
|
||||
)
|
||||
|
||||
# engage ze REPL
|
||||
# B~()
|
||||
else:
|
||||
pdb, undo_sigint = mk_mpdb()
|
||||
|
||||
# we entered the global ``breakpoint()`` built-in from sync
|
||||
# code?
|
||||
Lock.local_task_in_debug = 'sync'
|
||||
|
||||
pdb.set_trace(frame=frame)
|
||||
|
||||
|
||||
async def _pause(
|
||||
|
||||
debug_func: Callable = _set_trace,
|
||||
|
||||
# NOTE: must be passed in the `.pause_from_sync()` case!
|
||||
pdb: MultiActorPdb|None = None,
|
||||
release_lock_signal: trio.Event | None = None,
|
||||
|
||||
# TODO: allow caller to pause despite task cancellation,
|
||||
# exactly the same as wrapping with:
|
||||
|
@ -796,9 +691,9 @@ async def _pause(
|
|||
# => the REMAINING ISSUE is that the scope's .__exit__() frame
|
||||
# is always show in the debugger on entry.. and there seems to
|
||||
# be no way to override it?..
|
||||
#
|
||||
# shield: bool = False,
|
||||
|
||||
shield: bool = False,
|
||||
hide_tb: bool = True,
|
||||
task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED
|
||||
|
||||
) -> None:
|
||||
|
@ -810,16 +705,10 @@ async def _pause(
|
|||
Hopefully we won't need this in the long run.
|
||||
|
||||
'''
|
||||
__tracebackhide__: bool = hide_tb
|
||||
actor: Actor = current_actor()
|
||||
try:
|
||||
__tracebackhide__: bool = True
|
||||
actor = current_actor()
|
||||
pdb, undo_sigint = mk_mpdb()
|
||||
task_name: str = trio.lowlevel.current_task().name
|
||||
except RuntimeError as rte:
|
||||
if actor.is_infected_aio():
|
||||
raise RuntimeError(
|
||||
'`tractor.pause[_from_sync]()` not yet supported '
|
||||
'for infected `asyncio` mode!'
|
||||
) from rte
|
||||
|
||||
if (
|
||||
not Lock.local_pdb_complete
|
||||
|
@ -827,14 +716,10 @@ async def _pause(
|
|||
):
|
||||
Lock.local_pdb_complete = trio.Event()
|
||||
|
||||
if debug_func is not None:
|
||||
debug_func = partial(
|
||||
debug_func,
|
||||
)
|
||||
|
||||
if pdb is None:
|
||||
pdb: MultiActorPdb = mk_mpdb()
|
||||
|
||||
# TODO: need a more robust check for the "root" actor
|
||||
if (
|
||||
not is_root_process()
|
||||
|
@ -882,7 +767,6 @@ async def _pause(
|
|||
actor.uid,
|
||||
)
|
||||
Lock.repl = pdb
|
||||
|
||||
except RuntimeError:
|
||||
Lock.release()
|
||||
|
||||
|
@ -927,10 +811,16 @@ async def _pause(
|
|||
# TODO: do we want to support using this **just** for the
|
||||
# locking / common code (prolly to help address #320)?
|
||||
#
|
||||
if debug_func is None:
|
||||
task_status.started(Lock)
|
||||
# if debug_func is None:
|
||||
# assert release_lock_signal, (
|
||||
# 'Must pass `release_lock_signal: trio.Event` if no '
|
||||
# 'trace func provided!'
|
||||
# )
|
||||
# print(f"{actor.uid} ENTERING WAIT")
|
||||
# with trio.CancelScope(shield=True):
|
||||
# await release_lock_signal.wait()
|
||||
|
||||
else:
|
||||
# else:
|
||||
# block here one (at the appropriate frame *up*) where
|
||||
# ``breakpoint()`` was awaited and begin handling stdio.
|
||||
log.debug('Entering sync world of the `pdb` REPL..')
|
||||
|
@ -972,7 +862,8 @@ async def _pause(
|
|||
|
||||
async def pause(
|
||||
|
||||
debug_func: Callable|None = _set_trace,
|
||||
debug_func: Callable = _set_trace,
|
||||
release_lock_signal: trio.Event | None = None,
|
||||
|
||||
# TODO: allow caller to pause despite task cancellation,
|
||||
# exactly the same as wrapping with:
|
||||
|
@ -981,11 +872,10 @@ async def pause(
|
|||
# => the REMAINING ISSUE is that the scope's .__exit__() frame
|
||||
# is always show in the debugger on entry.. and there seems to
|
||||
# be no way to override it?..
|
||||
#
|
||||
shield: bool = False,
|
||||
task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED,
|
||||
# shield: bool = False,
|
||||
|
||||
**_pause_kwargs,
|
||||
shield: bool = False,
|
||||
task_status: TaskStatus[trio.Event] = trio.TASK_STATUS_IGNORED
|
||||
|
||||
) -> None:
|
||||
'''
|
||||
|
@ -1030,166 +920,89 @@ async def pause(
|
|||
task_status.started(cs)
|
||||
return await _pause(
|
||||
debug_func=debug_func,
|
||||
release_lock_signal=release_lock_signal,
|
||||
shield=True,
|
||||
task_status=task_status,
|
||||
**_pause_kwargs
|
||||
)
|
||||
else:
|
||||
return await _pause(
|
||||
debug_func=debug_func,
|
||||
release_lock_signal=release_lock_signal,
|
||||
shield=False,
|
||||
task_status=task_status,
|
||||
**_pause_kwargs
|
||||
)
|
||||
|
||||
|
||||
_gb_mod: None|ModuleType|False = None
|
||||
|
||||
|
||||
def maybe_import_greenback(
|
||||
raise_not_found: bool = True,
|
||||
force_reload: bool = False,
|
||||
|
||||
) -> ModuleType|False:
|
||||
# be cached-fast on module-already-inited
|
||||
global _gb_mod
|
||||
|
||||
if _gb_mod is False:
|
||||
return False
|
||||
|
||||
elif (
|
||||
_gb_mod is not None
|
||||
and not force_reload
|
||||
):
|
||||
return _gb_mod
|
||||
|
||||
try:
|
||||
import greenback
|
||||
_gb_mod = greenback
|
||||
return greenback
|
||||
|
||||
except ModuleNotFoundError as mnf:
|
||||
log.debug(
|
||||
'`greenback` is not installed.\n'
|
||||
'No sync debug support!\n'
|
||||
)
|
||||
_gb_mod = False
|
||||
|
||||
if raise_not_found:
|
||||
raise RuntimeError(
|
||||
'The `greenback` lib is required to use `tractor.pause_from_sync()`!\n'
|
||||
'https://github.com/oremanj/greenback\n'
|
||||
) from mnf
|
||||
|
||||
return False
|
||||
|
||||
|
||||
async def maybe_init_greenback(
|
||||
**kwargs,
|
||||
) -> None|ModuleType:
|
||||
|
||||
if mod := maybe_import_greenback(**kwargs):
|
||||
await mod.ensure_portal()
|
||||
log.info(
|
||||
'`greenback` portal opened!\n'
|
||||
'Sync debug support activated!\n'
|
||||
)
|
||||
return mod
|
||||
|
||||
return None
|
||||
|
||||
|
||||
# TODO: allow pausing from sync code.
|
||||
# normally by remapping python's builtin breakpoint() hook to this
|
||||
# runtime aware version which takes care of all .
|
||||
def pause_from_sync(
|
||||
hide_tb: bool = False,
|
||||
) -> None:
|
||||
|
||||
__tracebackhide__: bool = hide_tb
|
||||
def pause_from_sync() -> None:
|
||||
print("ENTER SYNC PAUSE")
|
||||
actor: tractor.Actor = current_actor(
|
||||
err_on_no_runtime=False,
|
||||
)
|
||||
log.debug(
|
||||
f'{actor.uid}: JUST ENTERED `tractor.pause_from_sync()`'
|
||||
f'|_{actor}\n'
|
||||
)
|
||||
if not actor:
|
||||
raise RuntimeError(
|
||||
'Not inside the `tractor`-runtime?\n'
|
||||
'`tractor.pause_from_sync()` is not functional without a wrapping\n'
|
||||
'- `async with tractor.open_nursery()` or,\n'
|
||||
'- `async with tractor.open_root_actor()`\n'
|
||||
)
|
||||
if actor:
|
||||
try:
|
||||
import greenback
|
||||
# __tracebackhide__ = True
|
||||
|
||||
# NOTE: once supported, remove this AND the one
|
||||
# inside `._pause()`!
|
||||
if actor.is_infected_aio():
|
||||
raise RuntimeError(
|
||||
'`tractor.pause[_from_sync]()` not yet supported '
|
||||
'for infected `asyncio` mode!'
|
||||
)
|
||||
|
||||
# raises on not-found by default
|
||||
greenback: ModuleType = maybe_import_greenback()
|
||||
mdb: MultiActorPdb = mk_mpdb()
|
||||
# task_can_release_tty_lock = trio.Event()
|
||||
|
||||
# run async task which will lock out the root proc's TTY.
|
||||
if not Lock.is_main_trio_thread():
|
||||
|
||||
# TODO: we could also check for a non-`.to_thread` context
|
||||
# using `trio.from_thread.check_cancelled()` (says
|
||||
# oremanj) wherein we get the following outputs:
|
||||
#
|
||||
# `RuntimeError`: non-`.to_thread` spawned thread
|
||||
# noop: non-cancelled `.to_thread`
|
||||
# `trio.Cancelled`: cancelled `.to_thread`
|
||||
#
|
||||
trio.from_thread.run(
|
||||
partial(
|
||||
# spawn bg task which will lock out the TTY, we poll
|
||||
# just below until the release event is reporting that task as
|
||||
# waiting.. not the most ideal but works for now ;)
|
||||
greenback.await_(
|
||||
actor._service_n.start(partial(
|
||||
pause,
|
||||
debug_func=None,
|
||||
pdb=mdb,
|
||||
hide_tb=hide_tb,
|
||||
# release_lock_signal=task_can_release_tty_lock,
|
||||
))
|
||||
)
|
||||
)
|
||||
# TODO: maybe the `trio.current_task()` id/name if avail?
|
||||
Lock.local_task_in_debug: str = str(threading.current_thread().name)
|
||||
|
||||
else: # we are presumably the `trio.run()` + main thread
|
||||
greenback.await_(
|
||||
pause(
|
||||
debug_func=None,
|
||||
pdb=mdb,
|
||||
hide_tb=hide_tb,
|
||||
)
|
||||
)
|
||||
Lock.local_task_in_debug: str = current_task().name
|
||||
except ModuleNotFoundError:
|
||||
log.warning('NO GREENBACK FOUND')
|
||||
else:
|
||||
log.warning('Not inside actor-runtime')
|
||||
|
||||
# TODO: ensure we aggressively make the user aware about
|
||||
# entering the global ``breakpoint()`` built-in from sync
|
||||
db, undo_sigint = mk_mpdb()
|
||||
Lock.local_task_in_debug = 'sync'
|
||||
# db.config.enable_hidden_frames = True
|
||||
|
||||
# we entered the global ``breakpoint()`` built-in from sync
|
||||
# code?
|
||||
_set_trace(
|
||||
actor=actor,
|
||||
pdb=mdb,
|
||||
hide_tb=hide_tb,
|
||||
extra_frames_up_when_async=1,
|
||||
frame: FrameType | None = sys._getframe()
|
||||
# print(f'FRAME: {str(frame)}')
|
||||
# assert not db._is_hidden(frame)
|
||||
|
||||
# TODO? will we ever need it?
|
||||
# -> the gb._await() won't be affected by cancellation?
|
||||
# shield=shield,
|
||||
)
|
||||
# LEGACY NOTE on next LOC's frame showing weirdness..
|
||||
#
|
||||
# XXX NOTE XXX no other LOC can be here without it
|
||||
# showing up in the REPL's last stack frame !?!
|
||||
# -[ ] tried to use `@pdbp.hideframe` decoration but
|
||||
# still doesn't work
|
||||
frame: FrameType = frame.f_back # type: ignore
|
||||
# print(f'FRAME: {str(frame)}')
|
||||
# if not db._is_hidden(frame):
|
||||
# pdbp.set_trace()
|
||||
# db._hidden_frames.append(
|
||||
# (frame, frame.f_lineno)
|
||||
# )
|
||||
db.set_trace(frame=frame)
|
||||
# NOTE XXX: see the `@pdbp.hideframe` decoration
|
||||
# on `Lock.unshield_sigint()`.. I have NO CLUE why
|
||||
# the next instruction's def frame is being shown
|
||||
# in the tb but it seems to be something wonky with
|
||||
# the way `pdb` core works?
|
||||
# undo_sigint()
|
||||
|
||||
# Lock.global_actor_in_debug = actor.uid
|
||||
# Lock.release()
|
||||
# task_can_release_tty_lock.set()
|
||||
|
||||
|
||||
# using the "pause" semantics instead since
|
||||
# that better covers actually somewhat "pausing the runtime"
|
||||
# for this particular paralell task to do debugging B)
|
||||
# pp = pause # short-hand for "pause point"
|
||||
|
||||
|
||||
# NOTE prefer a new "pause" semantic since it better describes
|
||||
# "pausing the actor's runtime" for this particular
|
||||
# paralell task to do debugging in a REPL.
|
||||
async def breakpoint(**kwargs):
|
||||
log.warning(
|
||||
'`tractor.breakpoint()` is deprecated!\n'
|
||||
|
|
|
@ -23,31 +23,12 @@ into each ``trio.Nursery`` except it links the lifetimes of memory space
|
|||
disjoint, parallel executing tasks in separate actors.
|
||||
|
||||
'''
|
||||
from __future__ import annotations
|
||||
import multiprocessing as mp
|
||||
from signal import (
|
||||
signal,
|
||||
SIGUSR1,
|
||||
)
|
||||
import traceback
|
||||
from typing import TYPE_CHECKING
|
||||
|
||||
import trio
|
||||
from tractor import (
|
||||
_state,
|
||||
log as logmod,
|
||||
)
|
||||
|
||||
log = logmod.get_logger(__name__)
|
||||
|
||||
|
||||
if TYPE_CHECKING:
|
||||
from tractor._spawn import ProcessType
|
||||
from tractor import (
|
||||
Actor,
|
||||
ActorNursery,
|
||||
)
|
||||
|
||||
|
||||
@trio.lowlevel.disable_ki_protection
|
||||
def dump_task_tree() -> None:
|
||||
|
@ -60,15 +41,9 @@ def dump_task_tree() -> None:
|
|||
recurse_child_tasks=True
|
||||
)
|
||||
)
|
||||
log = get_console_log(
|
||||
name=__name__,
|
||||
level='cancel',
|
||||
)
|
||||
actor: Actor = _state.current_actor()
|
||||
log.devx(
|
||||
f'Dumping `stackscope` tree for actor\n'
|
||||
f'{actor.name}: {actor}\n'
|
||||
f' |_{mp.current_process()}\n\n'
|
||||
log = get_console_log('cancel')
|
||||
log.pdb(
|
||||
f'Dumping `stackscope` tree:\n\n'
|
||||
f'{tree_str}\n'
|
||||
)
|
||||
# import logging
|
||||
|
@ -81,13 +56,8 @@ def dump_task_tree() -> None:
|
|||
# ).exception("Error printing task tree")
|
||||
|
||||
|
||||
def signal_handler(
|
||||
sig: int,
|
||||
frame: object,
|
||||
|
||||
relay_to_subs: bool = True,
|
||||
|
||||
) -> None:
|
||||
def signal_handler(sig: int, frame: object) -> None:
|
||||
import traceback
|
||||
try:
|
||||
trio.lowlevel.current_trio_token(
|
||||
).run_sync_soon(dump_task_tree)
|
||||
|
@ -95,26 +65,6 @@ def signal_handler(
|
|||
# not in async context -- print a normal traceback
|
||||
traceback.print_stack()
|
||||
|
||||
if not relay_to_subs:
|
||||
return
|
||||
|
||||
an: ActorNursery
|
||||
for an in _state.current_actor()._actoruid2nursery.values():
|
||||
|
||||
subproc: ProcessType
|
||||
subactor: Actor
|
||||
for subactor, subproc, _ in an._children.values():
|
||||
log.devx(
|
||||
f'Relaying `SIGUSR1`[{sig}] to sub-actor\n'
|
||||
f'{subactor}\n'
|
||||
f' |_{subproc}\n'
|
||||
)
|
||||
|
||||
if isinstance(subproc, trio.Process):
|
||||
subproc.send_signal(sig)
|
||||
|
||||
elif isinstance(subproc, mp.Process):
|
||||
subproc._send_signal(sig)
|
||||
|
||||
|
||||
def enable_stack_on_sig(
|
||||
|
@ -132,6 +82,3 @@ def enable_stack_on_sig(
|
|||
# NOTE: not the above can be triggered from
|
||||
# a (xonsh) shell using:
|
||||
# kill -SIGUSR1 @$(pgrep -f '<cmd>')
|
||||
#
|
||||
# for example if you were looking to trace a `pytest` run
|
||||
# kill -SIGUSR1 @$(pgrep -f 'pytest')
|
||||
|
|
|
@ -21,11 +21,6 @@ Log like a forester!
|
|||
from collections.abc import Mapping
|
||||
import sys
|
||||
import logging
|
||||
from logging import (
|
||||
LoggerAdapter,
|
||||
Logger,
|
||||
StreamHandler,
|
||||
)
|
||||
import colorlog # type: ignore
|
||||
|
||||
import trio
|
||||
|
@ -53,19 +48,20 @@ LOG_FORMAT = (
|
|||
|
||||
DATE_FORMAT = '%b %d %H:%M:%S'
|
||||
|
||||
# FYI, ERROR is 40
|
||||
CUSTOM_LEVELS: dict[str, int] = {
|
||||
LEVELS: dict[str, int] = {
|
||||
'TRANSPORT': 5,
|
||||
'RUNTIME': 15,
|
||||
'DEVX': 17,
|
||||
'CANCEL': 18,
|
||||
'CANCEL': 16,
|
||||
'PDB': 500,
|
||||
}
|
||||
# _custom_levels: set[str] = {
|
||||
# lvlname.lower for lvlname in LEVELS.keys()
|
||||
# }
|
||||
|
||||
STD_PALETTE = {
|
||||
'CRITICAL': 'red',
|
||||
'ERROR': 'red',
|
||||
'PDB': 'white',
|
||||
'DEVX': 'cyan',
|
||||
'WARNING': 'yellow',
|
||||
'INFO': 'green',
|
||||
'CANCEL': 'yellow',
|
||||
|
@ -82,7 +78,7 @@ BOLD_PALETTE = {
|
|||
|
||||
# TODO: this isn't showing the correct '{filename}'
|
||||
# as it did before..
|
||||
class StackLevelAdapter(LoggerAdapter):
|
||||
class StackLevelAdapter(logging.LoggerAdapter):
|
||||
|
||||
def transport(
|
||||
self,
|
||||
|
@ -90,8 +86,7 @@ class StackLevelAdapter(LoggerAdapter):
|
|||
|
||||
) -> None:
|
||||
'''
|
||||
IPC transport level msg IO; generally anything below
|
||||
`._ipc.Channel` and friends.
|
||||
IPC level msg-ing.
|
||||
|
||||
'''
|
||||
return self.log(5, msg)
|
||||
|
@ -107,11 +102,11 @@ class StackLevelAdapter(LoggerAdapter):
|
|||
msg: str,
|
||||
) -> None:
|
||||
'''
|
||||
Cancellation sequencing, mostly for runtime reporting.
|
||||
Cancellation logging, mostly for runtime reporting.
|
||||
|
||||
'''
|
||||
return self.log(
|
||||
level=22,
|
||||
level=16,
|
||||
msg=msg,
|
||||
# stacklevel=4,
|
||||
)
|
||||
|
@ -121,21 +116,11 @@ class StackLevelAdapter(LoggerAdapter):
|
|||
msg: str,
|
||||
) -> None:
|
||||
'''
|
||||
`pdb`-REPL (debugger) related statuses.
|
||||
Debugger logging.
|
||||
|
||||
'''
|
||||
return self.log(500, msg)
|
||||
|
||||
def devx(
|
||||
self,
|
||||
msg: str,
|
||||
) -> None:
|
||||
'''
|
||||
"Developer experience" sub-sys statuses.
|
||||
|
||||
'''
|
||||
return self.log(17, msg)
|
||||
|
||||
def log(
|
||||
self,
|
||||
level,
|
||||
|
@ -151,7 +136,8 @@ class StackLevelAdapter(LoggerAdapter):
|
|||
if self.isEnabledFor(level):
|
||||
stacklevel: int = 3
|
||||
if (
|
||||
level in CUSTOM_LEVELS.values()
|
||||
level in LEVELS.values()
|
||||
# or level in _custom_levels
|
||||
):
|
||||
stacklevel: int = 4
|
||||
|
||||
|
@ -198,30 +184,8 @@ class StackLevelAdapter(LoggerAdapter):
|
|||
)
|
||||
|
||||
|
||||
# TODO IDEAs:
|
||||
# -[ ] move to `.devx.pformat`?
|
||||
# -[ ] do per task-name and actor-name color coding
|
||||
# -[ ] unique color per task-id and actor-uuid
|
||||
def pformat_task_uid(
|
||||
id_part: str = 'tail'
|
||||
):
|
||||
'''
|
||||
Return `str`-ified unique for a `trio.Task` via a combo of its
|
||||
`.name: str` and `id()` truncated output.
|
||||
|
||||
'''
|
||||
task: trio.Task = trio.lowlevel.current_task()
|
||||
tid: str = str(id(task))
|
||||
if id_part == 'tail':
|
||||
tid_part: str = tid[-6:]
|
||||
else:
|
||||
tid_part: str = tid[:6]
|
||||
|
||||
return f'{task.name}[{tid_part}]'
|
||||
|
||||
|
||||
_conc_name_getters = {
|
||||
'task': pformat_task_uid,
|
||||
'task': lambda: trio.lowlevel.current_task().name,
|
||||
'actor': lambda: current_actor(),
|
||||
'actor_name': lambda: current_actor().name,
|
||||
'actor_uid': lambda: current_actor().uid[1][:6],
|
||||
|
@ -229,10 +193,7 @@ _conc_name_getters = {
|
|||
|
||||
|
||||
class ActorContextInfo(Mapping):
|
||||
'''
|
||||
Dyanmic lookup for local actor and task names.
|
||||
|
||||
'''
|
||||
"Dyanmic lookup for local actor and task names"
|
||||
_context_keys = (
|
||||
'task',
|
||||
'actor',
|
||||
|
@ -263,7 +224,6 @@ def get_logger(
|
|||
'''Return the package log or a sub-logger for ``name`` if provided.
|
||||
|
||||
'''
|
||||
log: Logger
|
||||
log = rlog = logging.getLogger(_root_name)
|
||||
|
||||
if (
|
||||
|
@ -306,7 +266,7 @@ def get_logger(
|
|||
logger = StackLevelAdapter(log, ActorContextInfo())
|
||||
|
||||
# additional levels
|
||||
for name, val in CUSTOM_LEVELS.items():
|
||||
for name, val in LEVELS.items():
|
||||
logging.addLevelName(val, name)
|
||||
|
||||
# ensure customs levels exist as methods
|
||||
|
@ -318,7 +278,7 @@ def get_logger(
|
|||
def get_console_log(
|
||||
level: str | None = None,
|
||||
**kwargs,
|
||||
) -> LoggerAdapter:
|
||||
) -> logging.LoggerAdapter:
|
||||
'''Get the package logger and enable a handler which writes to stderr.
|
||||
|
||||
Yeah yeah, i know we can use ``DictConfig``. You do it.
|
||||
|
@ -343,7 +303,7 @@ def get_console_log(
|
|||
None,
|
||||
)
|
||||
):
|
||||
handler = StreamHandler()
|
||||
handler = logging.StreamHandler()
|
||||
formatter = colorlog.ColoredFormatter(
|
||||
LOG_FORMAT,
|
||||
datefmt=DATE_FORMAT,
|
||||
|
@ -363,19 +323,3 @@ def get_loglevel() -> str:
|
|||
|
||||
# global module logger for tractor itself
|
||||
log = get_logger('tractor')
|
||||
|
||||
|
||||
def at_least_level(
|
||||
log: Logger|LoggerAdapter,
|
||||
level: int|str,
|
||||
) -> bool:
|
||||
'''
|
||||
Predicate to test if a given level is active.
|
||||
|
||||
'''
|
||||
if isinstance(level, str):
|
||||
level: int = CUSTOM_LEVELS[level.upper()]
|
||||
|
||||
if log.getEffectiveLevel() <= level:
|
||||
return True
|
||||
return False
|
||||
|
|
|
@ -33,14 +33,10 @@ from typing import (
|
|||
import trio
|
||||
from outcome import Error
|
||||
|
||||
from tractor.log import get_logger
|
||||
from tractor._state import (
|
||||
current_actor,
|
||||
debug_mode,
|
||||
)
|
||||
from tractor.devx import _debug
|
||||
from tractor._exceptions import AsyncioCancelled
|
||||
from tractor.trionics._broadcast import (
|
||||
from .log import get_logger
|
||||
from ._state import current_actor
|
||||
from ._exceptions import AsyncioCancelled
|
||||
from .trionics._broadcast import (
|
||||
broadcast_receiver,
|
||||
BroadcastReceiver,
|
||||
)
|
||||
|
@ -68,9 +64,9 @@ class LinkedTaskChannel(trio.abc.Channel):
|
|||
_trio_exited: bool = False
|
||||
|
||||
# set after ``asyncio.create_task()``
|
||||
_aio_task: asyncio.Task|None = None
|
||||
_aio_err: BaseException|None = None
|
||||
_broadcaster: BroadcastReceiver|None = None
|
||||
_aio_task: asyncio.Task | None = None
|
||||
_aio_err: BaseException | None = None
|
||||
_broadcaster: BroadcastReceiver | None = None
|
||||
|
||||
async def aclose(self) -> None:
|
||||
await self._from_aio.aclose()
|
||||
|
@ -162,9 +158,7 @@ def _run_asyncio_task(
|
|||
'''
|
||||
__tracebackhide__ = True
|
||||
if not current_actor().is_infected_aio():
|
||||
raise RuntimeError(
|
||||
"`infect_asyncio` mode is not enabled!?"
|
||||
)
|
||||
raise RuntimeError("`infect_asyncio` mode is not enabled!?")
|
||||
|
||||
# ITC (inter task comms), these channel/queue names are mostly from
|
||||
# ``asyncio``'s perspective.
|
||||
|
@ -193,7 +187,7 @@ def _run_asyncio_task(
|
|||
|
||||
cancel_scope = trio.CancelScope()
|
||||
aio_task_complete = trio.Event()
|
||||
aio_err: BaseException|None = None
|
||||
aio_err: BaseException | None = None
|
||||
|
||||
chan = LinkedTaskChannel(
|
||||
aio_q, # asyncio.Queue
|
||||
|
@ -259,7 +253,7 @@ def _run_asyncio_task(
|
|||
if not inspect.isawaitable(coro):
|
||||
raise TypeError(f"No support for invoking {coro}")
|
||||
|
||||
task: asyncio.Task = asyncio.create_task(
|
||||
task = asyncio.create_task(
|
||||
wait_on_coro_final_result(
|
||||
to_trio,
|
||||
coro,
|
||||
|
@ -268,18 +262,6 @@ def _run_asyncio_task(
|
|||
)
|
||||
chan._aio_task = task
|
||||
|
||||
# XXX TODO XXX get this actually workin.. XD
|
||||
# maybe setup `greenback` for `asyncio`-side task REPLing
|
||||
if (
|
||||
debug_mode()
|
||||
and
|
||||
(greenback := _debug.maybe_import_greenback(
|
||||
force_reload=True,
|
||||
raise_not_found=False,
|
||||
))
|
||||
):
|
||||
greenback.bestow_portal(task)
|
||||
|
||||
def cancel_trio(task: asyncio.Task) -> None:
|
||||
'''
|
||||
Cancel the calling ``trio`` task on error.
|
||||
|
@ -287,7 +269,7 @@ def _run_asyncio_task(
|
|||
'''
|
||||
nonlocal chan
|
||||
aio_err = chan._aio_err
|
||||
task_err: BaseException|None = None
|
||||
task_err: BaseException | None = None
|
||||
|
||||
# only to avoid ``asyncio`` complaining about uncaptured
|
||||
# task exceptions
|
||||
|
@ -367,11 +349,11 @@ async def translate_aio_errors(
|
|||
'''
|
||||
trio_task = trio.lowlevel.current_task()
|
||||
|
||||
aio_err: BaseException|None = None
|
||||
aio_err: BaseException | None = None
|
||||
|
||||
# TODO: make thisi a channel method?
|
||||
def maybe_raise_aio_err(
|
||||
err: Exception|None = None
|
||||
err: Exception | None = None
|
||||
) -> None:
|
||||
aio_err = chan._aio_err
|
||||
if (
|
||||
|
@ -549,16 +531,6 @@ def run_as_asyncio_guest(
|
|||
loop = asyncio.get_running_loop()
|
||||
trio_done_fut = asyncio.Future()
|
||||
|
||||
if debug_mode():
|
||||
# XXX make it obvi we know this isn't supported yet!
|
||||
log.error(
|
||||
'Attempting to enter unsupported `greenback` init '
|
||||
'from `asyncio` task..'
|
||||
)
|
||||
await _debug.maybe_init_greenback(
|
||||
force_reload=True,
|
||||
)
|
||||
|
||||
def trio_done_callback(main_outcome):
|
||||
|
||||
if isinstance(main_outcome, Error):
|
||||
|
|
Loading…
Reference in New Issue