Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
16 changes: 16 additions & 0 deletions distributed/tests/test_worker_client.py
Original file line number Diff line number Diff line change
Expand Up @@ -297,3 +297,19 @@ def f():

result = yield c.submit(f)
assert result


@gen_cluster()
def test_submit_different_names(s, a, b):
# https://github.com/dask/distributed/issues/2058
da = pytest.importorskip('dask.array')
c = yield Client('localhost:' + s.address.split(":")[-1], loop=s.loop,
asynchronous=True)
try:
X = c.persist(da.random.uniform(size=(100, 10), chunks=50))
yield wait(X)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fine for now, but in the future I recommend foo.persist() over c.persist(foo)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just for aesthetic reasons really


fut = yield c.submit(lambda x: x.sum().compute(), X)
assert fut > 0
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I recommend avoiding the use of the normal context manager in async tests. If you were to use async with c that would work well, but we can't use that within tests that have to be within python 2 friendly files. Instead I recommend using try/finally.

Also, if you're not going to use the client created in the gen_cluster decorator then you probably want the following:

@gen_cluster()
def test_foo(s, a, b):
    c = yield Client(...)

    try:
        ...
    finally:
        yield c.close()

There are also some examples of this in test_client.py

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK, thanks. I was just using the context manager so that it was cleaned up.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Understood, but the context manager is synchronous API. There is no way for us to yield c.close() within the normal context mananger. I think that it just starts the close coroutine, but doesn't wait for it. You'll either need to use async with or explicitly call c.close() to be sure that it closes up.

finally:
yield c.close()
22 changes: 19 additions & 3 deletions distributed/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
from tornado.ioloop import IOLoop
from tornado.locks import Event

from . import profile
from . import profile, comm
from .batched import BatchedSend
from .comm import get_address_host, get_local_address_for, connect
from .comm.utils import offload
Expand Down Expand Up @@ -2596,11 +2596,25 @@ def get_worker():
raise ValueError("No workers found")


def get_client(address=None, timeout=3):
""" Get a client while within a task
def get_client(address=None, timeout=3, resolve_address=True):
"""Get a client while within a task.

This client connects to the same scheduler to which the worker is connected

Parameters
----------
address : str, optional
The address of the scheduler to connect to. Defaults to the scheduler
the worker is connected to.
timeout : int, default 3
Timeout (in seconds) for getting the Client
resolve_address : bool, default True
Whether to resolve `address` to its canonical form.

Returns
-------
Client

Examples
--------
>>> def f():
Expand All @@ -2619,6 +2633,8 @@ def get_client(address=None, timeout=3):
worker_client
secede
"""
if address and resolve_address:
address = comm.resolve_address(address)
try:
worker = get_worker()
except ValueError: # could not find worker
Expand Down