Compare commits

..

No commits in common. "to_asyncio_eoc_signal" and "main" have entirely different histories.

30 changed files with 760 additions and 2033 deletions

View File

@ -317,6 +317,7 @@ def test_subactor_breakpoint(
assert in_prompt_msg(
child, [
'MessagingError:',
'RemoteActorError:',
"('breakpoint_forever'",
'bdb.BdbQuit',

View File

@ -121,11 +121,9 @@ def test_shield_pause(
child.pid,
signal.SIGINT,
)
from tractor._supervise import _shutdown_msg
expect(
child,
# 'Shutting down actor runtime',
_shutdown_msg,
'Shutting down actor runtime',
timeout=6,
)
assert_before(

View File

@ -1,113 +0,0 @@
'''
Unit-ish tests for specific IPC transport protocol backends.
'''
from __future__ import annotations
from pathlib import Path
import pytest
import trio
import tractor
from tractor import (
Actor,
_state,
_addr,
)
@pytest.fixture
def bindspace_dir_str() -> str:
bs_dir_str: str = '/run/user/1000/doggy'
bs_dir = Path(bs_dir_str)
assert not bs_dir.is_dir()
yield bs_dir_str
# delete it on suite teardown.
# ?TODO? should we support this internally
# or is leaking it ok?
if bs_dir.is_dir():
bs_dir.rmdir()
def test_uds_bindspace_created_implicitly(
debug_mode: bool,
bindspace_dir_str: str,
):
registry_addr: tuple = (
f'{bindspace_dir_str}',
'registry@doggy.sock',
)
bs_dir_str: str = registry_addr[0]
# XXX, ensure bindspace-dir DNE beforehand!
assert not Path(bs_dir_str).is_dir()
async def main():
async with tractor.open_nursery(
enable_transports=['uds'],
registry_addrs=[registry_addr],
debug_mode=debug_mode,
) as _an:
# XXX MUST be created implicitly by
# `.ipc._uds.start_listener()`!
assert Path(bs_dir_str).is_dir()
root: Actor = tractor.current_actor()
assert root.is_registrar
assert registry_addr in root.reg_addrs
assert (
registry_addr
in
_state._runtime_vars['_registry_addrs']
)
assert (
_addr.wrap_address(registry_addr)
in
root.registry_addrs
)
trio.run(main)
def test_uds_double_listen_raises_connerr(
debug_mode: bool,
bindspace_dir_str: str,
):
registry_addr: tuple = (
f'{bindspace_dir_str}',
'registry@doggy.sock',
)
async def main():
async with tractor.open_nursery(
enable_transports=['uds'],
registry_addrs=[registry_addr],
debug_mode=debug_mode,
) as _an:
# runtime up
root: Actor = tractor.current_actor()
from tractor.ipc._uds import (
start_listener,
UDSAddress,
)
ya_bound_addr: UDSAddress = root.registry_addrs[0]
try:
await start_listener(
addr=ya_bound_addr,
)
except ConnectionError as connerr:
assert type(src_exc := connerr.__context__) is OSError
assert 'Address already in use' in src_exc.args
# complete, exit test.
else:
pytest.fail('It dint raise a connerr !?')
trio.run(main)

View File

@ -573,16 +573,14 @@ def test_basic_interloop_channel_stream(
fan_out: bool,
):
async def main():
# TODO, figure out min timeout here!
with trio.fail_after(6):
async with tractor.open_nursery() as an:
portal = await an.run_in_actor(
stream_from_aio,
infect_asyncio=True,
fan_out=fan_out,
)
# should raise RAE diectly
await portal.result()
async with tractor.open_nursery() as an:
portal = await an.run_in_actor(
stream_from_aio,
infect_asyncio=True,
fan_out=fan_out,
)
# should raise RAE diectly
await portal.result()
trio.run(main)
@ -1090,108 +1088,6 @@ def test_sigint_closes_lifetime_stack(
trio.run(main)
# ?TODO asyncio.Task fn-deco?
# -[ ] do sig checkingat import time like @context?
# -[ ] maybe name it @aio_task ??
# -[ ] chan: to_asyncio.InterloopChannel ??
async def raise_before_started(
# from_trio: asyncio.Queue,
# to_trio: trio.abc.SendChannel,
chan: to_asyncio.LinkedTaskChannel,
) -> None:
'''
`asyncio.Task` entry point which RTEs before calling
`to_trio.send_nowait()`.
'''
await asyncio.sleep(0.2)
raise RuntimeError('Some shite went wrong before `.send_nowait()`!!')
# to_trio.send_nowait('Uhh we shouldve RTE-d ^^ ??')
chan.started_nowait('Uhh we shouldve RTE-d ^^ ??')
await asyncio.sleep(float('inf'))
@tractor.context
async def caching_ep(
ctx: tractor.Context,
):
log = tractor.log.get_logger('caching_ep')
log.info('syncing via `ctx.started()`')
await ctx.started()
# XXX, allocate the `open_channel_from()` inside
# a `.trionics.maybe_open_context()`.
chan: to_asyncio.LinkedTaskChannel
async with (
tractor.trionics.maybe_open_context(
acm_func=tractor.to_asyncio.open_channel_from,
kwargs={
'target': raise_before_started,
# ^XXX, kwarg to `open_channel_from()`
},
# lock around current actor task access
key=tractor.current_actor().uid,
) as (cache_hit, (clients, chan)),
):
if cache_hit:
log.error(
'Re-using cached `.open_from_channel()` call!\n'
)
else:
log.info(
'Allocating SHOULD-FAIL `.open_from_channel()`\n'
)
await trio.sleep_forever()
def test_aio_side_raises_before_started(
reg_addr: tuple[str, int],
debug_mode: bool,
loglevel: str,
):
'''
Simulates connection-err from `piker.brokers.ib.api`..
Ensure any error raised by child-`asyncio.Task` BEFORE
`chan.started()`
'''
# delay = 999 if debug_mode else 1
async def main():
with trio.fail_after(3):
an: tractor.ActorNursery
async with tractor.open_nursery(
debug_mode=debug_mode,
loglevel=loglevel,
) as an:
p: tractor.Portal = await an.start_actor(
'lchan_cacher_that_raises_fast',
enable_modules=[__name__],
infect_asyncio=True,
)
async with p.open_context(
caching_ep,
) as (ctx, first):
assert not first
with pytest.raises(
expected_exception=(RemoteActorError),
) as excinfo:
trio.run(main)
# ensure `asyncio.Task` exception is bubbled
# allll the way erp!!
rae = excinfo.value
assert rae.boxed_type is RuntimeError
# TODO: debug_mode tests once we get support for `asyncio`!
#
# -[ ] need tests to wrap both scripts:

View File

@ -410,6 +410,7 @@ def test_peer_canceller(
'''
async def main():
async with tractor.open_nursery(
# NOTE: to halt the peer tasks on ctxc, uncomment this.
debug_mode=debug_mode,
) as an:
canceller: Portal = await an.start_actor(

View File

@ -1,6 +1,5 @@
'''
Suites for our `.trionics.maybe_open_context()` multi-task
shared-cached `@acm` API.
Async context manager cache api testing: ``trionics.maybe_open_context():``
'''
from contextlib import asynccontextmanager as acm
@ -10,15 +9,6 @@ from typing import Awaitable
import pytest
import trio
import tractor
from tractor.trionics import (
maybe_open_context,
)
from tractor.log import (
get_console_log,
get_logger,
)
log = get_logger(__name__)
_resource: int = 0
@ -62,7 +52,7 @@ def test_resource_only_entered_once(key_on):
# different task names per task will be used
kwargs = {'task_name': name}
async with maybe_open_context(
async with tractor.trionics.maybe_open_context(
maybe_increment_counter,
kwargs=kwargs,
key=key,
@ -82,13 +72,11 @@ def test_resource_only_entered_once(key_on):
with trio.move_on_after(0.5):
async with (
tractor.open_root_actor(),
trio.open_nursery() as tn,
trio.open_nursery() as n,
):
for i in range(10):
tn.start_soon(
enter_cached_mngr,
f'task_{i}',
)
n.start_soon(enter_cached_mngr, f'task_{i}')
await trio.sleep(0.001)
trio.run(main)
@ -110,34 +98,23 @@ async def streamer(
@acm
async def open_stream() -> Awaitable[
tuple[
tractor.ActorNursery,
tractor.MsgStream,
]
]:
async def open_stream() -> Awaitable[tractor.MsgStream]:
try:
async with tractor.open_nursery() as an:
portal = await an.start_actor(
'streamer',
enable_modules=[__name__],
)
try:
async with (
portal.open_context(streamer) as (ctx, first),
ctx.open_stream() as stream,
):
print('Entered open_stream() caller')
yield an, stream
print('Exited open_stream() caller')
async with (
portal.open_context(streamer) as (ctx, first),
ctx.open_stream() as stream,
):
yield stream
finally:
print(
'Cancelling streamer with,\n'
'=> `Portal.cancel_actor()`'
)
await portal.cancel_actor()
print('Cancelled streamer')
print('Cancelling streamer')
await portal.cancel_actor()
print('Cancelled streamer')
except Exception as err:
print(
@ -150,15 +127,11 @@ async def open_stream() -> Awaitable[
@acm
async def maybe_open_stream(taskname: str):
async with maybe_open_context(
async with tractor.trionics.maybe_open_context(
# NOTE: all secondary tasks should cache hit on the same key
acm_func=open_stream,
) as (
cache_hit,
(an, stream)
):
# when the actor + portal + ctx + stream has already been
# allocated we want to just bcast to this task.
) as (cache_hit, stream):
if cache_hit:
print(f'{taskname} loaded from cache')
@ -166,43 +139,10 @@ async def maybe_open_stream(taskname: str):
# if this feed is already allocated by the first
# task that entereed
async with stream.subscribe() as bstream:
yield an, bstream
print(
f'cached task exited\n'
f')>\n'
f' |_{taskname}\n'
)
# we should always unreg the "cloned" bcrc for this
# consumer-task
assert id(bstream) not in bstream._state.subs
yield bstream
else:
# yield the actual stream
try:
yield an, stream
finally:
print(
f'NON-cached task exited\n'
f')>\n'
f' |_{taskname}\n'
)
first_bstream = stream._broadcaster
bcrx_state = first_bstream._state
subs: dict[int, int] = bcrx_state.subs
if len(subs) == 1:
assert id(first_bstream) in subs
# ^^TODO! the bcrx should always de-allocate all subs,
# including the implicit first one allocated on entry
# by the first subscribing peer task, no?
#
# -[ ] adjust `MsgStream.subscribe()` to do this mgmt!
# |_ allows reverting `MsgStream.receive()` to the
# non-bcaster method.
# |_ we can decide whether to reset `._broadcaster`?
#
# await tractor.pause(shield=True)
yield stream
def test_open_local_sub_to_stream(
@ -219,24 +159,16 @@ def test_open_local_sub_to_stream(
if debug_mode:
timeout = 999
print(f'IN debug_mode, setting large timeout={timeout!r}..')
async def main():
full = list(range(1000))
an: tractor.ActorNursery|None = None
num_tasks: int = 10
async def get_sub_and_pull(taskname: str):
nonlocal an
stream: tractor.MsgStream
async with (
maybe_open_stream(taskname) as (
an,
stream,
),
maybe_open_stream(taskname) as stream,
):
if '0' in taskname:
assert isinstance(stream, tractor.MsgStream)
@ -248,159 +180,34 @@ def test_open_local_sub_to_stream(
first = await stream.receive()
print(f'{taskname} started with value {first}')
seq: list[int] = []
seq = []
async for msg in stream:
seq.append(msg)
assert set(seq).issubset(set(full))
# end of @acm block
print(f'{taskname} finished')
root: tractor.Actor
with trio.fail_after(timeout) as cs:
# TODO: turns out this isn't multi-task entrant XD
# We probably need an indepotent entry semantic?
async with tractor.open_root_actor(
debug_mode=debug_mode,
# maybe_enable_greenback=True,
#
# ^TODO? doesn't seem to mk breakpoint() usage work
# bc each bg task needs to open a portal??
# - [ ] we should consider making this part of
# our taskman defaults?
# |_see https://github.com/goodboy/tractor/pull/363
#
) as root:
assert root.is_registrar
):
async with (
trio.open_nursery() as tn,
):
for i in range(num_tasks):
for i in range(10):
tn.start_soon(
get_sub_and_pull,
f'task_{i}',
)
await trio.sleep(0.001)
print('all consumer tasks finished!')
# ?XXX, ensure actor-nursery is shutdown or we might
# hang here due to a minor task deadlock/race-condition?
#
# - seems that all we need is a checkpoint to ensure
# the last suspended task, which is inside
# `.maybe_open_context()`, can do the
# `Portal.cancel_actor()` call?
#
# - if that bg task isn't resumed, then this blocks
# timeout might hit before that?
#
if root.ipc_server.has_peers():
await trio.lowlevel.checkpoint()
# alt approach, cancel the entire `an`
# await tractor.pause()
# await an.cancel()
# end of runtime scope
print('root actor terminated.')
print('all consumer tasks finished')
if cs.cancelled_caught:
pytest.fail(
'Should NOT time out in `open_root_actor()` ?'
)
print('exiting main.')
trio.run(main)
@acm
async def cancel_outer_cs(
cs: trio.CancelScope|None = None,
delay: float = 0,
):
# on first task delay this enough to block
# the 2nd task but then cancel it mid sleep
# so that the tn.start() inside the key-err handler block
# is cancelled and would previously corrupt the
# mutext state.
log.info(f'task entering sleep({delay})')
await trio.sleep(delay)
if cs:
log.info('task calling cs.cancel()')
cs.cancel()
trio.lowlevel.checkpoint()
yield
await trio.sleep_forever()
def test_lock_not_corrupted_on_fast_cancel(
debug_mode: bool,
loglevel: str,
):
'''
Verify that if the caching-task (the first to enter
`maybe_open_context()`) is cancelled mid-cache-miss, the embedded
mutex can never be left in a corrupted state.
That is, the lock is always eventually released ensuring a peer
(cache-hitting) task will never,
- be left to inf-block/hang on the `lock.acquire()`.
- try to release the lock when still owned by the caching-task
due to it having erronously exited without calling
`lock.release()`.
'''
delay: float = 1.
async def use_moc(
cs: trio.CancelScope|None,
delay: float,
):
log.info('task entering moc')
async with maybe_open_context(
cancel_outer_cs,
kwargs={
'cs': cs,
'delay': delay,
},
) as (cache_hit, _null):
if cache_hit:
log.info('2nd task entered')
else:
log.info('1st task entered')
await trio.sleep_forever()
async def main():
with trio.fail_after(delay + 2):
async with (
tractor.open_root_actor(
debug_mode=debug_mode,
loglevel=loglevel,
),
trio.open_nursery() as tn,
):
get_console_log('info')
log.info('yo starting')
cs = tn.cancel_scope
tn.start_soon(
use_moc,
cs,
delay,
name='child',
)
with trio.CancelScope() as rent_cs:
await use_moc(
cs=rent_cs,
delay=delay,
)
trio.run(main)

View File

@ -117,9 +117,11 @@ def test_acm_embedded_nursery_propagates_enter_err(
async with (
trio.open_nursery() as tn,
tractor.trionics.maybe_raise_from_masking_exc(
tn=tn,
unmask_from=(
(trio.Cancelled,) if unmask_from_canc
else ()
trio.Cancelled
if unmask_from_canc
else None
),
)
):
@ -134,7 +136,8 @@ def test_acm_embedded_nursery_propagates_enter_err(
with tractor.devx.maybe_open_crash_handler(
pdb=debug_mode,
) as bxerr:
assert not bxerr.value
if bxerr:
assert not bxerr.value
async with (
wraps_tn_that_always_cancels() as tn,
@ -142,12 +145,11 @@ def test_acm_embedded_nursery_propagates_enter_err(
assert not tn.cancel_scope.cancel_called
assert 0
if debug_mode:
assert (
(err := bxerr.value)
and
type(err) is AssertionError
)
assert (
(err := bxerr.value)
and
type(err) is AssertionError
)
with pytest.raises(ExceptionGroup) as excinfo:
trio.run(_main)

View File

@ -101,9 +101,6 @@ from ._state import (
debug_mode,
_ctxvar_Context,
)
from .trionics import (
collapse_eg,
)
# ------ - ------
if TYPE_CHECKING:
from ._portal import Portal
@ -154,7 +151,7 @@ class Context:
2 cancel-scope-linked, communicating and parallel executing
`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 "child" side a context is
actor from a `Portal`. On the "callee" side a context is
always allocated inside `._rpc._invoke()`.
TODO: more detailed writeup on cancellation, error and
@ -222,8 +219,8 @@ class Context:
# `._runtime.invoke()`.
_remote_func_type: str | None = None
# NOTE: (for now) only set (a portal) on the parent side since
# the child doesn't generally need a ref to one and should
# NOTE: (for now) only set (a portal) on the caller side since
# the callee doesn't generally need a ref to one and should
# normally need to explicitly ask for handle to its peer if
# more the the `Context` is needed?
_portal: Portal | None = None
@ -252,12 +249,12 @@ class Context:
_outcome_msg: Return|Error|ContextCancelled = Unresolved
# on a clean exit there should be a final value
# delivered from the far end "child" task, so
# delivered from the far end "callee" task, so
# this value is only set on one side.
# _result: Any | int = None
_result: PayloadT|Unresolved = Unresolved
# if the local "parent" task errors this value is always set
# if the local "caller" task errors this value is always set
# to the error that was captured in the
# `Portal.open_context().__aexit__()` teardown block OR, in
# 2 special cases when an (maybe) expected remote error
@ -293,7 +290,7 @@ class Context:
# a `ContextCancelled` due to a call to `.cancel()` triggering
# "graceful closure" on either side:
# - `._runtime._invoke()` will check this flag before engaging
# the crash handler REPL in such cases where the "child"
# the crash handler REPL in such cases where the "callee"
# raises the cancellation,
# - `.devx.debug.lock_stdio_for_peer()` will set it to `False` if
# the global tty-lock has been configured to filter out some
@ -307,8 +304,8 @@ class Context:
_stream_opened: bool = False
_stream: MsgStream|None = None
# the parent-task's calling-fn's frame-info, the frame above
# `Portal.open_context()`, for introspection/logging.
# caller of `Portal.open_context()` for
# logging purposes mostly
_caller_info: CallerInfo|None = None
# overrun handling machinery
@ -529,11 +526,11 @@ class Context:
'''
Exactly the value of `self._scope.cancelled_caught`
(delegation) and should only be (able to be read as)
`True` for a `.side == "parent"` ctx wherein the
`True` for a `.side == "caller"` ctx wherein the
`Portal.open_context()` block was exited due to a call to
`._scope.cancel()` - which should only ocurr in 2 cases:
- a parent side calls `.cancel()`, the far side cancels
- a caller side calls `.cancel()`, the far side cancels
and delivers back a `ContextCancelled` (making
`.cancel_acked == True`) and `._scope.cancel()` is
called by `._maybe_cancel_and_set_remote_error()` which
@ -542,20 +539,20 @@ class Context:
=> `._scope.cancelled_caught == True` by normal `trio`
cs semantics.
- a parent side is delivered a `._remote_error:
- a caller side is delivered a `._remote_error:
RemoteActorError` via `._deliver_msg()` and a transitive
call to `_maybe_cancel_and_set_remote_error()` calls
`._scope.cancel()` and that cancellation eventually
results in `trio.Cancelled`(s) caught in the
`.open_context()` handling around the @acm's `yield`.
Only as an FYI, in the "child" side case it can also be
Only as an FYI, in the "callee" side case it can also be
set but never is readable by any task outside the RPC
machinery in `._invoke()` since,:
- when a child side calls `.cancel()`, `._scope.cancel()`
- when a callee side calls `.cancel()`, `._scope.cancel()`
is called immediately and handled specially inside
`._invoke()` to raise a `ContextCancelled` which is then
sent to the parent side.
sent to the caller side.
However, `._scope.cancelled_caught` can NEVER be
accessed/read as `True` by any RPC invoked task since it
@ -666,7 +663,7 @@ class Context:
when called/closed by actor local task(s).
NOTEs:
- It is expected that the parent has previously unwrapped
- It is expected that the caller has previously unwrapped
the remote error using a call to `unpack_error()` and
provides that output exception value as the input
`error` argument *here*.
@ -676,7 +673,7 @@ class Context:
`Portal.open_context()` (ideally) we want to interrupt
any ongoing local tasks operating within that
`Context`'s cancel-scope so as to be notified ASAP of
the remote error and engage any parent handling (eg.
the remote error and engage any caller handling (eg.
for cross-process task supervision).
- In some cases we may want to raise the remote error
@ -743,8 +740,6 @@ class Context:
# cancelled, NOT their reported canceller. IOW in the
# latter case we're cancelled by someone else getting
# cancelled.
#
# !TODO, switching to `Actor.aid` here!
if (canc := error.canceller) == self._actor.uid:
whom: str = 'us'
self._canceller = canc
@ -886,11 +881,6 @@ class Context:
@property
def repr_caller(self) -> str:
'''
Render a "namespace-path" style representation of the calling
task-fn.
'''
ci: CallerInfo|None = self._caller_info
if ci:
return (
@ -904,7 +894,7 @@ class Context:
def repr_api(self) -> str:
return 'Portal.open_context()'
# TODO: use `.dev._frame_stack` scanning to find caller fn!
# TODO: use `.dev._frame_stack` scanning to find caller!
# ci: CallerInfo|None = self._caller_info
# if ci:
# return (
@ -939,7 +929,7 @@ class Context:
=> That is, an IPC `Context` (this) **does not**
have the same semantics as a `trio.CancelScope`.
If the parent (who entered the `Portal.open_context()`)
If the caller (who entered the `Portal.open_context()`)
desires that the internal block's cancel-scope be
cancelled it should open its own `trio.CancelScope` and
manage it as needed.
@ -950,7 +940,7 @@ class Context:
self.cancel_called = True
header: str = (
f'Cancelling ctx from {side!r}-side\n'
f'Cancelling ctx from {side.upper()}-side\n'
)
reminfo: str = (
# ' =>\n'
@ -958,7 +948,7 @@ class Context:
f'\n'
f'c)=> {self.chan.uid}\n'
f' |_[{self.dst_maddr}\n'
f' >> {self.repr_rpc}\n'
f' >>{self.repr_rpc}\n'
# f' >> {self._nsf}() -> {codec}[dict]:\n\n'
# TODO: pull msg-type from spec re #320
)
@ -1022,7 +1012,7 @@ class Context:
# `_invoke()` RPC task.
#
# NOTE: on this side we ALWAYS cancel the local scope
# since the parent expects a `ContextCancelled` to be sent
# since the caller expects a `ContextCancelled` to be sent
# from `._runtime._invoke()` back to the other side. The
# logic for catching the result of the below
# `._scope.cancel()` is inside the `._runtime._invoke()`
@ -1195,8 +1185,8 @@ class Context:
) -> Any|Exception:
'''
From some (parent) side task, wait for and return the final
result from the remote (child) side's task.
From some (caller) side task, wait for and return the final
result from the remote (callee) side's task.
This provides a mechanism for one task running in some actor to wait
on another task at the other side, in some other actor, to terminate.
@ -1605,7 +1595,7 @@ class Context:
raise err
# TODO: maybe a flag to by-pass encode op if already done
# here in parent?
# here in caller?
await self.chan.send(started_msg)
# set msg-related internal runtime-state
@ -1681,7 +1671,7 @@ class Context:
XXX RULES XXX
------ - ------
- NEVER raise remote errors from this method; a calling runtime-task.
- NEVER raise remote errors from this method; a runtime task caller.
An error "delivered" to a ctx should always be raised by
the corresponding local task operating on the
`Portal`/`Context` APIs.
@ -1757,7 +1747,7 @@ class Context:
else:
report = (
'Queueing OVERRUN msg on parent task:\n\n'
'Queueing OVERRUN msg on caller task:\n\n'
+ report
)
log.debug(report)
@ -1953,12 +1943,12 @@ async def open_context_from_portal(
IPC protocol.
The yielded `tuple` is a pair delivering a `tractor.Context`
and any first value "sent" by the "child" task via a call
and any first value "sent" by the "callee" task via a call
to `Context.started(<value: Any>)`; this side of the
context does not unblock until the "child" task calls
context does not unblock until the "callee" task calls
`.started()` in similar style to `trio.Nursery.start()`.
When the "child" (side that is "called"/started by a call
to *this* method) returns, the parent side (this) unblocks
When the "callee" (side that is "called"/started by a call
to *this* method) returns, the caller side (this) unblocks
and any final value delivered from the other end can be
retrieved using the `Contex.wait_for_result()` api.
@ -1971,7 +1961,7 @@ async def open_context_from_portal(
__tracebackhide__: bool = hide_tb
# denote this frame as a "runtime frame" for stack
# introspection where we report the parent code in logging
# introspection where we report the caller code in logging
# and error message content.
# NOTE: 2 bc of the wrapping `@acm`
__runtimeframe__: int = 2 # noqa
@ -2030,11 +2020,13 @@ async def open_context_from_portal(
# placeholder for any exception raised in the runtime
# or by user tasks which cause this context's closure.
scope_err: BaseException|None = None
ctxc_from_child: ContextCancelled|None = None
ctxc_from_callee: ContextCancelled|None = None
try:
async with (
collapse_eg(),
trio.open_nursery() as tn,
trio.open_nursery(
strict_exception_groups=False,
) as tn,
msgops.maybe_limit_plds(
ctx=ctx,
spec=ctx_meta.get('pld_spec'),
@ -2109,7 +2101,7 @@ async def open_context_from_portal(
# that we can re-use it around the `yield` ^ here
# or vice versa?
#
# maybe TODO NOTE: between the parent exiting and
# maybe TODO NOTE: between the caller exiting and
# arriving here the far end may have sent a ctxc-msg or
# other error, so the quetion is whether we should check
# for it here immediately and maybe raise so as to engage
@ -2175,16 +2167,16 @@ async def open_context_from_portal(
# request in which case we DO let the error bubble to the
# opener.
#
# 2-THIS "parent" task somewhere invoked `Context.cancel()`
# and received a `ContextCanclled` from the "child"
# 2-THIS "caller" task somewhere invoked `Context.cancel()`
# and received a `ContextCanclled` from the "callee"
# task, in which case we mask the `ContextCancelled` from
# bubbling to this "parent" (much like how `trio.Nursery`
# bubbling to this "caller" (much like how `trio.Nursery`
# swallows any `trio.Cancelled` bubbled by a call to
# `Nursery.cancel_scope.cancel()`)
except ContextCancelled as ctxc:
scope_err = ctxc
ctx._local_error: BaseException = scope_err
ctxc_from_child = ctxc
ctxc_from_callee = ctxc
# XXX TODO XXX: FIX THIS debug_mode BUGGGG!!!
# using this code and then resuming the REPL will
@ -2221,11 +2213,11 @@ async def open_context_from_portal(
# the above `._scope` can be cancelled due to:
# 1. an explicit self cancel via `Context.cancel()` or
# `Actor.cancel()`,
# 2. any "child"-side remote error, possibly also a cancellation
# 2. any "callee"-side remote error, possibly also a cancellation
# request by some peer,
# 3. any "parent" (aka THIS scope's) local error raised in the above `yield`
# 3. any "caller" (aka THIS scope's) local error raised in the above `yield`
except (
# CASE 3: standard local error in this parent/yieldee
# CASE 3: standard local error in this caller/yieldee
Exception,
# CASES 1 & 2: can manifest as a `ctx._scope_nursery`
@ -2239,9 +2231,9 @@ async def open_context_from_portal(
# any `Context._maybe_raise_remote_err()` call.
#
# 2.-`BaseExceptionGroup[ContextCancelled | RemoteActorError]`
# from any error delivered from the "child" side
# from any error delivered from the "callee" side
# AND a group-exc is only raised if there was > 1
# tasks started *here* in the "parent" / opener
# tasks started *here* in the "caller" / opener
# block. If any one of those tasks calls
# `.wait_for_result()` or `MsgStream.receive()`
# `._maybe_raise_remote_err()` will be transitively
@ -2254,8 +2246,8 @@ async def open_context_from_portal(
trio.Cancelled, # NOTE: NOT from inside the ctx._scope
KeyboardInterrupt,
) as rent_err:
scope_err = rent_err
) as caller_err:
scope_err = caller_err
ctx._local_error: BaseException = scope_err
# XXX: ALWAYS request the context to CANCEL ON any ERROR.
@ -2265,7 +2257,7 @@ async def open_context_from_portal(
# await debug.pause()
# log.cancel(
match scope_err:
case trio.Cancelled():
case trio.Cancelled:
logmeth = log.cancel
# XXX explicitly report on any non-graceful-taskc cases
@ -2294,9 +2286,9 @@ async def open_context_from_portal(
'Calling `ctx.cancel()`!\n'
)
# we don't need to cancel the child if it already
# we don't need to cancel the callee if it already
# told us it's cancelled ;p
if ctxc_from_child is None:
if ctxc_from_callee is None:
try:
await ctx.cancel()
except (
@ -2327,8 +2319,8 @@ async def open_context_from_portal(
# via a call to
# `Context._maybe_cancel_and_set_remote_error()`.
# As per `Context._deliver_msg()`, that error IS
# ALWAYS SET any time "child" side fails and causes
# "parent side" cancellation via a `ContextCancelled` here.
# ALWAYS SET any time "callee" side fails and causes "caller
# side" cancellation via a `ContextCancelled` here.
try:
result_or_err: Exception|Any = await ctx.wait_for_result()
except BaseException as berr:
@ -2364,7 +2356,7 @@ async def open_context_from_portal(
)
case (None, _):
log.runtime(
'Context returned final result from child task:\n'
'Context returned final result from callee task:\n'
f'<= peer: {uid}\n'
f' |_ {nsf}()\n\n'
@ -2459,7 +2451,7 @@ async def open_context_from_portal(
)
# TODO: should we add a `._cancel_req_received`
# flag to determine if the child manually called
# flag to determine if the callee manually called
# `ctx.cancel()`?
# -[ ] going to need a cid check no?
@ -2515,7 +2507,7 @@ def mk_context(
recv_chan: trio.MemoryReceiveChannel
send_chan, recv_chan = trio.open_memory_channel(msg_buffer_size)
# TODO: only scan parent-info if log level so high!
# TODO: only scan caller-info if log level so high!
from .devx._frame_stack import find_caller_info
caller_info: CallerInfo|None = find_caller_info()

View File

@ -28,10 +28,7 @@ from typing import (
from contextlib import asynccontextmanager as acm
from tractor.log import get_logger
from .trionics import (
gather_contexts,
collapse_eg,
)
from .trionics import gather_contexts
from .ipc import _connect_chan, Channel
from ._addr import (
UnwrappedAddress,
@ -90,6 +87,7 @@ async def get_registry(
yield regstr_ptl
@acm
async def get_root(
**kwargs,
@ -255,12 +253,9 @@ async def find_actor(
for addr in registry_addrs
)
portals: list[Portal]
async with (
collapse_eg(),
gather_contexts(
mngrs=maybe_portals,
) as portals,
):
async with gather_contexts(
mngrs=maybe_portals,
) as portals:
# log.runtime(
# 'Gathered portals:\n'
# f'{portals}'

View File

@ -21,7 +21,7 @@ Sub-process entry points.
from __future__ import annotations
from functools import partial
import multiprocessing as mp
# import os
import os
from typing import (
Any,
TYPE_CHECKING,
@ -38,7 +38,6 @@ from .devx import (
_frame_stack,
pformat,
)
# from .msg import pretty_struct
from .to_asyncio import run_as_asyncio_guest
from ._addr import UnwrappedAddress
from ._runtime import (
@ -128,13 +127,20 @@ def _trio_main(
if actor.loglevel is not None:
get_console_log(actor.loglevel)
actor_info: str = (
f'|_{actor}\n'
f' uid: {actor.uid}\n'
f' pid: {os.getpid()}\n'
f' parent_addr: {parent_addr}\n'
f' loglevel: {actor.loglevel}\n'
)
log.info(
f'Starting `trio` subactor from parent @ '
f'{parent_addr}\n'
'Starting new `trio` subactor\n'
+
pformat.nest_from_op(
input_op='>(', # see syntax ideas above
text=f'{actor}',
text=actor_info,
nest_indent=2, # since "complete"
)
)
logmeth = log.info
@ -143,7 +149,7 @@ def _trio_main(
+
pformat.nest_from_op(
input_op=')>', # like a "closed-to-play"-icon from super perspective
text=f'{actor}',
text=actor_info,
nest_indent=1,
)
)
@ -161,7 +167,7 @@ def _trio_main(
+
pformat.nest_from_op(
input_op='c)>', # closed due to cancel (see above)
text=f'{actor}',
text=actor_info,
)
)
except BaseException as err:
@ -171,7 +177,7 @@ def _trio_main(
+
pformat.nest_from_op(
input_op='x)>', # closed by error
text=f'{actor}',
text=actor_info,
)
)
# NOTE since we raise a tb will already be shown on the

View File

@ -39,10 +39,7 @@ import warnings
import trio
from .trionics import (
maybe_open_nursery,
collapse_eg,
)
from .trionics import maybe_open_nursery
from ._state import (
current_actor,
)
@ -118,10 +115,6 @@ class Portal:
@property
def chan(self) -> Channel:
'''
Ref to this ctx's underlying `tractor.ipc.Channel`.
'''
return self._chan
@property
@ -181,17 +174,10 @@ class Portal:
# not expecting a "main" result
if self._expect_result_ctx is None:
peer_id: str = f'{self.channel.aid.reprol()!r}'
log.warning(
f'Portal to peer {peer_id} will not deliver a final result?\n'
f'\n'
f'Context.result() can only be called by the parent of '
f'a sub-actor when it was spawned with '
f'`ActorNursery.run_in_actor()`'
f'\n'
f'Further this `ActorNursery`-method-API will deprecated in the'
f'near fututre!\n'
)
f"Portal for {self.channel.aid} not expecting a final"
" result?\nresult() should only be called if subactor"
" was spawned with `ActorNursery.run_in_actor()`")
return NoResult
# expecting a "main" result
@ -224,7 +210,6 @@ class Portal:
typname: str = type(self).__name__
log.warning(
f'`{typname}.result()` is DEPRECATED!\n'
f'\n'
f'Use `{typname}.wait_for_result()` instead!\n'
)
return await self.wait_for_result(
@ -236,10 +221,8 @@ class Portal:
# terminate all locally running async generator
# IPC calls
if self._streams:
peer_id: str = f'{self.channel.aid.reprol()!r}'
report: str = (
f'Cancelling all msg-streams with {peer_id}\n'
)
log.cancel(
f"Cancelling all streams with {self.channel.aid}")
for stream in self._streams.copy():
try:
await stream.aclose()
@ -248,18 +231,10 @@ class Portal:
# (unless of course at some point down the road we
# won't expect this to always be the case or need to
# detect it for respawning purposes?)
report += (
f'->) {stream!r} already closed\n'
)
log.cancel(report)
log.debug(f"{stream} was already closed.")
async def aclose(self):
log.debug(
f'Closing portal\n'
f'>}}\n'
f'|_{self}\n'
)
log.debug(f"Closing {self}")
# TODO: once we move to implementing our own `ReceiveChannel`
# (including remote task cancellation inside its `.aclose()`)
# we'll need to .aclose all those channels here
@ -285,22 +260,23 @@ class Portal:
__runtimeframe__: int = 1 # noqa
chan: Channel = self.channel
peer_id: str = f'{self.channel.aid.reprol()!r}'
if not chan.connected():
log.runtime(
'Peer {peer_id} is already disconnected\n'
'-> skipping cancel request..\n'
'This channel is already closed, skipping cancel request..'
)
return False
reminfo: str = (
f'c)=> {self.channel.aid}\n'
f' |_{chan}\n'
)
log.cancel(
f'Sending actor-runtime-cancel-req to peer\n'
f'\n'
f'c)=> {peer_id}\n'
f'Requesting actor-runtime cancel for peer\n\n'
f'{reminfo}'
)
# XXX the one spot we set it?
chan._cancel_called: bool = True
self.channel._cancel_called: bool = True
try:
# send cancel cmd - might not get response
# XXX: sure would be nice to make this work with
@ -321,9 +297,8 @@ class Portal:
# may timeout and we never get an ack (obvi racy)
# but that doesn't mean it wasn't cancelled.
log.debug(
f'May have failed to cancel peer?\n'
f'\n'
f'c)=?> {peer_id}\n'
'May have failed to cancel peer?\n'
f'{reminfo}'
)
# if we get here some weird cancellation case happened
@ -341,22 +316,22 @@ class Portal:
TransportClosed,
) as tpt_err:
ipc_borked_report: str = (
f'IPC for actor already closed/broken?\n\n'
f'\n'
f'c)=x> {peer_id}\n'
report: str = (
f'IPC chan for actor already closed or broken?\n\n'
f'{self.channel.aid}\n'
f' |_{self.channel}\n'
)
match tpt_err:
case TransportClosed():
log.debug(ipc_borked_report)
log.debug(report)
case _:
ipc_borked_report += (
report += (
f'\n'
f'Unhandled low-level transport-closed/error during\n'
f'Portal.cancel_actor()` request?\n'
f'<{type(tpt_err).__name__}( {tpt_err} )>\n'
)
log.warning(ipc_borked_report)
log.warning(report)
return False
@ -513,13 +488,10 @@ class Portal:
with trio.CancelScope(shield=True):
await ctx.cancel()
except trio.ClosedResourceError as cre:
except trio.ClosedResourceError:
# if the far end terminates before we send a cancel the
# underlying transport-channel may already be closed.
log.cancel(
f'Context.cancel() -> {cre!r}\n'
f'cid: {ctx.cid!r} already closed?\n'
)
log.cancel(f'Context {ctx} was already closed?')
# XXX: should this always be done?
# await recv_chan.aclose()
@ -586,13 +558,14 @@ async def open_portal(
assert actor
was_connected: bool = False
async with (
collapse_eg(),
maybe_open_nursery(
tn,
shield=shield,
) as tn,
):
async with maybe_open_nursery(
tn,
shield=shield,
strict_exception_groups=False,
# ^XXX^ TODO? soo roll our own then ??
# -> since we kinda want the "if only one `.exception` then
# just raise that" interface?
) as tn:
if not channel.connected():
await channel.connect()

View File

@ -37,11 +37,16 @@ import warnings
import trio
from . import _runtime
from ._runtime import (
Actor,
Arbiter,
# TODO: rename and make a non-actor subtype?
# Arbiter as Registry,
async_main,
)
from .devx import (
debug,
_frame_stack,
pformat as _pformat,
)
from . import _spawn
from . import _state
@ -58,7 +63,6 @@ from ._addr import (
)
from .trionics import (
is_multi_cancelled,
collapse_eg,
)
from ._exceptions import (
RuntimeFailure,
@ -97,7 +101,7 @@ async def maybe_block_bp(
):
logger.info(
f'Found `greenback` installed @ {maybe_mod}\n'
f'Enabling `tractor.pause_from_sync()` support!\n'
'Enabling `tractor.pause_from_sync()` support!\n'
)
os.environ['PYTHONBREAKPOINT'] = (
'tractor.devx.debug._sync_pause_from_builtin'
@ -192,19 +196,13 @@ async def open_root_actor(
# read-only state to sublayers?
# extra_rt_vars: dict|None = None,
) -> _runtime.Actor:
) -> Actor:
'''
Initialize the `tractor` runtime by starting a "root actor" in
a parent-most Python process.
All (disjoint) actor-process-trees-as-programs are created via
this entrypoint.
Runtime init entry point for ``tractor``.
'''
# XXX NEVER allow nested actor-trees!
if already_actor := _state.current_actor(
err_on_no_runtime=False,
):
if already_actor := _state.current_actor(err_on_no_runtime=False):
rtvs: dict[str, Any] = _state._runtime_vars
root_mailbox: list[str, int] = rtvs['_root_mailbox']
registry_addrs: list[list[str, int]] = rtvs['_registry_addrs']
@ -274,20 +272,14 @@ async def open_root_actor(
DeprecationWarning,
stacklevel=2,
)
uw_reg_addrs = [arbiter_addr]
registry_addrs = [arbiter_addr]
uw_reg_addrs = registry_addrs
if not uw_reg_addrs:
uw_reg_addrs: list[UnwrappedAddress] = default_lo_addrs(
if not registry_addrs:
registry_addrs: list[UnwrappedAddress] = default_lo_addrs(
enable_transports
)
# must exist by now since all below code is dependent
assert uw_reg_addrs
registry_addrs: list[Address] = [
wrap_address(uw_addr)
for uw_addr in uw_reg_addrs
]
assert registry_addrs
loglevel = (
loglevel
@ -336,10 +328,10 @@ async def open_root_actor(
enable_stack_on_sig()
# closed into below ping task-func
ponged_addrs: list[Address] = []
ponged_addrs: list[UnwrappedAddress] = []
async def ping_tpt_socket(
addr: Address,
addr: UnwrappedAddress,
timeout: float = 1,
) -> None:
'''
@ -359,22 +351,17 @@ async def open_root_actor(
# be better to eventually have a "discovery" protocol
# with basic handshake instead?
with trio.move_on_after(timeout):
async with _connect_chan(addr.unwrap()):
async with _connect_chan(addr):
ponged_addrs.append(addr)
except OSError:
# ?TODO, make this a "discovery" log level?
# TODO: make this a "discovery" log level?
logger.info(
f'No root-actor registry found @ {addr!r}\n'
f'No actor registry found @ {addr}\n'
)
# !TODO, this is basically just another (abstract)
# happy-eyeballs, so we should try for formalize it somewhere
# in a `.[_]discovery` ya?
#
async with trio.open_nursery() as tn:
for uw_addr in uw_reg_addrs:
addr: Address = wrap_address(uw_addr)
for addr in registry_addrs:
tn.start_soon(
ping_tpt_socket,
addr,
@ -396,35 +383,31 @@ async def open_root_actor(
f'Registry(s) seem(s) to exist @ {ponged_addrs}'
)
actor = _runtime.Actor(
actor = Actor(
name=name or 'anonymous',
uuid=mk_uuid(),
registry_addrs=ponged_addrs,
loglevel=loglevel,
enable_modules=enable_modules,
)
# **DO NOT** use the registry_addrs as the
# ipc-transport-server's bind-addrs as this is
# a new NON-registrar, ROOT-actor.
#
# XXX INSTEAD, bind random addrs using the same tpt
# proto.
# DO NOT use the registry_addrs as the transport server
# addrs for this new non-registar, root-actor.
for addr in ponged_addrs:
waddr: Address = wrap_address(addr)
trans_bind_addrs.append(
addr.get_random(
bindspace=addr.bindspace,
)
waddr.get_random(bindspace=waddr.bindspace)
)
# 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:
# NOTE that if the current actor IS THE REGISTAR, the
# following init steps are taken:
# - the tranport layer server is bound to each addr
# pair defined in provided registry_addrs, or the default.
trans_bind_addrs = uw_reg_addrs
trans_bind_addrs = registry_addrs
# - it is normally desirable for any registrar to stay up
# indefinitely until either all registered (child/sub)
@ -435,8 +418,7 @@ async def open_root_actor(
# https://github.com/goodboy/tractor/pull/348
# https://github.com/goodboy/tractor/issues/296
# TODO: rename as `RootActor` or is that even necessary?
actor = _runtime.Arbiter(
actor = Arbiter(
name=name or 'registrar',
uuid=mk_uuid(),
registry_addrs=registry_addrs,
@ -448,16 +430,6 @@ async def open_root_actor(
# `.trio.run()`.
actor._infected_aio = _state._runtime_vars['_is_infected_aio']
# 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.
raddrs: list[Address] = _state._runtime_vars['_root_addrs']
raddrs.extend(trans_bind_addrs)
# TODO, remove once we have also removed all usage;
# eventually all (root-)registry apis should expect > 1 addr.
_state._runtime_vars['_root_mailbox'] = raddrs[0]
# Start up main task set via core actor-runtime nurseries.
try:
# assign process-local actor
@ -465,27 +437,21 @@ async def open_root_actor(
# start local channel-server and fake the portal API
# NOTE: this won't block since we provide the nursery
report: str = f'Starting actor-runtime for {actor.aid.reprol()!r}\n'
if reg_addrs := actor.registry_addrs:
report += (
'-> Opening new registry @ '
+
'\n'.join(
f'{addr}' for addr in reg_addrs
)
)
logger.info(f'{report}\n')
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}'
)
# start runtime in a bg sub-task, yield to caller.
async with (
collapse_eg(),
trio.open_nursery() as root_tn,
# start the actor runtime in a new task
async with trio.open_nursery(
strict_exception_groups=False,
# ^XXX^ TODO? instead unpack any RAE as per "loose" style?
) as nursery:
# XXX, finally-footgun below?
# -> see note on why shielding.
# maybe_raise_from_masking_exc(),
):
# `_runtime.async_main()` creates an internal 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
@ -493,9 +459,9 @@ async def open_root_actor(
# "actor runtime" primitives are SC-compat and thus all
# transitively spawned actors/processes must be as
# well.
await root_tn.start(
await nursery.start(
partial(
_runtime.async_main,
async_main,
actor,
accept_addrs=trans_bind_addrs,
parent_addr=None
@ -543,7 +509,7 @@ async def open_root_actor(
raise
finally:
# NOTE/TODO?, not sure if we'll ever need this but it's
# NOTE: not sure if we'll ever need this but it's
# possibly better for even more determinism?
# logger.cancel(
# f'Waiting on {len(nurseries)} nurseries in root..')
@ -552,21 +518,12 @@ async def open_root_actor(
# for an in nurseries:
# tempn.start_soon(an.exited.wait)
op_nested_actor_repr: str = _pformat.nest_from_op(
input_op='>) ',
text=actor.pformat(),
nest_prefix='|_',
)
logger.info(
f'Closing down root actor\n'
f'{op_nested_actor_repr}'
f'>)\n'
f'|_{actor}\n'
)
# XXX, THIS IS A *finally-footgun*!
# -> though already shields iternally it can
# taskc here and mask underlying errors raised in
# the try-block above?
with trio.CancelScope(shield=True):
await actor.cancel(None) # self cancel
await actor.cancel(None) # self cancel
finally:
# revert all process-global runtime state
if (
@ -579,16 +536,10 @@ async def open_root_actor(
_state._current_actor = None
_state._last_actor_terminated = actor
sclang_repr: str = _pformat.nest_from_op(
input_op=')>',
text=actor.pformat(),
nest_prefix='|_',
nest_indent=1,
)
logger.info(
logger.runtime(
f'Root actor terminated\n'
f'{sclang_repr}'
f')>\n'
f' |_{actor}\n'
)

View File

@ -64,7 +64,6 @@ from .trionics import (
from .devx import (
debug,
add_div,
pformat as _pformat,
)
from . import _state
from .log import get_logger
@ -73,7 +72,7 @@ from .msg import (
MsgCodec,
PayloadT,
NamespacePath,
pretty_struct,
# pretty_struct,
_ops as msgops,
)
from tractor.msg.types import (
@ -221,18 +220,11 @@ async def _invoke_non_context(
task_status.started(ctx)
result = await coro
fname: str = func.__name__
op_nested_task: str = _pformat.nest_from_op(
input_op=f')> cid: {ctx.cid!r}',
text=f'{ctx._task}',
nest_indent=1, # under >
)
log.runtime(
f'RPC task complete\n'
f'\n'
f'{op_nested_task}\n'
f'\n'
f')> {fname}() -> {pformat(result)}\n'
'RPC complete:\n'
f'task: {ctx._task}\n'
f'|_cid={ctx.cid}\n'
f'|_{fname}() -> {pformat(result)}\n'
)
# NOTE: only send result if we know IPC isn't down
@ -284,10 +276,6 @@ async def _errors_relayed_via_ipc(
try:
yield # run RPC invoke body
except TransportClosed:
log.exception('Tpt disconnect during remote-exc relay?')
raise
# box and ship RPC errors for wire-transit via
# the task's requesting parent IPC-channel.
except (
@ -323,9 +311,6 @@ async def _errors_relayed_via_ipc(
and debug_kbis
)
)
# TODO? better then `debug_filter` below?
# and
# not isinstance(err, TransportClosed)
):
# XXX QUESTION XXX: is there any case where we'll
# want to debug IPC disconnects as a default?
@ -334,25 +319,13 @@ async def _errors_relayed_via_ipc(
# recovery logic - the only case is some kind of
# strange bug in our transport layer itself? Going
# to keep this open ended for now.
if _state.debug_mode():
log.exception(
f'RPC task crashed!\n'
f'Attempting to enter debugger\n'
f'\n'
f'{ctx}'
)
log.debug(
'RPC task crashed, attempting to enter debugger\n'
f'|_{ctx}'
)
entered_debug = await debug._maybe_enter_pm(
err,
api_frame=inspect.currentframe(),
# don't REPL any psuedo-expected tpt-disconnect
# debug_filter=lambda exc: (
# type (exc) not in {
# TransportClosed,
# }
# ),
)
if not entered_debug:
# if we prolly should have entered the REPL but
@ -469,7 +442,7 @@ async def _invoke(
kwargs: dict[str, Any],
is_rpc: bool = True,
hide_tb: bool = False,
hide_tb: bool = True,
return_msg_type: Return|CancelAck = Return,
task_status: TaskStatus[
@ -673,7 +646,8 @@ async def _invoke(
# scope ensures unasking of the `await coro` below
# *should* never be interfered with!!
maybe_raise_from_masking_exc(
unmask_from=(Cancelled,),
tn=tn,
unmask_from=Cancelled,
) as _mbme, # maybe boxed masked exc
):
ctx._scope_nursery = tn
@ -690,23 +664,9 @@ async def _invoke(
ctx._result = res
log.runtime(
f'Sending result msg and exiting {ctx.side!r}\n'
f'\n'
f'{pretty_struct.pformat(return_msg)}\n'
f'{return_msg}\n'
)
try:
await chan.send(return_msg)
except TransportClosed:
log.exception(
f"Failed send final result to 'parent'-side of IPC-ctx!\n"
f'\n'
f'{chan}\n'
f'Channel already disconnected ??\n'
f'\n'
f'{pretty_struct.pformat(return_msg)}'
)
# ?TODO? will this ever be true though?
if chan.connected():
raise
await chan.send(return_msg)
# NOTE: this happens IFF `ctx._scope.cancel()` is
# called by any of,
@ -796,6 +756,7 @@ async def _invoke(
BaseExceptionGroup,
BaseException,
trio.Cancelled,
) as _scope_err:
scope_err = _scope_err
if (
@ -871,12 +832,12 @@ async def _invoke(
else:
descr_str += f'\n{merr!r}\n'
else:
descr_str += f'\nwith final result {ctx.outcome!r}\n'
descr_str += f'\nand final result {ctx.outcome!r}\n'
logmeth(
f'{message}\n'
f'\n'
f'{descr_str}\n'
message
+
descr_str
)
@ -1043,6 +1004,8 @@ async def process_messages(
cid=cid,
kwargs=kwargs,
):
kwargs |= {'req_chan': chan}
# XXX NOTE XXX don't start entire actor
# runtime cancellation if this actor is
# currently in debug mode!
@ -1061,14 +1024,14 @@ async def process_messages(
cid,
chan,
actor.cancel,
kwargs | {'req_chan': chan},
kwargs,
is_rpc=False,
return_msg_type=CancelAck,
)
log.runtime(
'Cancelling RPC-msg-loop with peer\n'
f'->c}} {chan.aid.reprol()}@[{chan.maddr}]\n'
'Cancelling IPC transport msg-loop with peer:\n'
f'|_{chan}\n'
)
loop_cs.cancel()
break
@ -1081,7 +1044,7 @@ async def process_messages(
):
target_cid: str = kwargs['cid']
kwargs |= {
'requesting_aid': chan.aid,
'requesting_uid': chan.uid,
'ipc_msg': msg,
# XXX NOTE! ONLY the rpc-task-owning
@ -1117,34 +1080,21 @@ async def process_messages(
ns=ns,
func=funcname,
kwargs=kwargs, # type-spec this? see `msg.types`
uid=actor_uuid,
uid=actorid,
):
if actor_uuid != chan.aid.uid:
raise RuntimeError(
f'IPC <Start> msg <-> chan.aid mismatch!?\n'
f'Channel.aid = {chan.aid!r}\n'
f'Start.uid = {actor_uuid!r}\n'
)
# await debug.pause()
op_repr: str = 'Start <=) '
req_repr: str = _pformat.nest_from_op(
input_op=op_repr,
op_suffix='',
nest_prefix='',
text=f'{chan}',
nest_indent=len(op_repr)-1,
rm_from_first_ln='<',
# ^XXX, subtract -1 to account for
# <Channel
# ^_chevron to be stripped
)
start_status: str = (
'Handling RPC request\n'
f'{req_repr}\n'
f'\n'
f'->{{ ipc-context-id: {cid!r}\n'
f'->{{ nsp for fn: `{ns}.{funcname}({kwargs})`\n'
'Handling RPC `Start` request\n'
f'<= peer: {actorid}\n\n'
f' |_{chan}\n'
f' |_cid: {cid}\n\n'
# f' |_{ns}.{funcname}({kwargs})\n'
f'>> {actor.uid}\n'
f' |_{actor}\n'
f' -> nsp: `{ns}.{funcname}({kwargs})`\n'
# f' |_{ns}.{funcname}({kwargs})\n\n'
# f'{pretty_struct.pformat(msg)}\n'
)
# runtime-internal endpoint: `Actor.<funcname>`
@ -1173,6 +1123,10 @@ async def process_messages(
await chan.send(err_msg)
continue
start_status += (
f' -> func: {func}\n'
)
# schedule a task for the requested RPC function
# in the actor's main "service nursery".
#
@ -1180,7 +1134,7 @@ async def process_messages(
# supervision isolation? would avoid having to
# manage RPC tasks individually in `._rpc_tasks`
# table?
start_status += '->( scheduling new task..\n'
start_status += ' -> scheduling new task..\n'
log.runtime(start_status)
try:
ctx: Context = await actor._service_n.start(
@ -1264,24 +1218,12 @@ async def process_messages(
# END-OF `async for`:
# IPC disconnected via `trio.EndOfChannel`, likely
# due to a (graceful) `Channel.aclose()`.
chan_op_repr: str = '<=x] '
chan_repr: str = _pformat.nest_from_op(
input_op=chan_op_repr,
op_suffix='',
nest_prefix='',
text=chan.pformat(),
nest_indent=len(chan_op_repr)-1,
rm_from_first_ln='<',
)
log.runtime(
f'IPC channel disconnected\n'
f'{chan_repr}\n'
f'\n'
f'->c) cancelling RPC tasks.\n'
f'channel for {chan.uid} disconnected, cancelling RPC tasks\n'
f'|_{chan}\n'
)
await actor.cancel_rpc_tasks(
req_aid=actor.aid,
req_uid=actor.uid,
# a "self cancel" in terms of the lifetime of the
# IPC connection which is presumed to be the
# source of any requests for spawned tasks.
@ -1353,37 +1295,13 @@ async def process_messages(
finally:
# msg debugging for when he machinery is brokey
if msg is None:
message: str = 'Exiting RPC-loop without receiving a msg?'
message: str = 'Exiting IPC msg loop without receiving a msg?'
else:
task_op_repr: str = ')>'
task: trio.Task = trio.lowlevel.current_task()
# maybe add cancelled opt prefix
if task._cancel_status.effectively_cancelled:
task_op_repr = 'c' + task_op_repr
task_repr: str = _pformat.nest_from_op(
input_op=task_op_repr,
text=f'{task!r}',
nest_indent=1,
)
# chan_op_repr: str = '<=} '
# chan_repr: str = _pformat.nest_from_op(
# input_op=chan_op_repr,
# op_suffix='',
# nest_prefix='',
# text=chan.pformat(),
# nest_indent=len(chan_op_repr)-1,
# rm_from_first_ln='<',
# )
message: str = (
f'Exiting RPC-loop with final msg\n'
f'\n'
# f'{chan_repr}\n'
f'{task_repr}\n'
f'\n'
f'{pretty_struct.pformat(msg)}'
f'\n'
'Exiting IPC msg loop with final msg\n\n'
f'<= peer: {chan.uid}\n'
f' |_{chan}\n\n'
# f'{pretty_struct.pformat(msg)}'
)
log.runtime(message)

View File

@ -55,7 +55,6 @@ from typing import (
TYPE_CHECKING,
)
import uuid
import textwrap
from types import ModuleType
import warnings
@ -74,9 +73,6 @@ from tractor.msg import (
pretty_struct,
types as msgtypes,
)
from .trionics import (
collapse_eg,
)
from .ipc import (
Channel,
# IPCServer, # causes cycles atm..
@ -101,10 +97,7 @@ from ._exceptions import (
MsgTypeError,
unpack_error,
)
from .devx import (
debug,
pformat as _pformat
)
from .devx import debug
from ._discovery import get_registry
from ._portal import Portal
from . import _state
@ -213,7 +206,7 @@ class Actor:
*,
enable_modules: list[str] = [],
loglevel: str|None = None,
registry_addrs: list[Address]|None = None,
registry_addrs: list[UnwrappedAddress]|None = None,
spawn_method: str|None = None,
# TODO: remove!
@ -234,7 +227,7 @@ class Actor:
# state
self._cancel_complete = trio.Event()
self._cancel_called_by: 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
@ -256,12 +249,11 @@ class Actor:
if arbiter_addr is not None:
warnings.warn(
'`Actor(arbiter_addr=<blah>)` is now deprecated.\n'
'Use `registry_addrs: list[Address]` instead.',
'Use `registry_addrs: list[tuple]` instead.',
DeprecationWarning,
stacklevel=2,
)
registry_addrs: list[Address] = [wrap_address(arbiter_addr)]
registry_addrs: list[UnwrappedAddress] = [arbiter_addr]
# marked by the process spawning backend at startup
# will be None for the parent most process started manually
@ -300,10 +292,8 @@ class Actor:
# input via the validator.
self._reg_addrs: list[UnwrappedAddress] = []
if registry_addrs:
_state._runtime_vars['_registry_addrs'] = self.reg_addrs = [
addr.unwrap()
for addr in registry_addrs
]
self.reg_addrs: list[UnwrappedAddress] = registry_addrs
_state._runtime_vars['_registry_addrs'] = registry_addrs
@property
def aid(self) -> msgtypes.Aid:
@ -349,125 +339,46 @@ class Actor:
def pid(self) -> int:
return self._aid.pid
@property
def repr_state(self) -> str:
if self.cancel_complete:
return 'cancelled'
elif canceller := self.cancel_caller:
return f' and cancel-called by {canceller}'
else:
return 'running'
def pformat(
self,
ds: str = ': ',
indent: int = 0,
privates: bool = False,
) -> str:
fmtstr: str = f'|_id: {self.aid.reprol()!r}\n'
if privates:
aid_nest_prefix: str = '|_aid='
aid_field_repr: str = _pformat.nest_from_op(
input_op='',
text=pretty_struct.pformat(
struct=self.aid,
field_indent=2,
),
op_suffix='',
nest_prefix=aid_nest_prefix,
nest_indent=0,
)
fmtstr: str = f'{aid_field_repr}'
def pformat(self) -> str:
ds: str = '='
parent_uid: tuple|None = None
if rent_chan := self._parent_chan:
fmtstr += (
f"|_parent{ds}{rent_chan.aid.reprol()}\n"
)
parent_uid = rent_chan.uid
peers: list = []
server: _server.IPCServer = self.ipc_server
if server:
if privates:
server_repr: str = self._ipc_server.pformat(
privates=privates,
)
# create field ln as a key-header indented under
# and up to the section's key prefix.
# ^XXX if we were to indent `repr(Server)` to
# '<key>: '
# _here_^
server_repr: str = _pformat.nest_from_op(
input_op='', # nest as sub-obj
op_suffix='',
text=server_repr,
)
fmtstr += (
f"{server_repr}"
)
else:
fmtstr += (
f'|_ipc: {server.repr_state!r}\n'
)
peers: list[tuple] = list(server._peer_connected)
fmtstr += (
f'|_rpc: {len(self._rpc_tasks)} active tasks\n'
fmtstr: str = (
f' |_id: {self.aid!r}\n'
# f" aid{ds}{self.aid!r}\n"
f" parent{ds}{parent_uid}\n"
f'\n'
f' |_ipc: {len(peers)!r} connected peers\n'
f" peers{ds}{peers!r}\n"
f" ipc_server{ds}{self._ipc_server}\n"
f'\n'
f' |_rpc: {len(self._rpc_tasks)} tasks\n'
f" ctxs{ds}{len(self._contexts)}\n"
f'\n'
f' |_runtime: ._task{ds}{self._task!r}\n'
f' _spawn_method{ds}{self._spawn_method}\n'
f' _actoruid2nursery{ds}{self._actoruid2nursery}\n'
f' _forkserver_info{ds}{self._forkserver_info}\n'
f'\n'
f' |_state: "TODO: .repr_state()"\n'
f' _cancel_complete{ds}{self._cancel_complete}\n'
f' _cancel_called_by_remote{ds}{self._cancel_called_by_remote}\n'
f' _cancel_called{ds}{self._cancel_called}\n'
)
# TODO, actually fix the .repr_state impl/output?
# append ipc-ctx state summary
# ctxs: dict = self._contexts
# if ctxs:
# ctx_states: dict[str, int] = {}
# for ctx in self._contexts.values():
# ctx_state: str = ctx.repr_state
# cnt = ctx_states.setdefault(ctx_state, 0)
# ctx_states[ctx_state] = cnt + 1
# fmtstr += (
# f" ctxs{ds}{ctx_states}\n"
# )
# runtime-state
task_name: str = '<dne>'
if task := self._task:
task_name: str = task.name
fmtstr += (
# TODO, this just like ctx?
f'|_state: {self.repr_state!r}\n'
f' task: {task_name}\n'
f' loglevel: {self.loglevel!r}\n'
f' subactors_spawned: {len(self._actoruid2nursery)}\n'
return (
'<Actor(\n'
+
fmtstr
+
')>\n'
)
if not _state.is_root_process():
fmtstr += f' spawn_method: {self._spawn_method!r}\n'
if privates:
fmtstr += (
# f' actoruid2nursery{ds}{self._actoruid2nursery}\n'
f' cancel_complete{ds}{self._cancel_complete}\n'
f' cancel_called_by_remote{ds}{self._cancel_called_by}\n'
f' cancel_called{ds}{self._cancel_called}\n'
)
if fmtstr:
fmtstr: str = textwrap.indent(
text=fmtstr,
prefix=' '*(1 + indent),
)
_repr: str = (
f'<{type(self).__name__}(\n'
f'{fmtstr}'
f')>\n'
)
if indent:
_repr: str = textwrap.indent(
text=_repr,
prefix=' '*indent,
)
return _repr
__repr__ = pformat
@ -475,11 +386,7 @@ class Actor:
def reg_addrs(self) -> list[UnwrappedAddress]:
'''
List of (socket) addresses for all known (and contactable)
registry-service actors in "unwrapped" (i.e. IPC interchange
wire-compat) form.
If you are looking for the "wrapped" address form, use
`.registry_addrs` instead.
registry actors.
'''
return self._reg_addrs
@ -498,14 +405,8 @@ class Actor:
self._reg_addrs = addrs
@property
def registry_addrs(self) -> list[Address]:
return [wrap_address(uw_addr)
for uw_addr in self.reg_addrs]
def load_modules(
self,
) -> None:
'''
Load explicitly enabled python modules from local fs after
@ -552,14 +453,6 @@ class Actor:
)
raise
# ?TODO, factor this meth-iface into a new `.rpc` subsys primitive?
# - _get_rpc_func(),
# - _deliver_ctx_payload(),
# - get_context(),
# - start_remote_task(),
# - cancel_rpc_tasks(),
# - _cancel_task(),
#
def _get_rpc_func(self, ns, funcname):
'''
Try to lookup and return a target RPC func from the
@ -603,11 +496,11 @@ class Actor:
queue.
'''
aid: msgtypes.Aid = chan.aid
assert aid, f"`chan.aid` can't be {aid}"
uid: tuple[str, str] = chan.uid
assert uid, f"`chan.uid` can't be {uid}"
try:
ctx: Context = self._contexts[(
aid.uid,
uid,
cid,
# TODO: how to determine this tho?
@ -618,7 +511,7 @@ class Actor:
'Ignoring invalid IPC msg!?\n'
f'Ctx seems to not/no-longer exist??\n'
f'\n'
f'<=? {aid.reprol()!r}\n'
f'<=? {uid}\n'
f' |_{pretty_struct.pformat(msg)}\n'
)
match msg:
@ -667,7 +560,6 @@ class Actor:
msging session's lifetime.
'''
# ?TODO, use Aid here as well?
actor_uid = chan.uid
assert actor_uid
try:
@ -1016,22 +908,6 @@ class Actor:
None, # self cancel all rpc tasks
)
@property
def cancel_complete(self) -> bool:
return self._cancel_complete.is_set()
@property
def cancel_called(self) -> bool:
'''
Was this actor requested to cancel by a remote peer actor.
'''
return self._cancel_called_by is not None
@property
def cancel_caller(self) -> msgtypes.Aid|None:
return self._cancel_called_by
async def cancel(
self,
@ -1056,18 +932,20 @@ class Actor:
'''
(
requesting_aid, # Aid
requester_type, # str
requesting_uid,
requester_type,
req_chan,
log_meth,
) = (
req_chan.aid,
req_chan.uid,
'peer',
req_chan,
log.cancel,
) if req_chan else (
# a self cancel of ALL rpc tasks
self.aid,
self.uid,
'self',
self,
log.runtime,
@ -1075,14 +953,14 @@ class Actor:
# TODO: just use the new `Context.repr_rpc: str` (and
# other) repr fields instead of doing this all manual..
msg: str = (
f'Actor-runtime cancel request from {requester_type!r}\n'
f'Actor-runtime cancel request from {requester_type}\n\n'
f'<=c) {requesting_uid}\n'
f' |_{self}\n'
f'\n'
f'<=c)\n'
f'{self}'
)
# TODO: what happens here when we self-cancel tho?
self._cancel_called_by: tuple = requesting_aid
self._cancel_called_by_remote: tuple = requesting_uid
self._cancel_called = True
# cancel all ongoing rpc tasks
@ -1110,7 +988,7 @@ class Actor:
# self-cancel **all** ongoing RPC tasks
await self.cancel_rpc_tasks(
req_aid=requesting_aid,
req_uid=requesting_uid,
parent_chan=None,
)
@ -1127,11 +1005,19 @@ class Actor:
self._cancel_complete.set()
return True
# XXX: hard kill logic if needed?
# def _hard_mofo_kill(self):
# # If we're the root actor or zombied kill everything
# if self._parent_chan is None: # TODO: more robust check
# root = trio.lowlevel.current_root_task()
# for n in root.child_nurseries:
# n.cancel_scope.cancel()
async def _cancel_task(
self,
cid: str,
parent_chan: Channel,
requesting_aid: msgtypes.Aid|None,
requesting_uid: tuple[str, str]|None,
ipc_msg: dict|None|bool = False,
@ -1169,7 +1055,7 @@ class Actor:
log.runtime(
'Cancel request for invalid RPC task.\n'
'The task likely already completed or was never started!\n\n'
f'<= canceller: {requesting_aid}\n'
f'<= canceller: {requesting_uid}\n'
f'=> {cid}@{parent_chan.uid}\n'
f' |_{parent_chan}\n'
)
@ -1177,12 +1063,9 @@ class Actor:
log.cancel(
'Rxed cancel request for RPC task\n'
f'{ctx._task!r} <=c) {requesting_aid}\n'
f'|_>> {ctx.repr_rpc}\n'
# f'|_{ctx._task}\n'
# f' >> {ctx.repr_rpc}\n'
f'<=c) {requesting_uid}\n'
f' |_{ctx._task}\n'
f' >> {ctx.repr_rpc}\n'
# f'=> {ctx._task}\n'
# f' >> Actor._cancel_task() => {ctx._task}\n'
# f' |_ {ctx._task}\n\n'
@ -1203,9 +1086,9 @@ class Actor:
)
if (
ctx._canceller is None
and requesting_aid
and requesting_uid
):
ctx._canceller: tuple = requesting_aid.uid
ctx._canceller: tuple = requesting_uid
# TODO: pack the RPC `{'cmd': <blah>}` msg into a ctxc and
# then raise and pack it here?
@ -1231,7 +1114,7 @@ class Actor:
# wait for _invoke to mark the task complete
flow_info: str = (
f'<= canceller: {requesting_aid}\n'
f'<= canceller: {requesting_uid}\n'
f'=> ipc-parent: {parent_chan}\n'
f'|_{ctx}\n'
)
@ -1248,7 +1131,7 @@ class Actor:
async def cancel_rpc_tasks(
self,
req_aid: msgtypes.Aid,
req_uid: tuple[str, str],
# NOTE: when None is passed we cancel **all** rpc
# tasks running in this actor!
@ -1265,7 +1148,7 @@ class Actor:
if not tasks:
log.runtime(
'Actor has no cancellable RPC tasks?\n'
f'<= canceller: {req_aid.reprol()}\n'
f'<= canceller: {req_uid}\n'
)
return
@ -1305,7 +1188,7 @@ class Actor:
)
log.cancel(
f'Cancelling {descr} RPC tasks\n\n'
f'<=c) {req_aid} [canceller]\n'
f'<=c) {req_uid} [canceller]\n'
f'{rent_chan_repr}'
f'c)=> {self.uid} [cancellee]\n'
f' |_{self} [with {len(tasks)} tasks]\n'
@ -1333,7 +1216,7 @@ class Actor:
await self._cancel_task(
cid,
task_caller_chan,
requesting_aid=req_aid,
requesting_uid=req_uid,
)
if tasks:
@ -1361,13 +1244,25 @@ class Actor:
'''
return self.accept_addrs[0]
# TODO, this should delegate ONLY to the
# `._spawn_spec._runtime_vars: dict` / `._state` APIs?
#
# XXX, AH RIGHT that's why..
# it's bc we pass this as a CLI flag to the child.py precisely
# bc we need the bootstrapping pre `async_main()`.. but maybe
# keep this as an impl deat and not part of the pub iface impl?
def get_parent(self) -> Portal:
'''
Return a `Portal` to our parent.
'''
assert self._parent_chan, "No parent channel for this actor?"
return Portal(self._parent_chan)
def get_chans(
self,
uid: tuple[str, str],
) -> list[Channel]:
'''
Return all IPC channels to the actor with provided `uid`.
'''
return self._peers[uid]
def is_infected_aio(self) -> bool:
'''
If `True`, this actor is running `trio` in guest mode on
@ -1378,23 +1273,6 @@ class Actor:
'''
return self._infected_aio
# ?TODO, is this the right type for this method?
def get_parent(self) -> Portal:
'''
Return a `Portal` to our parent.
'''
assert self._parent_chan, "No parent channel for this actor?"
return Portal(self._parent_chan)
# XXX: hard kill logic if needed?
# def _hard_mofo_kill(self):
# # If we're the root actor or zombied kill everything
# if self._parent_chan is None: # TODO: more robust check
# root = trio.lowlevel.current_root_task()
# for n in root.child_nurseries:
# n.cancel_scope.cancel()
async def async_main(
actor: Actor,
@ -1438,8 +1316,6 @@ async def async_main(
# establish primary connection with immediate parent
actor._parent_chan: Channel|None = None
# is this a sub-actor?
# get runtime info from parent.
if parent_addr is not None:
(
actor._parent_chan,
@ -1474,18 +1350,18 @@ async def async_main(
# parent is kept alive as a resilient service until
# cancellation steps have (mostly) occurred in
# a deterministic way.
root_tn: trio.Nursery
async with (
collapse_eg(),
trio.open_nursery() as root_tn,
):
actor._root_n = root_tn
async with trio.open_nursery(
strict_exception_groups=False,
) as root_nursery:
actor._root_n = root_nursery
assert actor._root_n
ipc_server: _server.IPCServer
async with (
collapse_eg(),
trio.open_nursery() as service_nursery,
trio.open_nursery(
strict_exception_groups=False,
) as service_nursery,
_server.open_ipc_server(
parent_tn=service_nursery,
stream_handler_tn=service_nursery,
@ -1536,6 +1412,9 @@ async def async_main(
# TODO: why is this not with the root nursery?
try:
log.runtime(
'Booting IPC server'
)
eps: list = await ipc_server.listen_on(
accept_addrs=accept_addrs,
stream_handler_nursery=service_nursery,
@ -1567,6 +1446,18 @@ async def async_main(
# TODO, just read direct from ipc_server?
accept_addrs: list[UnwrappedAddress] = 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.
if _state._runtime_vars['_is_root']:
raddrs: list[Address] = _state._runtime_vars['_root_addrs']
for addr in accept_addrs:
waddr: Address = wrap_address(addr)
raddrs.append(addr)
else:
_state._runtime_vars['_root_mailbox'] = raddrs[0]
# Register with the arbiter if we're told its addr
log.runtime(
f'Registering `{actor.name}` => {pformat(accept_addrs)}\n'
@ -1584,7 +1475,6 @@ async def async_main(
except AssertionError:
await debug.pause()
# !TODO, get rid of the local-portal crap XD
async with get_registry(addr) as reg_portal:
for accept_addr in accept_addrs:
accept_addr = wrap_address(accept_addr)
@ -1609,7 +1499,7 @@ async def async_main(
# start processing parent requests until our channel
# server is 100% up and running.
if actor._parent_chan:
await root_tn.start(
await root_nursery.start(
partial(
_rpc.process_messages,
chan=actor._parent_chan,
@ -1621,9 +1511,8 @@ async def async_main(
# 'Blocking on service nursery to exit..\n'
)
log.runtime(
'Service nursery complete\n'
'\n'
'->} waiting on root nursery to complete..\n'
"Service nursery complete\n"
"Waiting on root nursery to complete"
)
# Blocks here as expected until the root nursery is
@ -1678,7 +1567,6 @@ async def async_main(
finally:
teardown_report: str = (
'Main actor-runtime task completed\n'
'\n'
)
# ?TODO? should this be in `._entry`/`._root` mods instead?
@ -1720,8 +1608,7 @@ async def async_main(
# Unregister actor from the registry-sys / registrar.
if (
is_registered
and
not actor.is_registrar
and not actor.is_registrar
):
failed: bool = False
for addr in actor.reg_addrs:
@ -1756,30 +1643,23 @@ async def async_main(
ipc_server.has_peers(check_chans=True)
):
teardown_report += (
f'-> Waiting for remaining peers to clear..\n'
f' {pformat(ipc_server._peers)}'
f'-> Waiting for remaining peers {ipc_server._peers} to clear..\n'
)
log.runtime(teardown_report)
await ipc_server.wait_for_no_more_peers()
await ipc_server.wait_for_no_more_peers(
shield=True,
)
teardown_report += (
'-]> all peer channels are complete.\n'
'-> All peer channels are complete\n'
)
# op_nested_actor_repr: str = _pformat.nest_from_op(
# input_op=')>',
# text=actor.pformat(),
# nest_prefix='|_',
# nest_indent=1, # under >
# )
teardown_report += (
'-)> actor runtime main task exit.\n'
# f'{op_nested_actor_repr}'
'Actor runtime exiting\n'
f'>)\n'
f'|_{actor}\n'
)
# if _state._runtime_vars['_is_root']:
# log.info(teardown_report)
# else:
log.runtime(teardown_report)
log.info(teardown_report)
# TODO: rename to `Registry` and move to `.discovery._registry`!

View File

@ -34,9 +34,9 @@ from typing import (
import trio
from trio import TaskStatus
from .devx import (
debug,
pformat as _pformat
from .devx.debug import (
maybe_wait_for_debugger,
acquire_debug_lock,
)
from tractor._state import (
current_actor,
@ -51,17 +51,14 @@ from tractor._portal import Portal
from tractor._runtime import Actor
from tractor._entry import _mp_main
from tractor._exceptions import ActorFailure
from tractor.msg import (
types as msgtypes,
pretty_struct,
from tractor.msg.types import (
Aid,
SpawnSpec,
)
if TYPE_CHECKING:
from ipc import (
_server,
Channel,
)
from ipc import IPCServer
from ._supervise import ActorNursery
ProcessType = TypeVar('ProcessType', mp.Process, trio.Process)
@ -331,21 +328,20 @@ async def soft_kill(
see `.hard_kill()`).
'''
chan: Channel = portal.channel
peer_aid: msgtypes.Aid = chan.aid
peer_aid: Aid = portal.channel.aid
try:
log.cancel(
f'Soft killing sub-actor via portal request\n'
f'\n'
f'c)=> {peer_aid.reprol()}@[{chan.maddr}]\n'
f' |_{proc}\n'
f'(c=> {peer_aid}\n'
f' |_{proc}\n'
)
# wait on sub-proc to signal termination
await wait_func(proc)
except trio.Cancelled:
with trio.CancelScope(shield=True):
await debug.maybe_wait_for_debugger(
await maybe_wait_for_debugger(
child_in_debug=_runtime_vars.get(
'_debug_mode', False
),
@ -469,7 +465,7 @@ async def trio_proc(
"--uid",
# TODO, how to pass this over "wire" encodings like
# cmdline args?
# -[ ] maybe we can add an `msgtypes.Aid.min_tuple()` ?
# -[ ] maybe we can add an `Aid.min_tuple()` ?
str(subactor.uid),
# Address the child must connect to on startup
"--parent_addr",
@ -487,14 +483,13 @@ async def trio_proc(
cancelled_during_spawn: bool = False
proc: trio.Process|None = None
ipc_server: _server.Server = actor_nursery._actor.ipc_server
ipc_server: IPCServer = actor_nursery._actor.ipc_server
try:
try:
proc: trio.Process = await trio.lowlevel.open_process(spawn_cmd, **proc_kwargs)
log.runtime(
f'Started new child subproc\n'
f'(>\n'
f' |_{proc}\n'
'Started new child\n'
f'|_{proc}\n'
)
# wait for actor to spawn and connect back to us
@ -512,10 +507,10 @@ async def trio_proc(
with trio.CancelScope(shield=True):
# don't clobber an ongoing pdb
if is_root_process():
await debug.maybe_wait_for_debugger()
await maybe_wait_for_debugger()
elif proc is not None:
async with debug.acquire_debug_lock(subactor.uid):
async with acquire_debug_lock(subactor.uid):
# soft wait on the proc to terminate
with trio.move_on_after(0.5):
await proc.wait()
@ -533,19 +528,14 @@ async def trio_proc(
# send a "spawning specification" which configures the
# initial runtime state of the child.
sspec = msgtypes.SpawnSpec(
sspec = SpawnSpec(
_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,
)
log.runtime(
f'Sending spawn spec to child\n'
f'{{}}=> {chan.aid.reprol()!r}\n'
f'\n'
f'{pretty_struct.pformat(sspec)}\n'
)
log.runtime(f'Sending spawn spec: {str(sspec)}')
await chan.send(sspec)
# track subactor in current nursery
@ -573,7 +563,7 @@ async def trio_proc(
# condition.
await soft_kill(
proc,
trio.Process.wait, # XXX, uses `pidfd_open()` below.
trio.Process.wait,
portal
)
@ -581,7 +571,8 @@ async def trio_proc(
# tandem if not done already
log.cancel(
'Cancelling portal result reaper task\n'
f'c)> {subactor.aid.reprol()!r}\n'
f'>c)\n'
f' |_{subactor.uid}\n'
)
nursery.cancel_scope.cancel()
@ -590,24 +581,21 @@ async def trio_proc(
# allowed! Do this **after** cancellation/teardown to avoid
# killing the process too early.
if proc:
reap_repr: str = _pformat.nest_from_op(
input_op='>x)',
text=subactor.pformat(),
)
log.cancel(
f'Hard reap sequence starting for subactor\n'
f'{reap_repr}'
f'>x)\n'
f' |_{subactor}@{subactor.uid}\n'
)
with trio.CancelScope(shield=True):
# don't clobber an ongoing pdb
if cancelled_during_spawn:
# Try again to avoid TTY clobbering.
async with debug.acquire_debug_lock(subactor.uid):
async with acquire_debug_lock(subactor.uid):
with trio.move_on_after(0.5):
await proc.wait()
await debug.maybe_wait_for_debugger(
await maybe_wait_for_debugger(
child_in_debug=_runtime_vars.get(
'_debug_mode', False
),
@ -636,7 +624,7 @@ async def trio_proc(
# acquire the lock and get notified of who has it,
# check that uid against our known children?
# this_uid: tuple[str, str] = current_actor().uid
# await debug.acquire_debug_lock(this_uid)
# await acquire_debug_lock(this_uid)
if proc.poll() is None:
log.cancel(f"Attempting to hard kill {proc}")
@ -739,7 +727,7 @@ async def mp_proc(
log.runtime(f"Started {proc}")
ipc_server: _server.Server = actor_nursery._actor.ipc_server
ipc_server: IPCServer = actor_nursery._actor.ipc_server
try:
# wait for actor to spawn and connect back to us
# channel should have handshake completed by the

View File

@ -21,6 +21,7 @@
from contextlib import asynccontextmanager as acm
from functools import partial
import inspect
from pprint import pformat
from typing import (
TYPE_CHECKING,
)
@ -30,10 +31,7 @@ import warnings
import trio
from .devx import (
debug,
pformat as _pformat,
)
from .devx.debug import maybe_wait_for_debugger
from ._addr import (
UnwrappedAddress,
mk_uuid,
@ -44,7 +42,6 @@ from ._runtime import Actor
from ._portal import Portal
from .trionics import (
is_multi_cancelled,
collapse_eg,
)
from ._exceptions import (
ContextCancelled,
@ -117,6 +114,7 @@ class ActorNursery:
]
] = {}
self.cancelled: bool = False
self._join_procs = trio.Event()
self._at_least_one_child_in_debug: bool = False
self.errors = errors
@ -134,53 +132,10 @@ class ActorNursery:
# TODO: remove the `.run_in_actor()` API and thus this 2ndary
# nursery when that API get's moved outside this primitive!
self._ria_nursery = ria_nursery
# TODO, factor this into a .hilevel api!
#
# portals spawned with ``run_in_actor()`` are
# cancelled when their "main" result arrives
self._cancel_after_result_on_exit: set = set()
# trio.Nursery-like cancel (request) statuses
self._cancelled_caught: bool = False
self._cancel_called: bool = False
@property
def cancel_called(self) -> bool:
'''
Records whether cancellation has been requested for this
actor-nursery by a call to `.cancel()` either due to,
- an explicit call by some actor-local-task,
- an implicit call due to an error/cancel emited inside
the `tractor.open_nursery()` block.
'''
return self._cancel_called
@property
def cancelled_caught(self) -> bool:
'''
Set when this nursery was able to cance all spawned subactors
gracefully via an (implicit) call to `.cancel()`.
'''
return self._cancelled_caught
# TODO! remove internal/test-suite usage!
@property
def cancelled(self) -> bool:
warnings.warn(
"`ActorNursery.cancelled` is now deprecated, use "
" `.cancel_called` instead.",
DeprecationWarning,
stacklevel=2,
)
return (
self._cancel_called
# and
# self._cancelled_caught
)
async def start_actor(
self,
name: str,
@ -244,7 +199,7 @@ class ActorNursery:
loglevel=loglevel,
# verbatim relay this actor's registrar addresses
registry_addrs=current_actor().registry_addrs,
registry_addrs=current_actor().reg_addrs,
)
parent_addr: UnwrappedAddress = self._actor.accept_addr
assert parent_addr
@ -358,7 +313,7 @@ class ActorNursery:
'''
__runtimeframe__: int = 1 # noqa
self._cancel_called = True
self.cancelled = True
# TODO: impl a repr for spawn more compact
# then `._children`..
@ -369,10 +324,9 @@ class ActorNursery:
server: IPCServer = self._actor.ipc_server
with trio.move_on_after(3) as cs:
async with (
collapse_eg(),
trio.open_nursery() as tn,
):
async with trio.open_nursery(
strict_exception_groups=False,
) as tn:
subactor: Actor
proc: trio.Process
@ -436,8 +390,6 @@ class ActorNursery:
) in children.values():
log.warning(f"Hard killing process {proc}")
proc.terminate()
else:
self._cancelled_caught
# mark ourselves as having (tried to have) cancelled all subactors
self._join_procs.set()
@ -467,10 +419,10 @@ async def _open_and_supervise_one_cancels_all_nursery(
# `ActorNursery.start_actor()`).
# errors from this daemon actor nursery bubble up to caller
async with (
collapse_eg(),
trio.open_nursery() as da_nursery,
):
async with trio.open_nursery(
strict_exception_groups=False,
# ^XXX^ TODO? instead unpack any RAE as per "loose" style?
) as da_nursery:
try:
# This is the inner level "run in actor" nursery. It is
# awaited first since actors spawned in this way (using
@ -480,10 +432,11 @@ async def _open_and_supervise_one_cancels_all_nursery(
# immediately raised for handling by a supervisor strategy.
# As such if the strategy propagates any error(s) upwards
# the above "daemon actor" nursery will be notified.
async with (
collapse_eg(),
trio.open_nursery() as ria_nursery,
):
async with trio.open_nursery(
strict_exception_groups=False,
# ^XXX^ TODO? instead unpack any RAE as per "loose" style?
) as ria_nursery:
an = ActorNursery(
actor,
ria_nursery,
@ -500,7 +453,7 @@ async def _open_and_supervise_one_cancels_all_nursery(
# the "hard join phase".
log.runtime(
'Waiting on subactors to complete:\n'
f'>}} {len(an._children)}\n'
f'{pformat(an._children)}\n'
)
an._join_procs.set()
@ -514,7 +467,7 @@ async def _open_and_supervise_one_cancels_all_nursery(
# will make the pdb repl unusable.
# Instead try to wait for pdb to be released before
# tearing down.
await debug.maybe_wait_for_debugger(
await maybe_wait_for_debugger(
child_in_debug=an._at_least_one_child_in_debug
)
@ -590,7 +543,7 @@ async def _open_and_supervise_one_cancels_all_nursery(
# XXX: yet another guard before allowing the cancel
# sequence in case a (single) child is in debug.
await debug.maybe_wait_for_debugger(
await maybe_wait_for_debugger(
child_in_debug=an._at_least_one_child_in_debug
)
@ -639,14 +592,9 @@ async def _open_and_supervise_one_cancels_all_nursery(
# final exit
_shutdown_msg: str = (
'Actor-runtime-shutdown'
)
# @api_frame
@acm
# @api_frame
async def open_nursery(
*, # named params only!
hide_tb: bool = True,
**kwargs,
# ^TODO, paramspec for `open_root_actor()`
@ -731,26 +679,17 @@ async def open_nursery(
):
__tracebackhide__: bool = False
op_nested_an_repr: str = _pformat.nest_from_op(
input_op=')>',
text=f'{an}',
# nest_prefix='|_',
nest_indent=1, # under >
msg: str = (
'Actor-nursery exited\n'
f'|_{an}\n'
)
an_msg: str = (
f'Actor-nursery exited\n'
f'{op_nested_an_repr}\n'
)
# keep noise low during std operation.
log.runtime(an_msg)
if implicit_runtime:
# shutdown runtime if it was started and report noisly
# that we're did so.
msg: str = (
'\n'
'\n'
f'{_shutdown_msg} )>\n'
)
msg += '=> Shutting down actor runtime <=\n'
log.info(msg)
else:
# keep noise low during std operation.
log.runtime(msg)

View File

@ -101,27 +101,11 @@ class Channel:
# ^XXX! ONLY set if a remote actor sends an `Error`-msg
self._closed: bool = False
# flag set by `Portal.cancel_actor()` indicating remote
# (possibly peer) cancellation of the far end actor runtime.
# flag set by ``Portal.cancel_actor()`` indicating remote
# (possibly peer) cancellation of the far end actor
# runtime.
self._cancel_called: bool = False
@property
def closed(self) -> bool:
'''
Was `.aclose()` successfully called?
'''
return self._closed
@property
def cancel_called(self) -> bool:
'''
Set when `Portal.cancel_actor()` is called on a portal which
wraps this IPC channel.
'''
return self._cancel_called
@property
def uid(self) -> tuple[str, str]:
'''
@ -185,27 +169,13 @@ class Channel:
addr,
**kwargs,
)
# XXX, for UDS *no!* since we recv the peer-pid and build out
# a new addr..
# assert transport.raddr == addr
assert transport.raddr == addr
chan = Channel(transport=transport)
# ?TODO, compact this into adapter level-methods?
# -[ ] would avoid extra repr-calcs if level not active?
# |_ how would the `calc_if_level` look though? func?
if log.at_least_level('runtime'):
from tractor.devx import (
pformat as _pformat,
)
chan_repr: str = _pformat.nest_from_op(
input_op='[>',
text=chan.pformat(),
nest_indent=1,
)
log.runtime(
f'Connected channel IPC transport\n'
f'{chan_repr}'
)
log.runtime(
f'Connected channel IPC transport\n'
f'[>\n'
f' |_{chan}\n'
)
return chan
@cm
@ -226,12 +196,9 @@ class Channel:
self._transport.codec = orig
# TODO: do a .src/.dst: str for maddrs?
def pformat(
self,
privates: bool = False,
) -> str:
def pformat(self) -> str:
if not self._transport:
return '<Channel( with inactive transport? )>'
return '<Channel with inactive transport?>'
tpt: MsgTransport = self._transport
tpt_name: str = type(tpt).__name__
@ -239,35 +206,26 @@ class Channel:
'connected' if self.connected()
else 'closed'
)
repr_str: str = (
return (
f'<Channel(\n'
f' |_status: {tpt_status!r}\n'
) + (
f' _closed={self._closed}\n'
f' _cancel_called={self._cancel_called}\n'
if privates else ''
) + ( # peer-actor (processs) section
f' |_peer: {self.aid.reprol()!r}\n'
if self.aid else ' |_peer: <unknown>\n'
) + (
f'\n'
f' |_peer: {self.aid}\n'
f'\n'
f' |_msgstream: {tpt_name}\n'
f' maddr: {tpt.maddr!r}\n'
f' proto: {tpt.laddr.proto_key!r}\n'
f' layer: {tpt.layer_key!r}\n'
f' codec: {tpt.codec_key!r}\n'
f' .laddr={tpt.laddr}\n'
f' .raddr={tpt.raddr}\n'
) + (
f' ._transport.stream={tpt.stream}\n'
f' ._transport.drained={tpt.drained}\n'
if privates else ''
) + (
f' proto={tpt.laddr.proto_key!r}\n'
f' layer={tpt.layer_key!r}\n'
f' laddr={tpt.laddr}\n'
f' raddr={tpt.raddr}\n'
f' codec={tpt.codec_key!r}\n'
f' stream={tpt.stream}\n'
f' maddr={tpt.maddr!r}\n'
f' drained={tpt.drained}\n'
f' _send_lock={tpt._send_lock.statistics()}\n'
if privates else ''
) + (
')>\n'
f')>\n'
)
return repr_str
# NOTE: making this return a value that can be passed to
# `eval()` is entirely **optional** FYI!
@ -289,10 +247,6 @@ class Channel:
def raddr(self) -> Address|None:
return self._transport.raddr if self._transport else None
@property
def maddr(self) -> str:
return self._transport.maddr if self._transport else '<no-tpt>'
# TODO: something like,
# `pdbp.hideframe_on(errors=[MsgTypeError])`
# instead of the `try/except` hack we have rn..
@ -303,7 +257,7 @@ class Channel:
self,
payload: Any,
hide_tb: bool = False,
hide_tb: bool = True,
) -> None:
'''
@ -480,8 +434,8 @@ class Channel:
await self.send(aid)
peer_aid: Aid = await self.recv()
log.runtime(
f'Received hanshake with peer\n'
f'<= {peer_aid.reprol(sin_uuid=False)}\n'
f'Received hanshake with peer actor,\n'
f'{peer_aid}\n'
)
# NOTE, we always are referencing the remote peer!
self.aid = peer_aid

View File

@ -17,16 +17,9 @@
Utils to tame mp non-SC madeness
'''
# !TODO! in 3.13 this can be disabled (the-same/similarly) using
# a flag,
# - [ ] soo if it works like this, drop this module entirely for
# 3.13+ B)
# |_https://docs.python.org/3/library/multiprocessing.shared_memory.html
#
def disable_mantracker():
'''
Disable all `multiprocessing` "resource tracking" machinery since
Disable all ``multiprocessing``` "resource tracking" machinery since
it's an absolute multi-threaded mess of non-SC madness.
'''

View File

@ -26,7 +26,7 @@ from contextlib import (
from functools import partial
from itertools import chain
import inspect
import textwrap
from pprint import pformat
from types import (
ModuleType,
)
@ -43,10 +43,7 @@ from trio import (
SocketListener,
)
from ..devx.pformat import (
ppfmt,
nest_from_op,
)
# from ..devx import debug
from .._exceptions import (
TransportClosed,
)
@ -144,8 +141,9 @@ async def maybe_wait_on_canced_subs(
):
log.cancel(
'Waiting on cancel request to peer\n'
f'c)=> {chan.aid.reprol()}@[{chan.maddr}]\n'
'Waiting on cancel request to peer..\n'
f'c)=>\n'
f' |_{chan.aid}\n'
)
# XXX: this is a soft wait on the channel (and its
@ -181,7 +179,7 @@ async def maybe_wait_on_canced_subs(
log.warning(
'Draining msg from disconnected peer\n'
f'{chan_info}'
f'{ppfmt(msg)}\n'
f'{pformat(msg)}\n'
)
# cid: str|None = msg.get('cid')
cid: str|None = msg.cid
@ -250,7 +248,7 @@ async def maybe_wait_on_canced_subs(
if children := local_nursery._children:
# indent from above local-nurse repr
report += (
f' |_{ppfmt(children)}\n'
f' |_{pformat(children)}\n'
)
log.warning(report)
@ -281,9 +279,8 @@ async def maybe_wait_on_canced_subs(
log.runtime(
f'Peer IPC broke but subproc is alive?\n\n'
f'<=x {chan.aid.reprol()}@[{chan.maddr}]\n'
f'\n'
f'{proc}\n'
f'<=x {chan.aid}@{chan.raddr}\n'
f' |_{proc}\n'
)
return local_nursery
@ -327,10 +324,9 @@ async def handle_stream_from_peer(
chan = Channel.from_stream(stream)
con_status: str = (
f'New inbound IPC transport connection\n'
f'<=( {stream!r}\n'
'New inbound IPC connection <=\n'
f'|_{chan}\n'
)
con_status_steps: str = ''
# initial handshake with peer phase
try:
@ -376,7 +372,7 @@ async def handle_stream_from_peer(
if _pre_chan := server._peers.get(uid):
familiar: str = 'pre-existing-peer'
uid_short: str = f'{uid[0]}[{uid[1][-6:]}]'
con_status_steps += (
con_status += (
f' -> Handshake with {familiar} `{uid_short}` complete\n'
)
@ -401,7 +397,7 @@ async def handle_stream_from_peer(
None,
)
if event:
con_status_steps += (
con_status += (
' -> Waking subactor spawn waiters: '
f'{event.statistics().tasks_waiting}\n'
f' -> Registered IPC chan for child actor {uid}@{chan.raddr}\n'
@ -412,7 +408,7 @@ async def handle_stream_from_peer(
event.set()
else:
con_status_steps += (
con_status += (
f' -> Registered IPC chan for peer actor {uid}@{chan.raddr}\n'
) # type: ignore
@ -426,15 +422,8 @@ async def handle_stream_from_peer(
# TODO: can we just use list-ref directly?
chans.append(chan)
con_status_steps += ' -> Entering RPC msg loop..\n'
log.runtime(
con_status
+
textwrap.indent(
con_status_steps,
prefix=' '*3, # align to first-ln
)
)
con_status += ' -> Entering RPC msg loop..\n'
log.runtime(con_status)
# Begin channel management - respond to remote requests and
# process received reponses.
@ -467,67 +456,41 @@ async def handle_stream_from_peer(
disconnected=disconnected,
)
# `Channel` teardown and closure sequence
# ``Channel`` teardown and closure sequence
# drop ref to channel so it can be gc-ed and disconnected
#
# -[x]TODO mk this be like
# <=x Channel(
# |_field: blah
# )>
op_repr: str = '<=x '
chan_repr: str = nest_from_op(
input_op=op_repr,
op_suffix='',
nest_prefix='',
text=chan.pformat(),
nest_indent=len(op_repr)-1,
rm_from_first_ln='<',
)
con_teardown_status: str = (
f'IPC channel disconnect\n'
f'\n'
f'{chan_repr}\n'
f'\n'
f'IPC channel disconnected:\n'
f'<=x uid: {chan.aid}\n'
f' |_{pformat(chan)}\n\n'
)
chans.remove(chan)
# TODO: do we need to be this pedantic?
if not chans:
con_teardown_status += (
f'-> No more channels with {chan.aid.reprol()!r}\n'
f'-> No more channels with {chan.aid}'
)
server._peers.pop(uid, None)
if peers := list(server._peers.values()):
peer_cnt: int = len(peers)
if (
(first := peers[0][0]) is not chan
and
not disconnected
and
peer_cnt > 1
):
con_teardown_status += (
f'-> Remaining IPC {peer_cnt-1!r} peers:\n'
peers_str: str = ''
for uid, chans in server._peers.items():
peers_str += (
f'uid: {uid}\n'
)
for i, chan in enumerate(chans):
peers_str += (
f' |_[{i}] {pformat(chan)}\n'
)
for chans in server._peers.values():
first: Channel = chans[0]
if not (
first is chan
and
disconnected
):
con_teardown_status += (
f' |_{first.aid.reprol()!r} -> {len(chans)!r} chans\n'
)
con_teardown_status += (
f'-> Remaining IPC {len(server._peers)} peers: {peers_str}\n'
)
# No more channels to other actors (at all) registered
# as connected.
if not server._peers:
con_teardown_status += (
'-> Signalling no more peer connections!\n'
'Signalling no more peer channel connections'
)
server._no_more_peers.set()
@ -616,10 +579,10 @@ async def handle_stream_from_peer(
class Endpoint(Struct):
'''
An instance of an IPC "bound" address where the lifetime of an
"ability to accept connections" and handle the subsequent
sequence-of-packets (maybe oriented as sessions) is determined by
the underlying nursery scope(s).
An instance of an IPC "bound" address where the lifetime of the
"ability to accept connections" (from clients) and then handle
those inbound sessions or sequences-of-packets is determined by
a (maybe pair of) nurser(y/ies).
'''
addr: Address
@ -637,24 +600,6 @@ class Endpoint(Struct):
MsgTransport, # handle to encoded-msg transport stream
] = {}
def pformat(
self,
indent: int = 0,
privates: bool = False,
) -> str:
type_repr: str = type(self).__name__
fmtstr: str = (
# !TODO, always be ns aware!
# f'|_netns: {netns}\n'
f' |.addr: {self.addr!r}\n'
f' |_peers: {len(self.peer_tpts)}\n'
)
return (
f'<{type_repr}(\n'
f'{fmtstr}'
f')>'
)
async def start_listener(self) -> SocketListener:
tpt_mod: ModuleType = inspect.getmodule(self.addr)
lstnr: SocketListener = await tpt_mod.start_listener(
@ -694,13 +639,11 @@ class Endpoint(Struct):
class Server(Struct):
_parent_tn: Nursery
_stream_handler_tn: Nursery
# level-triggered sig for whether "no peers are currently
# connected"; field is **always** set to an instance but
# initialized with `.is_set() == True`.
_no_more_peers: trio.Event
# active eps as allocated by `.listen_on()`
_endpoints: list[Endpoint] = []
# connection tracking & mgmt
@ -708,19 +651,12 @@ class Server(Struct):
str, # uaid
list[Channel], # IPC conns from peer
] = defaultdict(list)
# events-table with entries registered unset while the local
# actor is waiting on a new actor to inbound connect, often
# a parent waiting on its child just after spawn.
_peer_connected: dict[
tuple[str, str],
trio.Event,
] = {}
# syncs for setup/teardown sequences
# - null when not yet booted,
# - unset when active,
# - set when fully shutdown with 0 eps active.
_shutdown: trio.Event|None = None
# TODO, maybe just make `._endpoints: list[Endpoint]` and
@ -728,6 +664,7 @@ class Server(Struct):
# @property
# def addrs2eps(self) -> dict[Address, Endpoint]:
# ...
@property
def proto_keys(self) -> list[str]:
return [
@ -753,7 +690,7 @@ class Server(Struct):
# TODO: obvi a different server type when we eventually
# support some others XD
log.runtime(
f'Cancelling server(s) for tpt-protos\n'
f'Cancelling server(s) for\n'
f'{self.proto_keys!r}\n'
)
self._parent_tn.cancel_scope.cancel()
@ -780,14 +717,6 @@ class Server(Struct):
f'protos: {tpt_protos!r}\n'
)
def len_peers(
self,
) -> int:
return len([
chan.connected()
for chan in chain(*self._peers.values())
])
def has_peers(
self,
check_chans: bool = False,
@ -801,11 +730,13 @@ class Server(Struct):
has_peers
and
check_chans
and
(peer_cnt := self.len_peers())
):
has_peers: bool = (
peer_cnt > 0
any(chan.connected()
for chan in chain(
*self._peers.values()
)
)
and
has_peers
)
@ -814,14 +745,10 @@ class Server(Struct):
async def wait_for_no_more_peers(
self,
# XXX, should this even be allowed?
# -> i've seen it cause hangs on teardown
# in `test_resource_cache.py`
# _shield: bool = False,
shield: bool = False,
) -> None:
await self._no_more_peers.wait()
# with trio.CancelScope(shield=_shield):
# await self._no_more_peers.wait()
with trio.CancelScope(shield=shield):
await self._no_more_peers.wait()
async def wait_for_peer(
self,
@ -876,66 +803,30 @@ class Server(Struct):
return ev.is_set()
@property
def repr_state(self) -> str:
'''
A `str`-status describing the current state of this
IPC server in terms of the current operating "phase".
'''
status = 'server is active'
if self.has_peers():
peer_cnt: int = self.len_peers()
status: str = (
f'{peer_cnt!r} peer chans'
)
else:
status: str = 'No peer chans'
if self.is_shutdown():
status: str = 'server-shutdown'
return status
def pformat(
self,
privates: bool = False,
) -> str:
def pformat(self) -> str:
eps: list[Endpoint] = self._endpoints
# state_repr: str = (
# f'{len(eps)!r} endpoints active'
# )
fmtstr = (
f' |_state: {self.repr_state!r}\n'
state_repr: str = (
f'{len(eps)!r} IPC-endpoints active'
)
if privates:
fmtstr += f' no_more_peers: {self.has_peers()}\n'
if self._shutdown is not None:
shutdown_stats: EventStatistics = self._shutdown.statistics()
fmtstr += (
f' task_waiting_on_shutdown: {shutdown_stats}\n'
)
if eps := self._endpoints:
addrs: list[tuple] = [
ep.addr for ep in eps
]
repr_eps: str = ppfmt(addrs)
fmtstr = (
f' |_state: {state_repr}\n'
f' no_more_peers: {self.has_peers()}\n'
)
if self._shutdown is not None:
shutdown_stats: EventStatistics = self._shutdown.statistics()
fmtstr += (
f' |_endpoints: {repr_eps}\n'
# ^TODO? how to indent closing ']'..
f' task_waiting_on_shutdown: {shutdown_stats}\n'
)
if peers := self._peers:
fmtstr += (
f' |_peers: {len(peers)} connected\n'
)
fmtstr += (
# TODO, use the `ppfmt()` helper from `modden`!
f' |_endpoints: {pformat(self._endpoints)}\n'
f' |_peers: {len(self._peers)} connected\n'
)
return (
f'<Server(\n'
f'<IPCServer(\n'
f'{fmtstr}'
f')>\n'
)
@ -994,8 +885,8 @@ class Server(Struct):
)
log.runtime(
f'Binding endpoints\n'
f'{ppfmt(accept_addrs)}\n'
f'Binding to endpoints for,\n'
f'{accept_addrs}\n'
)
eps: list[Endpoint] = await self._parent_tn.start(
partial(
@ -1005,19 +896,13 @@ class Server(Struct):
listen_addrs=accept_addrs,
)
)
self._endpoints.extend(eps)
serv_repr: str = nest_from_op(
input_op='(>',
text=self.pformat(),
nest_indent=1,
)
log.runtime(
f'Started IPC server\n'
f'{serv_repr}'
f'Started IPC endpoints\n'
f'{eps}\n'
)
# XXX, a little sanity on new ep allocations
self._endpoints.extend(eps)
# XXX, just a little bit of sanity
group_tn: Nursery|None = None
ep: Endpoint
for ep in eps:
@ -1071,13 +956,9 @@ async def _serve_ipc_eps(
stream_handler_tn=stream_handler_tn,
)
try:
ep_sclang: str = nest_from_op(
input_op='>[',
text=f'{ep.pformat()}',
)
log.runtime(
f'Starting new endpoint listener\n'
f'{ep_sclang}\n'
f'{ep}\n'
)
listener: trio.abc.Listener = await ep.start_listener()
assert listener is ep._listener
@ -1115,6 +996,17 @@ async def _serve_ipc_eps(
handler_nursery=stream_handler_tn
)
)
# TODO, wow make this message better! XD
log.runtime(
'Started server(s)\n'
+
'\n'.join([f'|_{addr}' for addr in listen_addrs])
)
log.runtime(
f'Started IPC endpoints\n'
f'{eps}\n'
)
task_status.started(
eps,
)
@ -1157,7 +1049,8 @@ async def open_ipc_server(
try:
yield ipc_server
log.runtime(
'Server-tn running until terminated\n'
f'Waiting on server to shutdown or be cancelled..\n'
f'{ipc_server}'
)
# TODO? when if ever would we want/need this?
# with trio.CancelScope(shield=True):

View File

@ -789,11 +789,6 @@ def open_shm_list(
readonly=readonly,
)
# TODO, factor into a @actor_fixture acm-API?
# -[ ] also `@maybe_actor_fixture()` which inludes
# the .current_actor() convenience check?
# |_ orr can that just be in the sin-maybe-version?
#
# "close" attached shm on actor teardown
try:
actor = tractor.current_actor()

View File

@ -160,9 +160,10 @@ async def start_listener(
Start a TCP socket listener on the given `TCPAddress`.
'''
log.runtime(
f'Trying socket bind\n'
f'>[ {addr}\n'
log.info(
f'Attempting to bind TCP socket\n'
f'>[\n'
f'|_{addr}\n'
)
# ?TODO, maybe we should just change the lower-level call this is
# using internall per-listener?
@ -177,10 +178,11 @@ async def start_listener(
assert len(listeners) == 1
listener = listeners[0]
host, port = listener.socket.getsockname()[:2]
bound_addr: TCPAddress = type(addr).from_addr((host, port))
log.info(
f'Listening on TCP socket\n'
f'[> {bound_addr}\n'
f'[>\n'
f' |_{addr}\n'
)
return listener

View File

@ -18,9 +18,6 @@ Unix Domain Socket implementation of tractor.ipc._transport.MsgTransport protoco
'''
from __future__ import annotations
from contextlib import (
contextmanager as cm,
)
from pathlib import Path
import os
from socket import (
@ -32,7 +29,6 @@ from socket import (
)
import struct
from typing import (
Type,
TYPE_CHECKING,
ClassVar,
)
@ -103,6 +99,8 @@ class UDSAddress(
self.filedir
or
self.def_bindspace
# or
# get_rt_dir()
)
@property
@ -207,35 +205,12 @@ class UDSAddress(
f']'
)
@cm
def _reraise_as_connerr(
src_excs: tuple[Type[Exception]],
addr: UDSAddress,
):
try:
yield
except src_excs as src_exc:
raise ConnectionError(
f'Bad UDS socket-filepath-as-address ??\n'
f'{addr}\n'
f' |_sockpath: {addr.sockpath}\n'
f'\n'
f'from src: {src_exc!r}\n'
) from src_exc
async def start_listener(
addr: UDSAddress,
**kwargs,
) -> SocketListener:
'''
Start listening for inbound connections via
a `trio.SocketListener` (task) which `socket.bind()`s on `addr`.
Note, if the `UDSAddress.bindspace: Path` directory dne it is
implicitly created.
'''
# sock = addr._sock = socket.socket(
sock = socket.socket(
socket.AF_UNIX,
socket.SOCK_STREAM
@ -246,25 +221,17 @@ async def start_listener(
f'|_{addr}\n'
)
# ?TODO? should we use the `actor.lifetime_stack`
# to rm on shutdown?
bindpath: Path = addr.sockpath
if not (bs := addr.bindspace).is_dir():
log.info(
'Creating bindspace dir in file-sys\n'
f'>{{\n'
f'|_{bs!r}\n'
)
bs.mkdir()
with _reraise_as_connerr(
src_excs=(
FileNotFoundError,
OSError,
),
addr=addr
):
try:
await sock.bind(str(bindpath))
except (
FileNotFoundError,
) as fdne:
raise ConnectionError(
f'Bad UDS socket-filepath-as-address ??\n'
f'{addr}\n'
f' |_sockpath: {addr.sockpath}\n'
) from fdne
sock.listen(1)
log.info(
@ -389,30 +356,27 @@ class MsgpackUDSStream(MsgpackTransport):
# `.setsockopt()` call tells the OS provide it; the client
# pid can then be read on server/listen() side via
# `get_peer_info()` above.
with _reraise_as_connerr(
src_excs=(
FileNotFoundError,
),
addr=addr
):
try:
stream = await open_unix_socket_w_passcred(
str(sockpath),
**kwargs
)
except (
FileNotFoundError,
) as fdne:
raise ConnectionError(
f'Bad UDS socket-filepath-as-address ??\n'
f'{addr}\n'
f' |_sockpath: {sockpath}\n'
) from fdne
tpt_stream = MsgpackUDSStream(
stream = MsgpackUDSStream(
stream,
prefix_size=prefix_size,
codec=codec
)
# XXX assign from new addrs after peer-PID extract!
(
tpt_stream._laddr,
tpt_stream._raddr,
) = cls.get_stream_addrs(stream)
return tpt_stream
stream._raddr = addr
return stream
@classmethod
def get_stream_addrs(

View File

@ -81,35 +81,10 @@ BOLD_PALETTE = {
}
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
# TODO: this isn't showing the correct '{filename}'
# as it did before..
class StackLevelAdapter(LoggerAdapter):
def at_least_level(
self,
level: str,
) -> bool:
return at_least_level(
log=self,
level=level,
)
def transport(
self,
msg: str,
@ -426,3 +401,19 @@ def get_loglevel() -> str:
# global module logger for tractor itself
log: StackLevelAdapter = 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

View File

@ -210,14 +210,12 @@ class PldRx(Struct):
match msg:
case Return()|Error():
log.runtime(
f'Rxed final-outcome msg\n'
f'\n'
f'Rxed final outcome msg\n'
f'{msg}\n'
)
case Stop():
log.runtime(
f'Rxed stream stopped msg\n'
f'\n'
f'{msg}\n'
)
if passthrough_non_pld_msgs:
@ -263,9 +261,8 @@ class PldRx(Struct):
if (
type(msg) is Return
):
log.runtime(
log.info(
f'Rxed final result msg\n'
f'\n'
f'{msg}\n'
)
return self.decode_pld(
@ -307,13 +304,10 @@ class PldRx(Struct):
try:
pld: PayloadT = self._pld_dec.decode(pld)
log.runtime(
f'Decoded payload for\n'
# f'\n'
'Decoded msg payload\n\n'
f'{msg}\n'
# ^TODO?, ideally just render with `,
# pld={decode}` in the `msg.pformat()`??
f'where, '
f'{type(msg).__name__}.pld={pld!r}\n'
f'where payload decoded as\n'
f'|_pld={pld!r}\n'
)
return pld
except TypeError as typerr:
@ -500,8 +494,7 @@ def limit_plds(
finally:
log.runtime(
f'Reverted to previous payload-decoder\n'
f'\n'
'Reverted to previous payload-decoder\n\n'
f'{orig_pldec}\n'
)
# sanity on orig settings
@ -636,8 +629,7 @@ async def drain_to_final_msg(
(local_cs := rent_n.cancel_scope).cancel_called
):
log.cancel(
f'RPC-ctx cancelled by local-parent scope during drain!\n'
f'\n'
'RPC-ctx cancelled by local-parent scope during drain!\n\n'
f'c}}>\n'
f' |_{rent_n}\n'
f' |_.cancel_scope = {local_cs}\n'
@ -671,8 +663,7 @@ async def drain_to_final_msg(
# final result arrived!
case Return():
log.runtime(
f'Context delivered final draining msg\n'
f'\n'
'Context delivered final draining msg:\n'
f'{pretty_struct.pformat(msg)}'
)
ctx._result: Any = pld
@ -706,14 +697,12 @@ async def drain_to_final_msg(
):
log.cancel(
'Cancelling `MsgStream` drain since '
f'{reason}\n'
f'\n'
f'{reason}\n\n'
f'<= {ctx.chan.uid}\n'
f' |_{ctx._nsf}()\n'
f'\n'
f' |_{ctx._nsf}()\n\n'
f'=> {ctx._task}\n'
f' |_{ctx._stream}\n'
f'\n'
f' |_{ctx._stream}\n\n'
f'{pretty_struct.pformat(msg)}\n'
)
break
@ -750,8 +739,7 @@ async def drain_to_final_msg(
case Stop():
pre_result_drained.append(msg)
log.runtime( # normal/expected shutdown transaction
f'Remote stream terminated due to "stop" msg\n'
f'\n'
'Remote stream terminated due to "stop" msg:\n\n'
f'{pretty_struct.pformat(msg)}\n'
)
continue
@ -826,8 +814,7 @@ async def drain_to_final_msg(
else:
log.cancel(
f'Skipping `MsgStream` drain since final outcome is set\n'
f'\n'
'Skipping `MsgStream` drain since final outcome is set\n\n'
f'{ctx.outcome}\n'
)

View File

@ -154,39 +154,6 @@ class Aid(
# should also include at least `.pid` (equiv to port for tcp)
# and/or host-part always?
@property
def uid(self) -> tuple[str, str]:
'''
Legacy actor "unique-id" pair format.
'''
return (
self.name,
self.uuid,
)
def reprol(
self,
sin_uuid: bool = True,
) -> str:
if not sin_uuid:
return (
f'{self.name}[{self.uuid[:6]}]@{self.pid!r}'
)
return (
f'{self.name}@{self.pid!r}'
)
# mk hashable via `.uuid`
def __hash__(self) -> int:
return hash(self.uuid)
def __eq__(self, other: Aid) -> bool:
return self.uuid == other.uuid
# use pretty fmt since often repr-ed for console/log
__repr__ = pretty_struct.Struct.__repr__
class SpawnSpec(
pretty_struct.Struct,

View File

@ -130,7 +130,6 @@ class LinkedTaskChannel(
_trio_task: trio.Task
_aio_task_complete: trio.Event
_closed_by_aio_task: bool = False
_suppress_graceful_exits: bool = True
_trio_err: BaseException|None = None
@ -209,15 +208,10 @@ class LinkedTaskChannel(
async def aclose(self) -> None:
await self._from_aio.aclose()
# ?TODO? async version of this?
def started_nowait(
def started(
self,
val: Any = None,
) -> None:
'''
Synchronize aio-sde with its trio-parent.
'''
self._aio_started_val = val
return self._to_trio.send_nowait(val)
@ -248,7 +242,6 @@ class LinkedTaskChannel(
# cycle on the trio side?
# await trio.lowlevel.checkpoint()
return await self._from_aio.receive()
except BaseException as err:
async with translate_aio_errors(
chan=self,
@ -326,7 +319,7 @@ def _run_asyncio_task(
qsize: int = 1,
provide_channels: bool = False,
suppress_graceful_exits: bool = True,
hide_tb: bool = True,
hide_tb: bool = False,
**kwargs,
) -> LinkedTaskChannel:
@ -354,6 +347,18 @@ def _run_asyncio_task(
# value otherwise it would just return ;P
assert qsize > 1
if provide_channels:
assert 'to_trio' in args
# allow target func to accept/stream results manually by name
if 'to_trio' in args:
kwargs['to_trio'] = to_trio
if 'from_trio' in args:
kwargs['from_trio'] = from_trio
coro = func(**kwargs)
trio_task: trio.Task = trio.lowlevel.current_task()
trio_cs = trio.CancelScope()
aio_task_complete = trio.Event()
@ -368,25 +373,6 @@ def _run_asyncio_task(
_suppress_graceful_exits=suppress_graceful_exits,
)
# allow target func to accept/stream results manually by name
if 'to_trio' in args:
kwargs['to_trio'] = to_trio
if 'from_trio' in args:
kwargs['from_trio'] = from_trio
if 'chan' in args:
kwargs['chan'] = chan
if provide_channels:
assert (
'to_trio' in args
or
'chan' in args
)
coro = func(**kwargs)
async def wait_on_coro_final_result(
to_trio: trio.MemorySendChannel,
coro: Awaitable,
@ -459,15 +445,9 @@ def _run_asyncio_task(
f'Task exited with final result: {result!r}\n'
)
# only close the aio (child) side which will relay
# a `trio.EndOfChannel` to the trio (parent) side.
#
# XXX NOTE, that trio-side MUST then in such cases
# check for a `chan._aio_err` and raise it!!
# only close the sender side which will relay
# a `trio.EndOfChannel` to the trio (consumer) side.
to_trio.close()
# specially mark the closure as due to the
# asyncio.Task terminating!
chan._closed_by_aio_task = True
aio_task_complete.set()
log.runtime(
@ -665,9 +645,8 @@ def _run_asyncio_task(
not trio_cs.cancel_called
):
log.cancel(
f'Cancelling trio-side due to aio-side src exc\n'
f'\n'
f'{curr_aio_err!r}\n'
f'Cancelling `trio` side due to aio-side src exc\n'
f'{curr_aio_err}\n'
f'\n'
f'(c>\n'
f' |_{trio_task}\n'
@ -779,7 +758,6 @@ async def translate_aio_errors(
aio_done_before_trio: bool = aio_task.done()
assert aio_task
trio_err: BaseException|None = None
eoc: trio.EndOfChannel|None = None
try:
yield # back to one of the cross-loop apis
except trio.Cancelled as taskc:
@ -811,50 +789,12 @@ async def translate_aio_errors(
# )
# raise
# XXX EoC is a special SIGNAL from the aio-side here!
# There are 2 cases to handle:
# 1. the "EoC passthrough" case.
# - the aio-task actually closed the channel "gracefully" and
# the trio-task should unwind any ongoing channel
# iteration/receiving,
# |_this exc-translator wraps calls to `LinkedTaskChannel.receive()`
# in which case we want to relay the actual "end-of-chan" for
# iteration purposes.
#
# 2. relaying the "asyncio.Task termination" case.
# - if the aio-task terminates, maybe with an error, AND the
# `open_channel_from()` API was used, it will always signal
# that termination.
# |_`wait_on_coro_final_result()` always calls
# `to_trio.close()` when `provide_channels=True` so we need to
# always check if there is an aio-side exc which needs to be
# relayed to the parent trio side!
# |_in this case the special `chan._closed_by_aio_task` is
# ALWAYS set.
#
except trio.EndOfChannel as _eoc:
eoc = _eoc
if (
chan._closed_by_aio_task
and
aio_err
):
log.cancel(
f'The asyncio-child task terminated due to error\n'
f'{aio_err!r}\n'
)
chan._trio_to_raise = aio_err
trio_err = chan._trio_err = eoc
#
# await tractor.pause(shield=True)
#
# ?TODO?, raise something like a,
# chan._trio_to_raise = AsyncioErrored()
# BUT, with the tb rewritten to reflect the underlying
# call stack?
else:
trio_err = chan._trio_err = eoc
# XXX always passthrough EoC since this translator is often
# called from `LinkedTaskChannel.receive()` which we want
# passthrough and further we have no special meaning for it in
# terms of relaying errors or signals from the aio side!
except trio.EndOfChannel as eoc:
trio_err = chan._trio_err = eoc
raise eoc
# NOTE ALSO SEE the matching note in the `cancel_trio()` asyncio
@ -1107,7 +1047,7 @@ async def translate_aio_errors(
#
if wait_on_aio_task:
await chan._aio_task_complete.wait()
log.debug(
log.info(
'asyncio-task is done and unblocked trio-side!\n'
)
@ -1124,17 +1064,11 @@ async def translate_aio_errors(
trio_to_raise: (
AsyncioCancelled|
AsyncioTaskExited|
Exception| # relayed from aio-task
None
) = chan._trio_to_raise
raise_from: Exception = (
trio_err if (aio_err is trio_to_raise)
else aio_err
)
if not suppress_graceful_exits:
raise trio_to_raise from raise_from
raise trio_to_raise from (aio_err or trio_err)
if trio_to_raise:
match (
@ -1167,7 +1101,7 @@ async def translate_aio_errors(
)
return
case _:
raise trio_to_raise from raise_from
raise trio_to_raise from (aio_err or trio_err)
# Check if the asyncio-side is the cause of the trio-side
# error.
@ -1233,6 +1167,7 @@ async def run_task(
@acm
async def open_channel_from(
target: Callable[..., Any],
suppress_graceful_exits: bool = True,
**target_kwargs,
@ -1266,6 +1201,7 @@ async def open_channel_from(
# deliver stream handle upward
yield first, chan
except trio.Cancelled as taskc:
# await tractor.pause(shield=True) # ya it worx ;)
if cs.cancel_called:
if isinstance(chan._trio_to_raise, AsyncioCancelled):
log.cancel(

View File

@ -31,7 +31,7 @@ from ._broadcast import (
)
from ._beg import (
collapse_eg as collapse_eg,
get_collapsed_eg as get_collapsed_eg,
maybe_collapse_eg as maybe_collapse_eg,
is_multi_cancelled as is_multi_cancelled,
)
from ._taskc import (

View File

@ -15,9 +15,8 @@
# along with this program. If not, see <https://www.gnu.org/licenses/>.
'''
`BaseExceptionGroup` utils and helpers pertaining to
first-class-`trio` from a "historical" perspective, like "loose
exception group" task-nurseries.
`BaseExceptionGroup` related utils and helpers pertaining to
first-class-`trio` from a historical perspective B)
'''
from contextlib import (
@ -25,84 +24,27 @@ from contextlib import (
)
from typing import (
Literal,
Type,
)
import trio
# from trio._core._concat_tb import (
# concat_tb,
# )
# XXX NOTE
# taken verbatim from `trio._core._run` except,
# - remove the NONSTRICT_EXCEPTIONGROUP_NOTE deprecation-note
# guard-check; we know we want an explicit collapse.
# - mask out tb rewriting in collapse case, i don't think it really
# matters?
#
def collapse_exception_group(
excgroup: BaseExceptionGroup[BaseException],
) -> BaseException:
"""Recursively collapse any single-exception groups into that single contained
exception.
"""
exceptions = list(excgroup.exceptions)
modified = False
for i, exc in enumerate(exceptions):
if isinstance(exc, BaseExceptionGroup):
new_exc = collapse_exception_group(exc)
if new_exc is not exc:
modified = True
exceptions[i] = new_exc
if (
len(exceptions) == 1
and isinstance(excgroup, BaseExceptionGroup)
# XXX trio's loose-setting condition..
# and NONSTRICT_EXCEPTIONGROUP_NOTE in getattr(excgroup, "__notes__", ())
):
# exceptions[0].__traceback__ = concat_tb(
# excgroup.__traceback__,
# exceptions[0].__traceback__,
# )
return exceptions[0]
elif modified:
return excgroup.derive(exceptions)
else:
return excgroup
def get_collapsed_eg(
def maybe_collapse_eg(
beg: BaseExceptionGroup,
bp: bool = False,
) -> BaseException|None:
) -> BaseException:
'''
If the input beg can collapse to a single sub-exception which is
itself **not** an eg, return it.
If the input beg can collapse to a single non-eg sub-exception,
return it instead.
'''
maybe_exc = collapse_exception_group(beg)
if maybe_exc is beg:
return None
return maybe_exc
if len(excs := beg.exceptions) == 1:
return excs[0]
return beg
@acm
async def collapse_eg(
hide_tb: bool = True,
# XXX, for ex. will always show begs containing single taskc
ignore: set[Type[BaseException]] = {
# trio.Cancelled,
},
add_notes: bool = True,
):
async def collapse_eg():
'''
If `BaseExceptionGroup` raised in the body scope is
"collapse-able" (in the same way that
@ -110,44 +52,15 @@ async def collapse_eg(
only raise the lone emedded non-eg in in place.
'''
__tracebackhide__: bool = hide_tb
try:
yield
except BaseExceptionGroup as _beg:
beg = _beg
except* BaseException as beg:
if (
(exc := get_collapsed_eg(beg))
and
type(exc) not in ignore
):
exc := maybe_collapse_eg(beg)
) is not beg:
raise exc
# TODO? report number of nested groups it was collapsed
# *from*?
if add_notes:
from_group_note: str = (
'( ^^^ this exc was collapsed from a group ^^^ )\n'
)
if (
from_group_note
not in
getattr(exc, "__notes__", ())
):
exc.add_note(from_group_note)
# raise exc
# ^^ this will leave the orig beg tb above with the
# "during the handling of <beg> the following.."
# So, instead do..
#
if cause := exc.__cause__:
raise exc from cause
else:
# suppress "during handling of <the beg>"
# output in tb/console.
raise exc from None
# keep original
raise # beg
raise beg
def is_multi_cancelled(

View File

@ -41,9 +41,6 @@ import trio
from tractor._state import current_actor
from tractor.log import get_logger
# from ._beg import collapse_eg
# from ._taskc import (
# maybe_raise_from_masking_exc,
# )
if TYPE_CHECKING:
@ -109,9 +106,6 @@ async def _enter_and_wait(
async def gather_contexts(
mngrs: Sequence[AsyncContextManager[T]],
# caller can provide their own scope
tn: trio.Nursery|None = None,
) -> AsyncGenerator[
tuple[
T | None,
@ -154,45 +148,39 @@ async def gather_contexts(
'`.trionics.gather_contexts()` input mngrs is empty?\n'
'\n'
'Did try to use inline generator syntax?\n'
'Check that list({mngrs}) works!\n'
# 'or sequence-type intead!\n'
# 'Use a non-lazy iterator or sequence-type intead!\n'
'Use a non-lazy iterator or sequence-type intead!\n'
)
try:
async with (
#
# ?TODO, does including these (eg-collapsing,
# taskc-unmasking) improve tb noise-reduction/legibility?
#
# collapse_eg(),
maybe_open_nursery(
nursery=tn,
) as tn,
# maybe_raise_from_masking_exc(),
):
for mngr in mngrs:
tn.start_soon(
_enter_and_wait,
mngr,
unwrapped,
all_entered,
parent_exit,
seed,
)
async with (
# collapse_eg(),
trio.open_nursery(
strict_exception_groups=False,
# ^XXX^ TODO? soo roll our own then ??
# -> since we kinda want the "if only one `.exception` then
# just raise that" interface?
) as tn,
):
for mngr in mngrs:
tn.start_soon(
_enter_and_wait,
mngr,
unwrapped,
all_entered,
parent_exit,
seed,
)
# deliver control to caller once all ctx-managers have
# started (yielded back to us).
await all_entered.wait()
# deliver control once all managers have started up
await all_entered.wait()
try:
yield tuple(unwrapped.values())
finally:
# XXX NOTE: this is ABSOLUTELY REQUIRED to avoid
# the following wacky bug:
# <tractorbugurlhere>
parent_exit.set()
finally:
# XXX NOTE: this is ABSOLUTELY REQUIRED to avoid
# the following wacky bug:
# <tractorbugurlhere>
parent_exit.set()
# Per actor task caching helpers.
# Further potential examples of interest:
@ -245,9 +233,6 @@ async def maybe_open_context(
kwargs: dict = {},
key: Hashable | Callable[..., Hashable] = None,
# caller can provide their own scope
tn: trio.Nursery|None = None,
) -> AsyncIterator[tuple[bool, T]]:
'''
Maybe open an async-context-manager (acm) if there is not already
@ -280,23 +265,7 @@ async def maybe_open_context(
# have it not be closed until all consumers have exited (which is
# currently difficult to implement any other way besides using our
# pre-allocated runtime instance..)
if tn:
# TODO, assert tn is eventual parent of this task!
task: trio.Task = trio.lowlevel.current_task()
task_tn: trio.Nursery = task.parent_nursery
if not tn._cancel_status.encloses(
task_tn._cancel_status
):
raise RuntimeError(
f'Mis-nesting of task under provided {tn} !?\n'
f'Current task is NOT a child(-ish)!!\n'
f'\n'
f'task: {task}\n'
f'task_tn: {task_tn}\n'
)
service_n = tn
else:
service_n: trio.Nursery = current_actor()._service_n
service_n: trio.Nursery = current_actor()._service_n
# TODO: is there any way to allocate
# a 'stays-open-till-last-task-finshed nursery?
@ -304,70 +273,32 @@ async def maybe_open_context(
# async with maybe_open_nursery(_Cache.service_n) as service_n:
# _Cache.service_n = service_n
cache_miss_ke: KeyError|None = None
maybe_taskc: trio.Cancelled|None = None
try:
# **critical section** that should prevent other tasks from
# checking the _Cache until complete otherwise the scheduler
# may switch and by accident we create more then one resource.
yielded = _Cache.values[ctx_key]
except KeyError as _ke:
# XXX, stay mutexed up to cache-miss yield
try:
cache_miss_ke = _ke
log.debug(
f'Allocating new @acm-func entry\n'
f'ctx_key={ctx_key}\n'
f'acm_func={acm_func}\n'
)
mngr = acm_func(**kwargs)
resources = _Cache.resources
assert not resources.get(ctx_key), f'Resource exists? {ctx_key}'
resources[ctx_key] = (service_n, trio.Event())
yielded: Any = await service_n.start(
_Cache.run_ctx,
mngr,
ctx_key,
)
_Cache.users += 1
finally:
# XXX, since this runs from an `except` it's a checkpoint
# whih can be `trio.Cancelled`-masked.
#
# NOTE, in that case the mutex is never released by the
# (first and) caching task and **we can't** simply shield
# bc that will inf-block on the `await
# no_more_users.wait()`.
#
# SO just always unlock!
lock.release()
except KeyError:
log.debug(f'Allocating new {acm_func} for {ctx_key}')
mngr = acm_func(**kwargs)
resources = _Cache.resources
assert not resources.get(ctx_key), f'Resource exists? {ctx_key}'
resources[ctx_key] = (service_n, trio.Event())
try:
yield (
False, # cache_hit = "no"
yielded,
)
except trio.Cancelled as taskc:
maybe_taskc = taskc
log.cancel(
f'Cancelled from cache-miss entry\n'
f'\n'
f'ctx_key: {ctx_key!r}\n'
f'mngr: {mngr!r}\n'
)
# XXX, always unset ke from cancelled context
# since we never consider it a masked exc case!
# - bc this can be called directly ty `._rpc._invoke()`?
#
if maybe_taskc.__context__ is cache_miss_ke:
maybe_taskc.__context__ = None
raise taskc
# sync up to the mngr's yielded value
yielded = await service_n.start(
_Cache.run_ctx,
mngr,
ctx_key,
)
_Cache.users += 1
lock.release()
yield False, yielded
else:
_Cache.users += 1
log.debug(
log.runtime(
f'Re-using cached resource for user {_Cache.users}\n\n'
f'{ctx_key!r} -> {type(yielded)}\n'
@ -377,19 +308,9 @@ async def maybe_open_context(
# f'{ctx_key!r} -> {yielded!r}\n'
)
lock.release()
yield (
True, # cache_hit = "yes"
yielded,
)
yield True, yielded
finally:
if lock.locked():
stats: trio.LockStatistics = lock.statistics()
log.error(
f'Lock left locked by last owner !?\n'
f'{stats}\n'
)
_Cache.users -= 1
if yielded is not None:

View File

@ -22,10 +22,7 @@ from __future__ import annotations
from contextlib import (
asynccontextmanager as acm,
)
from typing import (
Type,
TYPE_CHECKING,
)
from typing import TYPE_CHECKING
import trio
from tractor.log import get_logger
@ -68,6 +65,7 @@ def find_masked_excs(
#
@acm
async def maybe_raise_from_masking_exc(
tn: trio.Nursery|None = None,
unmask_from: (
BaseException|
tuple[BaseException]
@ -76,26 +74,15 @@ async def maybe_raise_from_masking_exc(
raise_unmasked: bool = True,
extra_note: str = (
'This can occurr when,\n'
'\n'
' - a `trio.Nursery/CancelScope` embeds a `finally/except:`-block '
'which execs an un-shielded checkpoint!'
' - a `trio.Nursery` scope embeds a `finally:`-block '
'which executes a checkpoint!'
#
# ^TODO? other cases?
),
always_warn_on: tuple[Type[BaseException]] = (
always_warn_on: tuple[BaseException] = (
trio.Cancelled,
),
# don't ever unmask or warn on any masking pair,
# {<masked-excT-key> -> <masking-excT-value>}
never_warn_on: dict[
Type[BaseException],
Type[BaseException],
] = {
KeyboardInterrupt: trio.Cancelled,
trio.Cancelled: trio.Cancelled,
},
# ^XXX, special case(s) where we warn-log bc likely
# there will be no operational diff since the exc
# is always expected to be consumed.
@ -117,91 +104,81 @@ async def maybe_raise_from_masking_exc(
individual sub-excs but maintain the eg-parent's form right?
'''
if not isinstance(unmask_from, tuple):
raise ValueError(
f'Invalid unmask_from = {unmask_from!r}\n'
f'Must be a `tuple[Type[BaseException]]`.\n'
)
from tractor.devx.debug import (
BoxedMaybeException,
pause,
)
boxed_maybe_exc = BoxedMaybeException(
raise_on_exit=raise_unmasked,
)
matching: list[BaseException]|None = None
try:
yield boxed_maybe_exc
return
except BaseException as _bexc:
bexc = _bexc
if isinstance(bexc, BaseExceptionGroup):
matches: ExceptionGroup
matches, _ = bexc.split(unmask_from)
if matches:
matching = matches.exceptions
maybe_eg: ExceptionGroup|None
elif (
unmask_from
and
type(bexc) in unmask_from
):
matching = [bexc]
if tn:
try: # handle egs
yield boxed_maybe_exc
return
except* unmask_from as _maybe_eg:
maybe_eg = _maybe_eg
matches: ExceptionGroup
matches, _ = maybe_eg.split(
unmask_from
)
if not matches:
raise
matching: list[BaseException] = matches.exceptions
else:
try: # handle non-egs
yield boxed_maybe_exc
return
except unmask_from as _maybe_exc:
maybe_exc = _maybe_exc
matching: list[BaseException] = [
maybe_exc
]
# XXX, only unmask-ed for debuggin!
# TODO, remove eventually..
except BaseException as _berr:
berr = _berr
await pause(shield=True)
raise berr
if matching is None:
raise
masked: list[tuple[BaseException, BaseException]] = []
for exc_match in matching:
if exc_ctx := find_masked_excs(
maybe_masker=exc_match,
unmask_from=set(unmask_from),
unmask_from={unmask_from},
):
masked.append((
exc_ctx,
exc_match,
))
masked.append((exc_ctx, exc_match))
boxed_maybe_exc.value = exc_match
note: str = (
f'\n'
f'^^WARNING^^\n'
f'the above {type(exc_ctx)!r} was masked by a {type(exc_match)!r}\n'
f'^^WARNING^^ the above {exc_ctx!r} was masked by a {unmask_from!r}\n'
)
if extra_note:
note += (
f'\n'
f'{extra_note}\n'
)
exc_ctx.add_note(note)
do_warn: bool = (
never_warn_on.get(
type(exc_ctx) # masking type
)
is not
type(exc_match) # masked type
)
if do_warn:
exc_ctx.add_note(note)
if (
do_warn
and
type(exc_match) in always_warn_on
):
if type(exc_match) in always_warn_on:
log.warning(note)
if (
do_warn
and
raise_unmasked
):
# await tractor.pause(shield=True)
if raise_unmasked:
if len(masked) < 2:
raise exc_ctx from exc_match
# ??TODO, see above but, possibly unmasking sub-exc
# entries if there are > 1
# else:
# await pause(shield=True)
else:
# ?TODO, see above but, possibly unmasking sub-exc
# entries if there are > 1
await pause(shield=True)
else:
raise