-
-
Notifications
You must be signed in to change notification settings - Fork 719
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Allow actor exceptions to propagate #4232
Changes from 15 commits
c845a6c
9081029
8c1628c
23269b8
96a2e02
a3d0cff
6eadd9e
877b562
0a079d1
8242454
776e408
a1a2620
1e6e929
95631d4
8c7d2ed
16fd6d7
28b6460
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -8,7 +8,13 @@ | |
|
||
from distributed import Actor, ActorFuture, Client, Future, Nanny, wait | ||
from distributed.metrics import time | ||
from distributed.utils_test import cluster, gen_cluster | ||
from distributed.utils_test import ( # noqa: F401 | ||
async_wait_for, | ||
cluster, | ||
cluster_fixture, | ||
gen_cluster, | ||
loop, | ||
) | ||
|
||
|
||
class Counter: | ||
|
@@ -465,7 +471,6 @@ def f(block, ps=None): | |
print(format_time(end - start)) | ||
|
||
|
||
@pytest.mark.flaky(reruns=10, reruns_delay=5) | ||
martindurant marked this conversation as resolved.
Show resolved
Hide resolved
|
||
@gen_cluster(client=True) | ||
async def test_compute(c, s, a, b): | ||
@dask.delayed | ||
|
@@ -485,10 +490,7 @@ def check(counter, blanks): | |
result = await c.compute(final, actors=counter) | ||
assert result == 0 + 1 + 2 + 3 + 4 | ||
|
||
start = time() | ||
while a.data or b.data: | ||
await asyncio.sleep(0.01) | ||
assert time() < start + 30 | ||
await async_wait_for(lambda: a.data or b.data, timeout=10) | ||
martindurant marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
|
||
def test_compute_sync(client): | ||
|
@@ -581,3 +583,48 @@ async def test_async_deadlock(client, s, a, b): | |
ac2 = await client.submit(UsesCounter, actor=True, workers=[ac._address]) | ||
|
||
assert (await ac2.ado_inc(ac)) == 1 | ||
|
||
|
||
def test_exception(): | ||
class MyException(Exception): | ||
pass | ||
|
||
class Broken: | ||
def method(self): | ||
raise MyException | ||
|
||
@property | ||
def prop(self): | ||
raise MyException | ||
|
||
with cluster(nworkers=2) as (cl, w): | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is there a reason to use There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It was to test the sync API, which is more typical for actors. I added an async version immediately below (could remove this one, if you like). There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
To be clear here, the sync api is more common for everything. We prefer the async tests because they are faster/easier on CI and allow for greater debuggability. Adding sync tests too is fine if we want to be extra careful, but in general we prefer async tests. In general if async tests I usually have confidence that sync works just as well, unless I'm explicitly writing code to handle synchronization. What's here is great though. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
I don't mind keeping it or not. Writing the sync version first probably reflects how I initially tested by hand. It was some time ago, so I can't remember if there was any other reason, given that the async version is effectively identical and works just fine. |
||
client = Client(cl["address"]) | ||
ac = client.submit(Broken, actor=True).result() | ||
acfut = ac.method() | ||
with pytest.raises(MyException): | ||
acfut.result() | ||
|
||
with pytest.raises(MyException): | ||
ac.prop | ||
|
||
|
||
@gen_cluster(client=True) | ||
async def test_exception_async(client, s, a, b): | ||
class MyException(Exception): | ||
pass | ||
|
||
class Broken: | ||
def method(self): | ||
raise MyException | ||
|
||
@property | ||
def prop(self): | ||
raise MyException | ||
|
||
ac = await client.submit(Broken, actor=True) | ||
acfut = ac.method() | ||
with pytest.raises(MyException): | ||
await acfut | ||
|
||
with pytest.raises(MyException): | ||
await ac.prop |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Following #4888, all pytest fixtures in
distributed.utils_test
are globally available. So there's no need for thecluster_fixture
orloop
imports (and hopefully we can drop# noqa: F401
too).