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

Update Scheduler.rebalance return value when data is missing #3670

Merged
merged 3 commits into from
Apr 4, 2020
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
26 changes: 15 additions & 11 deletions distributed/client.py
Original file line number Diff line number Diff line change
Expand Up @@ -587,7 +587,7 @@ def __init__(
deserializers=None,
extensions=DEFAULT_EXTENSIONS,
direct_to_workers=None,
**kwargs
**kwargs,
):
if timeout == no_default:
timeout = dask.config.get("distributed.comm.timeouts.connect")
Expand Down Expand Up @@ -960,7 +960,7 @@ async def _start(self, timeout=no_default, **kwargs):
self.cluster = await LocalCluster(
loop=self.loop,
asynchronous=self._asynchronous,
**self._startup_kwargs
**self._startup_kwargs,
)
except (OSError, socket.error) as e:
if e.errno != errno.EADDRINUSE:
Expand All @@ -970,7 +970,7 @@ async def _start(self, timeout=no_default, **kwargs):
scheduler_port=0,
loop=self.loop,
asynchronous=True,
**self._startup_kwargs
**self._startup_kwargs,
)

# Wait for all workers to be ready
Expand Down Expand Up @@ -1422,7 +1422,7 @@ def submit(
actor=False,
actors=False,
pure=None,
**kwargs
**kwargs,
):

""" Submit a function application to the scheduler
Expand Down Expand Up @@ -1542,7 +1542,7 @@ def map(
actor=False,
actors=False,
pure=None,
**kwargs
**kwargs,
):
""" Map a function on a sequence of arguments

Expand Down Expand Up @@ -2538,7 +2538,7 @@ def get(
priority=0,
fifo_timeout="60s",
actors=None,
**kwargs
**kwargs,
):
""" Compute dask graph

Expand Down Expand Up @@ -2669,7 +2669,7 @@ def compute(
fifo_timeout="60s",
actors=None,
traverse=True,
**kwargs
**kwargs,
):
""" Compute dask collections on cluster

Expand Down Expand Up @@ -2817,7 +2817,7 @@ def persist(
priority=0,
fifo_timeout="60s",
actors=None,
**kwargs
**kwargs,
):
""" Persist dask collections on cluster

Expand Down Expand Up @@ -3013,6 +3013,10 @@ async def _rebalance(self, futures=None, workers=None):
await _wait(futures)
keys = list({tokey(f.key) for f in self.futures_of(futures)})
result = await self.scheduler.rebalance(keys=keys, workers=workers)
if result["status"] == "missing-data":
raise ValueError(
f"During rebalance {len(result['keys'])} keys were found to be missing"
)
assert result["status"] == "OK"

def rebalance(self, futures=None, workers=None, **kwargs):
Expand All @@ -3023,7 +3027,7 @@ def rebalance(self, futures=None, workers=None, **kwargs):
depending on keyword arguments.

This operation is generally not well tested against normal operation of
the scheduler. It it not recommended to use it while waiting on
the scheduler. It is not recommended to use it while waiting on
computations.

Parameters
Expand Down Expand Up @@ -3085,7 +3089,7 @@ def replicate(self, futures, n=None, workers=None, branching_factor=2, **kwargs)
n=n,
workers=workers,
branching_factor=branching_factor,
**kwargs
**kwargs,
)

def nthreads(self, workers=None, **kwargs):
Expand Down Expand Up @@ -3505,7 +3509,7 @@ def retire_workers(self, workers=None, close_workers=True, **kwargs):
self.scheduler.retire_workers,
workers=workers,
close_workers=close_workers,
**kwargs
**kwargs,
)

def set_metadata(self, key, value):
Expand Down
3 changes: 2 additions & 1 deletion distributed/scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
second,
compose,
groupby,
concat,
)
from tornado.ioloop import IOLoop

Expand Down Expand Up @@ -3103,7 +3104,7 @@ async def rebalance(self, comm=None, keys=None, workers=None):
if not all(r["status"] == "OK" for r in result):
return {
"status": "missing-data",
"keys": sum([r["keys"] for r in result if "keys" in r], []),
"keys": tuple(concat(r["keys"].keys() for r in result)),
}

for sender, recipient, ts in msgs:
Expand Down
21 changes: 15 additions & 6 deletions distributed/tests/test_client.py
Original file line number Diff line number Diff line change
Expand Up @@ -2876,6 +2876,15 @@ def test_rebalance_unprepared(c, s, a, b):
s.validate_state()


@gen_cluster(client=True)
async def test_rebalance_raises_missing_data(c, s, a, b):
with pytest.raises(ValueError, match=f"keys were found to be missing"):
futures = await c.scatter(range(100))
keys = [f.key for f in futures]
del futures
await c.rebalance(keys)


@gen_cluster(client=True)
def test_receive_lost_key(c, s, a, b):
x = c.submit(inc, 1, workers=[a.address])
Expand Down Expand Up @@ -4864,8 +4873,8 @@ def test_bytes_keys(c, s, a, b):

@gen_cluster(client=True)
def test_unicode_ascii_keys(c, s, a, b):
uni_type = type(u"")
key = u"inc-123"
uni_type = type("")
key = "inc-123"
future = c.submit(inc, 1, key=key)
result = yield future
assert type(future.key) is uni_type
Expand All @@ -4876,8 +4885,8 @@ def test_unicode_ascii_keys(c, s, a, b):

@gen_cluster(client=True)
def test_unicode_keys(c, s, a, b):
uni_type = type(u"")
key = u"inc-123\u03bc"
uni_type = type("")
key = "inc-123\u03bc"
future = c.submit(inc, 1, key=key)
result = yield future
assert type(future.key) is uni_type
Expand All @@ -4889,8 +4898,8 @@ def test_unicode_keys(c, s, a, b):
result2 = yield future2
assert result2 == 3

future3 = yield c.scatter({u"data-123": 123})
result3 = yield future3[u"data-123"]
future3 = yield c.scatter({"data-123": 123})
result3 = yield future3["data-123"]
assert result3 == 123


Expand Down