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

[data] Trigger UDF.__del__ when an actor is about to exit #49929

Merged
merged 3 commits into from
Jan 24, 2025

Conversation

raulchen
Copy link
Contributor

Why are these changes needed?

A stateful UDF may depend on external resources that may need to be cleaned up at the end.
This PR allows performing the cleanup operations via UDF.__del__

Note, this only ensures cleanup is performed when the job exists gracefully.
If the driver or the actor is forcefully killed, __del__ will not be called.

Related issue number

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

@raulchen raulchen requested a review from a team as a code owner January 18, 2025 00:19
Signed-off-by: Hao Chen <[email protected]>
Copy link
Collaborator

@comaniac comaniac left a comment

Choose a reason for hiding this comment

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

Test locally and it works like charm

Comment on lines +413 to +424
def on_exit(self):
"""Called when the actor is about to exist.
This enables performing cleanup operations via `UDF.__del__`.

Note, this only ensures cleanup is performed when the job exists gracefully.
If the driver or the actor is forcefully killed, `__del__` will not be called.
"""
# `_map_actor_context` is a global variable that references the UDF object.
# Delete it to trigger `UDF.__del__`.
del ray.data._map_actor_context
ray.data._map_actor_context = None

Copy link
Member

Choose a reason for hiding this comment

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

Is there a reason we need to introduce an explicit method rather than overriding _MapWorker.__del__? I imagine if we override __del__, we'll implicitly do the cleanup when we run del self._running_actors[actor].

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I also tried that. But it looks like that Ray Core doesn't call Actor.__del__ when killing an actor. cc @jjyao

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok, I found the ray core issue #50048 cc @jjyao
I can fix it if the solution looks good to you.

@raulchen raulchen added the go add ONLY when ready to merge, run all tests label Jan 24, 2025
@raulchen raulchen enabled auto-merge (squash) January 24, 2025 01:18
Signed-off-by: Hao Chen <[email protected]>
@github-actions github-actions bot disabled auto-merge January 24, 2025 21:51
@raulchen raulchen enabled auto-merge (squash) January 24, 2025 21:52
@raulchen raulchen merged commit 9b9843f into ray-project:master Jan 24, 2025
6 checks passed
@raulchen raulchen deleted the actor-map-clean-up branch January 24, 2025 23:27
erictang000 pushed a commit to erictang000/ray that referenced this pull request Jan 27, 2025
…ect#49929)

## Why are these changes needed?

A stateful UDF may depend on external resources that may need to be
cleaned up at the end.
This PR allows performing the cleanup operations via `UDF.__del__`

Note, this only ensures cleanup is performed when the job exists
gracefully.
If the driver or the actor is forcefully killed, `__del__` will not be

---------

Signed-off-by: Hao Chen <[email protected]>
Signed-off-by: Eric Tang <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
go add ONLY when ready to merge, run all tests
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants