-
-
Notifications
You must be signed in to change notification settings - Fork 727
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
Published Future retrieved from async client is synchronous #3227
Comments
Hrm, interesting. What is the other client that it created? How does this
client come about?
…On Tue, Nov 12, 2019 at 4:12 AM crusaderky ***@***.***> wrote:
Possibly related to #2336
<#2336>.
I'm using an asynchronous client connected to a remote cluster.
A publish_dataset -> get_dataset roundtrip on a Future causes it to be
attached to a synchronous client.
client = await distributed.Client("localhost:8786", asynchronous=True)
a, = client.compute([delayed(1, pure=True)])print(a) # <Future: status: pending, key: int-62645d78d66e2508256b7ab60a38b944>print(a.result()) # <coroutine object Future._result at 0x11ad10dd0>print(await a) # 1
await client.publish_dataset(foo=a)
b = await client.get_dataset("foo")
print(b) # <Future: status: pending, key: int-62645d78d66e2508256b7ab60a38b944>print(client.asynchronous, b.client.asynchronous) # True, Falseprint(b.result()) # 1print(await b) # AttributeError: 'int' object has no attribute '__await__'
Switching to a LocalCluster weirdly makes the issue disappear.
Workarounds:
- Force b.client = client
- Use client.gather([b]) instead, which (involuntarily?) seems to
ignore b.client.
—
You are receiving this because you are subscribed to this thread.
Reply to this email directly, view it on GitHub
<#3227?email_source=notifications&email_token=AACKZTFS54JKEZD7A6Q7UG3QTKMSXA5CNFSM4JMCTMO2YY3PNVWWK3TUL52HS4DFUVEXG43VMWVGG33NNVSW45C7NFSM4HYVVGRA>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AACKZTAX2Q7TS2KIUHOPQZ3QTKMSXANCNFSM4JMCTMOQ>
.
|
Closing this as stale. @crusaderky if you have other information that would lead to resolving this please post an we'll reopen. |
@mrocklin I don't quite understand why this was closed - I wrote foolproof instructions on how to reproduce the issue. Sorry I don't have time right now to look into it, but I feel that marking it as stale after 2 weeks feels a bit excessive? |
Reopening is easy :) We can certainly do that if you'll be able to provide more information on the other client. |
@TomAugspurger , there is no missing information; anybody can reproduce the issue and start debugging it... |
Mmm, perhaps I misread. The "other client" must be the client associated
with the future, not another Client created by a user.
…On Thu, Dec 19, 2019 at 11:46 AM crusaderky ***@***.***> wrote:
@TomAugspurger <https://github.com/TomAugspurger> , there is no missing
information; anybody can reproduce the issue and start debugging it...
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#3227?email_source=notifications&email_token=AAKAOIW5RUYGMVBSBGLDA2LQZOXNVA5CNFSM4JMCTMO2YY3PNVWWK3TUL52HS4DFVREXG43VMVBW63LNMVXHJKTDN5WW2ZLOORPWSZGOEHKMELQ#issuecomment-567591470>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AAKAOIXY5XHHR5YXO67CAA3QZOXNVANCNFSM4JMCTMOQ>
.
|
Found it.
distributed/distributed/worker.py Lines 3106 to 3112 in 74f10aa
On line 3107,
which also explains why everything works with LocalCluster. Changing my POC code to use the IP instead of localhost makes in fact the problem disappear. Working on a fix now. On a more high level note, I feel the design of using a global variable in asyncio code will always be bug prone and it makes a great case for contextvars. I won't try introducing them as part of the bugfix though as it would cause severe scope creep. |
Sounds like definitely also causes #2336 |
distributed-2.7.0
Possibly related to #2336.
I'm using an asynchronous client connected to a remote cluster.
A publish_dataset -> get_dataset roundtrip on a Future causes it to be attached to a synchronous client.
Switching to a LocalCluster weirdly makes the issue disappear.
Workarounds:
b.client = client
client.gather([b])
instead, which (involuntarily?) seems to ignore b.client.The text was updated successfully, but these errors were encountered: