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

[Ray core] Stopped job leaks worker #44897

Closed
codingl2k1 opened this issue Apr 22, 2024 · 5 comments · Fixed by #44214
Closed

[Ray core] Stopped job leaks worker #44897

codingl2k1 opened this issue Apr 22, 2024 · 5 comments · Fixed by #44214
Assignees
Labels
bug Something that is supposed to be working; but isn't core Issues that should be addressed in Ray Core p0.5 uueeehhh

Comments

@codingl2k1
Copy link

codingl2k1 commented Apr 22, 2024

What happened + What you expected to happen

No idle worker exists for the stopped job.

Versions / Dependencies

2.10.0

Reproduction script

No reliable script to reproduce this issue.

Issue Severity

High

@codingl2k1 codingl2k1 added bug Something that is supposed to be working; but isn't triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Apr 22, 2024
@codingl2k1
Copy link
Author

The issue is that the leaked idle workers are not in the idle_of_all_languages_ of the worker pool. However, in the dashboard, it shows "IDLE".

@anyscalesam anyscalesam added the core Issues that should be addressed in Ray Core label Apr 23, 2024
@codingl2k1
Copy link
Author

Leaked again. From the state dump, I can see that there are two Python workers not idle, but they are shown as IDLE on the dashboard. The two registered jobs have already been stopped. No job running now.

[state-dump] WorkerPool:
[state-dump] - registered jobs: 2
[state-dump] - process_failed_job_config_missing: 0
[state-dump] - process_failed_rate_limited: 0
[state-dump] - process_failed_pending_registration: 0
[state-dump] - process_failed_runtime_env_setup_failed: 0
[state-dump] - num CPP workers: 0
[state-dump] - num CPP drivers: 0
[state-dump] - num object spill callbacks queued: 0
[state-dump] - num object restore queued: 0
[state-dump] - num util functions queued: 0
[state-dump] - num PYTHON workers: 2
[state-dump] - num PYTHON drivers: 0
[state-dump] - num object spill callbacks queued: 0
[state-dump] - num object restore queued: 0
[state-dump] - num util functions queued: 0
[state-dump] - num idle workers: 0

@codingl2k1
Copy link
Author

codingl2k1 commented Apr 24, 2024

Both of the leaked workers have received the force kill request. They have released all reference counts but are waiting for the flight tasks.

task_manager.cc:387: This worker is still managing 90 in flight tasks, waiting for them to finish before shutting down.

I am not sure why 908 + 90 != 1002, but there is no Completing task after the received the Exit signal and all the Add pending task with 0 retries and 0 oom retries.

grep -c "Adding pending task" python-core-worker-a758bca1c0a1cf40c2c29449d44810d8a3b371082c9830891d1c28ec_380790.log
1002
grep -c "Completing task" python-core-worker-a758bca1c0a1cf40c2c29449d44810d8a3b371082c9830891d1c28ec_380790.log
908

@codingl2k1
Copy link
Author

From the source code: https://github.com/ray-project/ray/blob/master/src/ray/core_worker/core_worker.cc#L4236
Even the core worker has received a force exit, it still use the Exit instead of ForceExit. I think here should call the ForceExit in HandleExit if the force_exit is true. https://github.com/ray-project/ray/blob/master/src/ray/core_worker/core_worker.cc#L925

@jjyao
Copy link
Contributor

jjyao commented Apr 29, 2024

@codingl2k1 you are absolutely right and I have a PR to fix it #44214

@jjyao jjyao added P1 Issue that should be fixed within a few weeks and removed triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Apr 29, 2024
@anyscalesam anyscalesam added p0.5 uueeehhh and removed P1 Issue that should be fixed within a few weeks labels May 10, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something that is supposed to be working; but isn't core Issues that should be addressed in Ray Core p0.5 uueeehhh
Projects
None yet
Development

Successfully merging a pull request may close this issue.

3 participants