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
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
Original file line number Diff line number Diff line change
Expand Up @@ -410,6 +410,18 @@ def submit(
def __repr__(self):
return f"MapWorker({self.src_fn_name})"

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

Comment on lines +413 to +424
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.


@dataclass
class _ActorState:
Expand Down Expand Up @@ -738,6 +750,9 @@ def _remove_actor(self, actor: ray.actor.ActorHandle):
# garbage collect the actor, instead of using ray.kill.
# Because otherwise the actor cannot be restarted upon lineage reconstruction.
if actor in self._running_actors:
# Call `on_exit` to trigger `UDF.__del__` which may perform
# cleanup operations.
actor.on_exit.remote()
del self._running_actors[actor]

def _get_location(self, bundle: RefBundle) -> Optional[NodeIdStr]:
Expand Down
3 changes: 3 additions & 0 deletions python/ray/data/tests/test_actor_pool_map_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,9 @@ def __init__(self, node_id: str = "node1"):
def get_location(self) -> str:
return self.node_id

def on_exit(self):
pass


class TestActorPool(unittest.TestCase):
def setup_class(self):
Expand Down
25 changes: 25 additions & 0 deletions python/ray/data/tests/test_map.py
Original file line number Diff line number Diff line change
Expand Up @@ -1500,6 +1500,31 @@ def test_random_sample_checks(ray_start_regular_shared):
ray.data.range(1).random_sample(10)


def test_actor_udf_cleanup(ray_start_regular_shared, tmp_path):
"""Test that for the actor map operator, the UDF object is deleted properly."""
test_file = tmp_path / "test.txt"

# Simulate the case that the UDF depends on some external resources that
# need to be cleaned up.
class StatefulUDF:
def __init__(self):
with open(test_file, "w") as f:
f.write("test")

def __call__(self, row):
return row

def __del__(self):
# Delete the file when the UDF is deleted.
os.remove(test_file)

ds = ray.data.range(10)
ds = ds.map(StatefulUDF, concurrency=1)
assert sorted(extract_values("id", ds.take_all())) == list(range(10))

wait_for_condition(lambda: not os.path.exists(test_file))


# NOTE: All tests above share a Ray cluster, while the tests below do not. These
# tests should only be carefully reordered to retain this invariant!
def test_actor_pool_strategy_default_num_actors(shutdown_only):
Expand Down
Loading