Skip to content

Commit

Permalink
Transition task when victimized by stealing
Browse files Browse the repository at this point in the history
  • Loading branch information
fjetter committed Jan 12, 2021
1 parent 99ad636 commit e1b92e4
Show file tree
Hide file tree
Showing 2 changed files with 29 additions and 4 deletions.
6 changes: 6 additions & 0 deletions distributed/tests/test_stress.py
Original file line number Diff line number Diff line change
Expand Up @@ -228,6 +228,12 @@ async def test_close_connections(c, s, *workers):
x = x.rechunk((1000, 1))

future = c.compute(x.sum())

# need to wait to ensure the scheduler actually registered anything
# processing. Computation itself should take a few seconds so this should be
# a safe period to wait.
await asyncio.sleep(0.05)

while any(s.processing.values()):
await asyncio.sleep(0.5)
worker = random.choice(list(workers))
Expand Down
27 changes: 23 additions & 4 deletions distributed/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -2259,16 +2259,22 @@ def steal_request(self, key):
# There may be a race condition between stealing and releasing a task.
# In this case the self.tasks is already cleared. The `None` will be
# registered as `already-computing` on the other end
if key in self.tasks:
state = self.tasks[key].state
ts = self.tasks.get(key)
if ts:
state = ts.state
else:
state = None

response = {"op": "steal-response", "key": key, "state": state}
self.batched_stream.send(response)

if state in ("ready", "waiting", "constrained"):
self.release_key(key, reason="steal_request")
self.transition(ts, "waiting")
# FIXME GH4413 We no longer want to execute this task but there is
# no state to identify this. We still need to track it,
# otherwise this worker may never fetch it in case it is still
# required
ts.runspec = None

def release_key(self, key, reason=None, cause=None, report=True):
try:
Expand Down Expand Up @@ -2582,7 +2588,17 @@ async def execute(self, key, report=False):
executor_error = e
raise

if ts.state not in ("executing", "long-running"):
# We'll need to check again for the task state since it may have
# changed since the execution was kicked off. In particular, it may
# have been canceled and released already in which case we'll have
# to drop the result immediately
key = ts.key
ts = self.tasks.get(key)

if ts is None:
logger.debug(
"Dropping result for %s since task has already been released." % key
)
return

result["key"] = ts.key
Expand Down Expand Up @@ -2940,6 +2956,9 @@ def validate_state(self):
try:
assert len(self.data_needed) == len(set(self.data_needed))
waiting_keys = set()

assert self.executing_count >= 0

for ts in self.tasks.values():
assert ts.state is not None
# check that worker has task
Expand Down

0 comments on commit e1b92e4

Please sign in to comment.