Skip to content

Commit

Permalink
[tune] Evict object cache for actor re-use when search ended (ray-pro…
Browse files Browse the repository at this point in the history
…ject#33593)

Similar regression that was originally fixed in ray-project#31974, but re-surfaced after ray-project#33045.

With actor re-use, we speculatively keep one cached actor around in case it is needed when new trials are added (e.g. if we add trials one-by-one). However, we should only do this when the search has not ended, as otherwise we keep an extra actor until the end of the experiment, wasting resources. This leads the tune_scale_up_down release test fail.

In our refactor to ray-project#33045 we generalized object caching, but the eviction logic here explicitly keeps one object in cache. Our previous implementation in RayTrialExecutor relied on not calling the eviction function at all when trials were still coming up, essentially not adjusting the number of cached actors down. This is rarely a problem in practice, but does not make a clean contract when separated out into a component.

In this PR, we change the logic as follows: When the search ended, no trials are pending execution, and we don't want to explicitly cache an actor, we force eviction of all cached objects.

We are refactoring our execution backend, thus I believe it's sufficient to keep the release test to catch this regression. In the new backend we can add light weight unit tests to capture this behavior.

Signed-off-by: Kai Fricke <[email protected]>
  • Loading branch information
krfricke authored Mar 22, 2023
1 parent 979b9db commit 02c8c95
Show file tree
Hide file tree
Showing 2 changed files with 15 additions and 0 deletions.
10 changes: 10 additions & 0 deletions python/ray/tune/execution/ray_trial_executor.py
Original file line number Diff line number Diff line change
Expand Up @@ -840,6 +840,16 @@ def _cleanup_cached_actors(
# (if the search ended).
return

if (
search_ended
and not self._staged_trials
and self._actor_cache.total_max_objects == 0
):
# If there are no more trials coming in, no trials are pending execution,
# and we don't explicitly want to cache objects, we can evict the full
# cache.
force_all = True

for actor, acquired_resources in self._actor_cache.flush_cached_objects(
force_all=force_all
):
Expand Down
5 changes: 5 additions & 0 deletions python/ray/tune/utils/object_cache.py
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,11 @@ def __init__(self, may_keep_one: bool = True):
def num_cached_objects(self):
return self._num_cached_objects

@property
def total_max_objects(self):
# Counter.total() is only available for python 3.10+
return sum(self._max_num_objects.values())

def increase_max(self, key: T, by: int = 1) -> None:
"""Increase number of max objects for this key.
Expand Down

0 comments on commit 02c8c95

Please sign in to comment.