Skip to content
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

Closed
crusaderky opened this issue Nov 12, 2019 · 8 comments · Fixed by #3729
Closed

Published Future retrieved from async client is synchronous #3227

crusaderky opened this issue Nov 12, 2019 · 8 comments · Fixed by #3729

Comments

@crusaderky
Copy link
Collaborator

crusaderky commented Nov 12, 2019

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.

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, False
print(b.result())  # 1
print(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.
@crusaderky crusaderky changed the title published Future retrieved from async client is synchronous Published Future retrieved from async client is synchronous Nov 12, 2019
@mrocklin
Copy link
Member

mrocklin commented Nov 12, 2019 via email

@mrocklin
Copy link
Member

mrocklin commented Dec 7, 2019

Closing this as stale. @crusaderky if you have other information that would lead to resolving this please post an we'll reopen.

@mrocklin mrocklin closed this as completed Dec 7, 2019
@crusaderky
Copy link
Collaborator Author

@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?

@TomAugspurger
Copy link
Member

Reopening is easy :) We can certainly do that if you'll be able to provide more information on the other client.

@TomAugspurger TomAugspurger reopened this Dec 19, 2019
@crusaderky
Copy link
Collaborator Author

@TomAugspurger , there is no missing information; anybody can reproduce the issue and start debugging it...

@TomAugspurger
Copy link
Member

TomAugspurger commented Dec 19, 2019 via email

@crusaderky
Copy link
Collaborator Author

Found it.

Client.get_dataset causes a Client instance to be unpickled, which in turn invokes distributed.worker.get_client().

client = _get_global_client() # TODO: assumes the same scheduler
if client and (not address or client.scheduler.address == address):
return client
elif address:
from .client import Client
return Client(address, timeout=timeout)

On line 3107,

  • client.scheduler.address = 'tcp://localhost:8786'
  • address = 'tcp://127.0.0.1:8786'

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.

@martindurant
Copy link
Member

Sounds like definitely also causes #2336

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging a pull request may close this issue.

4 participants