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

Follow-up on removing report and safe from Worker.close #6423

Merged
merged 12 commits into from
May 24, 2022

Conversation

gjoseph92
Copy link
Collaborator

@gjoseph92 gjoseph92 commented May 23, 2022

Closes #6422

xref #6363

cc @fjetter @jrbourbeau

  • Tests added / passed
  • Passes pre-commit run --all-files

Comment on lines +4764 to +4765
else:
self.running.discard(ws)
Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Resolves https://github.com/dask/distributed/pull/6363/files#r878475185

This is equivalent to main before the PR:

def handle_worker_status_change(
self, status: str, worker: str, stimulus_id: str
) -> None:
ws = self.workers.get(worker)
if not ws:
return
prev_status = ws.status
ws.status = Status.lookup[status] # type: ignore
if ws.status == prev_status:
return
self.log_event(
ws.address,
{
"action": "worker-status-change",
"prev-status": prev_status.name,
"status": status,
},
)
if ws.status == Status.running:
self.running.add(ws)
recs = self.bulk_schedule_after_adding_worker(ws)
if recs:
client_msgs: dict = {}
worker_msgs: dict = {}
self._transitions(recs, client_msgs, worker_msgs, stimulus_id)
self.send_all(client_msgs, worker_msgs)
else:
self.running.discard(ws)

Copy link
Member

Choose a reason for hiding this comment

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

I think this is required for some tests. https://github.com/dask/distributed/pull/6363/files#r879719934

I agree than an explicit test would be better but I expect something to break. If CI is happy, I'm happy, though.



@gen_cluster(Worker=Nanny, nthreads=[("", 1)], timeout=10)
async def test_scheduler_crash_doesnt_restart(s, a):
Copy link
Collaborator Author

Choose a reason for hiding this comment

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

This fails on main

Comment on lines -1443 to -1445
async def terminate(self, **kwargs):
return await self.close(nanny=True, **kwargs)

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

I opted to remove Worker.terminate since it won't be used now that the Worker.close default is to also close the Nanny. I think we will want two different methods eventually, but found the naming of terminate vs close very confusing—it should be something like terminate vs maybe_restart?

If others would prefer to see terminate stick around, I'm happy to remove this commit and leave it.

@log_errors
async def close(
self,
timeout=30,
executor_wait=True,
nanny=False,
nanny=True,
Copy link
Collaborator Author

Choose a reason for hiding this comment

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

@fjetter
Copy link
Member

fjetter commented May 23, 2022

I'm 100% fine with reverting the default for close(nanny=True). I toggled this because I think the worker should not even have the ability to close the nanny and wanted to introduce small, incremental changes to this. That was premature and we should indeed revert it.

Regarding the remove_worker on status change, I believe we require this. If this is blocking the PR I would propose to not revert this to the original code and keep my version. We may want to add a couple of tests but I think we should merge the nanny toggle reversion regardless and the tests should be a follow up PR (i.e. not necessarily release blocking)

@pentschev
Copy link
Member

pentschev commented May 23, 2022

Just FYI, this doesn't resolve #6320 , both Dask-CUDA tests hang and the equivalent tests that were added in #6363 distributed/cli/tests/test_dask_worker.py::test_error_during_startup and distributed/deploy/tests/test_local.py::test_localcluster_start_exception are also failing in CI.

@github-actions
Copy link
Contributor

github-actions bot commented May 23, 2022

Unit Test Results

       15 files  +       15         15 suites  +15   6h 26m 7s ⏱️ + 6h 26m 7s
  2 809 tests +  2 809    2 729 ✔️ +  2 729    79 💤 +  79  1 +1 
20 826 runs  +20 826  19 896 ✔️ +19 896  929 💤 +929  1 +1 

For more details on these failures, see this check.

Results for commit e3ab594. ± Comparison against base commit d84485b.

♻️ This comment has been updated with latest results.

gjoseph92 added 2 commits May 23, 2022 14:58
This was breaking because:
* Worker failed to start
* `Server.start` then calls `self.close()`
* `Worker.close` was calling the `close_gracefully` RPC on the Nanny
* For some reason, the RPC on the Nanny was unresponsive. I can't find an explanation for this. The event loop was not blocked. The listener seemed to be running. This definitely needs more investigation but I didn't want to look into it right now.
* The worker was waiting forever on the nanny to respond, the nanny was waiting forever on the worker to close.
@jrbourbeau
Copy link
Member

the equivalent tests that were added in #6363 distributed/cli/tests/test_dask_worker.py::test_error_during_startup and distributed/deploy/tests/test_local.py::test_localcluster_start_exception are also failing in CI.

Can confirm these now pass for me locally with Gabe's latest updates. @pentschev do those Dask-CUDA tests also pass?

@quasiben
Copy link
Member

Can confirm these now pass for me locally with Gabe's latest updates. @pentschev do those Dask-CUDA tests also pass?

I'm still seeing this fail/hang on test_localcluster_start_exception

py.test -s -v distributed/deploy/tests/test_local.py::test_localcluster_start_exception --runslow

@quasiben
Copy link
Member

Oops, forgot to pull latest changes. This passes as does my testing of Dask-CUDA

@quasiben
Copy link
Member

@pentschev is asleep but also confirmed that this fixes the immediate issues in dask-cuda

@gjoseph92
Copy link
Collaborator Author

Seems like this is good to go? Do the failures seem flaky to others?

#6426 is a follow-up issue, but not a blocker.

@jrbourbeau jrbourbeau requested a review from fjetter May 24, 2022 00:53
@jrbourbeau
Copy link
Member

Thanks @gjoseph92! Given the current time, let's give @fjetter a chance to take a look and then once this PR is in I'll push out the release (hopefully early morning CT)

@jakirkham jakirkham requested a review from pentschev May 24, 2022 06:09
Comment on lines +1453 to +1466
if self.status in (Status.closed, Status.closing, Status.failed):
await self.finished()
return

if self.status == Status.init:
# If the worker is still in startup/init and is started by a nanny,
# this means the nanny itself is not up, yet. If the Nanny isn't up,
# yet, it's server will not accept any incoming RPC requests and
# will block until the startup is finished.
# Therefore, this worker trying to communicate with the Nanny during
# startup is not possible and we cannot close it.
# In this case, the Nanny will automatically close after inspecting
# the worker status
nanny = False
Copy link
Member

Choose a reason for hiding this comment

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

I added another commit to be more explicit about the stati and added a more thorough explanation of why this is happening

cc @gjoseph92

@fjetter
Copy link
Member

fjetter commented May 24, 2022

I'm not sure if the test failure #6430 is related

@jrbourbeau jrbourbeau changed the title Fix #6363 issues Follow-up on removing report and safe from Worker.close May 24, 2022
@jrbourbeau
Copy link
Member

Thanks all!

@jrbourbeau jrbourbeau merged commit 30a872f into dask:main May 24, 2022
@gjoseph92 gjoseph92 deleted the fix-6363 branch May 24, 2022 16:03
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 this pull request may close these issues.

Regressions and potential bugs from #6363
5 participants