From abb130745868303d537e1ffb647431aef12257f4 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Sun, 23 Jul 2023 13:38:45 -0700 Subject: [PATCH 001/108] Pipe storage context to Trainable (used now for Trainable syncing) Signed-off-by: Justin Yu --- python/ray/tune/experiment/config_parser.py | 1 + python/ray/tune/experiment/trial.py | 119 ++++++++++++-------- python/ray/tune/trainable/trainable.py | 44 ++++++-- 3 files changed, 104 insertions(+), 60 deletions(-) diff --git a/python/ray/tune/experiment/config_parser.py b/python/ray/tune/experiment/config_parser.py index 9dbd51427004..659104adc153 100644 --- a/python/ray/tune/experiment/config_parser.py +++ b/python/ray/tune/experiment/config_parser.py @@ -236,5 +236,6 @@ def _create_trial_from_spec( log_to_file=spec.get("log_to_file"), # str(None) doesn't create None max_failures=args.max_failures, + storage=spec.get("storage"), **trial_kwargs, ) diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index a496a92391cd..5bd70bbd5d71 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -27,6 +27,7 @@ import ray.cloudpickle as cloudpickle from ray.exceptions import RayActorError, RayTaskError +from ray.train._internal.storage import _use_storage_context, StorageContext from ray.tune import TuneError from ray.tune.error import _TuneRestoreError from ray.tune.execution.checkpoint_manager import _CheckpointManager @@ -61,6 +62,7 @@ from ray.util.debug import log_once from ray._private.utils import binary_to_hex, hex_to_binary + DEBUG_PRINT_INTERVAL = 5 _DEFAULT_WIN_MAX_PATH_LENGTH = 260 logger = logging.getLogger(__name__) @@ -247,7 +249,11 @@ def _get_trainable_kwargs( "logger_creator": logger_creator, } - if trial.uses_cloud_checkpointing: + if _use_storage_context(): + assert trial.storage + assert trial.storage.trial_dir_name + kwargs["storage"] = trial.storage + elif trial.uses_cloud_checkpointing: # We keep these kwargs separate for backwards compatibility # with trainables that don't provide these keyword arguments kwargs["remote_checkpoint_dir"] = trial.remote_path @@ -327,6 +333,7 @@ def __init__( *, config: Optional[Dict] = None, trial_id: Optional[str] = None, + storage: Optional[StorageContext] = None, experiment_path: Optional[str] = None, experiment_dir_name: Optional[str] = None, evaluated_params: Optional[Dict] = None, @@ -368,64 +375,75 @@ def __init__( self.trainable_name = trainable_name self.trial_id = Trial.generate_id() if trial_id is None else trial_id - # Sync config - self.sync_config = sync_config or SyncConfig() - # Set to pass through on `Trial.reset()` self._orig_experiment_path = experiment_path self._orig_experiment_dir_name = experiment_dir_name - local_experiment_path, remote_experiment_path = _split_remote_local_path( - experiment_path, None - ) + self.storage = copy.copy(storage) + if _use_storage_context(): + # NOTE: storage is None on restoration and is manually set afterwards. + # See `TuneControllerBase.restore_from_dir`. + if storage: + self.sync_config = self.storage.sync_config + self._local_experiment_path = self.storage.experiment_cache_path + self._remote_experiment_path = self.storage.experiment_fs_path + else: + # Sync config + self.sync_config = sync_config or SyncConfig() - # Backwards compatibility for `local_dir` - if local_dir: - if local_experiment_path: - raise ValueError( - "Only one of `local_dir` or `experiment_path` " - "can be passed to `Trial()`." - ) - local_experiment_path = local_dir - - # Derive experiment dir name from local path - if not experiment_dir_name and local_experiment_path: - # Maybe derive experiment dir name from local storage dir - experiment_dir_name = Path(local_experiment_path).name - elif not experiment_dir_name: - experiment_dir_name = DEFAULT_EXPERIMENT_NAME - - # Set default experiment dir name - if not local_experiment_path: - local_experiment_path = str( - Path(_get_defaults_results_dir()) / experiment_dir_name + local_experiment_path, remote_experiment_path = _split_remote_local_path( + experiment_path, None ) - os.makedirs(local_experiment_path, exist_ok=True) - # Set remote experiment path if upload_dir is set - if self.sync_config.upload_dir: - if remote_experiment_path: - if not remote_experiment_path.startswith(self.sync_config.upload_dir): + # Backwards compatibility for `local_dir` + if local_dir: + if local_experiment_path: raise ValueError( - f"Both a `SyncConfig.upload_dir` and an `experiment_path` " - f"pointing to remote storage were passed, but they do not " - f"point to the same location. Got: " - f"`experiment_path={experiment_path}` and " - f"`SyncConfig.upload_dir={self.sync_config.upload_dir}`. " + "Only one of `local_dir` or `experiment_path` " + "can be passed to `Trial()`." ) - warnings.warn( - "If `experiment_path` points to a remote storage location, " - "do not set `SyncConfig.upload_dir`. ", - DeprecationWarning, - ) - else: - remote_experiment_path = str( - URI(self.sync_config.upload_dir) / experiment_dir_name + local_experiment_path = local_dir + + # Derive experiment dir name from local path + if not experiment_dir_name and local_experiment_path: + # Maybe derive experiment dir name from local storage dir + experiment_dir_name = Path(local_experiment_path).name + elif not experiment_dir_name: + experiment_dir_name = DEFAULT_EXPERIMENT_NAME + + # Set default experiment dir name + if not local_experiment_path: + local_experiment_path = str( + Path(_get_defaults_results_dir()) / experiment_dir_name ) + os.makedirs(local_experiment_path, exist_ok=True) + + # Set remote experiment path if upload_dir is set + if self.sync_config.upload_dir: + if remote_experiment_path: + if not remote_experiment_path.startswith( + self.sync_config.upload_dir + ): + raise ValueError( + f"Both a `SyncConfig.upload_dir` and an `experiment_path` " + f"pointing to remote storage were passed, but they do not " + f"point to the same location. Got: " + f"`experiment_path={experiment_path}` and " + f"`SyncConfig.upload_dir={self.sync_config.upload_dir}`. " + ) + warnings.warn( + "If `experiment_path` points to a remote storage location, " + "do not set `SyncConfig.upload_dir`. ", + DeprecationWarning, + ) + else: + remote_experiment_path = str( + URI(self.sync_config.upload_dir) / experiment_dir_name + ) - # Finally, set properties - self._local_experiment_path = local_experiment_path - self._remote_experiment_path = remote_experiment_path + # Finally, set properties + self._local_experiment_path = local_experiment_path + self._remote_experiment_path = remote_experiment_path self.config = config or {} # Save a copy of the original unresolved config so that we can swap @@ -840,6 +858,11 @@ def init_local_path(self): ) logdir_path.mkdir(parents=True, exist_ok=True) + if _use_storage_context(): + # Populate the storage context with the trial dir name we just generated. + assert self.storage + self.storage.trial_dir_name = self.relative_logdir + self.invalidate_json_state() def update_resources(self, resources: Union[dict, PlacementGroupFactory]): diff --git a/python/ray/tune/trainable/trainable.py b/python/ray/tune/trainable/trainable.py index 4a36216dca5e..1d8345ff86db 100644 --- a/python/ray/tune/trainable/trainable.py +++ b/python/ray/tune/trainable/trainable.py @@ -24,6 +24,11 @@ TIME_THIS_ITER_S, TRAINING_ITERATION, ) +from ray.train._internal.storage import ( + _use_storage_context, + StorageContext, + init_shared_storage_context, +) from ray.tune.result import ( DEBUG_METRICS, DEFAULT_RESULTS_DIR, @@ -112,6 +117,7 @@ def __init__( logger_creator: Callable[[Dict[str, Any]], "Logger"] = None, # Deprecated (2.7) remote_checkpoint_dir: Optional[str] = None, sync_config: Optional[SyncConfig] = None, + storage: Optional[StorageContext] = None, ): """Initialize a Trainable. @@ -178,19 +184,33 @@ def __init__( log_sys_usage = self.config.get("log_sys_usage", False) self._monitor = UtilMonitor(start=log_sys_usage) - self.remote_checkpoint_dir = remote_checkpoint_dir - # If no sync_config is provided, but we save to a remote_checkpoint_dir, - # then provide a default syncer. `upload_dir` here is just a dummy directory - # that tells the SyncConfig to create a default syncer. - self.sync_config = sync_config or SyncConfig( - upload_dir=self.remote_checkpoint_dir, syncer="auto" - ) + self._storage = storage + # Set a globally accessible storage context on the remote Trainable process + # This is accessible from the training loop thread for FunctionTrainable's + init_shared_storage_context(storage) - # Resolves syncer="auto" to an actual syncer cloud storage is used - # If sync_config.syncer is a custom Syncer instance, this is a no-op. - self.sync_config.syncer = get_node_to_storage_syncer( - self.sync_config, self.remote_checkpoint_dir - ) + if _use_storage_context(): + assert storage + assert storage.trial_fs_path + logger.debug(f"StorageContext on the TRAINABLE:\n{storage}") + storage._check_validation_file() + + self.remote_checkpoint_dir = storage.trial_fs_path + self.sync_config = storage.sync_config + else: + self.remote_checkpoint_dir = remote_checkpoint_dir + # If no sync_config is provided, but we save to a remote_checkpoint_dir, + # then provide a default syncer. `upload_dir` here is just a dummy directory + # that tells the SyncConfig to create a default syncer. + self.sync_config = sync_config or SyncConfig( + upload_dir=self.remote_checkpoint_dir, syncer="auto" + ) + + # Resolves syncer="auto" to an actual syncer cloud storage is used + # If sync_config.syncer is a custom Syncer instance, this is a no-op. + self.sync_config.syncer = get_node_to_storage_syncer( + self.sync_config, self.remote_checkpoint_dir + ) self.sync_num_retries = int(os.getenv("TUNE_CHECKPOINT_CLOUD_RETRY_NUM", "2")) self.sync_sleep_time = float( From f6ff90a393843129a3cd4c29b5b5bf8ce482aac9 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 27 Jul 2023 15:00:24 -0700 Subject: [PATCH 002/108] Don't use the storage context in the trial/trainable Signed-off-by: Justin Yu --- python/ray/tune/experiment/trial.py | 118 ++++++++++++------------- python/ray/tune/trainable/trainable.py | 34 ++++--- 2 files changed, 75 insertions(+), 77 deletions(-) diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index 5bd70bbd5d71..7b200485f152 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -253,7 +253,8 @@ def _get_trainable_kwargs( assert trial.storage assert trial.storage.trial_dir_name kwargs["storage"] = trial.storage - elif trial.uses_cloud_checkpointing: + + if trial.uses_cloud_checkpointing: # We keep these kwargs separate for backwards compatibility # with trainables that don't provide these keyword arguments kwargs["remote_checkpoint_dir"] = trial.remote_path @@ -379,71 +380,70 @@ def __init__( self._orig_experiment_path = experiment_path self._orig_experiment_dir_name = experiment_dir_name + # Create a copy, since `init_local_path` updates the context with the + # generated trial dirname. self.storage = copy.copy(storage) - if _use_storage_context(): - # NOTE: storage is None on restoration and is manually set afterwards. - # See `TuneControllerBase.restore_from_dir`. - if storage: - self.sync_config = self.storage.sync_config - self._local_experiment_path = self.storage.experiment_cache_path - self._remote_experiment_path = self.storage.experiment_fs_path - else: - # Sync config - self.sync_config = sync_config or SyncConfig() - local_experiment_path, remote_experiment_path = _split_remote_local_path( - experiment_path, None - ) + # TODO(justinvyu): For now, explicitly avoid using the storage context + # to replace the Trial path handling. This should be re-worked + # when adding new persistence mode support for Tune Trainables. + + # Sync config + self.sync_config = sync_config or SyncConfig() + + local_experiment_path, remote_experiment_path = _split_remote_local_path( + experiment_path, None + ) - # Backwards compatibility for `local_dir` - if local_dir: - if local_experiment_path: + # Backwards compatibility for `local_dir` + if local_dir: + if local_experiment_path: + raise ValueError( + "Only one of `local_dir` or `experiment_path` " + "can be passed to `Trial()`." + ) + local_experiment_path = local_dir + + # Derive experiment dir name from local path + if not experiment_dir_name and local_experiment_path: + # Maybe derive experiment dir name from local storage dir + experiment_dir_name = Path(local_experiment_path).name + elif not experiment_dir_name: + experiment_dir_name = DEFAULT_EXPERIMENT_NAME + + # Set default experiment dir name + if not local_experiment_path: + local_experiment_path = str( + Path(_get_defaults_results_dir()) / experiment_dir_name + ) + os.makedirs(local_experiment_path, exist_ok=True) + + # Set remote experiment path if upload_dir is set + if self.sync_config.upload_dir: + if remote_experiment_path: + if not remote_experiment_path.startswith( + self.sync_config.upload_dir + ): raise ValueError( - "Only one of `local_dir` or `experiment_path` " - "can be passed to `Trial()`." + f"Both a `SyncConfig.upload_dir` and an `experiment_path` " + f"pointing to remote storage were passed, but they do not " + f"point to the same location. Got: " + f"`experiment_path={experiment_path}` and " + f"`SyncConfig.upload_dir={self.sync_config.upload_dir}`. " ) - local_experiment_path = local_dir - - # Derive experiment dir name from local path - if not experiment_dir_name and local_experiment_path: - # Maybe derive experiment dir name from local storage dir - experiment_dir_name = Path(local_experiment_path).name - elif not experiment_dir_name: - experiment_dir_name = DEFAULT_EXPERIMENT_NAME - - # Set default experiment dir name - if not local_experiment_path: - local_experiment_path = str( - Path(_get_defaults_results_dir()) / experiment_dir_name + warnings.warn( + "If `experiment_path` points to a remote storage location, " + "do not set `SyncConfig.upload_dir`. ", + DeprecationWarning, + ) + else: + remote_experiment_path = str( + URI(self.sync_config.upload_dir) / experiment_dir_name ) - os.makedirs(local_experiment_path, exist_ok=True) - - # Set remote experiment path if upload_dir is set - if self.sync_config.upload_dir: - if remote_experiment_path: - if not remote_experiment_path.startswith( - self.sync_config.upload_dir - ): - raise ValueError( - f"Both a `SyncConfig.upload_dir` and an `experiment_path` " - f"pointing to remote storage were passed, but they do not " - f"point to the same location. Got: " - f"`experiment_path={experiment_path}` and " - f"`SyncConfig.upload_dir={self.sync_config.upload_dir}`. " - ) - warnings.warn( - "If `experiment_path` points to a remote storage location, " - "do not set `SyncConfig.upload_dir`. ", - DeprecationWarning, - ) - else: - remote_experiment_path = str( - URI(self.sync_config.upload_dir) / experiment_dir_name - ) - # Finally, set properties - self._local_experiment_path = local_experiment_path - self._remote_experiment_path = remote_experiment_path + # Finally, set properties + self._local_experiment_path = local_experiment_path + self._remote_experiment_path = remote_experiment_path self.config = config or {} # Save a copy of the original unresolved config so that we can swap diff --git a/python/ray/tune/trainable/trainable.py b/python/ray/tune/trainable/trainable.py index 1d8345ff86db..d9119f1e699a 100644 --- a/python/ray/tune/trainable/trainable.py +++ b/python/ray/tune/trainable/trainable.py @@ -185,9 +185,6 @@ def __init__( self._monitor = UtilMonitor(start=log_sys_usage) self._storage = storage - # Set a globally accessible storage context on the remote Trainable process - # This is accessible from the training loop thread for FunctionTrainable's - init_shared_storage_context(storage) if _use_storage_context(): assert storage @@ -195,22 +192,23 @@ def __init__( logger.debug(f"StorageContext on the TRAINABLE:\n{storage}") storage._check_validation_file() - self.remote_checkpoint_dir = storage.trial_fs_path - self.sync_config = storage.sync_config - else: - self.remote_checkpoint_dir = remote_checkpoint_dir - # If no sync_config is provided, but we save to a remote_checkpoint_dir, - # then provide a default syncer. `upload_dir` here is just a dummy directory - # that tells the SyncConfig to create a default syncer. - self.sync_config = sync_config or SyncConfig( - upload_dir=self.remote_checkpoint_dir, syncer="auto" - ) + # Set a globally accessible storage context on the remote Trainable process + # This is accessible from the training loop thread for FunctionTrainable's + init_shared_storage_context(storage) - # Resolves syncer="auto" to an actual syncer cloud storage is used - # If sync_config.syncer is a custom Syncer instance, this is a no-op. - self.sync_config.syncer = get_node_to_storage_syncer( - self.sync_config, self.remote_checkpoint_dir - ) + self.remote_checkpoint_dir = remote_checkpoint_dir + # If no sync_config is provided, but we save to a remote_checkpoint_dir, + # then provide a default syncer. `upload_dir` here is just a dummy directory + # that tells the SyncConfig to create a default syncer. + self.sync_config = sync_config or SyncConfig( + upload_dir=self.remote_checkpoint_dir, syncer="auto" + ) + + # Resolves syncer="auto" to an actual syncer cloud storage is used + # If sync_config.syncer is a custom Syncer instance, this is a no-op. + self.sync_config.syncer = get_node_to_storage_syncer( + self.sync_config, self.remote_checkpoint_dir + ) self.sync_num_retries = int(os.getenv("TUNE_CHECKPOINT_CLOUD_RETRY_NUM", "2")) self.sync_sleep_time = float( From 562369fcab353754fdfb1a74e6dcf79c4f6abb5f Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 27 Jul 2023 15:02:25 -0700 Subject: [PATCH 003/108] Disable all trainable syncing in new codepath Signed-off-by: Justin Yu --- python/ray/tune/trainable/trainable.py | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/python/ray/tune/trainable/trainable.py b/python/ray/tune/trainable/trainable.py index d9119f1e699a..6f2d947de865 100644 --- a/python/ray/tune/trainable/trainable.py +++ b/python/ray/tune/trainable/trainable.py @@ -622,6 +622,9 @@ def _should_upload_artifacts(self) -> bool: return bool(artifact_files) def _maybe_save_artifacts_to_cloud(self) -> bool: + if _use_storage_context(): + return False + if not self._should_upload_artifacts: return False @@ -656,6 +659,9 @@ def _maybe_save_to_cloud(self, local_dir: str, exclude: List[str] = None) -> boo Returns: bool: True if (successfully) saved to cloud """ + if _use_storage_context(): + return False + if not self.uses_cloud_checkpointing: return False @@ -697,6 +703,9 @@ def _maybe_load_checkpoint_from_cloud(self, checkpoint_path: str) -> bool: Return: bool: True if the checkpoint was synced down successfully from cloud. """ + if _use_storage_context(): + return False + if os.path.exists(checkpoint_path): try: TrainableUtil.find_checkpoint_dir(checkpoint_path) @@ -726,6 +735,9 @@ def _maybe_load_checkpoint_from_cloud(self, checkpoint_path: str) -> bool: return success def _maybe_load_artifacts_from_cloud(self) -> bool: + if _use_storage_context(): + return False + if not self.sync_config.sync_artifacts: return False @@ -753,6 +765,9 @@ def _maybe_load_artifacts_from_cloud(self) -> bool: def _maybe_load_from_cloud( self, remote_dir: str, local_dir: str, exclude: List[str] = None ) -> bool: + if _use_storage_context(): + return False + if not self.uses_cloud_checkpointing or not list_at_uri(remote_dir): return False From 95a3d202aa5c3445c47687d92689855ec040778a Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Sun, 23 Jul 2023 16:37:29 -0700 Subject: [PATCH 004/108] Pipe storage context to Train workers (not actually used yet) Signed-off-by: Justin Yu Update to use the new checkpoint id attribute Signed-off-by: Justin Yu Add todo comment to remove legacy path Signed-off-by: Justin Yu --- .../ray/train/_internal/backend_executor.py | 5 ++ python/ray/train/_internal/session.py | 11 +++- python/ray/train/trainer.py | 50 +++++++++++++------ 3 files changed, 49 insertions(+), 17 deletions(-) diff --git a/python/ray/train/_internal/backend_executor.py b/python/ray/train/_internal/backend_executor.py index 3a67ec941b83..d3c04a5d30cc 100644 --- a/python/ray/train/_internal/backend_executor.py +++ b/python/ray/train/_internal/backend_executor.py @@ -17,6 +17,7 @@ init_session, shutdown_session, ) +from ray.train._internal.storage import _use_storage_context, get_storage_context from ray.train._internal.utils import check_for_failure from ray.train._internal.worker_group import WorkerGroup from ray.train.backend import BackendConfig @@ -378,6 +379,7 @@ def initialize_session( encode_data_fn, checkpoint_keep_all_ranks, checkpoint_upload_from_workers, + storage, ): try: init_session( @@ -395,6 +397,7 @@ def initialize_session( enable_lazy_checkpointing=use_lazy_checkpointing, checkpoint_keep_all_ranks=checkpoint_keep_all_ranks, checkpoint_upload_from_workers=(checkpoint_upload_from_workers), + storage=storage, ) except ValueError: raise TrainBackendError( @@ -440,6 +443,8 @@ def initialize_session( checkpoint_upload_from_workers=( self._checkpoint_upload_from_workers ), + # Pass the Trainable's shared storage context to the Train workers + storage=get_storage_context() if _use_storage_context() else None, ) ) diff --git a/python/ray/train/_internal/session.py b/python/ray/train/_internal/session.py index 69e25a1e26b3..f956e071bd35 100644 --- a/python/ray/train/_internal/session.py +++ b/python/ray/train/_internal/session.py @@ -41,7 +41,7 @@ from ray.train.error import SessionMisuseError from ray.util.annotations import DeveloperAPI, PublicAPI from ray.util.debug import log_once - +from ray.train._internal.storage import _use_storage_context, StorageContext if TYPE_CHECKING: from ray.data import DataIterator @@ -106,6 +106,7 @@ def __init__( enable_lazy_checkpointing: bool = True, checkpoint_keep_all_ranks: bool = False, checkpoint_upload_from_workers: bool = False, + storage: Optional[StorageContext] = None, ): self.dataset_shard = dataset_shard @@ -121,6 +122,14 @@ def __init__( self.enable_lazy_checkpointing = enable_lazy_checkpointing self.checkpoint_keep_all_ranks = checkpoint_keep_all_ranks self.checkpoint_upload_from_workers = checkpoint_upload_from_workers + + if _use_storage_context(): + assert storage + logger.debug(f"StorageContext on TRAIN WORKER {world_rank}:\n{storage}") + storage._check_validation_file() + + self.storage = storage + # Only used if checkpoint_upload_from_workers is True. self.checkpoint_uri = None diff --git a/python/ray/train/trainer.py b/python/ray/train/trainer.py index 6921977be114..42d084fbafd6 100644 --- a/python/ray/train/trainer.py +++ b/python/ray/train/trainer.py @@ -30,6 +30,8 @@ ) from ray.tune.trainable.util import TrainableUtil from ray.util.annotations import DeveloperAPI +from ray.train._internal.storage import _use_storage_context, get_storage_context + T = TypeVar("T") S = TypeVar("S") @@ -101,8 +103,31 @@ def _start_training( ) ) - # Session has started. Set current cloud checkpoint dir if necessary. - if self._checkpoint_strategy._checkpoint_upload_from_workers: + # Workers need to start out with a path to write the first checkpoint to. + self._send_next_checkpoint_path_to_workers() + + def _send_next_checkpoint_path_to_workers(self): + # NOTE: Always upload to storage from workers in the new persistence path + # (no need to check for the `checkpoint_upload_from_workers` flag) + if _use_storage_context(): + storage = get_storage_context() + + # NOTE: Idea: this checkpoint dir name should be customizable + # and created on the fly when the checkpoint is reported with metrics. + # Ex: lambda metrics: f"checkpoint_iter={metrics['training_iteration']}" + storage.current_checkpoint_id = ( + self._checkpoint_manager._latest_checkpoint_id + ) + logger.debug( + f"Setting next checkpoint path to: {storage.checkpoint_fs_path}" + ) + + # TODO(justinvyu): This checkpoint_path is NOT a URI anymore. + # It's just a path relative to the storage filesystem. + # `session.report` needs to be updated to upload using pyarrow.fs.copy_files + self._backend_executor._set_checkpoint_uri(storage.checkpoint_fs_path) + + elif self._checkpoint_strategy._checkpoint_upload_from_workers: self._backend_executor._set_checkpoint_uri( self.__get_cloud_checkpoint_dir() ) @@ -201,10 +226,7 @@ def _fetch_next_result(self) -> Optional[List[Dict]]: # TODO(jungong) : It would be nicer if we find a cleaner way # to sync the current cloud checkpointing directory between # Tuner, Trainable, and Trainers. - if self._checkpoint_strategy._checkpoint_upload_from_workers: - self._backend_executor._set_checkpoint_uri( - self.__get_cloud_checkpoint_dir() - ) + self._send_next_checkpoint_path_to_workers() # Iterate until next REPORT call or training has finished. else: raise TrainBackendError( @@ -214,6 +236,7 @@ def _fetch_next_result(self) -> Optional[List[Dict]]: f"{[type in TrainingResultType]}" ) + # TODO(justinvyu): Remove unused code def _finish_checkpointing(self): while True: results = self._backend_executor.get_next_results() @@ -225,11 +248,10 @@ def _finish_checkpointing(self): self._checkpoint_manager._process_checkpoints( results, decode_checkpoint_fn=self._backend._decode_data ) - if self._checkpoint_strategy._checkpoint_upload_from_workers: - self._backend_executor._set_checkpoint_uri( - self.__get_cloud_checkpoint_dir() - ) + # TODO: Is this needed? I don't think this is ever called... + self._send_next_checkpoint_path_to_workers() + # TODO(justinvyu): Remove unused code def _finish_training(self): """Finish training and return final results. Propagate any exceptions. @@ -251,6 +273,7 @@ def _finish_training(self): def is_finished(self) -> bool: return self._finished_training + # TODO(justinvyu): Remove unused code def get_final_results(self, force: bool = False) -> List[T]: """Gets the training func return values from each worker. @@ -281,12 +304,7 @@ def get_final_results(self, force: bool = False) -> List[T]: return self._final_results - # This is extremely hacky and fragile. - # TODO(jungong) : We should refactor things so Tuner, Trinable, and - # Trainers have a consistent view of the current cloud checkpointing - # directory. - # We should probably also refactor things so Syncer and SyncConfig - # are available everywhere session is available. + # TODO(justinvyu): Remove legacy path. def __get_cloud_checkpoint_dir(self): if not self._storage_path: # Can't run cloud upload if storage path is not set. From 484e67fe1de043a026b22658f2a29505a95206e4 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Sun, 23 Jul 2023 17:01:40 -0700 Subject: [PATCH 005/108] Fix race condition for setting checkpoint_uri Signed-off-by: Justin Yu --- python/ray/train/_internal/backend_executor.py | 4 ++++ python/ray/train/trainer.py | 5 ++--- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/python/ray/train/_internal/backend_executor.py b/python/ray/train/_internal/backend_executor.py index d3c04a5d30cc..58a4076c5a8d 100644 --- a/python/ray/train/_internal/backend_executor.py +++ b/python/ray/train/_internal/backend_executor.py @@ -346,6 +346,7 @@ def start_training( datasets: Dict[str, Dataset], data_config: DataConfig, checkpoint: Optional[Checkpoint] = None, + on_session_init: Callable[[], None] = None, ) -> None: """Executes a training function on all workers in a separate thread. @@ -452,6 +453,9 @@ def initialize_session( self.get_with_failure_handling(futures) + if on_session_init: + on_session_init() + # Run the training function asynchronously in its own thread. def train_async(): session = get_session() diff --git a/python/ray/train/trainer.py b/python/ray/train/trainer.py index 42d084fbafd6..ab75b708cb3e 100644 --- a/python/ray/train/trainer.py +++ b/python/ray/train/trainer.py @@ -100,12 +100,11 @@ def _start_training( datasets=datasets, data_config=data_config, checkpoint=checkpoint, + # Workers need to start out with a path to write the first checkpoint to + on_session_init=self._send_next_checkpoint_path_to_workers, ) ) - # Workers need to start out with a path to write the first checkpoint to. - self._send_next_checkpoint_path_to_workers() - def _send_next_checkpoint_path_to_workers(self): # NOTE: Always upload to storage from workers in the new persistence path # (no need to check for the `checkpoint_upload_from_workers` flag) From 214866990c2d4337e606a9c1bf9ca5e6d505e028 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 27 Jul 2023 15:30:50 -0700 Subject: [PATCH 006/108] Fix cyclical import Signed-off-by: Justin Yu --- python/ray/train/_internal/storage.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/ray/train/_internal/storage.py b/python/ray/train/_internal/storage.py index 6b2a7943141f..a845c7b87998 100644 --- a/python/ray/train/_internal/storage.py +++ b/python/ray/train/_internal/storage.py @@ -29,7 +29,6 @@ from ray.air._internal.uri_utils import URI, is_uri from ray.tune.syncer import Syncer, SyncConfig, _BackgroundSyncer from ray.tune.result import _get_defaults_results_dir -from ray.tune.trainable.util import TrainableUtil logger = logging.getLogger(__file__) @@ -506,6 +505,8 @@ def checkpoint_fs_path(self) -> str: Raises a ValueError if `current_checkpoint_id` is not set beforehand. """ + from ray.tune.trainable.util import TrainableUtil + if self.current_checkpoint_id is None: raise RuntimeError( "Should not access `checkpoint_fs_path` without setting " From 8c856b816b1d57d4d9541fdcc7742a5892d21917 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 27 Jul 2023 15:31:01 -0700 Subject: [PATCH 007/108] Add simple trainer test Signed-off-by: Justin Yu --- .../ray/train/tests/test_new_persistence.py | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index b1efd09d5121..e468e9de24a3 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -10,6 +10,7 @@ from ray import air, train, tune from ray.air.tests.test_checkpoints import mock_s3_bucket_uri from ray.train._internal.storage import _download_from_fs_path +from ray.train.data_parallel_trainer import DataParallelTrainer @contextmanager @@ -150,6 +151,33 @@ def test_tuner(monkeypatch, storage_path_type, tmp_path): assert len(list(exp_dir.glob("tuner.pkl"))) == 1 +def test_trainer(tmp_path): + """For now, this is just a dummy test to inspect that the storage context + has been passed to the train workers properly.""" + storage_path = str(tmp_path / "fake_nfs") + + def dummy_train_fn(config): + from ray.air._internal.session import _get_session + from ray.train._internal.session import _TrainSession + + train_session = _get_session() + print(train_session.storage) + + assert isinstance(train_session, _TrainSession) + assert train_session.storage + assert train_session.checkpoint_uri.startswith(storage_path) + + trainer = DataParallelTrainer( + dummy_train_fn, + scaling_config=train.ScalingConfig(num_workers=2), + run_config=train.RunConfig( + storage_path=storage_path, + name="trainer_new_persistence", + ), + ) + trainer.fit() + + if __name__ == "__main__": import sys From 78c525faa43dd9fa4ff8965c8b3456ccf82363b4 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 27 Jul 2023 15:35:16 -0700 Subject: [PATCH 008/108] Add legacy prefix to train session checkpoint uri Signed-off-by: Justin Yu --- python/ray/train/_internal/session.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/python/ray/train/_internal/session.py b/python/ray/train/_internal/session.py index f956e071bd35..2f5f8dc7c59d 100644 --- a/python/ray/train/_internal/session.py +++ b/python/ray/train/_internal/session.py @@ -131,7 +131,7 @@ def __init__( self.storage = storage # Only used if checkpoint_upload_from_workers is True. - self.checkpoint_uri = None + self.legacy_checkpoint_uri = None # Function to encode checkpoint dict before sending to the driver. if not encode_data_fn: @@ -376,18 +376,18 @@ def checkpoint(self, checkpoint: Checkpoint): upload_from_workers = ( checkpoint_type == "local_path" and self.checkpoint_upload_from_workers - and self.checkpoint_uri + and self.legacy_checkpoint_uri ) if upload_from_workers: self._create_checkpoint_file_list(checkpoint) logger.info( f"Uploading checkpoint files from worker rank {self.world_rank} " - f"to cloud URI {self.checkpoint_uri}." + f"to cloud URI {self.legacy_checkpoint_uri}." ) # We want to upload the files directly to cloud storage, # so that they won't need to be shipped to the driver node # via object store. - checkpoint.to_uri(self.checkpoint_uri) + checkpoint.to_uri(self.legacy_checkpoint_uri) logger.info("Done uploading checkpoint files.") self._remove_uploaded_checkpoint_files(checkpoint) @@ -433,7 +433,7 @@ def _set_checkpoint_uri(self, uri: str): Args: uri: URI to the location where next checkpoint should be saved. """ - self.checkpoint_uri = uri + self.legacy_checkpoint_uri = uri def report(self, metrics: Dict, checkpoint: Optional[Checkpoint] = None) -> None: # TODO(xwjiang): tons of optimizations. From e97f4715d4f3f65f2a3612cf04d56eb193c47d92 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 27 Jul 2023 16:15:47 -0700 Subject: [PATCH 009/108] Add new checkpoint class Signed-off-by: Justin Yu --- python/ray/train/checkpoint.py | 234 +++++++++++++++++++++++++++++++++ 1 file changed, 234 insertions(+) create mode 100644 python/ray/train/checkpoint.py diff --git a/python/ray/train/checkpoint.py b/python/ray/train/checkpoint.py new file mode 100644 index 000000000000..1774996f7525 --- /dev/null +++ b/python/ray/train/checkpoint.py @@ -0,0 +1,234 @@ +import contextlib +import logging +import os +import platform +import shutil +import tempfile +import traceback +from pathlib import Path +from typing import TYPE_CHECKING, Any, Dict, Iterator, Optional, Union + +import pyarrow.fs + +from ray import cloudpickle as pickle +from ray.air._internal.filelock import TempFileLock +from ray.util.annotations import PublicAPI + +logger = logging.getLogger(__name__) + +_METADATA_FILE_NAME = ".metadata.pkl" +_CHECKPOINT_DIR_PREFIX = "checkpoint_tmp_" + + +@PublicAPI(stability="beta") +class Checkpoint: + """A reference to data persisted in local or remote storage. + + Access checkpoint contents locally using `checkpoint.to_directory()`. + + Attributes: + path: A path on the filesystem containing the checkpoint contents. + filesystem: PyArrow FileSystem that can be used to access data at the `path`. + """ + + def __init__( + self, + path: Union[str, os.PathLike], + filesystem: Optional["pyarrow.fs.FileSystem"] = None, + ): + """Construct a Checkpoint. + + Args: + path: A local path or remote URI containing the checkpoint data. + If a filesystem is provided, then this path must NOT be a URI. + It should be a path on the filesystem with the prefix already stripped. + filesystem: PyArrow FileSystem to use to access data at the path. + If not specified, this is inferred from the URI scheme. + """ + self.path = path + self.filesystem = filesystem + + if path and not filesystem: + self.filesystem, self.path = pyarrow.fs.FileSystem.from_uri(path) + + def __repr__(self): + return f"Checkpoint(filesystem={self.filesystem}, path={self.path})" + + def get_metadata(self) -> Dict[str, Any]: + """Return the metadata dict stored with the checkpoint. + + If no metadata is stored, an empty dict is returned.""" + + path = os.path.join(self.path, _METADATA_FILE_NAME) + with self.filesystem.open_input_file(path) as f: + return pickle.loads(f.readall()) + + def set_metadata(self, metadata: Dict[str, Any]) -> None: + """Update the metadata stored with this checkpoint. + + This will overwrite any existing metadata stored with this checkpoint. + """ + + path = os.path.join(self.path, _METADATA_FILE_NAME) + with self.filesystem.open_output_stream(path) as f: + f.write(pickle.dumps(metadata)) + + @staticmethod + def from_directory(path: Union[str, os.PathLike]) -> "Checkpoint": + """Create checkpoint object from a local directory. + + Args: + path: Directory containing checkpoint data. The caller promises to + not delete the directory (gifts ownership of the directory to this + Checkpoint). + + Returns: + Checkpoint: checkpoint object. + """ + return Checkpoint(path, filesystem=pyarrow.fs.LocalFileSystem()) + + def to_directory(self, path: Optional[str] = None) -> str: + """Write checkpoint data to directory. + + Args: + path: Target directory to restore data in. If not specified, + will create a temporary directory. + + Returns: + str: Directory containing checkpoint data. + """ + import pyarrow + + user_provided_path = path is not None + path = path if user_provided_path else self._get_temporary_checkpoint_dir() + path = os.path.normpath(str(path)) + _make_dir(path, acquire_del_lock=not user_provided_path) + + try: + # Timeout 0 means there will be only one attempt to acquire + # the file lock. If it cannot be aquired, a TimeoutError + # will be thrown. + with TempFileLock(f"{path}.lock", timeout=0): + pyarrow.fs.copy_files(self.path, path) + except TimeoutError: + # if the directory is already locked, then wait but do not do anything. + with TempFileLock(f"{path}.lock", timeout=-1): + pass + if not os.path.exists(path): + raise RuntimeError( + f"Checkpoint directory {path} does not exist, " + "even though it should have been created by " + "another process. Please raise an issue on GitHub: " + "https://github.com/ray-project/ray/issues" + ) + + return path + + @contextlib.contextmanager + def as_directory(self) -> Iterator[str]: + """Return checkpoint directory path in a context. + + This function makes checkpoint data available as a directory while avoiding + unnecessary copies and left-over temporary data. + + If the current path is local, it will return the existing path. If it is + not, it will create a temporary directory, + which will be deleted after the context is exited. + + Users should treat the returned checkpoint directory as read-only and avoid + changing any data within it, as it might get deleted when exiting the context. + + Example: + + .. code-block:: python + + with checkpoint.as_directory() as checkpoint_dir: + # Do some read-only processing of files within checkpoint_dir + pass + + # At this point, if a temporary directory was created, it will have + # been deleted. + + """ + import pyarrow + + if isinstance(self.path, pyarrow.fs.LocalFileSystem): + yield self._local_path + else: + temp_dir = self.to_directory() + del_lock_path = _get_del_lock_path(temp_dir) + yield temp_dir + + # Cleanup + try: + os.remove(del_lock_path) + except Exception: + logger.warning( + f"Could not remove {del_lock_path} deletion file lock. " + f"Traceback:\n{traceback.format_exc()}" + ) + + # In the edge case (process crash before del lock file is removed), + # we do not remove the directory at all. + # Since it's in /tmp, this is not that big of a deal. + # check if any lock files are remaining + temp_dir_base_name = Path(temp_dir).name + if not list( + Path(temp_dir).parent.glob(_get_del_lock_path(temp_dir_base_name, "*")) + ): + try: + # Timeout 0 means there will be only one attempt to acquire + # the file lock. If it cannot be aquired, a TimeoutError + # will be thrown. + with TempFileLock(f"{temp_dir}.lock", timeout=0): + shutil.rmtree(temp_dir, ignore_errors=True) + except TimeoutError: + pass + + def _get_temporary_checkpoint_dir(self) -> str: + """Return the name for the temporary checkpoint dir.""" + tmp_dir_path = tempfile.gettempdir() + checkpoint_dir_name = _CHECKPOINT_DIR_PREFIX + self._uuid.hex + if platform.system() == "Windows": + # Max path on Windows is 260 chars, -1 for joining \ + # Also leave a little for the del lock + del_lock_name = _get_del_lock_path("") + checkpoint_dir_name = ( + _CHECKPOINT_DIR_PREFIX + + self._uuid.hex[ + -259 + + len(_CHECKPOINT_DIR_PREFIX) + + len(tmp_dir_path) + + len(del_lock_name) : + ] + ) + if not checkpoint_dir_name.startswith(_CHECKPOINT_DIR_PREFIX): + raise RuntimeError( + "Couldn't create checkpoint directory due to length " + "constraints. Try specifing a shorter checkpoint path." + ) + return os.path.join(tmp_dir_path, checkpoint_dir_name) + + def __fspath__(self): + raise TypeError( + "You cannot use `Checkpoint` objects directly as paths. " + "Use `Checkpoint.to_directory()` or `Checkpoint.as_directory()` instead." + ) + + +def _make_dir(path: str, acquire_del_lock: bool = False) -> None: + """Create the temporary checkpoint dir in ``path``.""" + if acquire_del_lock: + # Each process drops a deletion lock file it then cleans up. + # If there are no lock files left, the last process + # will remove the entire directory. + del_lock_path = _get_del_lock_path(path) + open(del_lock_path, "a").close() + + os.makedirs(path, exist_ok=True) + + +def _get_del_lock_path(path: str, pid: str = None) -> str: + """Get the path to the deletion lock file.""" + pid = pid if pid is not None else os.getpid() + return f"{path}.del_lock_{pid}" From 64945befc88b5e0003966836194efab81b0c68fc Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 27 Jul 2023 17:12:50 -0700 Subject: [PATCH 010/108] New train session report implementation using new checkpoint Signed-off-by: Justin Yu --- python/ray/train/_internal/session.py | 54 +++++++++++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/python/ray/train/_internal/session.py b/python/ray/train/_internal/session.py index 2f5f8dc7c59d..3fadd5788ea0 100644 --- a/python/ray/train/_internal/session.py +++ b/python/ray/train/_internal/session.py @@ -37,6 +37,7 @@ TIME_TOTAL_S, LAZY_CHECKPOINT_MARKER_FILE, ) +from ray.train.checkpoint import Checkpoint as NewCheckpoint from ray.train.error import SessionMisuseError from ray.util.annotations import DeveloperAPI, PublicAPI @@ -435,6 +436,56 @@ def _set_checkpoint_uri(self, uri: str): """ self.legacy_checkpoint_uri = uri + def new_checkpoint(self, checkpoint: NewCheckpoint): + import pyarrow.fs + + assert isinstance(checkpoint, NewCheckpoint) + assert isinstance(checkpoint.filesystem, pyarrow.fs.LocalFileSystem) + + # Upload checkpoint files. + pyarrow.fs.copy_files( + source=checkpoint.path, + destination=self.storage.checkpoint_fs_path, + source_filesystem=checkpoint.filesystem, + destination_filesystem=self.storage.storage_filesystem, + ) + # Delete local checkpoint files. + checkpoint.filesystem.delete_dir(checkpoint.path) + + # Report the uploaded checkpoint location for internal book-keeping. + checkpoint_to_report = NewCheckpoint( + filesystem=self.storage.storage_filesystem, + path=self.storage.checkpoint_fs_path, + ) + + metadata = self._auto_fill_checkpoint_metrics({}) + + # Save the rank of the worker that created this checkpoint. + metadata.update({CHECKPOINT_RANK_KEY: self.world_rank}) + checkpoint_to_report.set_metadata(metadata) + + result = TrainingResult( + type=TrainingResultType.CHECKPOINT, + data=checkpoint_to_report, + ) + + # Add result to a thread-safe queue. + self.result_queue.put(result, block=True) + + # Acquire lock to stop the training thread until + # checkpoint has been processed. + self.continue_lock.acquire() + + def new_report( + self, metrics: Dict, checkpoint: Optional[NewCheckpoint] = None + ) -> None: + if checkpoint: + self.new_checkpoint(checkpoint) + + # TODO(justinvyu): Unify checkpoint / report logic to just report a single + # (metrics, Checkpoint) result for the consumer to handle. + self._report_legacy(**metrics) + def report(self, metrics: Dict, checkpoint: Optional[Checkpoint] = None) -> None: # TODO(xwjiang): tons of optimizations. @@ -452,6 +503,9 @@ def report(self, metrics: Dict, checkpoint: Optional[Checkpoint] = None) -> None "store your Torch objects." ) + if _use_storage_context(): + return self.new_report(metrics, checkpoint=checkpoint) + if checkpoint: self.checkpoint(checkpoint) self._report_legacy(**metrics) From c6480c9c44e9556c47a8ee7421e1e3cccc5489fd Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 27 Jul 2023 17:33:46 -0700 Subject: [PATCH 011/108] Simplify checkpoint propagation from user code (in worker) -> trainer -> driver Signed-off-by: Justin Yu --- python/ray/train/data_parallel_trainer.py | 13 ++++++-- python/ray/train/trainer.py | 36 +++++++++++++++++++---- 2 files changed, 42 insertions(+), 7 deletions(-) diff --git a/python/ray/train/data_parallel_trainer.py b/python/ray/train/data_parallel_trainer.py index 5879411987da..a43ec2bcf910 100644 --- a/python/ray/train/data_parallel_trainer.py +++ b/python/ray/train/data_parallel_trainer.py @@ -6,7 +6,7 @@ from ray._private.thirdparty.tabulate.tabulate import tabulate import ray -from ray import tune +from ray import train, tune from ray.air.checkpoint import Checkpoint from ray.air._internal.checkpointing import add_preprocessor_to_checkpoint from ray.air.config import DatasetConfig, RunConfig, ScalingConfig, CheckpointConfig @@ -17,6 +17,7 @@ from ray.train._internal.backend_executor import BackendExecutor, TrialInfo from ray.train._internal.checkpoint import TuneCheckpointManager from ray.train._internal.data_config import DataConfig, _LegacyDataConfigWrapper +from ray.train._internal.storage import _use_storage_context from ray.train._internal.utils import construct_train_func from ray.train.constants import TRAIN_DATASET_KEY, WILDCARD_KEY from ray.train.trainer import BaseTrainer, GenDataset @@ -427,7 +428,15 @@ def _report(self, training_iterator: TrainingIterator) -> None: for results in training_iterator: # TODO(ml-team): add ability to report results from multiple workers. first_worker_results = results[0] - tune.report(**first_worker_results) + if _use_storage_context(): + assert ( + isinstance(first_worker_results, tuple) + and len(first_worker_results) == 2 + ) + metrics, checkpoint = first_worker_results + train.report(metrics, checkpoint=checkpoint) + else: + tune.report(**first_worker_results) def training_loop(self) -> None: scaling_config = self._validate_scaling_config(self.scaling_config) diff --git a/python/ray/train/trainer.py b/python/ray/train/trainer.py index ab75b708cb3e..3811aae5f3b2 100644 --- a/python/ray/train/trainer.py +++ b/python/ray/train/trainer.py @@ -18,7 +18,8 @@ from ray.train._internal.checkpoint import ( CheckpointManager, ) -from ray.train._internal.session import TrainingResultType +from ray.train._internal.session import TrainingResult, TrainingResultType +from ray.train.checkpoint import Checkpoint as NewCheckpoint # Ray Train should be usable even if Tune is not installed. from ray.train._internal.utils import ActorWrapper @@ -65,6 +66,11 @@ def __init__( self._checkpoint_manager = checkpoint_manager self._checkpoint_strategy = checkpoint_strategy self._storage_path = storage_path + + # TODO(justinvyu): report/checkpoint should be combined into a single + # TrainingResult event. There's no need to do these one at a time. + self._checkpoint_to_report = None + self._start_training( train_func=train_func, run_dir=run_dir, @@ -191,6 +197,16 @@ def __next__(self): pass raise + def _process_checkpoint_results(self, checkpoint_results: List[TrainingResult]): + checkpoints = [ + checkpoint_result.data for checkpoint_result in checkpoint_results + ] + assert all(isinstance(checkpoint, NewCheckpoint) for checkpoint in checkpoints) + + # All we need to do is track which checkpoint to use for book-keeping. + # Let's use rank 0, and attach metadata about the other checkpoints. + self._checkpoint_to_report = checkpoints[0] + def _fetch_next_result(self) -> Optional[List[Dict]]: """Fetch next results produced by ``session.report()`` from each worker. @@ -210,12 +226,22 @@ def _fetch_next_result(self) -> Optional[List[Dict]]: first_result = results[0] result_type = first_result.type if result_type is TrainingResultType.REPORT: - result_data = [r.data for r in results] + if _use_storage_context(): + # TODO(justinvyu): This tuple is what TrainingResult should be. + result_data = [ + (r.data, None if rank > 0 else self._checkpoint_to_report) + for rank, r in enumerate(results) + ] + else: + result_data = [r.data for r in results] return result_data elif result_type is TrainingResultType.CHECKPOINT: - self._checkpoint_manager._process_checkpoints( - results, decode_checkpoint_fn=self._backend._decode_data - ) + if _use_storage_context(): + self._process_checkpoint_results(results) + else: + self._checkpoint_manager._process_checkpoints( + results, decode_checkpoint_fn=self._backend._decode_data + ) # Note(jungong) : This is kinda funky. We update the cloud # checkpoint dir on every distributed worker right after From c681ccb38bbe9db2eab101a0331b81b0bf1d591a Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 27 Jul 2023 17:43:06 -0700 Subject: [PATCH 012/108] New tune session.report Signed-off-by: Justin Yu --- .../ray/tune/trainable/function_trainable.py | 30 ++++++++++++++----- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/python/ray/tune/trainable/function_trainable.py b/python/ray/tune/trainable/function_trainable.py index 55615a62e16f..bcfc88602b56 100644 --- a/python/ray/tune/trainable/function_trainable.py +++ b/python/ray/tune/trainable/function_trainable.py @@ -250,20 +250,36 @@ def _start(self): self._last_report_time = time.time() def report(self, metrics: Dict, *, checkpoint: Optional[Checkpoint] = None) -> None: + from ray.train._internal.storage import _use_storage_context + # TODO(xwjiang): Tons of optimizations. self._air_session_has_reported = True - if checkpoint: - training_iteration = self._get_training_iteration() - checkpoint_dir = self.make_checkpoint_dir(step=training_iteration) - self.set_checkpoint(checkpoint_dir) - checkpoint.to_directory(checkpoint_dir) - # TODO(krfricke): Remove this once support is added in Checkpoint. - open(os.path.join(checkpoint_dir, ".is_checkpoint"), "a").close() + + # TODO(justinvyu): With a unified session, we'll still run into this doubled + # report problem. This should be fixed by checking if the checkpoint has been + # uploaded already (via some marker), then skipping the repeat upload. + if _use_storage_context(): + self._last_checkpoint = checkpoint + else: + if checkpoint: + training_iteration = self._get_training_iteration() + checkpoint_dir = self.make_checkpoint_dir(step=training_iteration) + self.set_checkpoint(checkpoint_dir) + checkpoint.to_directory(checkpoint_dir) + # TODO(krfricke): Remove this once support is added in Checkpoint. + open(os.path.join(checkpoint_dir, ".is_checkpoint"), "a").close() self.__call__(**metrics) @property def loaded_checkpoint(self) -> Optional[Checkpoint]: if self._last_checkpoint: + from ray.train._internal.storage import _use_storage_context + from ray.train.checkpoint import Checkpoint as NewCheckpoint + + if _use_storage_context(): + assert isinstance(self._last_checkpoint, NewCheckpoint) + return self._last_checkpoint + assert isinstance(self._last_checkpoint, str) return Checkpoint.from_directory(self._last_checkpoint) return None From 795bafe6afb55f8edcb89ec07e22f67f8f88747c Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 28 Jul 2023 00:57:00 -0700 Subject: [PATCH 013/108] Save direction works with new checkpoint API Signed-off-by: Justin Yu Fix lint Signed-off-by: Justin Yu --- python/ray/train/_internal/session.py | 16 ++++++++++- python/ray/train/checkpoint.py | 2 +- python/ray/train/data_parallel_trainer.py | 6 ++++ python/ray/train/trainer.py | 17 ++++++----- .../ray/tune/execution/checkpoint_manager.py | 12 ++++++++ python/ray/tune/execution/trial_runner.py | 28 +++++++++++++------ python/ray/tune/experiment/trial.py | 4 +-- .../ray/tune/trainable/function_trainable.py | 23 +++++++++++++-- python/ray/tune/trainable/trainable.py | 9 +++++- 9 files changed, 93 insertions(+), 24 deletions(-) diff --git a/python/ray/train/_internal/session.py b/python/ray/train/_internal/session.py index 3fadd5788ea0..b2cc6a4d430c 100644 --- a/python/ray/train/_internal/session.py +++ b/python/ray/train/_internal/session.py @@ -434,7 +434,11 @@ def _set_checkpoint_uri(self, uri: str): Args: uri: URI to the location where next checkpoint should be saved. """ - self.legacy_checkpoint_uri = uri + if _use_storage_context(): + # TODO(justinvyu): Fix up the name. + self.storage.current_checkpoint_id = uri + else: + self.legacy_checkpoint_uri = uri def new_checkpoint(self, checkpoint: NewCheckpoint): import pyarrow.fs @@ -443,6 +447,16 @@ def new_checkpoint(self, checkpoint: NewCheckpoint): assert isinstance(checkpoint.filesystem, pyarrow.fs.LocalFileSystem) # Upload checkpoint files. + print( + "Uploading checkpoint files to storage path:" + "\n{source}\n{destination}\n{source_fs}\n{dest_fs}".format( + source=checkpoint.path, + destination=self.storage.checkpoint_fs_path, + source_fs=checkpoint.filesystem, + dest_fs=self.storage.storage_filesystem, + ) + ) + self.storage.storage_filesystem.create_dir(self.storage.checkpoint_fs_path) pyarrow.fs.copy_files( source=checkpoint.path, destination=self.storage.checkpoint_fs_path, diff --git a/python/ray/train/checkpoint.py b/python/ray/train/checkpoint.py index 1774996f7525..20a48c5976c2 100644 --- a/python/ray/train/checkpoint.py +++ b/python/ray/train/checkpoint.py @@ -6,7 +6,7 @@ import tempfile import traceback from pathlib import Path -from typing import TYPE_CHECKING, Any, Dict, Iterator, Optional, Union +from typing import Any, Dict, Iterator, Optional, Union import pyarrow.fs diff --git a/python/ray/train/data_parallel_trainer.py b/python/ray/train/data_parallel_trainer.py index a43ec2bcf910..a59209477536 100644 --- a/python/ray/train/data_parallel_trainer.py +++ b/python/ray/train/data_parallel_trainer.py @@ -434,6 +434,12 @@ def _report(self, training_iterator: TrainingIterator) -> None: and len(first_worker_results) == 2 ) metrics, checkpoint = first_worker_results + print( + "Calling _StatusReporter.report with (metrics, checkpoint): ", + metrics, + "\n", + checkpoint, + ) train.report(metrics, checkpoint=checkpoint) else: tune.report(**first_worker_results) diff --git a/python/ray/train/trainer.py b/python/ray/train/trainer.py index 3811aae5f3b2..854f38e10ada 100644 --- a/python/ray/train/trainer.py +++ b/python/ray/train/trainer.py @@ -71,6 +71,10 @@ def __init__( # TrainingResult event. There's no need to do these one at a time. self._checkpoint_to_report = None + # TODO(justinvyu): Is this the best way to do this? Need to save this + # as part of checkpoint metadata and load it back on restore. + self._latest_checkpoint_id = 0 + self._start_training( train_func=train_func, run_dir=run_dir, @@ -120,17 +124,15 @@ def _send_next_checkpoint_path_to_workers(self): # NOTE: Idea: this checkpoint dir name should be customizable # and created on the fly when the checkpoint is reported with metrics. # Ex: lambda metrics: f"checkpoint_iter={metrics['training_iteration']}" - storage.current_checkpoint_id = ( - self._checkpoint_manager._latest_checkpoint_id - ) - logger.debug( - f"Setting next checkpoint path to: {storage.checkpoint_fs_path}" - ) + storage.current_checkpoint_id = self._latest_checkpoint_id + print(f"Setting next checkpoint path to: {storage.checkpoint_fs_path}") # TODO(justinvyu): This checkpoint_path is NOT a URI anymore. # It's just a path relative to the storage filesystem. # `session.report` needs to be updated to upload using pyarrow.fs.copy_files - self._backend_executor._set_checkpoint_uri(storage.checkpoint_fs_path) + self._backend_executor._set_checkpoint_uri(storage.current_checkpoint_id) + + self._latest_checkpoint_id += 1 elif self._checkpoint_strategy._checkpoint_upload_from_workers: self._backend_executor._set_checkpoint_uri( @@ -201,6 +203,7 @@ def _process_checkpoint_results(self, checkpoint_results: List[TrainingResult]): checkpoints = [ checkpoint_result.data for checkpoint_result in checkpoint_results ] + print("TrainingIterator processed checkpoints: ", checkpoints) assert all(isinstance(checkpoint, NewCheckpoint) for checkpoint in checkpoints) # All we need to do is track which checkpoint to use for book-keeping. diff --git a/python/ray/tune/execution/checkpoint_manager.py b/python/ray/tune/execution/checkpoint_manager.py index 20d62de5d49b..cdf3d991d45c 100644 --- a/python/ray/tune/execution/checkpoint_manager.py +++ b/python/ray/tune/execution/checkpoint_manager.py @@ -51,8 +51,20 @@ def handle_checkpoint(self, checkpoint: _TrackedCheckpoint): ) self._process_persistent_checkpoint(checkpoint) + def new_handle_checkpoint(self, checkpoint): + # TODO(justinvyu): Support checkpoint book-keeping for the new checkpoint API. + # (keeping track of the top k heap, getting the latest checkpoint, etc.) + print("Trial checkpoint manager handling the checkpoint:\n", checkpoint) + pass + def on_checkpoint(self, checkpoint: _TrackedCheckpoint): """Ray Tune's entry point to handle a checkpoint.""" + from ray.train._internal.storage import _use_storage_context + from ray.train.checkpoint import Checkpoint as NewCheckpoint + + if _use_storage_context() and isinstance(checkpoint, NewCheckpoint): + return self.new_handle_checkpoint(checkpoint) + # Todo (krfricke): Replace with handle_checkpoint. self.handle_checkpoint(checkpoint) diff --git a/python/ray/tune/execution/trial_runner.py b/python/ray/tune/execution/trial_runner.py index f13e060572ea..e0501e62984c 100644 --- a/python/ray/tune/execution/trial_runner.py +++ b/python/ray/tune/execution/trial_runner.py @@ -967,15 +967,27 @@ def _process_trial_save( """ logger.debug("Trial %s: Processing trial save.", trial) + from ray.train.checkpoint import Checkpoint as NewCheckpoint + try: - trial.saving_to.dir_or_data = checkpoint_value - self._callbacks.on_checkpoint( - iteration=self._iteration, - trials=self._trials, - trial=trial, - checkpoint=trial.saving_to, - ) - trial.on_checkpoint(trial.saving_to) + if _use_storage_context() and isinstance(checkpoint_value, NewCheckpoint): + self._callbacks.on_checkpoint( + iteration=self._iteration, + trials=self._trials, + trial=trial, + checkpoint=checkpoint_value, + ) + trial.on_checkpoint(checkpoint_value) + else: + trial.saving_to.dir_or_data = checkpoint_value + self._callbacks.on_checkpoint( + iteration=self._iteration, + trials=self._trials, + trial=trial, + checkpoint=trial.saving_to, + ) + trial.on_checkpoint(trial.saving_to) + self._checkpoint_manager.on_trial_checkpoint(trial) if trial.checkpoint.storage_mode != CheckpointStorage.MEMORY: self._mark_trial_to_checkpoint(trial) diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index 7b200485f152..b502022a6064 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -421,9 +421,7 @@ def __init__( # Set remote experiment path if upload_dir is set if self.sync_config.upload_dir: if remote_experiment_path: - if not remote_experiment_path.startswith( - self.sync_config.upload_dir - ): + if not remote_experiment_path.startswith(self.sync_config.upload_dir): raise ValueError( f"Both a `SyncConfig.upload_dir` and an `experiment_path` " f"pointing to remote storage were passed, but they do not " diff --git a/python/ray/tune/trainable/function_trainable.py b/python/ray/tune/trainable/function_trainable.py index bcfc88602b56..e8b1bf3ddb01 100644 --- a/python/ray/tune/trainable/function_trainable.py +++ b/python/ray/tune/trainable/function_trainable.py @@ -251,6 +251,7 @@ def _start(self): def report(self, metrics: Dict, *, checkpoint: Optional[Checkpoint] = None) -> None: from ray.train._internal.storage import _use_storage_context + from ray.train.checkpoint import Checkpoint as NewCheckpoint # TODO(xwjiang): Tons of optimizations. self._air_session_has_reported = True @@ -258,7 +259,12 @@ def report(self, metrics: Dict, *, checkpoint: Optional[Checkpoint] = None) -> N # TODO(justinvyu): With a unified session, we'll still run into this doubled # report problem. This should be fixed by checking if the checkpoint has been # uploaded already (via some marker), then skipping the repeat upload. - if _use_storage_context(): + if _use_storage_context() and isinstance(checkpoint, NewCheckpoint): + print( + "_StatusReporter.report called with checkpoint:\n", + checkpoint, + ) + self._fresh_checkpoint = True self._last_checkpoint = checkpoint else: if checkpoint: @@ -276,8 +282,9 @@ def loaded_checkpoint(self) -> Optional[Checkpoint]: from ray.train._internal.storage import _use_storage_context from ray.train.checkpoint import Checkpoint as NewCheckpoint - if _use_storage_context(): - assert isinstance(self._last_checkpoint, NewCheckpoint) + if _use_storage_context() and isinstance( + self._last_checkpoint, NewCheckpoint + ): return self._last_checkpoint assert isinstance(self._last_checkpoint, str) @@ -481,6 +488,16 @@ def save_checkpoint(self, checkpoint_dir: str = ""): checkpoint = self._status_reporter.get_checkpoint() + from ray.train._internal.storage import _use_storage_context + from ray.train.checkpoint import Checkpoint as NewCheckpoint + + if _use_storage_context() and isinstance(checkpoint, NewCheckpoint): + print( + "Returning checkpoint from FunctionTrainable.save_checkpoint:\n", + checkpoint, + ) + return checkpoint + if not checkpoint: # We drop a marker here to indicate that the checkpoint is empty checkpoint = FuncCheckpointUtil.mk_null_checkpoint_dir(self.logdir) diff --git a/python/ray/tune/trainable/trainable.py b/python/ray/tune/trainable/trainable.py index 6f2d947de865..25ea02d19da4 100644 --- a/python/ray/tune/trainable/trainable.py +++ b/python/ray/tune/trainable/trainable.py @@ -501,6 +501,13 @@ def save( # User saves checkpoint checkpoint_dict_or_path = self.save_checkpoint(checkpoint_dir) + from ray.train.checkpoint import Checkpoint as NewCheckpoint + + if _use_storage_context() and isinstance( + checkpoint_dict_or_path, NewCheckpoint + ): + return checkpoint_dict_or_path + if checkpoint_dict_or_path is None: # checkpoint_dict_or_path can only be None in class trainables. # In that case the default is to use the root checkpoint directory. @@ -510,7 +517,7 @@ def save( # checkpoint_dir is only None in function trainables. In that case, # checkpoint_dict_or_path points to the already saved checkpoint dir. # This will be considered the root dir. - assert isinstance(checkpoint_dict_or_path, str) + assert isinstance(checkpoint_dict_or_path, str), checkpoint_dict_or_path checkpoint_dir = checkpoint_dict_or_path # Get trainable metadata From 8a084bc7dee720a8ba83b59c681d181fffa052c5 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 28 Jul 2023 10:10:13 -0700 Subject: [PATCH 014/108] Update test with e2e trainer test Signed-off-by: Justin Yu Fix lint Signed-off-by: Justin Yu --- .../ray/train/tests/test_new_persistence.py | 113 ++++++++++++++---- 1 file changed, 90 insertions(+), 23 deletions(-) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index e468e9de24a3..1f1573e16d21 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -7,9 +7,10 @@ import pyarrow.fs -from ray import air, train, tune +from ray import train, tune from ray.air.tests.test_checkpoints import mock_s3_bucket_uri from ray.train._internal.storage import _download_from_fs_path +from ray.train.checkpoint import Checkpoint as NewCheckpoint from ray.train.data_parallel_trainer import DataParallelTrainer @@ -26,6 +27,17 @@ def enable_new_persistence_mode(monkeypatch): def train_fn(config): + in_trainer = config.get("in_trainer", False) + if in_trainer: + from ray.air._internal.session import _get_session + from ray.train._internal.session import _TrainSession + + train_session = _get_session() + + assert isinstance(train_session, _TrainSession) + assert train_session.storage + assert train_session.storage.checkpoint_fs_path + start = 0 checkpoint = train.get_context().get_checkpoint() @@ -37,16 +49,23 @@ def train_fn(config): start = state["iter"] + 1 for i in range(start, config.get("num_iterations", 5)): - time.sleep(0.5) + time.sleep(0.25) - with open(f"artifact-{i}.txt", "w") as f: + checkpoint_file_name = "dummy.pkl" + artifact_file_name = f"artifact-{i}.txt" + if in_trainer: + rank = train.get_context().get_world_rank() + checkpoint_file_name = f"dummy-{rank}.pkl" + artifact_file_name = f"artifact-{i}-{rank}.txt" + + with open(artifact_file_name, "w") as f: f.write(f"{i}") temp_dir = tempfile.mkdtemp() - with open(os.path.join(temp_dir, "dummy.pkl"), "wb") as f: + with open(os.path.join(temp_dir, checkpoint_file_name), "wb") as f: pickle.dump({"iter": i}, f) - train.report({"iter": i}, checkpoint=air.Checkpoint.from_directory(temp_dir)) + train.report({"iter": i}, checkpoint=NewCheckpoint.from_directory(temp_dir)) @pytest.mark.parametrize("storage_path_type", [None, "nfs", "cloud", "custom_fs"]) @@ -151,32 +170,80 @@ def test_tuner(monkeypatch, storage_path_type, tmp_path): assert len(list(exp_dir.glob("tuner.pkl"))) == 1 -def test_trainer(tmp_path): - """For now, this is just a dummy test to inspect that the storage context - has been passed to the train workers properly.""" - storage_path = str(tmp_path / "fake_nfs") - - def dummy_train_fn(config): - from ray.air._internal.session import _get_session - from ray.train._internal.session import _TrainSession +@pytest.mark.parametrize("storage_path_type", [None]) +def test_trainer(tmp_path, monkeypatch, storage_path_type): + """ + TODO(justinvyu): Test for these once implemented: + - artifacts + - restoration + - accessing checkpoints from Result object + - top k checkpoints (num_to_keep) + + trainer_new_persistence + ├── experiment_state-2023-07-28_10-00-38.json + ├── basic-variant-state-2023-07-28_10-00-38.json + ├── trainer.pkl + ├── tuner.pkl + └── DataParallelTrainer_46367_00000_0_... + ├── events.out.tfevents... + ├── params.json + ├── params.pkl + ├── progress.csv + ├── result.json + ├── checkpoint_000000 + │ ├── dummy-0.pkl + │ └── dummy-1.pkl + ├── ... + ├── rank_0 <- TODO: remove these rank folders? + │ ├── artifact-0-0.txt + │ ├── artifact-1-0.txt + │ └── ... + └── rank_1 + ├── artifact-0-1.txt + ├── artifact-1-1.txt + └── ... + """ - train_session = _get_session() - print(train_session.storage) + LOCAL_CACHE_DIR = tmp_path / "ray_results" + monkeypatch.setenv("RAY_AIR_LOCAL_CACHE_DIR", str(LOCAL_CACHE_DIR)) + exp_name = "trainer_new_persistence" - assert isinstance(train_session, _TrainSession) - assert train_session.storage - assert train_session.checkpoint_uri.startswith(storage_path) + if storage_path_type is None: + storage_path = None + elif storage_path_type == "nfs": + storage_path = str(tmp_path / "fake_nfs") trainer = DataParallelTrainer( - dummy_train_fn, + train_fn, + train_loop_config={"in_trainer": True}, scaling_config=train.ScalingConfig(num_workers=2), - run_config=train.RunConfig( - storage_path=storage_path, - name="trainer_new_persistence", - ), + run_config=train.RunConfig(storage_path=storage_path, name=exp_name, verbose=0), ) trainer.fit() + local_inspect_dir = tmp_path / "inspect" + if storage_path: + # if storage_filesystem: + # fs, fs_path = storage_filesystem, storage_path + # else: + fs, fs_path = pyarrow.fs.FileSystem.from_uri(storage_path) + _download_from_fs_path( + fs=fs, fs_path=fs_path, local_path=str(local_inspect_dir) + ) + else: + local_inspect_dir = LOCAL_CACHE_DIR + + assert len(list(local_inspect_dir.glob("*"))) == 1 # Only expect 1 experiment dir + exp_dir = local_inspect_dir / exp_name + + # Files synced by the driver + assert len(list(exp_dir.glob("basic-variant-state-*"))) == 1 + assert len(list(exp_dir.glob("experiment_state-*"))) == 1 + assert len(list(exp_dir.glob("tuner.pkl"))) == 1 + assert len(list(exp_dir.glob("trainer.pkl"))) == 1 + + # Files synced by the worker + if __name__ == "__main__": import sys From 725d802fec6a296d3dd40426fcc0d78cd1c2cab2 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 28 Jul 2023 10:11:12 -0700 Subject: [PATCH 015/108] Make callback supporting new checkpoint a todo for now Signed-off-by: Justin Yu --- python/ray/tune/execution/trial_runner.py | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/python/ray/tune/execution/trial_runner.py b/python/ray/tune/execution/trial_runner.py index e0501e62984c..7f52cd49be7a 100644 --- a/python/ray/tune/execution/trial_runner.py +++ b/python/ray/tune/execution/trial_runner.py @@ -971,12 +971,8 @@ def _process_trial_save( try: if _use_storage_context() and isinstance(checkpoint_value, NewCheckpoint): - self._callbacks.on_checkpoint( - iteration=self._iteration, - trials=self._trials, - trial=trial, - checkpoint=checkpoint_value, - ) + # TODO(justinvyu): Update callbacks to take in a new Checkpoint + # rather than a _TrackedCheckpoint, then call the on_checkpoint hook. trial.on_checkpoint(checkpoint_value) else: trial.saving_to.dir_or_data = checkpoint_value From 877acb9fcbcb5b784a620e000d369c8c8373d724 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 28 Jul 2023 10:11:16 -0700 Subject: [PATCH 016/108] Remove unnecessary comment Signed-off-by: Justin Yu --- python/ray/train/trainer.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/python/ray/train/trainer.py b/python/ray/train/trainer.py index 854f38e10ada..7f88657047d8 100644 --- a/python/ray/train/trainer.py +++ b/python/ray/train/trainer.py @@ -127,9 +127,6 @@ def _send_next_checkpoint_path_to_workers(self): storage.current_checkpoint_id = self._latest_checkpoint_id print(f"Setting next checkpoint path to: {storage.checkpoint_fs_path}") - # TODO(justinvyu): This checkpoint_path is NOT a URI anymore. - # It's just a path relative to the storage filesystem. - # `session.report` needs to be updated to upload using pyarrow.fs.copy_files self._backend_executor._set_checkpoint_uri(storage.current_checkpoint_id) self._latest_checkpoint_id += 1 From 88042b33fc23405fac3acf122e5f3702c820b4e3 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 28 Jul 2023 13:53:10 -0700 Subject: [PATCH 017/108] Separate out the new set checkpoint id from the old set checkpoint uri Signed-off-by: Justin Yu --- python/ray/train/_internal/backend_executor.py | 18 +++++++++++++++--- python/ray/train/_internal/session.py | 8 ++------ python/ray/train/trainer.py | 4 ++-- 3 files changed, 19 insertions(+), 11 deletions(-) diff --git a/python/ray/train/_internal/backend_executor.py b/python/ray/train/_internal/backend_executor.py index 58a4076c5a8d..fd6e7da09e76 100644 --- a/python/ray/train/_internal/backend_executor.py +++ b/python/ray/train/_internal/backend_executor.py @@ -518,12 +518,24 @@ def get_next(): return results - def _set_checkpoint_uri(self, uri: str): + def _set_checkpoint_id(self, checkpoint_id: int): + """Update the checkpoint id in the StorageContext of all workers. + + This determines the path that the next checkpoint will be saved to.""" + + def set_checkpoint_id(): + session = _get_session("_set_checkpoint_id") + session.storage.current_checkpoint_id = checkpoint_id + + futures = self.worker_group.execute_async(set_checkpoint_id) + self.get_with_failure_handling(futures) + + def _set_legacy_checkpoint_uri(self, uri: str): """Tell remote sessions where to upload the chekcpoint.""" def set_uri(): - session = _get_session("_set_checkpoint_uri") - session._set_checkpoint_uri(uri) + session = _get_session("_set_legacy_checkpoint_uri") + session._set_legacy_checkpoint_uri(uri) futures = self.worker_group.execute_async(set_uri) self.get_with_failure_handling(futures) diff --git a/python/ray/train/_internal/session.py b/python/ray/train/_internal/session.py index b2cc6a4d430c..115c813449b3 100644 --- a/python/ray/train/_internal/session.py +++ b/python/ray/train/_internal/session.py @@ -428,17 +428,13 @@ def checkpoint(self, checkpoint: Checkpoint): # checkpoint has been processed. self.continue_lock.acquire() - def _set_checkpoint_uri(self, uri: str): + def _set_legacy_checkpoint_uri(self, uri: str): """Tell session where to save the next directory checkpoint on the cloud. Args: uri: URI to the location where next checkpoint should be saved. """ - if _use_storage_context(): - # TODO(justinvyu): Fix up the name. - self.storage.current_checkpoint_id = uri - else: - self.legacy_checkpoint_uri = uri + self.legacy_checkpoint_uri = uri def new_checkpoint(self, checkpoint: NewCheckpoint): import pyarrow.fs diff --git a/python/ray/train/trainer.py b/python/ray/train/trainer.py index 7f88657047d8..691eeaf40009 100644 --- a/python/ray/train/trainer.py +++ b/python/ray/train/trainer.py @@ -127,12 +127,12 @@ def _send_next_checkpoint_path_to_workers(self): storage.current_checkpoint_id = self._latest_checkpoint_id print(f"Setting next checkpoint path to: {storage.checkpoint_fs_path}") - self._backend_executor._set_checkpoint_uri(storage.current_checkpoint_id) + self._backend_executor._set_checkpoint_id(storage.current_checkpoint_id) self._latest_checkpoint_id += 1 elif self._checkpoint_strategy._checkpoint_upload_from_workers: - self._backend_executor._set_checkpoint_uri( + self._backend_executor._set_legacy_checkpoint_uri( self.__get_cloud_checkpoint_dir() ) From a6cd9dc913026d869331b009045dc2c96dd9d97d Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 31 Jul 2023 12:10:14 -0700 Subject: [PATCH 018/108] Update id -> index Signed-off-by: Justin Yu --- python/ray/train/trainer.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/ray/train/trainer.py b/python/ray/train/trainer.py index 45af370b4839..07b033041b97 100644 --- a/python/ray/train/trainer.py +++ b/python/ray/train/trainer.py @@ -73,7 +73,7 @@ def __init__( # TODO(justinvyu): Is this the best way to do this? Need to save this # as part of checkpoint metadata and load it back on restore. - self._latest_checkpoint_id = 0 + self._latest_checkpoint_index = 0 self._start_training( train_func=train_func, @@ -124,14 +124,14 @@ def _send_next_checkpoint_path_to_workers(self): # NOTE: Idea: this checkpoint dir name should be customizable # and created on the fly when the checkpoint is reported with metrics. # Ex: lambda metrics: f"checkpoint_iter={metrics['training_iteration']}" - storage.current_checkpoint_index = self._latest_checkpoint_id + storage.current_checkpoint_index = self._latest_checkpoint_index print(f"Setting next checkpoint path to: {storage.checkpoint_fs_path}") self._backend_executor._set_checkpoint_index( storage.current_checkpoint_index ) - self._latest_checkpoint_id += 1 + self._latest_checkpoint_index += 1 elif self._checkpoint_strategy._checkpoint_upload_from_workers: self._backend_executor._set_legacy_checkpoint_uri( From 01f34bb02cc4e61c7edf47d035e94dcffa7d5256 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 31 Jul 2023 12:59:35 -0700 Subject: [PATCH 019/108] Address comments on error to raise with old ckpt type Signed-off-by: Justin Yu --- python/ray/train/_internal/session.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/python/ray/train/_internal/session.py b/python/ray/train/_internal/session.py index 115c813449b3..46261c9f49a1 100644 --- a/python/ray/train/_internal/session.py +++ b/python/ray/train/_internal/session.py @@ -439,8 +439,11 @@ def _set_legacy_checkpoint_uri(self, uri: str): def new_checkpoint(self, checkpoint: NewCheckpoint): import pyarrow.fs - assert isinstance(checkpoint, NewCheckpoint) - assert isinstance(checkpoint.filesystem, pyarrow.fs.LocalFileSystem) + if not isinstance(checkpoint, NewCheckpoint): + raise ValueError( + "You must pass a `ray.train.checkpoint.Checkpoint` " + "object to `train.report`. `ray.air.Checkpoint` is deprecated." + ) # Upload checkpoint files. print( From 65e7a27fb95bee848a81bc047d7759a1ac871dde Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 31 Jul 2023 13:22:17 -0700 Subject: [PATCH 020/108] Move checkpoint upload logic to a helper fn of storage ctx Signed-off-by: Justin Yu Fix lint for session.py Signed-off-by: Justin Yu --- python/ray/train/_internal/session.py | 33 +++---------------- python/ray/train/_internal/storage.py | 47 +++++++++++++++++++++++++++ 2 files changed, 52 insertions(+), 28 deletions(-) diff --git a/python/ray/train/_internal/session.py b/python/ray/train/_internal/session.py index 46261c9f49a1..bd6589d5ddf6 100644 --- a/python/ray/train/_internal/session.py +++ b/python/ray/train/_internal/session.py @@ -437,49 +437,26 @@ def _set_legacy_checkpoint_uri(self, uri: str): self.legacy_checkpoint_uri = uri def new_checkpoint(self, checkpoint: NewCheckpoint): - import pyarrow.fs - if not isinstance(checkpoint, NewCheckpoint): raise ValueError( "You must pass a `ray.train.checkpoint.Checkpoint` " "object to `train.report`. `ray.air.Checkpoint` is deprecated." ) - # Upload checkpoint files. - print( - "Uploading checkpoint files to storage path:" - "\n{source}\n{destination}\n{source_fs}\n{dest_fs}".format( - source=checkpoint.path, - destination=self.storage.checkpoint_fs_path, - source_fs=checkpoint.filesystem, - dest_fs=self.storage.storage_filesystem, - ) - ) - self.storage.storage_filesystem.create_dir(self.storage.checkpoint_fs_path) - pyarrow.fs.copy_files( - source=checkpoint.path, - destination=self.storage.checkpoint_fs_path, - source_filesystem=checkpoint.filesystem, - destination_filesystem=self.storage.storage_filesystem, - ) - # Delete local checkpoint files. - checkpoint.filesystem.delete_dir(checkpoint.path) + # Persist the reported checkpoint files to storage. + persisted_checkpoint = self.storage.persist_current_checkpoint(checkpoint) - # Report the uploaded checkpoint location for internal book-keeping. - checkpoint_to_report = NewCheckpoint( - filesystem=self.storage.storage_filesystem, - path=self.storage.checkpoint_fs_path, - ) + self.loaded_checkpoint = persisted_checkpoint metadata = self._auto_fill_checkpoint_metrics({}) # Save the rank of the worker that created this checkpoint. metadata.update({CHECKPOINT_RANK_KEY: self.world_rank}) - checkpoint_to_report.set_metadata(metadata) + persisted_checkpoint.set_metadata(metadata) result = TrainingResult( type=TrainingResultType.CHECKPOINT, - data=checkpoint_to_report, + data=persisted_checkpoint, ) # Add result to a thread-safe queue. diff --git a/python/ray/train/_internal/storage.py b/python/ray/train/_internal/storage.py index 9b50c3173892..2982463fdc7c 100644 --- a/python/ray/train/_internal/storage.py +++ b/python/ray/train/_internal/storage.py @@ -27,6 +27,7 @@ from ray.air._internal.filelock import TempFileLock from ray.air._internal.uri_utils import URI, is_uri +from ray.train.checkpoint import Checkpoint from ray.tune.syncer import Syncer, SyncConfig, _BackgroundSyncer from ray.tune.result import _get_defaults_results_dir @@ -458,6 +459,52 @@ def _check_validation_file(self): "to the configured storage path." ) + def persist_current_checkpoint(self, checkpoint: Checkpoint) -> Checkpoint: + """Persists a given checkpoint to the current checkpoint path on the filesystem. + + "Current" is defined by the `current_checkpoint_index` attribute of the + storage context. + + This method copies the checkpoint files to the storage location, + drops a marker at the storage path to indicate that the checkpoint + is completely uploaded, then deletes the original checkpoint directory. + For example, the original directory is typically a local temp directory. + + Args: + checkpoint: The checkpoint to persist to (fs, checkpoint_fs_path). + + Returns: + Checkpoint: A Checkpoint pointing to the persisted checkpoint location. + """ + logger.debug( + "Copying checkpoint files to storage path:\n" + "({source_fs}, {source}) -> ({dest_fs}, {destination})".format( + source=checkpoint.path, + destination=self.checkpoint_fs_path, + source_fs=checkpoint.filesystem, + dest_fs=self.storage_filesystem, + ) + ) + self.storage_filesystem.create_dir(self.checkpoint_fs_path) + _pyarrow_fs_copy_files( + source=checkpoint.path, + destination=self.checkpoint_fs_path, + source_filesystem=checkpoint.filesystem, + destination_filesystem=self.storage_filesystem, + ) + + # Delete local checkpoint files. + # TODO(justinvyu): What if checkpoint.path == self.checkpoint_fs_path? + # TODO(justinvyu): What if users don't want to delete the local checkpoint? + checkpoint.filesystem.delete_dir(checkpoint.path) + + uploaded_checkpoint = Checkpoint( + filesystem=self.storage_filesystem, + path=self.checkpoint_fs_path, + ) + logger.debug(f"Checkpoint successfully created at: {uploaded_checkpoint}") + return uploaded_checkpoint + @property def experiment_path(self) -> str: """The path the experiment directory, where the format matches the From f2a4c368b4a984c02c1bd32fc8c18b5b5217597d Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 31 Jul 2023 14:36:37 -0700 Subject: [PATCH 021/108] Drop a checkpoint marker after uploading Signed-off-by: Justin Yu Fix lint for storage.py Signed-off-by: Justin Yu --- python/ray/train/_internal/storage.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/python/ray/train/_internal/storage.py b/python/ray/train/_internal/storage.py index 2982463fdc7c..850b41a1b425 100644 --- a/python/ray/train/_internal/storage.py +++ b/python/ray/train/_internal/storage.py @@ -493,6 +493,12 @@ def persist_current_checkpoint(self, checkpoint: Checkpoint) -> Checkpoint: destination_filesystem=self.storage_filesystem, ) + # Drop a marker file to indicate that the checkpoint is completely uploaded. + # TODO(justinvyu): This may make more sense to do in session.report + uploaded_marker_file = os.path.join(self.checkpoint_fs_path, ".is_checkpoint") + with self.storage_filesystem.open_output_stream(uploaded_marker_file): + pass + # Delete local checkpoint files. # TODO(justinvyu): What if checkpoint.path == self.checkpoint_fs_path? # TODO(justinvyu): What if users don't want to delete the local checkpoint? From 49ee1262008fb8a54264e17e21c9a07f1ce331cc Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 31 Jul 2023 17:14:01 -0700 Subject: [PATCH 022/108] Add a simplified checkpoint manager Signed-off-by: Justin Yu --- .../ray/train/_internal/checkpoint_manager.py | 186 ++++++++++++++++++ 1 file changed, 186 insertions(+) create mode 100644 python/ray/train/_internal/checkpoint_manager.py diff --git a/python/ray/train/_internal/checkpoint_manager.py b/python/ray/train/_internal/checkpoint_manager.py new file mode 100644 index 000000000000..41b7794600a2 --- /dev/null +++ b/python/ray/train/_internal/checkpoint_manager.py @@ -0,0 +1,186 @@ +from dataclasses import dataclass +import heapq +import logging +import numbers +from typing import Any, Dict, List, Optional, Set, Tuple + +from ray._private.dict import flatten_dict +from ray.air.config import MAX +from ray.air._internal.util import is_nan +from ray.train import CheckpointConfig +from ray.train._internal.storage import _delete_fs_path +from ray.train.checkpoint import Checkpoint + + +logger = logging.getLogger(__name__) + + +@dataclass +class _TrackedCheckpoint: + """Checkpoint tracked by a checkpoint manager.""" + + checkpoint: Checkpoint + metrics: Dict[str, Any] + index: int = -1 + + +class _HeapCheckpointWrapper: + def __init__(self, priority: Any, tracked_checkpoint: _TrackedCheckpoint): + # Ex: 2 wrapped checkpoints, with checkpoint_score_order = MAX + # Priority of ckpt 1: (not_is_nan=True, metric=5, id=0) + # Priority of ckpt 2: (not_is_nan=True, metric=6, id=1) + # In this case, the min-heap should pop ckpt 1 first, since + # the metric is smaller. + self.priority = priority + self.tracked_checkpoint = tracked_checkpoint + + def __lt__(self, other): + return self.priority < other.priority + + def __repr__(self) -> str: + return ( + f"_HeapCheckpointWrapper(\n" + " priority={self.priority}, " + f"tracked_checkpoint={repr(self.tracked_checkpoint)}\n)" + ) + + +class _CheckpointManager: + """Checkpoint manager that handles checkpoint book-keeping for a trial. + + Args: + checkpoint_config: Defines how many and which checkpoints to keep. + latest_checkpoint_id: First checkpoint ID to use (e.g., in case we + resume an existing experiment). + """ + + def __init__( + self, + checkpoint_config: Optional[CheckpointConfig], + latest_checkpoint_index: int = 0, + ): + self._checkpoint_config = checkpoint_config or CheckpointConfig() + + # Incremental unique checkpoint ID of this run. + self._latest_checkpoint_index = latest_checkpoint_index + + # A heap that tracks the top K checkpoints. + self._top_k_checkpoints: List[_TrackedCheckpoint] = [] + + # The most recently registered checkpoint. + # The latest checkpoint should never be deleted immediately, + # even if it's the worst checkpoint. + self._latest_checkpoint: Optional[_TrackedCheckpoint] = None + + # Deletion of some checkpoints should be deferred. Specifically, if the + # latest persisted checkpoint should be deleted, we will only delete it + # once a new checkpoint came in (so that `_latest_checkpoint` is available). + self._checkpoints_to_clean_up: Set[_TrackedCheckpoint] = set() + + def register_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): + """Register new checkpoint and add to bookkeeping. + + This method will register a new checkpoint and add it to the internal + bookkeeping logic. This means the checkpoint manager will decide if + this checkpoint should be kept, and if older or worse performing + checkpoints should be deleted. + + Args: + checkpoints: Tracked checkpoint object to add to bookkeeping. + """ + # Mark the index of the checkpoint, for use in determining the + # order in which checkpoints were registered. + # TODO(justinvyu): Should this be unified with the checkpoint index + # tracked in the session / used in the checkpoint dirname? + tracked_checkpoint.index = self._latest_checkpoint_index + self._latest_checkpoint = tracked_checkpoint + + # Add to heap. + checkpoint_priority = self._get_checkpoint_score(tracked_checkpoint) + wrapped_checkpoint = _HeapCheckpointWrapper( + checkpoint_priority, tracked_checkpoint + ) + heapq.heappush(self._top_k_checkpoints, wrapped_checkpoint) + + # Delete the worst checkpoint if we're over the `num_to_keep` limit. + if ( + self._checkpoint_config.num_to_keep is not None + and len(self._top_k_checkpoints) > self._checkpoint_config.num_to_keep + ): + worst_checkpoint = heapq.heappop(self._top_k_checkpoints) + self._maybe_delete_checkpoint(worst_checkpoint.tracked_checkpoint) + + # Delete any checkpoints that were deferred. + self._cleanup_checkpoints() + + self._latest_checkpoint_id += 1 + + def _get_checkpoint_score( + self, checkpoint: _TrackedCheckpoint + ) -> Tuple[bool, numbers.Number, int]: + """Get scoring tuple for a checkpoint, according to checkpoint strategy. + + We sort checkpoints by this score. Checkpoints with a higher score are kept. + To achieve the desired ordering, we return a tuple of + (is_not_na: bool, metric: Number, index: int). + + The first index means that checkpoints that are NaN are rated worst. + The second index sorts the checkpoints by metric value. The third index + sorts checkpoints with the same metric value by their index - more recent + checkpoints are rated higher. + """ + checkpoint_score_attribute = self._checkpoint_config.checkpoint_score_attribute + if checkpoint_score_attribute: + flat_metrics = flatten_dict(checkpoint.metrics) + try: + checkpoint_result = flat_metrics[checkpoint_score_attribute] + except KeyError: + valid_keys = list(flat_metrics.keys()) + logger.error( + f"Result dict has no key: {checkpoint_score_attribute}. " + f"checkpoint_score_attr must be set to a key in the " + f"result dict. Valid keys are: {valid_keys}" + ) + checkpoint_result = float("-inf") + else: + checkpoint_result = float("-inf") + + checkpoint_score_order = self._checkpoint_config.checkpoint_score_order + order_factor = 1.0 if checkpoint_score_order == MAX else -1.0 + + checkpoint_score = order_factor * checkpoint_result + + if not isinstance(checkpoint_score, numbers.Number): + raise ValueError( + f"Unable to persist checkpoint for " + f"checkpoint_score_attribute: " + f"{checkpoint_score_attribute} with value " + f"{checkpoint_score}. " + f"This attribute must be numerical." + ) + + return ( + (not is_nan(checkpoint_score), checkpoint_score, checkpoint.index) + if not is_nan(checkpoint_score) + else (False, float("-inf"), checkpoint.index) + ) + + def _maybe_delete_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): + """Delete a checkpoint as long as it's not the most recent one.""" + if tracked_checkpoint == self._latest_persisted_checkpoint: + self._checkpoints_to_clean_up.add(tracked_checkpoint) + else: + self._delete_checkpoint(tracked_checkpoint) + + def _delete_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): + checkpoint = tracked_checkpoint.checkpoint + _delete_fs_path(fs=checkpoint.filesystem, fs_path=checkpoint.path) + self._checkpoints_to_clean_up.discard(tracked_checkpoint) + + def _cleanup_checkpoints(self): + """Delete any checkpoints that were deferred for deletion.""" + for tracked_checkpoint in list(self._checkpoints_to_clean_up): + self._maybe_delete_checkpoint(tracked_checkpoint) + + def __del__(self): + self._cleanup_checkpoints() From ffa0dd4b00781dc1e16a51d3b115a557bfc6f4f9 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 31 Jul 2023 17:59:33 -0700 Subject: [PATCH 023/108] Fixes to checkpoint manager Signed-off-by: Justin Yu --- .../ray/train/_internal/checkpoint_manager.py | 52 +++++++++++++++---- 1 file changed, 42 insertions(+), 10 deletions(-) diff --git a/python/ray/train/_internal/checkpoint_manager.py b/python/ray/train/_internal/checkpoint_manager.py index 41b7794600a2..f80f994ddc5b 100644 --- a/python/ray/train/_internal/checkpoint_manager.py +++ b/python/ray/train/_internal/checkpoint_manager.py @@ -1,4 +1,3 @@ -from dataclasses import dataclass import heapq import logging import numbers @@ -15,13 +14,15 @@ logger = logging.getLogger(__name__) -@dataclass class _TrackedCheckpoint: """Checkpoint tracked by a checkpoint manager.""" - checkpoint: Checkpoint - metrics: Dict[str, Any] - index: int = -1 + def __init__( + self, checkpoint: Checkpoint, metrics: Dict[str, Any], index: int = -1 + ): + self.checkpoint = checkpoint + self.metrics = metrics + self.index = index class _HeapCheckpointWrapper: @@ -34,7 +35,7 @@ def __init__(self, priority: Any, tracked_checkpoint: _TrackedCheckpoint): self.priority = priority self.tracked_checkpoint = tracked_checkpoint - def __lt__(self, other): + def __lt__(self, other: "_HeapCheckpointWrapper") -> bool: return self.priority < other.priority def __repr__(self) -> str: @@ -50,7 +51,7 @@ class _CheckpointManager: Args: checkpoint_config: Defines how many and which checkpoints to keep. - latest_checkpoint_id: First checkpoint ID to use (e.g., in case we + latest_checkpoint_index: First checkpoint ID to use (e.g., in case we resume an existing experiment). """ @@ -65,13 +66,18 @@ def __init__( self._latest_checkpoint_index = latest_checkpoint_index # A heap that tracks the top K checkpoints. - self._top_k_checkpoints: List[_TrackedCheckpoint] = [] + self._top_k_checkpoints: List[_HeapCheckpointWrapper] = [] # The most recently registered checkpoint. # The latest checkpoint should never be deleted immediately, # even if it's the worst checkpoint. self._latest_checkpoint: Optional[_TrackedCheckpoint] = None + # The best checkpoint seen so far. + # Since _top_k_checkpoints is a min-heap (to know who to evict), + # we still need to keep track of the best checkpoint independently. + self._best_checkpoint: Optional[_HeapCheckpointWrapper] = None + # Deletion of some checkpoints should be deferred. Specifically, if the # latest persisted checkpoint should be deleted, we will only delete it # once a new checkpoint came in (so that `_latest_checkpoint` is available). @@ -102,6 +108,13 @@ def register_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): ) heapq.heappush(self._top_k_checkpoints, wrapped_checkpoint) + # Update best checkpoint. + if ( + self._best_checkpoint is None + or wrapped_checkpoint.priority >= self._best_checkpoint.priority + ): + self._best_checkpoint = wrapped_checkpoint + # Delete the worst checkpoint if we're over the `num_to_keep` limit. if ( self._checkpoint_config.num_to_keep is not None @@ -113,7 +126,7 @@ def register_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): # Delete any checkpoints that were deferred. self._cleanup_checkpoints() - self._latest_checkpoint_id += 1 + self._latest_checkpoint_index += 1 def _get_checkpoint_score( self, checkpoint: _TrackedCheckpoint @@ -167,7 +180,7 @@ def _get_checkpoint_score( def _maybe_delete_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): """Delete a checkpoint as long as it's not the most recent one.""" - if tracked_checkpoint == self._latest_persisted_checkpoint: + if tracked_checkpoint == self._latest_checkpoint: self._checkpoints_to_clean_up.add(tracked_checkpoint) else: self._delete_checkpoint(tracked_checkpoint) @@ -182,5 +195,24 @@ def _cleanup_checkpoints(self): for tracked_checkpoint in list(self._checkpoints_to_clean_up): self._maybe_delete_checkpoint(tracked_checkpoint) + @property + def best_checkpoint(self) -> Optional[_TrackedCheckpoint]: + return ( + None + if self._best_checkpoint is None + else self._best_checkpoint.tracked_checkpoint + ) + + @property + def latest_checkpoint(self) -> Optional[_TrackedCheckpoint]: + return self._latest_checkpoint + + @property + def best_checkpoints(self) -> List[_TrackedCheckpoint]: + return [ + wrapped_checkpoint.tracked_checkpoint + for wrapped_checkpoint in self._top_k_checkpoints + ] + def __del__(self): self._cleanup_checkpoints() From 15553f7a666fe2b8e7cb0bffaa05618d06725ebb Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 31 Jul 2023 18:00:25 -0700 Subject: [PATCH 024/108] Add unit test for simplified checkpoint manager Signed-off-by: Justin Yu --- python/ray/train/BUILD | 8 ++ .../train/tests/test_checkpoint_manager.py | 101 ++++++++++++++++++ 2 files changed, 109 insertions(+) create mode 100644 python/ray/train/tests/test_checkpoint_manager.py diff --git a/python/ray/train/BUILD b/python/ray/train/BUILD index 4711c99bfaed..d8753a3afa3b 100644 --- a/python/ray/train/BUILD +++ b/python/ray/train/BUILD @@ -294,6 +294,14 @@ py_test( deps = [":train_lib"] ) +py_test( + name = "test_checkpoint_manager", + size = "small", + srcs = ["tests/test_checkpoint_manager.py"], + tags = ["team:ml", "exclusive"], + deps = [":train_lib"] +) + py_test( name = "test_data_parallel_trainer", size = "medium", diff --git a/python/ray/train/tests/test_checkpoint_manager.py b/python/ray/train/tests/test_checkpoint_manager.py new file mode 100644 index 000000000000..e27ecc825a43 --- /dev/null +++ b/python/ray/train/tests/test_checkpoint_manager.py @@ -0,0 +1,101 @@ +import random + +import pytest + +from ray.train import CheckpointConfig +from ray.train._internal.checkpoint_manager import ( + _CheckpointManager, + _TrackedCheckpoint, +) +from ray.train.checkpoint import Checkpoint + + +def test_unlimited_checkpoints(tmp_path): + manager = _CheckpointManager(checkpoint_config=CheckpointConfig(num_to_keep=None)) + + for i in range(10): + manager.register_checkpoint( + _TrackedCheckpoint( + checkpoint=Checkpoint.from_directory(str(tmp_path)), metrics={"iter": i} + ) + ) + + assert len(manager.best_checkpoints) == 10 + + +def test_limited_checkpoints(tmp_path): + manager = _CheckpointManager(checkpoint_config=CheckpointConfig(num_to_keep=2)) + + for i in range(10): + manager.register_checkpoint( + _TrackedCheckpoint( + checkpoint=Checkpoint.from_directory(str(tmp_path)), metrics={"iter": i} + ) + ) + + assert len(manager.best_checkpoints) == 2 + + # Keep the latest checkpoints if no metric is given. + assert { + tracked_checkpoint.metrics["iter"] + for tracked_checkpoint in manager.best_checkpoints + } == {8, 9} + + +@pytest.mark.parametrize("order", ["min", "max"]) +def test_keep_checkpoints_by_score(order, tmp_path): + num_to_keep = 2 + score_attribute = "score" + + manager = _CheckpointManager( + checkpoint_config=CheckpointConfig( + num_to_keep=num_to_keep, + checkpoint_score_attribute=score_attribute, + checkpoint_score_order=order, + ) + ) + + scores = [] + for i in range(10): + score = random.random() + manager.register_checkpoint( + _TrackedCheckpoint( + checkpoint=Checkpoint.from_directory(str(tmp_path)), + metrics={"iter": i, "score": score}, + ) + ) + scores.append(score) + + sorted_scores = sorted(scores, reverse=order == "max") + assert set(sorted_scores[:num_to_keep]) == { + tracked_checkpoint.metrics[score_attribute] + for tracked_checkpoint in manager.best_checkpoints + } + + +@pytest.mark.parametrize( + "metrics", + [ + {"nested": {"sub": {"attr": 5}}}, + {"nested": {"sub/attr": 5}}, + {"nested/sub": {"attr": 5}}, + {"nested/sub/attr": 5}, + ], +) +def test_nested_get_checkpoint_score(metrics): + manager = _CheckpointManager( + checkpoint_config=CheckpointConfig( + num_to_keep=2, + checkpoint_score_attribute="nested/sub/attr", + checkpoint_score_order="max", + ) + ) + + tracked_checkpoint = _TrackedCheckpoint(checkpoint=None, metrics=metrics, index=3) + assert manager._get_checkpoint_score(tracked_checkpoint) == (True, 5.0, 3) + + +if __name__ == "__main__": + import sys + + sys.exit(pytest.main(["-v", __file__])) From 00cc9d7ad16649a840e6cd3cbf5cbdaba91afc64 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 31 Jul 2023 18:19:09 -0700 Subject: [PATCH 025/108] Full test coverage Signed-off-by: Justin Yu --- .../train/tests/test_checkpoint_manager.py | 104 ++++++++++++++++-- 1 file changed, 97 insertions(+), 7 deletions(-) diff --git a/python/ray/train/tests/test_checkpoint_manager.py b/python/ray/train/tests/test_checkpoint_manager.py index e27ecc825a43..47cee07854db 100644 --- a/python/ray/train/tests/test_checkpoint_manager.py +++ b/python/ray/train/tests/test_checkpoint_manager.py @@ -1,4 +1,6 @@ +from pathlib import Path import random +from typing import List import pytest @@ -10,26 +12,40 @@ from ray.train.checkpoint import Checkpoint -def test_unlimited_checkpoints(tmp_path): +@pytest.fixture +def checkpoint_paths(tmp_path): + checkpoint_paths = [] + for i in range(10): + checkpoint_path = tmp_path / f"ckpt_{i}" + checkpoint_path.mkdir() + (checkpoint_path / "dummy.txt").write_text(f"{i}") + checkpoint_paths.append(checkpoint_path) + + yield [str(path) for path in checkpoint_paths] + + +def test_unlimited_checkpoints(checkpoint_paths: List[str]): manager = _CheckpointManager(checkpoint_config=CheckpointConfig(num_to_keep=None)) for i in range(10): manager.register_checkpoint( _TrackedCheckpoint( - checkpoint=Checkpoint.from_directory(str(tmp_path)), metrics={"iter": i} + checkpoint=Checkpoint.from_directory(checkpoint_paths[i]), + metrics={"iter": i}, ) ) assert len(manager.best_checkpoints) == 10 -def test_limited_checkpoints(tmp_path): +def test_limited_checkpoints(checkpoint_paths: List[str]): manager = _CheckpointManager(checkpoint_config=CheckpointConfig(num_to_keep=2)) for i in range(10): manager.register_checkpoint( _TrackedCheckpoint( - checkpoint=Checkpoint.from_directory(str(tmp_path)), metrics={"iter": i} + checkpoint=Checkpoint.from_directory(checkpoint_paths[i]), + metrics={"iter": i}, ) ) @@ -41,9 +57,16 @@ def test_limited_checkpoints(tmp_path): for tracked_checkpoint in manager.best_checkpoints } == {8, 9} + # The first 8 checkpoints should be deleted. + for i in range(8): + assert not Path(checkpoint_paths[i]).exists() + + assert Path(checkpoint_paths[8]).exists() + assert Path(checkpoint_paths[9]).exists() + @pytest.mark.parametrize("order", ["min", "max"]) -def test_keep_checkpoints_by_score(order, tmp_path): +def test_keep_checkpoints_by_score(order, checkpoint_paths): num_to_keep = 2 score_attribute = "score" @@ -60,8 +83,8 @@ def test_keep_checkpoints_by_score(order, tmp_path): score = random.random() manager.register_checkpoint( _TrackedCheckpoint( - checkpoint=Checkpoint.from_directory(str(tmp_path)), - metrics={"iter": i, "score": score}, + checkpoint=Checkpoint.from_directory(checkpoint_paths[i]), + metrics={"iter": i, score_attribute: score}, ) ) scores.append(score) @@ -72,6 +95,73 @@ def test_keep_checkpoints_by_score(order, tmp_path): for tracked_checkpoint in manager.best_checkpoints } + # Make sure the bottom checkpoints are deleted. + best_checkpoint_iters = { + tracked_checkpoint.metrics["iter"] + for tracked_checkpoint in manager.best_checkpoints + } + for i, checkpoint_path in enumerate(checkpoint_paths): + if i in best_checkpoint_iters or i == 9: + # The checkpoint should only exist if it's one of the top K or the latest. + assert Path(checkpoint_path).exists() + else: + assert not Path(checkpoint_path).exists() + + +def test_keep_latest_checkpoint(checkpoint_paths): + manager = _CheckpointManager( + checkpoint_config=CheckpointConfig( + num_to_keep=2, + checkpoint_score_attribute="score", + checkpoint_score_order="max", + ) + ) + + manager.register_checkpoint( + _TrackedCheckpoint( + checkpoint=Checkpoint.from_directory(checkpoint_paths[0]), + metrics={"score": 3.0}, + ) + ) + manager.register_checkpoint( + _TrackedCheckpoint( + checkpoint=Checkpoint.from_directory(checkpoint_paths[1]), + metrics={"score": 2.0}, + ) + ) + manager.register_checkpoint( + _TrackedCheckpoint( + checkpoint=Checkpoint.from_directory(checkpoint_paths[2]), + metrics={"score": 1.0}, + ) + ) + + assert len(manager.best_checkpoints) == 2 + + # The latest checkpoint with the lowest score should not be deleted yet. + assert manager.latest_checkpoint.metrics["score"] == 1.0 + + # The latest checkpoint with the lowest score should not be deleted yet. + assert Path(checkpoint_paths[2]).exists() + + manager.register_checkpoint( + _TrackedCheckpoint( + checkpoint=Checkpoint.from_directory(checkpoint_paths[3]), + metrics={"score": 0.0}, + ) + ) + # A newer checkpoint came in. Even though the new one has a lower score, there are + # already num_to_keep better checkpoints, so the previous one should be deleted. + assert not Path(checkpoint_paths[2]).exists() + + # Quick sanity check to make sure that the new checkpoint is kept. + assert manager.latest_checkpoint.metrics["score"] == 0.0 + assert Path(checkpoint_paths[3]).exists() + + # The original 2 checkpoints should still exist + assert Path(checkpoint_paths[0]).exists() + assert Path(checkpoint_paths[1]).exists() + @pytest.mark.parametrize( "metrics", From cb5990e8917a2ce9462c559cbe4e09100f3eb8af Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 31 Jul 2023 17:14:01 -0700 Subject: [PATCH 026/108] Add a simplified checkpoint manager Signed-off-by: Justin Yu --- .../ray/train/_internal/checkpoint_manager.py | 186 ++++++++++++++++++ 1 file changed, 186 insertions(+) create mode 100644 python/ray/train/_internal/checkpoint_manager.py diff --git a/python/ray/train/_internal/checkpoint_manager.py b/python/ray/train/_internal/checkpoint_manager.py new file mode 100644 index 000000000000..41b7794600a2 --- /dev/null +++ b/python/ray/train/_internal/checkpoint_manager.py @@ -0,0 +1,186 @@ +from dataclasses import dataclass +import heapq +import logging +import numbers +from typing import Any, Dict, List, Optional, Set, Tuple + +from ray._private.dict import flatten_dict +from ray.air.config import MAX +from ray.air._internal.util import is_nan +from ray.train import CheckpointConfig +from ray.train._internal.storage import _delete_fs_path +from ray.train.checkpoint import Checkpoint + + +logger = logging.getLogger(__name__) + + +@dataclass +class _TrackedCheckpoint: + """Checkpoint tracked by a checkpoint manager.""" + + checkpoint: Checkpoint + metrics: Dict[str, Any] + index: int = -1 + + +class _HeapCheckpointWrapper: + def __init__(self, priority: Any, tracked_checkpoint: _TrackedCheckpoint): + # Ex: 2 wrapped checkpoints, with checkpoint_score_order = MAX + # Priority of ckpt 1: (not_is_nan=True, metric=5, id=0) + # Priority of ckpt 2: (not_is_nan=True, metric=6, id=1) + # In this case, the min-heap should pop ckpt 1 first, since + # the metric is smaller. + self.priority = priority + self.tracked_checkpoint = tracked_checkpoint + + def __lt__(self, other): + return self.priority < other.priority + + def __repr__(self) -> str: + return ( + f"_HeapCheckpointWrapper(\n" + " priority={self.priority}, " + f"tracked_checkpoint={repr(self.tracked_checkpoint)}\n)" + ) + + +class _CheckpointManager: + """Checkpoint manager that handles checkpoint book-keeping for a trial. + + Args: + checkpoint_config: Defines how many and which checkpoints to keep. + latest_checkpoint_id: First checkpoint ID to use (e.g., in case we + resume an existing experiment). + """ + + def __init__( + self, + checkpoint_config: Optional[CheckpointConfig], + latest_checkpoint_index: int = 0, + ): + self._checkpoint_config = checkpoint_config or CheckpointConfig() + + # Incremental unique checkpoint ID of this run. + self._latest_checkpoint_index = latest_checkpoint_index + + # A heap that tracks the top K checkpoints. + self._top_k_checkpoints: List[_TrackedCheckpoint] = [] + + # The most recently registered checkpoint. + # The latest checkpoint should never be deleted immediately, + # even if it's the worst checkpoint. + self._latest_checkpoint: Optional[_TrackedCheckpoint] = None + + # Deletion of some checkpoints should be deferred. Specifically, if the + # latest persisted checkpoint should be deleted, we will only delete it + # once a new checkpoint came in (so that `_latest_checkpoint` is available). + self._checkpoints_to_clean_up: Set[_TrackedCheckpoint] = set() + + def register_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): + """Register new checkpoint and add to bookkeeping. + + This method will register a new checkpoint and add it to the internal + bookkeeping logic. This means the checkpoint manager will decide if + this checkpoint should be kept, and if older or worse performing + checkpoints should be deleted. + + Args: + checkpoints: Tracked checkpoint object to add to bookkeeping. + """ + # Mark the index of the checkpoint, for use in determining the + # order in which checkpoints were registered. + # TODO(justinvyu): Should this be unified with the checkpoint index + # tracked in the session / used in the checkpoint dirname? + tracked_checkpoint.index = self._latest_checkpoint_index + self._latest_checkpoint = tracked_checkpoint + + # Add to heap. + checkpoint_priority = self._get_checkpoint_score(tracked_checkpoint) + wrapped_checkpoint = _HeapCheckpointWrapper( + checkpoint_priority, tracked_checkpoint + ) + heapq.heappush(self._top_k_checkpoints, wrapped_checkpoint) + + # Delete the worst checkpoint if we're over the `num_to_keep` limit. + if ( + self._checkpoint_config.num_to_keep is not None + and len(self._top_k_checkpoints) > self._checkpoint_config.num_to_keep + ): + worst_checkpoint = heapq.heappop(self._top_k_checkpoints) + self._maybe_delete_checkpoint(worst_checkpoint.tracked_checkpoint) + + # Delete any checkpoints that were deferred. + self._cleanup_checkpoints() + + self._latest_checkpoint_id += 1 + + def _get_checkpoint_score( + self, checkpoint: _TrackedCheckpoint + ) -> Tuple[bool, numbers.Number, int]: + """Get scoring tuple for a checkpoint, according to checkpoint strategy. + + We sort checkpoints by this score. Checkpoints with a higher score are kept. + To achieve the desired ordering, we return a tuple of + (is_not_na: bool, metric: Number, index: int). + + The first index means that checkpoints that are NaN are rated worst. + The second index sorts the checkpoints by metric value. The third index + sorts checkpoints with the same metric value by their index - more recent + checkpoints are rated higher. + """ + checkpoint_score_attribute = self._checkpoint_config.checkpoint_score_attribute + if checkpoint_score_attribute: + flat_metrics = flatten_dict(checkpoint.metrics) + try: + checkpoint_result = flat_metrics[checkpoint_score_attribute] + except KeyError: + valid_keys = list(flat_metrics.keys()) + logger.error( + f"Result dict has no key: {checkpoint_score_attribute}. " + f"checkpoint_score_attr must be set to a key in the " + f"result dict. Valid keys are: {valid_keys}" + ) + checkpoint_result = float("-inf") + else: + checkpoint_result = float("-inf") + + checkpoint_score_order = self._checkpoint_config.checkpoint_score_order + order_factor = 1.0 if checkpoint_score_order == MAX else -1.0 + + checkpoint_score = order_factor * checkpoint_result + + if not isinstance(checkpoint_score, numbers.Number): + raise ValueError( + f"Unable to persist checkpoint for " + f"checkpoint_score_attribute: " + f"{checkpoint_score_attribute} with value " + f"{checkpoint_score}. " + f"This attribute must be numerical." + ) + + return ( + (not is_nan(checkpoint_score), checkpoint_score, checkpoint.index) + if not is_nan(checkpoint_score) + else (False, float("-inf"), checkpoint.index) + ) + + def _maybe_delete_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): + """Delete a checkpoint as long as it's not the most recent one.""" + if tracked_checkpoint == self._latest_persisted_checkpoint: + self._checkpoints_to_clean_up.add(tracked_checkpoint) + else: + self._delete_checkpoint(tracked_checkpoint) + + def _delete_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): + checkpoint = tracked_checkpoint.checkpoint + _delete_fs_path(fs=checkpoint.filesystem, fs_path=checkpoint.path) + self._checkpoints_to_clean_up.discard(tracked_checkpoint) + + def _cleanup_checkpoints(self): + """Delete any checkpoints that were deferred for deletion.""" + for tracked_checkpoint in list(self._checkpoints_to_clean_up): + self._maybe_delete_checkpoint(tracked_checkpoint) + + def __del__(self): + self._cleanup_checkpoints() From 2db9aae9de0f74718e23556d86b539db10dec5c5 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 31 Jul 2023 17:59:33 -0700 Subject: [PATCH 027/108] Fixes to checkpoint manager Signed-off-by: Justin Yu --- .../ray/train/_internal/checkpoint_manager.py | 52 +++++++++++++++---- 1 file changed, 42 insertions(+), 10 deletions(-) diff --git a/python/ray/train/_internal/checkpoint_manager.py b/python/ray/train/_internal/checkpoint_manager.py index 41b7794600a2..f80f994ddc5b 100644 --- a/python/ray/train/_internal/checkpoint_manager.py +++ b/python/ray/train/_internal/checkpoint_manager.py @@ -1,4 +1,3 @@ -from dataclasses import dataclass import heapq import logging import numbers @@ -15,13 +14,15 @@ logger = logging.getLogger(__name__) -@dataclass class _TrackedCheckpoint: """Checkpoint tracked by a checkpoint manager.""" - checkpoint: Checkpoint - metrics: Dict[str, Any] - index: int = -1 + def __init__( + self, checkpoint: Checkpoint, metrics: Dict[str, Any], index: int = -1 + ): + self.checkpoint = checkpoint + self.metrics = metrics + self.index = index class _HeapCheckpointWrapper: @@ -34,7 +35,7 @@ def __init__(self, priority: Any, tracked_checkpoint: _TrackedCheckpoint): self.priority = priority self.tracked_checkpoint = tracked_checkpoint - def __lt__(self, other): + def __lt__(self, other: "_HeapCheckpointWrapper") -> bool: return self.priority < other.priority def __repr__(self) -> str: @@ -50,7 +51,7 @@ class _CheckpointManager: Args: checkpoint_config: Defines how many and which checkpoints to keep. - latest_checkpoint_id: First checkpoint ID to use (e.g., in case we + latest_checkpoint_index: First checkpoint ID to use (e.g., in case we resume an existing experiment). """ @@ -65,13 +66,18 @@ def __init__( self._latest_checkpoint_index = latest_checkpoint_index # A heap that tracks the top K checkpoints. - self._top_k_checkpoints: List[_TrackedCheckpoint] = [] + self._top_k_checkpoints: List[_HeapCheckpointWrapper] = [] # The most recently registered checkpoint. # The latest checkpoint should never be deleted immediately, # even if it's the worst checkpoint. self._latest_checkpoint: Optional[_TrackedCheckpoint] = None + # The best checkpoint seen so far. + # Since _top_k_checkpoints is a min-heap (to know who to evict), + # we still need to keep track of the best checkpoint independently. + self._best_checkpoint: Optional[_HeapCheckpointWrapper] = None + # Deletion of some checkpoints should be deferred. Specifically, if the # latest persisted checkpoint should be deleted, we will only delete it # once a new checkpoint came in (so that `_latest_checkpoint` is available). @@ -102,6 +108,13 @@ def register_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): ) heapq.heappush(self._top_k_checkpoints, wrapped_checkpoint) + # Update best checkpoint. + if ( + self._best_checkpoint is None + or wrapped_checkpoint.priority >= self._best_checkpoint.priority + ): + self._best_checkpoint = wrapped_checkpoint + # Delete the worst checkpoint if we're over the `num_to_keep` limit. if ( self._checkpoint_config.num_to_keep is not None @@ -113,7 +126,7 @@ def register_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): # Delete any checkpoints that were deferred. self._cleanup_checkpoints() - self._latest_checkpoint_id += 1 + self._latest_checkpoint_index += 1 def _get_checkpoint_score( self, checkpoint: _TrackedCheckpoint @@ -167,7 +180,7 @@ def _get_checkpoint_score( def _maybe_delete_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): """Delete a checkpoint as long as it's not the most recent one.""" - if tracked_checkpoint == self._latest_persisted_checkpoint: + if tracked_checkpoint == self._latest_checkpoint: self._checkpoints_to_clean_up.add(tracked_checkpoint) else: self._delete_checkpoint(tracked_checkpoint) @@ -182,5 +195,24 @@ def _cleanup_checkpoints(self): for tracked_checkpoint in list(self._checkpoints_to_clean_up): self._maybe_delete_checkpoint(tracked_checkpoint) + @property + def best_checkpoint(self) -> Optional[_TrackedCheckpoint]: + return ( + None + if self._best_checkpoint is None + else self._best_checkpoint.tracked_checkpoint + ) + + @property + def latest_checkpoint(self) -> Optional[_TrackedCheckpoint]: + return self._latest_checkpoint + + @property + def best_checkpoints(self) -> List[_TrackedCheckpoint]: + return [ + wrapped_checkpoint.tracked_checkpoint + for wrapped_checkpoint in self._top_k_checkpoints + ] + def __del__(self): self._cleanup_checkpoints() From a2067b7576f751fd4295ea3dae9a9db06e13447c Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 31 Jul 2023 18:00:25 -0700 Subject: [PATCH 028/108] Add unit test for simplified checkpoint manager Signed-off-by: Justin Yu --- python/ray/train/BUILD | 8 ++ .../train/tests/test_checkpoint_manager.py | 101 ++++++++++++++++++ 2 files changed, 109 insertions(+) create mode 100644 python/ray/train/tests/test_checkpoint_manager.py diff --git a/python/ray/train/BUILD b/python/ray/train/BUILD index 4711c99bfaed..d8753a3afa3b 100644 --- a/python/ray/train/BUILD +++ b/python/ray/train/BUILD @@ -294,6 +294,14 @@ py_test( deps = [":train_lib"] ) +py_test( + name = "test_checkpoint_manager", + size = "small", + srcs = ["tests/test_checkpoint_manager.py"], + tags = ["team:ml", "exclusive"], + deps = [":train_lib"] +) + py_test( name = "test_data_parallel_trainer", size = "medium", diff --git a/python/ray/train/tests/test_checkpoint_manager.py b/python/ray/train/tests/test_checkpoint_manager.py new file mode 100644 index 000000000000..e27ecc825a43 --- /dev/null +++ b/python/ray/train/tests/test_checkpoint_manager.py @@ -0,0 +1,101 @@ +import random + +import pytest + +from ray.train import CheckpointConfig +from ray.train._internal.checkpoint_manager import ( + _CheckpointManager, + _TrackedCheckpoint, +) +from ray.train.checkpoint import Checkpoint + + +def test_unlimited_checkpoints(tmp_path): + manager = _CheckpointManager(checkpoint_config=CheckpointConfig(num_to_keep=None)) + + for i in range(10): + manager.register_checkpoint( + _TrackedCheckpoint( + checkpoint=Checkpoint.from_directory(str(tmp_path)), metrics={"iter": i} + ) + ) + + assert len(manager.best_checkpoints) == 10 + + +def test_limited_checkpoints(tmp_path): + manager = _CheckpointManager(checkpoint_config=CheckpointConfig(num_to_keep=2)) + + for i in range(10): + manager.register_checkpoint( + _TrackedCheckpoint( + checkpoint=Checkpoint.from_directory(str(tmp_path)), metrics={"iter": i} + ) + ) + + assert len(manager.best_checkpoints) == 2 + + # Keep the latest checkpoints if no metric is given. + assert { + tracked_checkpoint.metrics["iter"] + for tracked_checkpoint in manager.best_checkpoints + } == {8, 9} + + +@pytest.mark.parametrize("order", ["min", "max"]) +def test_keep_checkpoints_by_score(order, tmp_path): + num_to_keep = 2 + score_attribute = "score" + + manager = _CheckpointManager( + checkpoint_config=CheckpointConfig( + num_to_keep=num_to_keep, + checkpoint_score_attribute=score_attribute, + checkpoint_score_order=order, + ) + ) + + scores = [] + for i in range(10): + score = random.random() + manager.register_checkpoint( + _TrackedCheckpoint( + checkpoint=Checkpoint.from_directory(str(tmp_path)), + metrics={"iter": i, "score": score}, + ) + ) + scores.append(score) + + sorted_scores = sorted(scores, reverse=order == "max") + assert set(sorted_scores[:num_to_keep]) == { + tracked_checkpoint.metrics[score_attribute] + for tracked_checkpoint in manager.best_checkpoints + } + + +@pytest.mark.parametrize( + "metrics", + [ + {"nested": {"sub": {"attr": 5}}}, + {"nested": {"sub/attr": 5}}, + {"nested/sub": {"attr": 5}}, + {"nested/sub/attr": 5}, + ], +) +def test_nested_get_checkpoint_score(metrics): + manager = _CheckpointManager( + checkpoint_config=CheckpointConfig( + num_to_keep=2, + checkpoint_score_attribute="nested/sub/attr", + checkpoint_score_order="max", + ) + ) + + tracked_checkpoint = _TrackedCheckpoint(checkpoint=None, metrics=metrics, index=3) + assert manager._get_checkpoint_score(tracked_checkpoint) == (True, 5.0, 3) + + +if __name__ == "__main__": + import sys + + sys.exit(pytest.main(["-v", __file__])) From f1216f2ae641feed91eb024c868ae3c199e3e851 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 31 Jul 2023 18:19:09 -0700 Subject: [PATCH 029/108] Full test coverage Signed-off-by: Justin Yu --- .../train/tests/test_checkpoint_manager.py | 104 ++++++++++++++++-- 1 file changed, 97 insertions(+), 7 deletions(-) diff --git a/python/ray/train/tests/test_checkpoint_manager.py b/python/ray/train/tests/test_checkpoint_manager.py index e27ecc825a43..47cee07854db 100644 --- a/python/ray/train/tests/test_checkpoint_manager.py +++ b/python/ray/train/tests/test_checkpoint_manager.py @@ -1,4 +1,6 @@ +from pathlib import Path import random +from typing import List import pytest @@ -10,26 +12,40 @@ from ray.train.checkpoint import Checkpoint -def test_unlimited_checkpoints(tmp_path): +@pytest.fixture +def checkpoint_paths(tmp_path): + checkpoint_paths = [] + for i in range(10): + checkpoint_path = tmp_path / f"ckpt_{i}" + checkpoint_path.mkdir() + (checkpoint_path / "dummy.txt").write_text(f"{i}") + checkpoint_paths.append(checkpoint_path) + + yield [str(path) for path in checkpoint_paths] + + +def test_unlimited_checkpoints(checkpoint_paths: List[str]): manager = _CheckpointManager(checkpoint_config=CheckpointConfig(num_to_keep=None)) for i in range(10): manager.register_checkpoint( _TrackedCheckpoint( - checkpoint=Checkpoint.from_directory(str(tmp_path)), metrics={"iter": i} + checkpoint=Checkpoint.from_directory(checkpoint_paths[i]), + metrics={"iter": i}, ) ) assert len(manager.best_checkpoints) == 10 -def test_limited_checkpoints(tmp_path): +def test_limited_checkpoints(checkpoint_paths: List[str]): manager = _CheckpointManager(checkpoint_config=CheckpointConfig(num_to_keep=2)) for i in range(10): manager.register_checkpoint( _TrackedCheckpoint( - checkpoint=Checkpoint.from_directory(str(tmp_path)), metrics={"iter": i} + checkpoint=Checkpoint.from_directory(checkpoint_paths[i]), + metrics={"iter": i}, ) ) @@ -41,9 +57,16 @@ def test_limited_checkpoints(tmp_path): for tracked_checkpoint in manager.best_checkpoints } == {8, 9} + # The first 8 checkpoints should be deleted. + for i in range(8): + assert not Path(checkpoint_paths[i]).exists() + + assert Path(checkpoint_paths[8]).exists() + assert Path(checkpoint_paths[9]).exists() + @pytest.mark.parametrize("order", ["min", "max"]) -def test_keep_checkpoints_by_score(order, tmp_path): +def test_keep_checkpoints_by_score(order, checkpoint_paths): num_to_keep = 2 score_attribute = "score" @@ -60,8 +83,8 @@ def test_keep_checkpoints_by_score(order, tmp_path): score = random.random() manager.register_checkpoint( _TrackedCheckpoint( - checkpoint=Checkpoint.from_directory(str(tmp_path)), - metrics={"iter": i, "score": score}, + checkpoint=Checkpoint.from_directory(checkpoint_paths[i]), + metrics={"iter": i, score_attribute: score}, ) ) scores.append(score) @@ -72,6 +95,73 @@ def test_keep_checkpoints_by_score(order, tmp_path): for tracked_checkpoint in manager.best_checkpoints } + # Make sure the bottom checkpoints are deleted. + best_checkpoint_iters = { + tracked_checkpoint.metrics["iter"] + for tracked_checkpoint in manager.best_checkpoints + } + for i, checkpoint_path in enumerate(checkpoint_paths): + if i in best_checkpoint_iters or i == 9: + # The checkpoint should only exist if it's one of the top K or the latest. + assert Path(checkpoint_path).exists() + else: + assert not Path(checkpoint_path).exists() + + +def test_keep_latest_checkpoint(checkpoint_paths): + manager = _CheckpointManager( + checkpoint_config=CheckpointConfig( + num_to_keep=2, + checkpoint_score_attribute="score", + checkpoint_score_order="max", + ) + ) + + manager.register_checkpoint( + _TrackedCheckpoint( + checkpoint=Checkpoint.from_directory(checkpoint_paths[0]), + metrics={"score": 3.0}, + ) + ) + manager.register_checkpoint( + _TrackedCheckpoint( + checkpoint=Checkpoint.from_directory(checkpoint_paths[1]), + metrics={"score": 2.0}, + ) + ) + manager.register_checkpoint( + _TrackedCheckpoint( + checkpoint=Checkpoint.from_directory(checkpoint_paths[2]), + metrics={"score": 1.0}, + ) + ) + + assert len(manager.best_checkpoints) == 2 + + # The latest checkpoint with the lowest score should not be deleted yet. + assert manager.latest_checkpoint.metrics["score"] == 1.0 + + # The latest checkpoint with the lowest score should not be deleted yet. + assert Path(checkpoint_paths[2]).exists() + + manager.register_checkpoint( + _TrackedCheckpoint( + checkpoint=Checkpoint.from_directory(checkpoint_paths[3]), + metrics={"score": 0.0}, + ) + ) + # A newer checkpoint came in. Even though the new one has a lower score, there are + # already num_to_keep better checkpoints, so the previous one should be deleted. + assert not Path(checkpoint_paths[2]).exists() + + # Quick sanity check to make sure that the new checkpoint is kept. + assert manager.latest_checkpoint.metrics["score"] == 0.0 + assert Path(checkpoint_paths[3]).exists() + + # The original 2 checkpoints should still exist + assert Path(checkpoint_paths[0]).exists() + assert Path(checkpoint_paths[1]).exists() + @pytest.mark.parametrize( "metrics", From 9b9ff343b0336db040ffbefe2ecb8169ca798abd Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 1 Aug 2023 13:44:17 -0700 Subject: [PATCH 030/108] Simplify even more Signed-off-by: Justin Yu --- .../ray/train/_internal/checkpoint_manager.py | 204 ++++++++---------- .../train/tests/test_checkpoint_manager.py | 4 +- 2 files changed, 87 insertions(+), 121 deletions(-) diff --git a/python/ray/train/_internal/checkpoint_manager.py b/python/ray/train/_internal/checkpoint_manager.py index f80f994ddc5b..e7c1ed4a2ff7 100644 --- a/python/ray/train/_internal/checkpoint_manager.py +++ b/python/ray/train/_internal/checkpoint_manager.py @@ -1,7 +1,6 @@ -import heapq import logging import numbers -from typing import Any, Dict, List, Optional, Set, Tuple +from typing import Any, Callable, Dict, List, Optional, Tuple from ray._private.dict import flatten_dict from ray.air.config import MAX @@ -17,71 +16,68 @@ class _TrackedCheckpoint: """Checkpoint tracked by a checkpoint manager.""" - def __init__( - self, checkpoint: Checkpoint, metrics: Dict[str, Any], index: int = -1 - ): + def __init__(self, checkpoint: Checkpoint, metrics: Dict[str, Any]): self.checkpoint = checkpoint self.metrics = metrics - self.index = index -class _HeapCheckpointWrapper: - def __init__(self, priority: Any, tracked_checkpoint: _TrackedCheckpoint): - # Ex: 2 wrapped checkpoints, with checkpoint_score_order = MAX - # Priority of ckpt 1: (not_is_nan=True, metric=5, id=0) - # Priority of ckpt 2: (not_is_nan=True, metric=6, id=1) - # In this case, the min-heap should pop ckpt 1 first, since - # the metric is smaller. - self.priority = priority - self.tracked_checkpoint = tracked_checkpoint - - def __lt__(self, other: "_HeapCheckpointWrapper") -> bool: - return self.priority < other.priority - - def __repr__(self) -> str: - return ( - f"_HeapCheckpointWrapper(\n" - " priority={self.priority}, " - f"tracked_checkpoint={repr(self.tracked_checkpoint)}\n)" - ) +def _insert_into_sorted_list(list: List[Any], item: Any, key: Callable[[Any], Any]): + """Insert an item into a sorted list with a custom key function. + + Examples: + + >>> list = [] + >>> _insert_into_sorted_list(list, {"a": 1, "b": 0}, lambda x: x["a"]) + >>> list + [{'a': 1, 'b': 0}] + >>> _insert_into_sorted_list(list, {"a": 3, "b": 1}, lambda x: x["a"]) + >>> list + [{'a': 1, 'b': 0}, {'a': 3, 'b': 1}] + >>> _insert_into_sorted_list(list, {"a": 4, "b": 2}, lambda x: x["a"]) + >>> list + [{'a': 1, 'b': 0}, {'a': 3, 'b': 1}, {'a': 4, 'b': 2}] + >>> _insert_into_sorted_list(list, {"a": 1, "b": 3}, lambda x: x["a"]) + >>> list + [{'a': 1, 'b': 0}, {'a': 1, 'b': 3}, {'a': 3, 'b': 1}, {'a': 4, 'b': 2}] + """ + i = 0 + while i < len(list): + # Insert to the right of all duplicates. + if key(list[i]) > key(item): + break + i += 1 + list.insert(i, item) class _CheckpointManager: """Checkpoint manager that handles checkpoint book-keeping for a trial. + Main purpose of this abstraction is to keep the top K checkpoints based on + recency/a user-provided metric. + Args: checkpoint_config: Defines how many and which checkpoints to keep. - latest_checkpoint_index: First checkpoint ID to use (e.g., in case we - resume an existing experiment). """ - def __init__( - self, - checkpoint_config: Optional[CheckpointConfig], - latest_checkpoint_index: int = 0, - ): + def __init__(self, checkpoint_config: Optional[CheckpointConfig]): self._checkpoint_config = checkpoint_config or CheckpointConfig() - # Incremental unique checkpoint ID of this run. - self._latest_checkpoint_index = latest_checkpoint_index - - # A heap that tracks the top K checkpoints. - self._top_k_checkpoints: List[_HeapCheckpointWrapper] = [] - - # The most recently registered checkpoint. - # The latest checkpoint should never be deleted immediately, - # even if it's the worst checkpoint. - self._latest_checkpoint: Optional[_TrackedCheckpoint] = None + # List of checkpoints ordered by ascending score. + self._checkpoints: List[_TrackedCheckpoint] = [] - # The best checkpoint seen so far. - # Since _top_k_checkpoints is a min-heap (to know who to evict), - # we still need to keep track of the best checkpoint independently. - self._best_checkpoint: Optional[_HeapCheckpointWrapper] = None + # The latest registered checkpoint. + # This should never be immediately deleted upon registration, + # even if it's not in the top K checkpoints, based on score. + self._latest_checkpoint: _TrackedCheckpoint = None - # Deletion of some checkpoints should be deferred. Specifically, if the - # latest persisted checkpoint should be deleted, we will only delete it - # once a new checkpoint came in (so that `_latest_checkpoint` is available). - self._checkpoints_to_clean_up: Set[_TrackedCheckpoint] = set() + if ( + self._checkpoint_config.num_to_keep is not None + and self._checkpoint_config.num_to_keep <= 0 + ): + raise ValueError( + f"`num_to_keep` must >= 1, got: " + f"{self._checkpoint_config.num_to_keep}" + ) def register_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): """Register new checkpoint and add to bookkeeping. @@ -92,55 +88,50 @@ def register_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): checkpoints should be deleted. Args: - checkpoints: Tracked checkpoint object to add to bookkeeping. + checkpoint: Tracked checkpoint object to add to bookkeeping. """ - # Mark the index of the checkpoint, for use in determining the - # order in which checkpoints were registered. - # TODO(justinvyu): Should this be unified with the checkpoint index - # tracked in the session / used in the checkpoint dirname? - tracked_checkpoint.index = self._latest_checkpoint_index self._latest_checkpoint = tracked_checkpoint - # Add to heap. - checkpoint_priority = self._get_checkpoint_score(tracked_checkpoint) - wrapped_checkpoint = _HeapCheckpointWrapper( - checkpoint_priority, tracked_checkpoint - ) - heapq.heappush(self._top_k_checkpoints, wrapped_checkpoint) - - # Update best checkpoint. - if ( - self._best_checkpoint is None - or wrapped_checkpoint.priority >= self._best_checkpoint.priority - ): - self._best_checkpoint = wrapped_checkpoint + if self._checkpoint_config.checkpoint_score_attribute is not None: + # If we're ordering by a score, insert the checkpoint + # so that the list remains sorted. + _insert_into_sorted_list( + self._checkpoints, tracked_checkpoint, key=self._get_checkpoint_score + ) + else: + # If no metric is provided, just append (ordering by time of registration). + self._checkpoints.append(tracked_checkpoint) - # Delete the worst checkpoint if we're over the `num_to_keep` limit. - if ( - self._checkpoint_config.num_to_keep is not None - and len(self._top_k_checkpoints) > self._checkpoint_config.num_to_keep - ): - worst_checkpoint = heapq.heappop(self._top_k_checkpoints) - self._maybe_delete_checkpoint(worst_checkpoint.tracked_checkpoint) + if self._checkpoint_config.num_to_keep is not None: + # Delete the bottom (N - K) checkpoints + worst_checkpoints = set( + self._checkpoints[: -self._checkpoint_config.num_to_keep] + ) + # Except for the latest checkpoint. + checkpoints_to_delete = worst_checkpoints - {self._latest_checkpoint} - # Delete any checkpoints that were deferred. - self._cleanup_checkpoints() + # Update internal state before actually deleting them. + self._checkpoints = [ + tracked_checkpoint + for tracked_checkpoint in self._checkpoints + if tracked_checkpoint not in checkpoints_to_delete + ] - self._latest_checkpoint_index += 1 + for checkpoint_to_delete in checkpoints_to_delete: + checkpoint = checkpoint_to_delete.checkpoint + _delete_fs_path(fs=checkpoint.filesystem, fs_path=checkpoint.path) def _get_checkpoint_score( self, checkpoint: _TrackedCheckpoint - ) -> Tuple[bool, numbers.Number, int]: - """Get scoring tuple for a checkpoint, according to checkpoint strategy. + ) -> Tuple[bool, numbers.Number]: + """Get the score for a checkpoint, according to checkpoint config. - We sort checkpoints by this score. Checkpoints with a higher score are kept. - To achieve the desired ordering, we return a tuple of - (is_not_na: bool, metric: Number, index: int). + If `mode="min"`, the metric is negated so that the lowest score is + treated as the best. - The first index means that checkpoints that are NaN are rated worst. - The second index sorts the checkpoints by metric value. The third index - sorts checkpoints with the same metric value by their index - more recent - checkpoints are rated higher. + Returns: + Tuple: A tuple of (not_is_nan: bool, score: numbers.Number). + This score orders: nan values < float("-inf") < valid numeric metrics """ checkpoint_score_attribute = self._checkpoint_config.checkpoint_score_attribute if checkpoint_score_attribute: @@ -173,35 +164,14 @@ def _get_checkpoint_score( ) return ( - (not is_nan(checkpoint_score), checkpoint_score, checkpoint.index) + (not is_nan(checkpoint_score), checkpoint_score) if not is_nan(checkpoint_score) - else (False, float("-inf"), checkpoint.index) + else (False, float("-inf")) ) - def _maybe_delete_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): - """Delete a checkpoint as long as it's not the most recent one.""" - if tracked_checkpoint == self._latest_checkpoint: - self._checkpoints_to_clean_up.add(tracked_checkpoint) - else: - self._delete_checkpoint(tracked_checkpoint) - - def _delete_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): - checkpoint = tracked_checkpoint.checkpoint - _delete_fs_path(fs=checkpoint.filesystem, fs_path=checkpoint.path) - self._checkpoints_to_clean_up.discard(tracked_checkpoint) - - def _cleanup_checkpoints(self): - """Delete any checkpoints that were deferred for deletion.""" - for tracked_checkpoint in list(self._checkpoints_to_clean_up): - self._maybe_delete_checkpoint(tracked_checkpoint) - @property def best_checkpoint(self) -> Optional[_TrackedCheckpoint]: - return ( - None - if self._best_checkpoint is None - else self._best_checkpoint.tracked_checkpoint - ) + return self._checkpoints[-1] if self._checkpoints else None @property def latest_checkpoint(self) -> Optional[_TrackedCheckpoint]: @@ -209,10 +179,6 @@ def latest_checkpoint(self) -> Optional[_TrackedCheckpoint]: @property def best_checkpoints(self) -> List[_TrackedCheckpoint]: - return [ - wrapped_checkpoint.tracked_checkpoint - for wrapped_checkpoint in self._top_k_checkpoints - ] - - def __del__(self): - self._cleanup_checkpoints() + if self._checkpoint_config.num_to_keep is None: + return self._checkpoints + return self._checkpoints[-self._checkpoint_config.num_to_keep :] diff --git a/python/ray/train/tests/test_checkpoint_manager.py b/python/ray/train/tests/test_checkpoint_manager.py index 47cee07854db..9000487bd4c1 100644 --- a/python/ray/train/tests/test_checkpoint_manager.py +++ b/python/ray/train/tests/test_checkpoint_manager.py @@ -181,8 +181,8 @@ def test_nested_get_checkpoint_score(metrics): ) ) - tracked_checkpoint = _TrackedCheckpoint(checkpoint=None, metrics=metrics, index=3) - assert manager._get_checkpoint_score(tracked_checkpoint) == (True, 5.0, 3) + tracked_checkpoint = _TrackedCheckpoint(checkpoint=None, metrics=metrics) + assert manager._get_checkpoint_score(tracked_checkpoint) == (True, 5.0) if __name__ == "__main__": From 913af105baf6469e5cc6182642979805a1428d51 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 1 Aug 2023 16:34:01 -0700 Subject: [PATCH 031/108] Patch fix for circular imports Signed-off-by: Justin Yu --- python/ray/train/_internal/checkpoint_manager.py | 10 +++++++--- python/ray/train/_internal/session.py | 9 ++++----- python/ray/train/_internal/storage.py | 11 ++++++++--- 3 files changed, 19 insertions(+), 11 deletions(-) diff --git a/python/ray/train/_internal/checkpoint_manager.py b/python/ray/train/_internal/checkpoint_manager.py index f80f994ddc5b..ff2774747977 100644 --- a/python/ray/train/_internal/checkpoint_manager.py +++ b/python/ray/train/_internal/checkpoint_manager.py @@ -1,15 +1,17 @@ import heapq import logging import numbers -from typing import Any, Dict, List, Optional, Set, Tuple +from typing import Any, Dict, List, Optional, Set, Tuple, TYPE_CHECKING from ray._private.dict import flatten_dict from ray.air.config import MAX from ray.air._internal.util import is_nan -from ray.train import CheckpointConfig from ray.train._internal.storage import _delete_fs_path from ray.train.checkpoint import Checkpoint +if TYPE_CHECKING: + from ray.train import CheckpointConfig + logger = logging.getLogger(__name__) @@ -57,9 +59,11 @@ class _CheckpointManager: def __init__( self, - checkpoint_config: Optional[CheckpointConfig], + checkpoint_config: Optional["CheckpointConfig"], latest_checkpoint_index: int = 0, ): + from ray.train import CheckpointConfig + self._checkpoint_config = checkpoint_config or CheckpointConfig() # Incremental unique checkpoint ID of this run. diff --git a/python/ray/train/_internal/session.py b/python/ray/train/_internal/session.py index bd6589d5ddf6..5cdea19c4d7e 100644 --- a/python/ray/train/_internal/session.py +++ b/python/ray/train/_internal/session.py @@ -37,7 +37,6 @@ TIME_TOTAL_S, LAZY_CHECKPOINT_MARKER_FILE, ) -from ray.train.checkpoint import Checkpoint as NewCheckpoint from ray.train.error import SessionMisuseError from ray.util.annotations import DeveloperAPI, PublicAPI @@ -436,7 +435,9 @@ def _set_legacy_checkpoint_uri(self, uri: str): """ self.legacy_checkpoint_uri = uri - def new_checkpoint(self, checkpoint: NewCheckpoint): + def new_checkpoint(self, checkpoint): + from ray.train.checkpoint import Checkpoint as NewCheckpoint + if not isinstance(checkpoint, NewCheckpoint): raise ValueError( "You must pass a `ray.train.checkpoint.Checkpoint` " @@ -466,9 +467,7 @@ def new_checkpoint(self, checkpoint: NewCheckpoint): # checkpoint has been processed. self.continue_lock.acquire() - def new_report( - self, metrics: Dict, checkpoint: Optional[NewCheckpoint] = None - ) -> None: + def new_report(self, metrics: Dict, checkpoint=None) -> None: if checkpoint: self.new_checkpoint(checkpoint) diff --git a/python/ray/train/_internal/storage.py b/python/ray/train/_internal/storage.py index 7550d9413ed5..dca051a387b5 100644 --- a/python/ray/train/_internal/storage.py +++ b/python/ray/train/_internal/storage.py @@ -4,7 +4,7 @@ import os from pathlib import Path import shutil -from typing import Callable, Dict, List, Optional, Tuple +from typing import Callable, Dict, List, Optional, Tuple, TYPE_CHECKING try: import fsspec @@ -27,10 +27,12 @@ from ray.air._internal.filelock import TempFileLock from ray.air._internal.uri_utils import URI, is_uri -from ray.train.checkpoint import Checkpoint from ray.tune.syncer import Syncer, SyncConfig, _BackgroundSyncer from ray.tune.result import _get_defaults_results_dir +if TYPE_CHECKING: + from ray.train.checkpoint import Checkpoint + logger = logging.getLogger(__file__) @@ -467,7 +469,7 @@ def _check_validation_file(self): "to the configured storage path." ) - def persist_current_checkpoint(self, checkpoint: Checkpoint) -> Checkpoint: + def persist_current_checkpoint(self, checkpoint: "Checkpoint") -> "Checkpoint": """Persists a given checkpoint to the current checkpoint path on the filesystem. "Current" is defined by the `current_checkpoint_index` attribute of the @@ -484,6 +486,9 @@ def persist_current_checkpoint(self, checkpoint: Checkpoint) -> Checkpoint: Returns: Checkpoint: A Checkpoint pointing to the persisted checkpoint location. """ + # TODO(justinvyu): Fix this cyclical import. + from ray.train.checkpoint import Checkpoint + logger.debug( "Copying checkpoint files to storage path:\n" "({source_fs}, {source}) -> ({dest_fs}, {destination})".format( From 6b5d34eacb7b58169a0be4b628fb324984f761da Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 1 Aug 2023 17:49:52 -0700 Subject: [PATCH 032/108] Use new checkpoint manager in Tune ckpt book-keeping Signed-off-by: Justin Yu --- .../ray/tune/execution/checkpoint_manager.py | 12 ------- python/ray/tune/execution/trial_runner.py | 20 +++++++---- python/ray/tune/execution/tune_controller.py | 19 +++++++++- python/ray/tune/experiment/trial.py | 36 ++++++++++++++----- .../ray/tune/trainable/function_trainable.py | 27 ++++++++++---- python/ray/tune/trainable/trainable.py | 6 ++-- 6 files changed, 84 insertions(+), 36 deletions(-) diff --git a/python/ray/tune/execution/checkpoint_manager.py b/python/ray/tune/execution/checkpoint_manager.py index cdf3d991d45c..20d62de5d49b 100644 --- a/python/ray/tune/execution/checkpoint_manager.py +++ b/python/ray/tune/execution/checkpoint_manager.py @@ -51,20 +51,8 @@ def handle_checkpoint(self, checkpoint: _TrackedCheckpoint): ) self._process_persistent_checkpoint(checkpoint) - def new_handle_checkpoint(self, checkpoint): - # TODO(justinvyu): Support checkpoint book-keeping for the new checkpoint API. - # (keeping track of the top k heap, getting the latest checkpoint, etc.) - print("Trial checkpoint manager handling the checkpoint:\n", checkpoint) - pass - def on_checkpoint(self, checkpoint: _TrackedCheckpoint): """Ray Tune's entry point to handle a checkpoint.""" - from ray.train._internal.storage import _use_storage_context - from ray.train.checkpoint import Checkpoint as NewCheckpoint - - if _use_storage_context() and isinstance(checkpoint, NewCheckpoint): - return self.new_handle_checkpoint(checkpoint) - # Todo (krfricke): Replace with handle_checkpoint. self.handle_checkpoint(checkpoint) diff --git a/python/ray/tune/execution/trial_runner.py b/python/ray/tune/execution/trial_runner.py index 6ab9193f1541..c8cc15d50bfc 100644 --- a/python/ray/tune/execution/trial_runner.py +++ b/python/ray/tune/execution/trial_runner.py @@ -970,13 +970,19 @@ def _process_trial_save( """ logger.debug("Trial %s: Processing trial save.", trial) - from ray.train.checkpoint import Checkpoint as NewCheckpoint + from ray.train._internal.checkpoint_manager import ( + _TrackedCheckpoint as _NewTrackedCheckpoint, + ) try: - if _use_storage_context() and isinstance(checkpoint_value, NewCheckpoint): - # TODO(justinvyu): Update callbacks to take in a new Checkpoint - # rather than a _TrackedCheckpoint, then call the on_checkpoint hook. + if _use_storage_context() and isinstance( + checkpoint_value, _NewTrackedCheckpoint + ): + # TODO(justinvyu): Update callbacks to take in a _NewTrackedCheckpoint trial.on_checkpoint(checkpoint_value) + + self._checkpoint_manager.on_trial_checkpoint(trial) + self._mark_trial_to_checkpoint(trial) else: trial.saving_to.dir_or_data = checkpoint_value self._callbacks.on_checkpoint( @@ -987,9 +993,9 @@ 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._mark_trial_to_checkpoint(trial) + self._checkpoint_manager.on_trial_checkpoint(trial) + if trial.checkpoint.storage_mode != CheckpointStorage.MEMORY: + self._mark_trial_to_checkpoint(trial) except Exception as e: if ( isinstance(e, _HeadNodeSyncDeprecationWarning) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index d7fdc4f1df44..73d393fdc3af 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -14,7 +14,7 @@ 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.train._internal.storage import StorageContext +from ray.train._internal.storage import _use_storage_context, StorageContext from ray.exceptions import RayActorError from ray.tune.error import _AbortTrialExecution from ray.tune.execution.ray_trial_executor import _class_cache @@ -1026,6 +1026,19 @@ def _schedule_trial_save( result = result or trial.last_result + if _use_storage_context(): + assert ( + storage == CheckpointStorage.PERSISTENT + ), "Memory checkpoints are no longer supported in the new persistence mode." + self._schedule_trial_task( + trial=trial, + method_name="save", + on_result=self._on_saving_result, + on_error=self._trial_task_failure, + ) + # TODO(justinvyu): Remove the return value? + return + if storage == CheckpointStorage.MEMORY: future = self._schedule_trial_task( trial=trial, @@ -1060,6 +1073,10 @@ def _schedule_trial_save( def _schedule_trial_restore(self, trial: Trial) -> bool: checkpoint = trial.checkpoint + if _use_storage_context(): + # TODO(justinvyu): Skipping restoration altogether for now. + return False + if checkpoint.dir_or_data is None: logger.debug(f"Not restoring trial {trial}: No checkpoint found.") return False diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index b502022a6064..dbd1d1838a78 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -28,6 +28,10 @@ import ray.cloudpickle as cloudpickle from ray.exceptions import RayActorError, RayTaskError from ray.train._internal.storage import _use_storage_context, StorageContext +from ray.train._internal.checkpoint_manager import ( + _TrackedCheckpoint as _NewTrackedCheckpoint, + _CheckpointManager as _NewCheckpointManager, +) from ray.tune import TuneError from ray.tune.error import _TuneRestoreError from ray.tune.execution.checkpoint_manager import _CheckpointManager @@ -521,10 +525,15 @@ def __init__( self.checkpoint_config = checkpoint_config - self.checkpoint_manager = _CheckpointManager( - checkpoint_config=self.checkpoint_config, - delete_fn=_CheckpointDeleter(self._trainable_name(), self.runner), - ) + if _use_storage_context(): + self.checkpoint_manager = _NewCheckpointManager( + checkpoint_config=self.checkpoint_config + ) + else: + self.checkpoint_manager = _CheckpointManager( + checkpoint_config=self.checkpoint_config, + delete_fn=_CheckpointDeleter(self._trainable_name(), self.runner), + ) # Restoration fields self.restore_path = restore_path @@ -780,6 +789,8 @@ def checkpoint(self): If the trial is in ERROR state, the most recent PERSISTENT checkpoint is returned. """ + if _use_storage_context(): + return self.checkpoint_manager.latest_checkpoint if self.status == Trial.ERROR: checkpoint = self.checkpoint_manager.newest_persistent_checkpoint else: @@ -890,9 +901,10 @@ def set_runner(self, runner): self._default_result_or_future = runner.get_auto_filled_metrics.remote( debug_metrics_only=True ) - self.checkpoint_manager.set_delete_fn( - _CheckpointDeleter(self._trainable_name(), runner) - ) + if not _use_storage_context(): + self.checkpoint_manager.set_delete_fn( + _CheckpointDeleter(self._trainable_name(), runner) + ) # No need to invalidate state cache: runner is not stored in json # self.invalidate_json_state() @@ -1004,7 +1016,15 @@ def on_checkpoint(self, checkpoint: _TrackedCheckpoint): Args: checkpoint: Checkpoint taken. """ - self.checkpoint_manager.on_checkpoint(checkpoint) + if _use_storage_context(): + from ray.train._internal.checkpoint_manager import ( + _TrackedCheckpoint as _NewTrackedCheckpoint, + ) + + assert isinstance(checkpoint, _NewTrackedCheckpoint) + self.checkpoint_manager.register_checkpoint(checkpoint) + else: + self.checkpoint_manager.on_checkpoint(checkpoint) self.invalidate_json_state() def on_restore(self): diff --git a/python/ray/tune/trainable/function_trainable.py b/python/ray/tune/trainable/function_trainable.py index e8b1bf3ddb01..110dc9ff77f2 100644 --- a/python/ray/tune/trainable/function_trainable.py +++ b/python/ray/tune/trainable/function_trainable.py @@ -243,6 +243,9 @@ def has_new_checkpoint(self): return self._fresh_checkpoint def get_checkpoint(self): + # NOTE: This is not the same as `train.get_checkpoint`. + # This is used internally by `FunctionTrainable.save_checkpoint`. + # `loaded_checkpoint` is the checkpoint accessible by the user. self._fresh_checkpoint = False return self._last_checkpoint @@ -251,6 +254,9 @@ def _start(self): def report(self, metrics: Dict, *, checkpoint: Optional[Checkpoint] = None) -> None: from ray.train._internal.storage import _use_storage_context + from ray.train._internal.checkpoint_manager import ( + _TrackedCheckpoint as _NewTrackedCheckpoint, + ) from ray.train.checkpoint import Checkpoint as NewCheckpoint # TODO(xwjiang): Tons of optimizations. @@ -265,7 +271,12 @@ def report(self, metrics: Dict, *, checkpoint: Optional[Checkpoint] = None) -> N checkpoint, ) self._fresh_checkpoint = True - self._last_checkpoint = checkpoint + # TODO(justinvyu): `metrics` doesn't include the autofilled metrics + # like `training_iteration` and `time_total_s`. + # Should the session be the source of truth for these metrics? + self._last_checkpoint = _NewTrackedCheckpoint( + checkpoint=checkpoint, metrics=metrics + ) else: if checkpoint: training_iteration = self._get_training_iteration() @@ -280,12 +291,14 @@ def report(self, metrics: Dict, *, checkpoint: Optional[Checkpoint] = None) -> N def loaded_checkpoint(self) -> Optional[Checkpoint]: if self._last_checkpoint: from ray.train._internal.storage import _use_storage_context - from ray.train.checkpoint import Checkpoint as NewCheckpoint + from ray.train._internal.checkpoint_manager import ( + _TrackedCheckpoint as _NewTrackedCheckpoint, + ) if _use_storage_context() and isinstance( - self._last_checkpoint, NewCheckpoint + self._last_checkpoint, _NewTrackedCheckpoint ): - return self._last_checkpoint + return self._last_checkpoint.checkpoint assert isinstance(self._last_checkpoint, str) return Checkpoint.from_directory(self._last_checkpoint) @@ -489,9 +502,11 @@ def save_checkpoint(self, checkpoint_dir: str = ""): checkpoint = self._status_reporter.get_checkpoint() from ray.train._internal.storage import _use_storage_context - from ray.train.checkpoint import Checkpoint as NewCheckpoint + from ray.train._internal.checkpoint_manager import ( + _TrackedCheckpoint as _NewTrackedCheckpoint, + ) - if _use_storage_context() and isinstance(checkpoint, NewCheckpoint): + if _use_storage_context() and isinstance(checkpoint, _NewTrackedCheckpoint): print( "Returning checkpoint from FunctionTrainable.save_checkpoint:\n", checkpoint, diff --git a/python/ray/tune/trainable/trainable.py b/python/ray/tune/trainable/trainable.py index 25ea02d19da4..2e0c995028b1 100644 --- a/python/ray/tune/trainable/trainable.py +++ b/python/ray/tune/trainable/trainable.py @@ -501,10 +501,12 @@ def save( # User saves checkpoint checkpoint_dict_or_path = self.save_checkpoint(checkpoint_dir) - from ray.train.checkpoint import Checkpoint as NewCheckpoint + from ray.train._internal.checkpoint_manager import ( + _TrackedCheckpoint as _NewTrackedCheckpoint, + ) if _use_storage_context() and isinstance( - checkpoint_dict_or_path, NewCheckpoint + checkpoint_dict_or_path, _NewTrackedCheckpoint ): return checkpoint_dict_or_path From 24f441a1a912362294a2af0f14415c311ffbef2f Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 1 Aug 2023 17:50:09 -0700 Subject: [PATCH 033/108] Update result to return a train.Checkpoint to the user Signed-off-by: Justin Yu --- python/ray/tune/result_grid.py | 34 +++++++++++++++++++++++++--------- 1 file changed, 25 insertions(+), 9 deletions(-) diff --git a/python/ray/tune/result_grid.py b/python/ray/tune/result_grid.py index 500811a6cd53..6e31e1f196f5 100644 --- a/python/ray/tune/result_grid.py +++ b/python/ray/tune/result_grid.py @@ -4,6 +4,7 @@ from typing import Optional, Union from ray.air.result import Result +from ray.train._internal.storage import _use_storage_context from ray.cloudpickle import cloudpickle from ray.exceptions import RayTaskError from ray.tune.analysis import ExperimentAnalysis @@ -279,16 +280,31 @@ def _trial_to_result(self, trial: Trial) -> Result: if trial.uses_cloud_checkpointing else None ) - checkpoint = trial.checkpoint.to_air_checkpoint( - local_to_remote_path_fn, - ) - best_checkpoints = [ - ( - checkpoint.to_air_checkpoint(local_to_remote_path_fn), - checkpoint.metrics, + + if _use_storage_context(): + from ray.train._internal.checkpoint_manager import ( + _CheckpointManager as _NewCheckpointManager, ) - for checkpoint in trial.get_trial_checkpoints() - ] + + assert isinstance( + trial.checkpoint_manager, _NewCheckpointManager + ), trial.checkpoint + checkpoint = trial.checkpoint_manager.latest_checkpoint.checkpoint + best_checkpoints = [ + (tracked_checkpoint.checkpoint, tracked_checkpoint.metrics) + for tracked_checkpoint in trial.checkpoint_manager.best_checkpoints + ] + else: + checkpoint = trial.checkpoint.to_air_checkpoint( + local_to_remote_path_fn, + ) + best_checkpoints = [ + ( + checkpoint.to_air_checkpoint(local_to_remote_path_fn), + checkpoint.metrics, + ) + for checkpoint in trial.get_trial_checkpoints() + ] result = Result( checkpoint=checkpoint, From 504ed54bb4de92ad4874a434f8c0248d91babb32 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 1 Aug 2023 17:50:34 -0700 Subject: [PATCH 034/108] Update e2e test to try multiple ckpt configs for trainer test Signed-off-by: Justin Yu --- .../ray/train/tests/test_new_persistence.py | 163 +++++++++++++----- 1 file changed, 117 insertions(+), 46 deletions(-) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index 58c101d7bd14..840b432e5b47 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -5,15 +5,21 @@ import pytest import tempfile import time +from typing import Optional, Tuple import pyarrow.fs from ray import train, tune -from ray.air.tests.test_checkpoints import mock_s3_bucket_uri +from ray.air.constants import EXPR_RESULT_FILE from ray.train._internal.storage import _download_from_fs_path from ray.train.checkpoint import Checkpoint as NewCheckpoint from ray.train.data_parallel_trainer import DataParallelTrainer +from ray.air.tests.test_checkpoints import mock_s3_bucket_uri + + +_SCORE_KEY = "score" + @contextmanager def dummy_context_manager(): @@ -40,6 +46,28 @@ def _create_mock_custom_fs(custom_fs_root_dir: Path) -> pyarrow.fs.FileSystem: return storage_filesystem +@contextmanager +def _resolve_storage_type( + storage_path_type: str, tmp_path: Path +) -> Tuple[str, Optional[pyarrow.fs.FileSystem]]: + storage_path, storage_filesystem = None, None + + context_manager = ( + mock_s3_bucket_uri if storage_path_type == "cloud" else dummy_context_manager + ) + + with context_manager() as cloud_storage_path: + if storage_path_type == "nfs": + storage_path = str(tmp_path / "fake_nfs") + elif storage_path_type == "cloud": + storage_path = str(cloud_storage_path) + elif storage_path_type == "custom_fs": + storage_path = "mock_bucket" + storage_filesystem = _create_mock_custom_fs(tmp_path / "custom_fs") + + yield storage_path, storage_filesystem + + def train_fn(config): in_trainer = config.get("in_trainer", False) if in_trainer: @@ -79,7 +107,10 @@ def train_fn(config): with open(os.path.join(temp_dir, checkpoint_file_name), "wb") as f: pickle.dump({"iter": i}, f) - train.report({"iter": i}, checkpoint=NewCheckpoint.from_directory(temp_dir)) + train.report( + {"iter": i, _SCORE_KEY: i}, + checkpoint=NewCheckpoint.from_directory(temp_dir), + ) @pytest.mark.parametrize("storage_path_type", [None, "nfs", "cloud", "custom_fs"]) @@ -117,24 +148,12 @@ def test_tuner(monkeypatch, storage_path_type, tmp_path): LOCAL_CACHE_DIR = tmp_path / "ray_results" monkeypatch.setenv("RAY_AIR_LOCAL_CACHE_DIR", str(LOCAL_CACHE_DIR)) - context_manager = ( - mock_s3_bucket_uri if storage_path_type == "cloud" else dummy_context_manager - ) - exp_name = "simple_persistence_test" - with context_manager() as cloud_storage_path: - storage_filesystem = None - if storage_path_type is None: - storage_path = None - elif storage_path_type == "nfs": - storage_path = str(tmp_path / "fake_nfs") - elif storage_path_type == "cloud": - storage_path = str(cloud_storage_path) - elif storage_path_type == "custom_fs": - storage_path = "mock_bucket" - storage_filesystem = _create_mock_custom_fs(tmp_path / "custom_fs") - + with _resolve_storage_type(storage_path_type, tmp_path) as ( + storage_path, + storage_filesystem, + ): NUM_ITERATIONS = 6 # == num_checkpoints == num_artifacts NUM_TRIALS = 2 tuner = tune.Tuner( @@ -175,14 +194,27 @@ def test_tuner(monkeypatch, storage_path_type, tmp_path): assert len(list(exp_dir.glob("tuner.pkl"))) == 1 -@pytest.mark.parametrize("storage_path_type", [None]) -def test_trainer(tmp_path, monkeypatch, storage_path_type): +@pytest.mark.parametrize("storage_path_type", [None, "nfs", "cloud", "custom_fs"]) +@pytest.mark.parametrize( + "checkpoint_config", + [ + train.CheckpointConfig(), + train.CheckpointConfig(num_to_keep=2), + train.CheckpointConfig( + num_to_keep=1, + checkpoint_score_attribute=_SCORE_KEY, + checkpoint_score_order="max", + ), + ], +) +def test_trainer( + tmp_path, monkeypatch, storage_path_type, checkpoint_config: train.CheckpointConfig +): """ TODO(justinvyu): Test for these once implemented: - artifacts - - restoration + - restoration, train.get_checkpoint - accessing checkpoints from Result object - - top k checkpoints (num_to_keep) trainer_new_persistence ├── experiment_state-2023-07-28_10-00-38.json @@ -208,36 +240,55 @@ def test_trainer(tmp_path, monkeypatch, storage_path_type): ├── artifact-1-1.txt └── ... """ - LOCAL_CACHE_DIR = tmp_path / "ray_results" monkeypatch.setenv("RAY_AIR_LOCAL_CACHE_DIR", str(LOCAL_CACHE_DIR)) exp_name = "trainer_new_persistence" - if storage_path_type is None: - storage_path = None - elif storage_path_type == "nfs": - storage_path = str(tmp_path / "fake_nfs") - - trainer = DataParallelTrainer( - train_fn, - train_loop_config={"in_trainer": True}, - scaling_config=train.ScalingConfig(num_workers=2), - run_config=train.RunConfig(storage_path=storage_path, name=exp_name, verbose=0), - ) - trainer.fit() - - local_inspect_dir = tmp_path / "inspect" - if storage_path: - # if storage_filesystem: - # fs, fs_path = storage_filesystem, storage_path - # else: - fs, fs_path = pyarrow.fs.FileSystem.from_uri(storage_path) - _download_from_fs_path( - fs=fs, fs_path=fs_path, local_path=str(local_inspect_dir) + with _resolve_storage_type(storage_path_type, tmp_path) as ( + storage_path, + storage_filesystem, + ): + NUM_ITERATIONS = 6 + NUM_WORKERS = 2 + trainer = DataParallelTrainer( + train_fn, + train_loop_config={"in_trainer": True, "num_iterations": NUM_ITERATIONS}, + scaling_config=train.ScalingConfig(num_workers=2), + run_config=train.RunConfig( + storage_path=storage_path, + storage_filesystem=storage_filesystem, + name=exp_name, + verbose=0, + checkpoint_config=checkpoint_config, + ), ) - else: - local_inspect_dir = LOCAL_CACHE_DIR + result = trainer.fit() + local_inspect_dir = tmp_path / "inspect" + if storage_path: + if storage_filesystem: + fs, storage_fs_path = storage_filesystem, storage_path + else: + fs, storage_fs_path = pyarrow.fs.FileSystem.from_uri(storage_path) + _download_from_fs_path( + fs=fs, fs_path=storage_fs_path, local_path=str(local_inspect_dir) + ) + else: + fs, storage_fs_path = pyarrow.fs.LocalFileSystem(), str(LOCAL_CACHE_DIR) + local_inspect_dir = LOCAL_CACHE_DIR + + # First, inspect that the result object returns the correct paths. + # TODO(justinvyu): [custom_fs_path_expansion] + # This doesn't work for the `custom_fs` case right now + # because Result.path <- Trial.remote_path/local_path <- Experiment.path, + # which expands the storage path to an absolute path. + # We shouldn't expand the storage path to an absolute path if a custom fs is passed. + if not storage_filesystem: + _, trial_fs_path = pyarrow.fs.FileSystem.from_uri(result.path) + assert trial_fs_path.startswith(storage_fs_path) + assert result.checkpoint.path.startswith(trial_fs_path) + + # Second, inspect the contents of the storage path assert len(list(local_inspect_dir.glob("*"))) == 1 # Only expect 1 experiment dir exp_dir = local_inspect_dir / exp_name @@ -248,6 +299,26 @@ def test_trainer(tmp_path, monkeypatch, storage_path_type): assert len(list(exp_dir.glob("trainer.pkl"))) == 1 # Files synced by the worker + assert len(list(exp_dir.glob("DataParallelTrainer_*"))) == 1 + for trial_dir in exp_dir.glob("DataParallelTrainer_*"): + # If set, expect num_to_keep. Otherwise, expect to see all of them. + expected_num_checkpoints = checkpoint_config.num_to_keep or NUM_ITERATIONS + + assert len(list(trial_dir.glob("checkpoint_*"))) == expected_num_checkpoints + for checkpoint_dir in trial_dir.glob("checkpoint_*"): + # 1 checkpoint shard per worker. + assert len(list(checkpoint_dir.glob("dummy-*.pkl"))) == NUM_WORKERS + + # NOTE: These next 2 are technically synced by the driver. + # TODO(justinvyu): In a follow-up PR, artifacts will be synced by the workers. + # TODO(justinvyu): In a follow-up PR, the rank folders will be removed. + # TODO(justinvyu): [custom_fs_path_expansion] Same issue as above. + if not storage_filesystem: + assert ( + len(list(trial_dir.glob("rank_*/artifact-*"))) + == NUM_ITERATIONS * NUM_WORKERS + ) + assert len(list(trial_dir.glob(EXPR_RESULT_FILE))) == 1 if __name__ == "__main__": From b9eb88fa1857dd9b99c3f694c4177bb64c579499 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 1 Aug 2023 17:55:36 -0700 Subject: [PATCH 035/108] Fix lint for trial.py Signed-off-by: Justin Yu --- python/ray/tune/experiment/trial.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index dbd1d1838a78..52234f78f3e8 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -29,7 +29,6 @@ from ray.exceptions import RayActorError, RayTaskError from ray.train._internal.storage import _use_storage_context, StorageContext from ray.train._internal.checkpoint_manager import ( - _TrackedCheckpoint as _NewTrackedCheckpoint, _CheckpointManager as _NewCheckpointManager, ) from ray.tune import TuneError From 7cc74d9b1ab4a57e5405ef4a23f502f1968d8278 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 2 Aug 2023 09:03:58 -0700 Subject: [PATCH 036/108] Rename _TrackedCheckpoint -> _TrainingResult Signed-off-by: Justin Yu --- .../ray/train/_internal/checkpoint_manager.py | 58 ++++++++++--------- .../train/tests/test_checkpoint_manager.py | 34 +++++------ 2 files changed, 49 insertions(+), 43 deletions(-) diff --git a/python/ray/train/_internal/checkpoint_manager.py b/python/ray/train/_internal/checkpoint_manager.py index e7c1ed4a2ff7..6026aba75e1b 100644 --- a/python/ray/train/_internal/checkpoint_manager.py +++ b/python/ray/train/_internal/checkpoint_manager.py @@ -13,8 +13,8 @@ logger = logging.getLogger(__name__) -class _TrackedCheckpoint: - """Checkpoint tracked by a checkpoint manager.""" +class _TrainingResult: + """A (checkpoint, metrics) result reported by the user.""" def __init__(self, checkpoint: Checkpoint, metrics: Dict[str, Any]): self.checkpoint = checkpoint @@ -52,9 +52,12 @@ def _insert_into_sorted_list(list: List[Any], item: Any, key: Callable[[Any], An class _CheckpointManager: """Checkpoint manager that handles checkpoint book-keeping for a trial. - Main purpose of this abstraction is to keep the top K checkpoints based on + The main purpose of this abstraction is to keep the top K checkpoints based on recency/a user-provided metric. + NOTE: This class interacts with `_TrainingResult` objects, which are + (checkpoint, metrics) pairs. This is to order checkpoints by metrics. + Args: checkpoint_config: Defines how many and which checkpoints to keep. """ @@ -63,12 +66,12 @@ def __init__(self, checkpoint_config: Optional[CheckpointConfig]): self._checkpoint_config = checkpoint_config or CheckpointConfig() # List of checkpoints ordered by ascending score. - self._checkpoints: List[_TrackedCheckpoint] = [] + self._checkpoint_results: List[_TrainingResult] = [] # The latest registered checkpoint. # This should never be immediately deleted upon registration, # even if it's not in the top K checkpoints, based on score. - self._latest_checkpoint: _TrackedCheckpoint = None + self._latest_checkpoint_result: _TrainingResult = None if ( self._checkpoint_config.num_to_keep is not None @@ -79,7 +82,7 @@ def __init__(self, checkpoint_config: Optional[CheckpointConfig]): f"{self._checkpoint_config.num_to_keep}" ) - def register_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): + def register_checkpoint(self, checkpoint_result: _TrainingResult): """Register new checkpoint and add to bookkeeping. This method will register a new checkpoint and add it to the internal @@ -90,39 +93,42 @@ def register_checkpoint(self, tracked_checkpoint: _TrackedCheckpoint): Args: checkpoint: Tracked checkpoint object to add to bookkeeping. """ - self._latest_checkpoint = tracked_checkpoint + self._latest_checkpoint_result = checkpoint_result if self._checkpoint_config.checkpoint_score_attribute is not None: # If we're ordering by a score, insert the checkpoint # so that the list remains sorted. _insert_into_sorted_list( - self._checkpoints, tracked_checkpoint, key=self._get_checkpoint_score + self._checkpoint_results, + checkpoint_result, + key=self._get_checkpoint_score, ) else: # If no metric is provided, just append (ordering by time of registration). - self._checkpoints.append(tracked_checkpoint) + self._checkpoint_results.append(checkpoint_result) if self._checkpoint_config.num_to_keep is not None: # Delete the bottom (N - K) checkpoints - worst_checkpoints = set( - self._checkpoints[: -self._checkpoint_config.num_to_keep] + worst_results = set( + self._checkpoint_results[: -self._checkpoint_config.num_to_keep] ) # Except for the latest checkpoint. - checkpoints_to_delete = worst_checkpoints - {self._latest_checkpoint} + results_to_delete = worst_results - {self._latest_checkpoint_result} # Update internal state before actually deleting them. - self._checkpoints = [ - tracked_checkpoint - for tracked_checkpoint in self._checkpoints - if tracked_checkpoint not in checkpoints_to_delete + self._checkpoint_results = [ + checkpoint_result + for checkpoint_result in self._checkpoint_results + if checkpoint_result not in results_to_delete ] - for checkpoint_to_delete in checkpoints_to_delete: - checkpoint = checkpoint_to_delete.checkpoint + for checkpoint_result in results_to_delete: + checkpoint = checkpoint_result.checkpoint + logger.debug("Deleting checkpoint: ", checkpoint) _delete_fs_path(fs=checkpoint.filesystem, fs_path=checkpoint.path) def _get_checkpoint_score( - self, checkpoint: _TrackedCheckpoint + self, checkpoint: _TrainingResult ) -> Tuple[bool, numbers.Number]: """Get the score for a checkpoint, according to checkpoint config. @@ -170,15 +176,15 @@ def _get_checkpoint_score( ) @property - def best_checkpoint(self) -> Optional[_TrackedCheckpoint]: - return self._checkpoints[-1] if self._checkpoints else None + def best_checkpoint_result(self) -> Optional[_TrainingResult]: + return self._checkpoint_results[-1] if self._checkpoint_results else None @property - def latest_checkpoint(self) -> Optional[_TrackedCheckpoint]: - return self._latest_checkpoint + def latest_checkpoint_result(self) -> Optional[_TrainingResult]: + return self._latest_checkpoint_result @property - def best_checkpoints(self) -> List[_TrackedCheckpoint]: + def best_checkpoint_results(self) -> List[_TrainingResult]: if self._checkpoint_config.num_to_keep is None: - return self._checkpoints - return self._checkpoints[-self._checkpoint_config.num_to_keep :] + return self._checkpoint_results + return self._checkpoint_results[-self._checkpoint_config.num_to_keep :] diff --git a/python/ray/train/tests/test_checkpoint_manager.py b/python/ray/train/tests/test_checkpoint_manager.py index 9000487bd4c1..746f8ec93b28 100644 --- a/python/ray/train/tests/test_checkpoint_manager.py +++ b/python/ray/train/tests/test_checkpoint_manager.py @@ -7,7 +7,7 @@ from ray.train import CheckpointConfig from ray.train._internal.checkpoint_manager import ( _CheckpointManager, - _TrackedCheckpoint, + _TrainingResult, ) from ray.train.checkpoint import Checkpoint @@ -29,13 +29,13 @@ def test_unlimited_checkpoints(checkpoint_paths: List[str]): for i in range(10): manager.register_checkpoint( - _TrackedCheckpoint( + _TrainingResult( checkpoint=Checkpoint.from_directory(checkpoint_paths[i]), metrics={"iter": i}, ) ) - assert len(manager.best_checkpoints) == 10 + assert len(manager.best_checkpoint_results) == 10 def test_limited_checkpoints(checkpoint_paths: List[str]): @@ -43,18 +43,18 @@ def test_limited_checkpoints(checkpoint_paths: List[str]): for i in range(10): manager.register_checkpoint( - _TrackedCheckpoint( + _TrainingResult( checkpoint=Checkpoint.from_directory(checkpoint_paths[i]), metrics={"iter": i}, ) ) - assert len(manager.best_checkpoints) == 2 + assert len(manager.best_checkpoint_results) == 2 # Keep the latest checkpoints if no metric is given. assert { tracked_checkpoint.metrics["iter"] - for tracked_checkpoint in manager.best_checkpoints + for tracked_checkpoint in manager.best_checkpoint_results } == {8, 9} # The first 8 checkpoints should be deleted. @@ -82,7 +82,7 @@ def test_keep_checkpoints_by_score(order, checkpoint_paths): for i in range(10): score = random.random() manager.register_checkpoint( - _TrackedCheckpoint( + _TrainingResult( checkpoint=Checkpoint.from_directory(checkpoint_paths[i]), metrics={"iter": i, score_attribute: score}, ) @@ -92,13 +92,13 @@ def test_keep_checkpoints_by_score(order, checkpoint_paths): sorted_scores = sorted(scores, reverse=order == "max") assert set(sorted_scores[:num_to_keep]) == { tracked_checkpoint.metrics[score_attribute] - for tracked_checkpoint in manager.best_checkpoints + for tracked_checkpoint in manager.best_checkpoint_results } # Make sure the bottom checkpoints are deleted. best_checkpoint_iters = { tracked_checkpoint.metrics["iter"] - for tracked_checkpoint in manager.best_checkpoints + for tracked_checkpoint in manager.best_checkpoint_results } for i, checkpoint_path in enumerate(checkpoint_paths): if i in best_checkpoint_iters or i == 9: @@ -118,34 +118,34 @@ def test_keep_latest_checkpoint(checkpoint_paths): ) manager.register_checkpoint( - _TrackedCheckpoint( + _TrainingResult( checkpoint=Checkpoint.from_directory(checkpoint_paths[0]), metrics={"score": 3.0}, ) ) manager.register_checkpoint( - _TrackedCheckpoint( + _TrainingResult( checkpoint=Checkpoint.from_directory(checkpoint_paths[1]), metrics={"score": 2.0}, ) ) manager.register_checkpoint( - _TrackedCheckpoint( + _TrainingResult( checkpoint=Checkpoint.from_directory(checkpoint_paths[2]), metrics={"score": 1.0}, ) ) - assert len(manager.best_checkpoints) == 2 + assert len(manager.best_checkpoint_results) == 2 # The latest checkpoint with the lowest score should not be deleted yet. - assert manager.latest_checkpoint.metrics["score"] == 1.0 + assert manager.latest_checkpoint_result.metrics["score"] == 1.0 # The latest checkpoint with the lowest score should not be deleted yet. assert Path(checkpoint_paths[2]).exists() manager.register_checkpoint( - _TrackedCheckpoint( + _TrainingResult( checkpoint=Checkpoint.from_directory(checkpoint_paths[3]), metrics={"score": 0.0}, ) @@ -155,7 +155,7 @@ def test_keep_latest_checkpoint(checkpoint_paths): assert not Path(checkpoint_paths[2]).exists() # Quick sanity check to make sure that the new checkpoint is kept. - assert manager.latest_checkpoint.metrics["score"] == 0.0 + assert manager.latest_checkpoint_result.metrics["score"] == 0.0 assert Path(checkpoint_paths[3]).exists() # The original 2 checkpoints should still exist @@ -181,7 +181,7 @@ def test_nested_get_checkpoint_score(metrics): ) ) - tracked_checkpoint = _TrackedCheckpoint(checkpoint=None, metrics=metrics) + tracked_checkpoint = _TrainingResult(checkpoint=None, metrics=metrics) assert manager._get_checkpoint_score(tracked_checkpoint) == (True, 5.0) From 8da04775e6dca2b08c28a047678a7bacf11cfbfc Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 2 Aug 2023 09:43:27 -0700 Subject: [PATCH 037/108] Fixes after merging latest ckpt manager changes Signed-off-by: Justin Yu --- .../ray/train/_internal/checkpoint_manager.py | 11 ++++++++--- python/ray/tune/execution/trial_runner.py | 10 +++------- python/ray/tune/experiment/trial.py | 9 ++++----- python/ray/tune/result_grid.py | 11 +++++------ .../ray/tune/trainable/function_trainable.py | 18 ++++++------------ python/ray/tune/trainable/trainable.py | 6 ++---- 6 files changed, 28 insertions(+), 37 deletions(-) diff --git a/python/ray/train/_internal/checkpoint_manager.py b/python/ray/train/_internal/checkpoint_manager.py index 6026aba75e1b..45888c9a1a42 100644 --- a/python/ray/train/_internal/checkpoint_manager.py +++ b/python/ray/train/_internal/checkpoint_manager.py @@ -1,14 +1,16 @@ import logging import numbers -from typing import Any, Callable, Dict, List, Optional, Tuple +from typing import Any, Callable, Dict, List, Optional, Tuple, TYPE_CHECKING from ray._private.dict import flatten_dict from ray.air.config import MAX from ray.air._internal.util import is_nan -from ray.train import CheckpointConfig from ray.train._internal.storage import _delete_fs_path from ray.train.checkpoint import Checkpoint +if TYPE_CHECKING: + from ray.train import CheckpointConfig + logger = logging.getLogger(__name__) @@ -62,7 +64,10 @@ class _CheckpointManager: checkpoint_config: Defines how many and which checkpoints to keep. """ - def __init__(self, checkpoint_config: Optional[CheckpointConfig]): + def __init__(self, checkpoint_config: Optional["CheckpointConfig"]): + # TODO(justinvyu): Fix this circular dependency. + from ray.train import CheckpointConfig + self._checkpoint_config = checkpoint_config or CheckpointConfig() # List of checkpoints ordered by ascending score. diff --git a/python/ray/tune/execution/trial_runner.py b/python/ray/tune/execution/trial_runner.py index c8cc15d50bfc..906a9766e3d7 100644 --- a/python/ray/tune/execution/trial_runner.py +++ b/python/ray/tune/execution/trial_runner.py @@ -970,15 +970,11 @@ def _process_trial_save( """ logger.debug("Trial %s: Processing trial save.", trial) - from ray.train._internal.checkpoint_manager import ( - _TrackedCheckpoint as _NewTrackedCheckpoint, - ) + from ray.train._internal.checkpoint_manager import _TrainingResult try: - if _use_storage_context() and isinstance( - checkpoint_value, _NewTrackedCheckpoint - ): - # TODO(justinvyu): Update callbacks to take in a _NewTrackedCheckpoint + if _use_storage_context() and isinstance(checkpoint_value, _TrainingResult): + # TODO(justinvyu): Update callbacks to take in a _TrainingResult trial.on_checkpoint(checkpoint_value) self._checkpoint_manager.on_trial_checkpoint(trial) diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index 52234f78f3e8..e641be4c5261 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -789,7 +789,8 @@ def checkpoint(self): is returned. """ if _use_storage_context(): - return self.checkpoint_manager.latest_checkpoint + return self.checkpoint_manager.latest_checkpoint_result + if self.status == Trial.ERROR: checkpoint = self.checkpoint_manager.newest_persistent_checkpoint else: @@ -1016,11 +1017,9 @@ def on_checkpoint(self, checkpoint: _TrackedCheckpoint): checkpoint: Checkpoint taken. """ if _use_storage_context(): - from ray.train._internal.checkpoint_manager import ( - _TrackedCheckpoint as _NewTrackedCheckpoint, - ) + from ray.train._internal.checkpoint_manager import _TrainingResult - assert isinstance(checkpoint, _NewTrackedCheckpoint) + assert isinstance(checkpoint, _TrainingResult) self.checkpoint_manager.register_checkpoint(checkpoint) else: self.checkpoint_manager.on_checkpoint(checkpoint) diff --git a/python/ray/tune/result_grid.py b/python/ray/tune/result_grid.py index 6e31e1f196f5..f72315518812 100644 --- a/python/ray/tune/result_grid.py +++ b/python/ray/tune/result_grid.py @@ -286,13 +286,12 @@ def _trial_to_result(self, trial: Trial) -> Result: _CheckpointManager as _NewCheckpointManager, ) - assert isinstance( - trial.checkpoint_manager, _NewCheckpointManager - ), trial.checkpoint - checkpoint = trial.checkpoint_manager.latest_checkpoint.checkpoint + assert isinstance(trial.checkpoint_manager, _NewCheckpointManager) + checkpoint = trial.checkpoint_manager.latest_checkpoint_result.checkpoint + best_checkpoint_results = trial.checkpoint_manager.best_checkpoint_results best_checkpoints = [ - (tracked_checkpoint.checkpoint, tracked_checkpoint.metrics) - for tracked_checkpoint in trial.checkpoint_manager.best_checkpoints + (checkpoint_result.checkpoint, checkpoint_result.metrics) + for checkpoint_result in best_checkpoint_results ] else: checkpoint = trial.checkpoint.to_air_checkpoint( diff --git a/python/ray/tune/trainable/function_trainable.py b/python/ray/tune/trainable/function_trainable.py index 110dc9ff77f2..221634ca85f1 100644 --- a/python/ray/tune/trainable/function_trainable.py +++ b/python/ray/tune/trainable/function_trainable.py @@ -254,9 +254,7 @@ def _start(self): def report(self, metrics: Dict, *, checkpoint: Optional[Checkpoint] = None) -> None: from ray.train._internal.storage import _use_storage_context - from ray.train._internal.checkpoint_manager import ( - _TrackedCheckpoint as _NewTrackedCheckpoint, - ) + from ray.train._internal.checkpoint_manager import _TrainingResult from ray.train.checkpoint import Checkpoint as NewCheckpoint # TODO(xwjiang): Tons of optimizations. @@ -274,7 +272,7 @@ def report(self, metrics: Dict, *, checkpoint: Optional[Checkpoint] = None) -> N # TODO(justinvyu): `metrics` doesn't include the autofilled metrics # like `training_iteration` and `time_total_s`. # Should the session be the source of truth for these metrics? - self._last_checkpoint = _NewTrackedCheckpoint( + self._last_checkpoint = _TrainingResult( checkpoint=checkpoint, metrics=metrics ) else: @@ -291,12 +289,10 @@ def report(self, metrics: Dict, *, checkpoint: Optional[Checkpoint] = None) -> N def loaded_checkpoint(self) -> Optional[Checkpoint]: if self._last_checkpoint: from ray.train._internal.storage import _use_storage_context - from ray.train._internal.checkpoint_manager import ( - _TrackedCheckpoint as _NewTrackedCheckpoint, - ) + from ray.train._internal.checkpoint_manager import _TrainingResult if _use_storage_context() and isinstance( - self._last_checkpoint, _NewTrackedCheckpoint + self._last_checkpoint, _TrainingResult ): return self._last_checkpoint.checkpoint @@ -502,11 +498,9 @@ def save_checkpoint(self, checkpoint_dir: str = ""): checkpoint = self._status_reporter.get_checkpoint() from ray.train._internal.storage import _use_storage_context - from ray.train._internal.checkpoint_manager import ( - _TrackedCheckpoint as _NewTrackedCheckpoint, - ) + from ray.train._internal.checkpoint_manager import _TrainingResult - if _use_storage_context() and isinstance(checkpoint, _NewTrackedCheckpoint): + if _use_storage_context() and isinstance(checkpoint, _TrainingResult): print( "Returning checkpoint from FunctionTrainable.save_checkpoint:\n", checkpoint, diff --git a/python/ray/tune/trainable/trainable.py b/python/ray/tune/trainable/trainable.py index 2e0c995028b1..d5488b52a8e9 100644 --- a/python/ray/tune/trainable/trainable.py +++ b/python/ray/tune/trainable/trainable.py @@ -501,12 +501,10 @@ def save( # User saves checkpoint checkpoint_dict_or_path = self.save_checkpoint(checkpoint_dir) - from ray.train._internal.checkpoint_manager import ( - _TrackedCheckpoint as _NewTrackedCheckpoint, - ) + from ray.train._internal.checkpoint_manager import _TrainingResult if _use_storage_context() and isinstance( - checkpoint_dict_or_path, _NewTrackedCheckpoint + checkpoint_dict_or_path, _TrainingResult ): return checkpoint_dict_or_path From 255b149738e9028915722b0819854099765bea84 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 2 Aug 2023 10:35:54 -0700 Subject: [PATCH 038/108] Remove prints / convert to logger.debug Signed-off-by: Justin Yu --- python/ray/train/data_parallel_trainer.py | 8 +++----- python/ray/train/trainer.py | 8 ++++---- python/ray/tune/trainable/function_trainable.py | 9 +-------- 3 files changed, 8 insertions(+), 17 deletions(-) diff --git a/python/ray/train/data_parallel_trainer.py b/python/ray/train/data_parallel_trainer.py index a59209477536..ed3786a1e262 100644 --- a/python/ray/train/data_parallel_trainer.py +++ b/python/ray/train/data_parallel_trainer.py @@ -434,11 +434,9 @@ def _report(self, training_iterator: TrainingIterator) -> None: and len(first_worker_results) == 2 ) metrics, checkpoint = first_worker_results - print( - "Calling _StatusReporter.report with (metrics, checkpoint): ", - metrics, - "\n", - checkpoint, + logger.debug( + "Report (metrics, checkpoint) to the Tune session:\n" + f" metrics={metrics}\n checkpoint={checkpoint}" ) train.report(metrics, checkpoint=checkpoint) else: diff --git a/python/ray/train/trainer.py b/python/ray/train/trainer.py index 07b033041b97..7eb45f3a4299 100644 --- a/python/ray/train/trainer.py +++ b/python/ray/train/trainer.py @@ -202,11 +202,11 @@ def _process_checkpoint_results(self, checkpoint_results: List[TrainingResult]): checkpoints = [ checkpoint_result.data for checkpoint_result in checkpoint_results ] - print("TrainingIterator processed checkpoints: ", checkpoints) assert all(isinstance(checkpoint, NewCheckpoint) for checkpoint in checkpoints) - # All we need to do is track which checkpoint to use for book-keeping. - # Let's use rank 0, and attach metadata about the other checkpoints. + # We need to track one of the checkpoints for book-keeping. + # Let's use the rank 0 checkpoint. + # (They should all point to the same checkpoint path anyways.) self._checkpoint_to_report = checkpoints[0] def _fetch_next_result(self) -> Optional[List[Dict]]: @@ -229,7 +229,7 @@ def _fetch_next_result(self) -> Optional[List[Dict]]: result_type = first_result.type if result_type is TrainingResultType.REPORT: if _use_storage_context(): - # TODO(justinvyu): This tuple is what TrainingResult should be. + # TODO(justinvyu): Use the new _TrainingResult instead. result_data = [ (r.data, None if rank > 0 else self._checkpoint_to_report) for rank, r in enumerate(results) diff --git a/python/ray/tune/trainable/function_trainable.py b/python/ray/tune/trainable/function_trainable.py index 221634ca85f1..562bf59b1f69 100644 --- a/python/ray/tune/trainable/function_trainable.py +++ b/python/ray/tune/trainable/function_trainable.py @@ -264,10 +264,7 @@ def report(self, metrics: Dict, *, checkpoint: Optional[Checkpoint] = None) -> N # report problem. This should be fixed by checking if the checkpoint has been # uploaded already (via some marker), then skipping the repeat upload. if _use_storage_context() and isinstance(checkpoint, NewCheckpoint): - print( - "_StatusReporter.report called with checkpoint:\n", - checkpoint, - ) + logger.debug(f"Checkpoint received by the Tune session: {checkpoint}") self._fresh_checkpoint = True # TODO(justinvyu): `metrics` doesn't include the autofilled metrics # like `training_iteration` and `time_total_s`. @@ -501,10 +498,6 @@ def save_checkpoint(self, checkpoint_dir: str = ""): from ray.train._internal.checkpoint_manager import _TrainingResult if _use_storage_context() and isinstance(checkpoint, _TrainingResult): - print( - "Returning checkpoint from FunctionTrainable.save_checkpoint:\n", - checkpoint, - ) return checkpoint if not checkpoint: From 0971aca9d74ef55aa6db5135d99598d39b3bfbe1 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 2 Aug 2023 10:36:11 -0700 Subject: [PATCH 039/108] Don't set training iteration as the default checkpoint_score_attr Signed-off-by: Justin Yu --- python/ray/tune/experiment/trial.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index e641be4c5261..ebc97467d6d1 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -518,9 +518,11 @@ def __init__( # Checkpoint config checkpoint_config = checkpoint_config or CheckpointConfig() - checkpoint_config.checkpoint_score_attribute = ( - checkpoint_config.checkpoint_score_attribute or TRAINING_ITERATION - ) + if not _use_storage_context(): + # TODO(justinvyu): Why is this needed? + checkpoint_config.checkpoint_score_attribute = ( + checkpoint_config.checkpoint_score_attribute or TRAINING_ITERATION + ) self.checkpoint_config = checkpoint_config From d9804b003166ae0c33a0c9bf26b3ad16c1504f8c Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 2 Aug 2023 11:24:13 -0700 Subject: [PATCH 040/108] Fix test to reflect working dir change Signed-off-by: Justin Yu --- python/ray/train/tests/test_new_persistence.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index 840b432e5b47..2c6367812e4b 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -311,12 +311,10 @@ def test_trainer( # NOTE: These next 2 are technically synced by the driver. # TODO(justinvyu): In a follow-up PR, artifacts will be synced by the workers. - # TODO(justinvyu): In a follow-up PR, the rank folders will be removed. # TODO(justinvyu): [custom_fs_path_expansion] Same issue as above. if not storage_filesystem: assert ( - len(list(trial_dir.glob("rank_*/artifact-*"))) - == NUM_ITERATIONS * NUM_WORKERS + len(list(trial_dir.glob("artifact-*"))) == NUM_ITERATIONS * NUM_WORKERS ) assert len(list(trial_dir.glob(EXPR_RESULT_FILE))) == 1 From 318158f95fe8e11b27e91161b21d53a3a8ab8018 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 2 Aug 2023 12:09:07 -0700 Subject: [PATCH 041/108] Don't upload a .is_checkpoint marker Signed-off-by: Justin Yu --- python/ray/train/_internal/storage.py | 6 ------ 1 file changed, 6 deletions(-) diff --git a/python/ray/train/_internal/storage.py b/python/ray/train/_internal/storage.py index da45faac283d..f0c6302a8447 100644 --- a/python/ray/train/_internal/storage.py +++ b/python/ray/train/_internal/storage.py @@ -512,12 +512,6 @@ def persist_current_checkpoint(self, checkpoint: "Checkpoint") -> "Checkpoint": destination_filesystem=self.storage_filesystem, ) - # Drop a marker file to indicate that the checkpoint is completely uploaded. - # TODO(justinvyu): This may make more sense to do in session.report - uploaded_marker_file = os.path.join(self.checkpoint_fs_path, ".is_checkpoint") - with self.storage_filesystem.open_output_stream(uploaded_marker_file): - pass - # Delete local checkpoint files. # TODO(justinvyu): What if checkpoint.path == self.checkpoint_fs_path? # TODO(justinvyu): What if users don't want to delete the local checkpoint? From a54664c9f2077922de134dd60d813b0408cceff4 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 2 Aug 2023 12:11:22 -0700 Subject: [PATCH 042/108] Add back cwd check Signed-off-by: Justin Yu --- python/ray/train/tests/test_new_persistence.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index 2c6367812e4b..c16d3e18d285 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -80,6 +80,9 @@ def train_fn(config): assert train_session.storage assert train_session.storage.checkpoint_fs_path + # Check that the working dir for each worker is the shared trial dir. + assert os.getcwd() == train_session.storage.trial_local_path + start = 0 checkpoint = train.get_checkpoint() From c4263ec0edaab2c66679157ddb157c9d9ae04041 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 2 Aug 2023 12:36:16 -0700 Subject: [PATCH 043/108] Update the dir trees + better naming for ckpt shards and artifacts Signed-off-by: Justin Yu --- .../ray/train/tests/test_new_persistence.py | 39 +++++++++---------- 1 file changed, 19 insertions(+), 20 deletions(-) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index c16d3e18d285..76f65b6ebe75 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -88,7 +88,7 @@ def train_fn(config): checkpoint = train.get_checkpoint() if checkpoint: with checkpoint.as_directory() as checkpoint_dir: - with open(os.path.join(checkpoint_dir, "dummy.pkl"), "rb") as f: + with open(os.path.join(checkpoint_dir, "checkpoint.pkl"), "rb") as f: state = pickle.load(f) print("Loaded back state from checkpoint:", state) start = state["iter"] + 1 @@ -96,12 +96,12 @@ def train_fn(config): for i in range(start, config.get("num_iterations", 5)): time.sleep(0.25) - checkpoint_file_name = "dummy.pkl" - artifact_file_name = f"artifact-{i}.txt" + checkpoint_file_name = "checkpoint.pkl" + artifact_file_name = f"artifact-iter={i}.txt" if in_trainer: rank = train.get_context().get_world_rank() - checkpoint_file_name = f"dummy-{rank}.pkl" - artifact_file_name = f"artifact-{i}-{rank}.txt" + checkpoint_file_name = f"checkpoint_shard-rank={rank}.pkl" + artifact_file_name = f"artifact-rank={rank}-iter={i}.txt" with open(artifact_file_name, "w") as f: f.write(f"{i}") @@ -133,10 +133,10 @@ def test_tuner(monkeypatch, storage_path_type, tmp_path): ├── basic-variant-state.json ├── experiment_state.json ├── train_fn_a2b9e_00000_0_... - │ ├── artifact-0.txt <- Trial artifacts + │ ├── artifact-iter=0.txt <- Trial artifacts │ ├── ... │ ├── checkpoint_000000 <- Trial checkpoints - │ │ └── dummy.pkl + │ │ └── checkpoint.pkl │ ├── ... │ ├── events.out.tfevents... <- Driver artifacts (trial results) │ ├── params.json @@ -217,9 +217,8 @@ def test_trainer( TODO(justinvyu): Test for these once implemented: - artifacts - restoration, train.get_checkpoint - - accessing checkpoints from Result object - trainer_new_persistence + {storage_path}/{exp_name} ├── experiment_state-2023-07-28_10-00-38.json ├── basic-variant-state-2023-07-28_10-00-38.json ├── trainer.pkl @@ -231,17 +230,15 @@ def test_trainer( ├── progress.csv ├── result.json ├── checkpoint_000000 - │ ├── dummy-0.pkl - │ └── dummy-1.pkl + │ ├── checkpoint_shard-rank=0.pkl <- Worker checkpoint shards + │ └── checkpoint_shard-rank=1.pkl + ├── ... + ├── artifact-rank=0-iter=0.txt <- Worker artifacts + ├── artifact-rank=1-iter=0.txt ├── ... - ├── rank_0 <- TODO: remove these rank folders? - │ ├── artifact-0-0.txt - │ ├── artifact-1-0.txt - │ └── ... - └── rank_1 - ├── artifact-0-1.txt - ├── artifact-1-1.txt - └── ... + ├── artifact-rank=0-iter=1.txt + ├── artifact-rank=1-iter=1.txt + └── ... """ LOCAL_CACHE_DIR = tmp_path / "ray_results" monkeypatch.setenv("RAY_AIR_LOCAL_CACHE_DIR", str(LOCAL_CACHE_DIR)) @@ -310,7 +307,9 @@ def test_trainer( assert len(list(trial_dir.glob("checkpoint_*"))) == expected_num_checkpoints for checkpoint_dir in trial_dir.glob("checkpoint_*"): # 1 checkpoint shard per worker. - assert len(list(checkpoint_dir.glob("dummy-*.pkl"))) == NUM_WORKERS + assert ( + len(list(checkpoint_dir.glob("checkpoint_shard-*.pkl"))) == NUM_WORKERS + ) # NOTE: These next 2 are technically synced by the driver. # TODO(justinvyu): In a follow-up PR, artifacts will be synced by the workers. From 3a6eba6a51604f13f0ecc0076b7bb6dc8284d1e0 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 2 Aug 2023 14:29:16 -0700 Subject: [PATCH 044/108] A different fix for the circular dep Signed-off-by: Justin Yu --- python/ray/train/_internal/checkpoint_manager.py | 9 ++------- python/ray/tune/experiment/trial.py | 7 ++++--- 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/python/ray/train/_internal/checkpoint_manager.py b/python/ray/train/_internal/checkpoint_manager.py index 45888c9a1a42..1924bfeaab5a 100644 --- a/python/ray/train/_internal/checkpoint_manager.py +++ b/python/ray/train/_internal/checkpoint_manager.py @@ -5,12 +5,10 @@ from ray._private.dict import flatten_dict from ray.air.config import MAX from ray.air._internal.util import is_nan +from ray.train import CheckpointConfig from ray.train._internal.storage import _delete_fs_path from ray.train.checkpoint import Checkpoint -if TYPE_CHECKING: - from ray.train import CheckpointConfig - logger = logging.getLogger(__name__) @@ -64,10 +62,7 @@ class _CheckpointManager: checkpoint_config: Defines how many and which checkpoints to keep. """ - def __init__(self, checkpoint_config: Optional["CheckpointConfig"]): - # TODO(justinvyu): Fix this circular dependency. - from ray.train import CheckpointConfig - + def __init__(self, checkpoint_config: CheckpointConfig): self._checkpoint_config = checkpoint_config or CheckpointConfig() # List of checkpoints ordered by ascending score. diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index ebc97467d6d1..073563e473cb 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -28,9 +28,6 @@ import ray.cloudpickle as cloudpickle from ray.exceptions import RayActorError, RayTaskError from ray.train._internal.storage import _use_storage_context, StorageContext -from ray.train._internal.checkpoint_manager import ( - _CheckpointManager as _NewCheckpointManager, -) from ray.tune import TuneError from ray.tune.error import _TuneRestoreError from ray.tune.execution.checkpoint_manager import _CheckpointManager @@ -527,6 +524,10 @@ def __init__( self.checkpoint_config = checkpoint_config if _use_storage_context(): + from ray.train._internal.checkpoint_manager import ( + _CheckpointManager as _NewCheckpointManager, + ) + self.checkpoint_manager = _NewCheckpointManager( checkpoint_config=self.checkpoint_config ) From b65e9fed4d4cacd9270567c6456a137748306d66 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 2 Aug 2023 14:35:24 -0700 Subject: [PATCH 045/108] Update checkpoint -> _checkpoint imports Signed-off-by: Justin Yu --- python/ray/train/_internal/session.py | 2 +- python/ray/train/_internal/storage.py | 4 ++-- python/ray/train/tests/test_new_persistence.py | 2 +- python/ray/train/trainer.py | 2 +- python/ray/tune/trainable/function_trainable.py | 2 +- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/python/ray/train/_internal/session.py b/python/ray/train/_internal/session.py index 15223827a348..34c51d342964 100644 --- a/python/ray/train/_internal/session.py +++ b/python/ray/train/_internal/session.py @@ -441,7 +441,7 @@ def _set_legacy_checkpoint_uri(self, uri: str): self.legacy_checkpoint_uri = uri def new_checkpoint(self, checkpoint): - from ray.train.checkpoint import Checkpoint as NewCheckpoint + from ray.train._checkpoint import Checkpoint as NewCheckpoint if not isinstance(checkpoint, NewCheckpoint): raise ValueError( diff --git a/python/ray/train/_internal/storage.py b/python/ray/train/_internal/storage.py index f0c6302a8447..e9856994fac6 100644 --- a/python/ray/train/_internal/storage.py +++ b/python/ray/train/_internal/storage.py @@ -31,7 +31,7 @@ from ray.tune.result import _get_defaults_results_dir if TYPE_CHECKING: - from ray.train.checkpoint import Checkpoint + from ray.train._checkpoint import Checkpoint logger = logging.getLogger(__file__) @@ -493,7 +493,7 @@ def persist_current_checkpoint(self, checkpoint: "Checkpoint") -> "Checkpoint": Checkpoint: A Checkpoint pointing to the persisted checkpoint location. """ # TODO(justinvyu): Fix this cyclical import. - from ray.train.checkpoint import Checkpoint + from ray.train._checkpoint import Checkpoint logger.debug( "Copying checkpoint files to storage path:\n" diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index 76f65b6ebe75..9f95eece7a7c 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -12,7 +12,7 @@ from ray import train, tune from ray.air.constants import EXPR_RESULT_FILE from ray.train._internal.storage import _download_from_fs_path -from ray.train.checkpoint import Checkpoint as NewCheckpoint +from ray.train._checkpoint import Checkpoint as NewCheckpoint from ray.train.data_parallel_trainer import DataParallelTrainer from ray.air.tests.test_checkpoints import mock_s3_bucket_uri diff --git a/python/ray/train/trainer.py b/python/ray/train/trainer.py index 7eb45f3a4299..7f1c2c576a44 100644 --- a/python/ray/train/trainer.py +++ b/python/ray/train/trainer.py @@ -19,7 +19,7 @@ CheckpointManager, ) from ray.train._internal.session import TrainingResult, TrainingResultType -from ray.train.checkpoint import Checkpoint as NewCheckpoint +from ray.train._checkpoint import Checkpoint as NewCheckpoint # Ray Train should be usable even if Tune is not installed. from ray.train._internal.utils import ActorWrapper diff --git a/python/ray/tune/trainable/function_trainable.py b/python/ray/tune/trainable/function_trainable.py index 562bf59b1f69..ac234e135519 100644 --- a/python/ray/tune/trainable/function_trainable.py +++ b/python/ray/tune/trainable/function_trainable.py @@ -255,7 +255,7 @@ def _start(self): def report(self, metrics: Dict, *, checkpoint: Optional[Checkpoint] = None) -> None: from ray.train._internal.storage import _use_storage_context from ray.train._internal.checkpoint_manager import _TrainingResult - from ray.train.checkpoint import Checkpoint as NewCheckpoint + from ray.train._checkpoint import Checkpoint as NewCheckpoint # TODO(xwjiang): Tons of optimizations. self._air_session_has_reported = True From b89bd1cd4e559b459ba0425d81e7c49449e2e0e2 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 2 Aug 2023 14:36:16 -0700 Subject: [PATCH 046/108] fix lint Signed-off-by: Justin Yu --- python/ray/train/_internal/checkpoint_manager.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/train/_internal/checkpoint_manager.py b/python/ray/train/_internal/checkpoint_manager.py index 1924bfeaab5a..55d313d6d7cb 100644 --- a/python/ray/train/_internal/checkpoint_manager.py +++ b/python/ray/train/_internal/checkpoint_manager.py @@ -1,6 +1,6 @@ import logging import numbers -from typing import Any, Callable, Dict, List, Optional, Tuple, TYPE_CHECKING +from typing import Any, Callable, Dict, List, Optional, Tuple from ray._private.dict import flatten_dict from ray.air.config import MAX From 3b784d74aa30375412bddc183c8b29fd66980e6c Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 2 Aug 2023 14:40:52 -0700 Subject: [PATCH 047/108] Revert all changes to ckpt manager Signed-off-by: Justin Yu --- python/ray/train/_internal/checkpoint_manager.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/train/_internal/checkpoint_manager.py b/python/ray/train/_internal/checkpoint_manager.py index 064b55b85efa..b0c9b777d3fa 100644 --- a/python/ray/train/_internal/checkpoint_manager.py +++ b/python/ray/train/_internal/checkpoint_manager.py @@ -62,7 +62,7 @@ class _CheckpointManager: checkpoint_config: Defines how many and which checkpoints to keep. """ - def __init__(self, checkpoint_config: CheckpointConfig): + def __init__(self, checkpoint_config: Optional[CheckpointConfig]): self._checkpoint_config = checkpoint_config or CheckpointConfig() # List of checkpoints ordered by ascending score. From 49c1ead7e5bc88cec763224b7773119bbbd486c2 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 2 Aug 2023 14:44:20 -0700 Subject: [PATCH 048/108] Don't set checkpoint user metadata Signed-off-by: Justin Yu --- python/ray/train/_internal/session.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/train/_internal/session.py b/python/ray/train/_internal/session.py index 34c51d342964..e76cff4094be 100644 --- a/python/ray/train/_internal/session.py +++ b/python/ray/train/_internal/session.py @@ -458,11 +458,11 @@ def new_checkpoint(self, checkpoint): # Save the rank of the worker that created this checkpoint. metadata.update({CHECKPOINT_RANK_KEY: self.world_rank}) - persisted_checkpoint.set_metadata(metadata) result = TrainingResult( type=TrainingResultType.CHECKPOINT, data=persisted_checkpoint, + metadata=metadata, ) # Add result to a thread-safe queue. From 7177940eaace621afbf7ecd9653189b96f729047 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 2 Aug 2023 14:44:48 -0700 Subject: [PATCH 049/108] Remove remaining print Signed-off-by: Justin Yu --- python/ray/train/trainer.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/ray/train/trainer.py b/python/ray/train/trainer.py index 7f1c2c576a44..833a9537acc5 100644 --- a/python/ray/train/trainer.py +++ b/python/ray/train/trainer.py @@ -125,7 +125,6 @@ def _send_next_checkpoint_path_to_workers(self): # and created on the fly when the checkpoint is reported with metrics. # Ex: lambda metrics: f"checkpoint_iter={metrics['training_iteration']}" storage.current_checkpoint_index = self._latest_checkpoint_index - print(f"Setting next checkpoint path to: {storage.checkpoint_fs_path}") self._backend_executor._set_checkpoint_index( storage.current_checkpoint_index From ae8a9ecb9944fbd1372bd6e9f9b82e74ddcca243 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 00:05:54 -0700 Subject: [PATCH 050/108] Add trial_path property to storage ctx Signed-off-by: Justin Yu --- python/ray/train/_internal/storage.py | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/python/ray/train/_internal/storage.py b/python/ray/train/_internal/storage.py index e9856994fac6..ff1d29f00f32 100644 --- a/python/ray/train/_internal/storage.py +++ b/python/ray/train/_internal/storage.py @@ -320,6 +320,14 @@ class StorageContext: For example, on the driver, the storage context is initialized, only knowing the experiment path. On the Trainable actor, the trial_dir_name is accessible. + There are 3 path concepts: + 1. *_path: A path or URI. This is the same format as the `storage_path` specified + by the user. This may need to be URI-joined. + 2. *_fs_path: A path relative to the storage filesystem. This is a regular path + that can be joined with `os.path.join`. + 3. *_local_path: The path on the local filesystem where results are saved to + before persisting to storage. + Example with storage_path="mock:///bucket/path": >>> from ray.train._internal.storage import StorageContext @@ -553,6 +561,16 @@ def experiment_local_path(self) -> str: """ return os.path.join(self.storage_local_path, self.experiment_dir_name) + @property + def trial_path(self) -> str: + """The path the experiment directory, where the format matches the + original `storage_path` format specified by the user. + + Ex: If the user passed in storage_path="s3://bucket/path?param=1", then + this property returns "s3://bucket/path/exp_name?param=1". + """ + return str(URI(self.experiment_path) / self.trial_dir_name) + @property def trial_local_path(self) -> str: """The local filesystem path to the trial directory. From c1c8441649693bd57887aa684da4a268b1cbb349 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 00:06:48 -0700 Subject: [PATCH 051/108] Use storage context for all experiment/trial path properties Signed-off-by: Justin Yu --- python/ray/tune/experiment/experiment.py | 30 ++-- python/ray/tune/experiment/trial.py | 166 ++++++++++++++--------- python/ray/tune/result_grid.py | 20 +-- 3 files changed, 131 insertions(+), 85 deletions(-) diff --git a/python/ray/tune/experiment/experiment.py b/python/ray/tune/experiment/experiment.py index f86986627bdc..4721fe7760a3 100644 --- a/python/ray/tune/experiment/experiment.py +++ b/python/ray/tune/experiment/experiment.py @@ -298,6 +298,17 @@ def __init__( stdout_file, stderr_file = _validate_log_to_file(log_to_file) + self.storage = None + if _use_storage_context(): + storage = StorageContext( + storage_path=storage_path, + storage_filesystem=storage_filesystem, + sync_config=sync_config, + experiment_dir_name=self.dir_name, + ) + self.storage = storage + logger.debug(f"StorageContext on the DRIVER:\n{storage}") + spec = { "run": self._run_identifier, "stop": stopping_criteria, @@ -317,19 +328,8 @@ def __init__( "restore": os.path.abspath(os.path.expanduser(restore)) if restore else None, + "storage": self.storage, } - - self.storage = None - if _use_storage_context(): - storage = StorageContext( - storage_path=storage_path, - storage_filesystem=storage_filesystem, - sync_config=sync_config, - experiment_dir_name=self.dir_name, - ) - self.storage = spec["storage"] = storage - logger.debug(f"StorageContext on the DRIVER:\n{storage}") - self.spec = spec @classmethod @@ -487,6 +487,9 @@ def stopper(self): @property def local_path(self) -> Optional[str]: + if _use_storage_context(): + return self.storage.experiment_local_path + if not self._local_storage_path: return None return str(Path(self._local_storage_path) / self.dir_name) @@ -499,6 +502,9 @@ def local_dir(self): @property def remote_path(self) -> Optional[str]: + if _use_storage_context(): + return self.storage.experiment_path + if not self._remote_storage_path: return None return str(URI(self._remote_storage_path) / self.dir_name) diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index 073563e473cb..1886a3958d4c 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -376,72 +376,83 @@ def __init__( self.trainable_name = trainable_name self.trial_id = Trial.generate_id() if trial_id is None else trial_id - # Set to pass through on `Trial.reset()` - self._orig_experiment_path = experiment_path - self._orig_experiment_dir_name = experiment_dir_name - # Create a copy, since `init_local_path` updates the context with the # generated trial dirname. self.storage = copy.copy(storage) - # TODO(justinvyu): For now, explicitly avoid using the storage context - # to replace the Trial path handling. This should be re-worked - # when adding new persistence mode support for Tune Trainables. + if _use_storage_context(): + assert self.storage - # Sync config - self.sync_config = sync_config or SyncConfig() + # TODO(justinvyu): Rename these to legacy. + self._orig_experiment_path = None + self._orig_experiment_dir_name = None + self._local_experiment_path = None + self._remote_experiment_path = None + self.experiment_dir_name = None + self.sync_config = None + else: + # Set to pass through on `Trial.reset()` + self._orig_experiment_path = experiment_path + self._orig_experiment_dir_name = experiment_dir_name - local_experiment_path, remote_experiment_path = _split_remote_local_path( - experiment_path, None - ) + self.experiment_dir_name = experiment_dir_name - # Backwards compatibility for `local_dir` - if local_dir: - if local_experiment_path: - raise ValueError( - "Only one of `local_dir` or `experiment_path` " - "can be passed to `Trial()`." - ) - local_experiment_path = local_dir - - # Derive experiment dir name from local path - if not experiment_dir_name and local_experiment_path: - # Maybe derive experiment dir name from local storage dir - experiment_dir_name = Path(local_experiment_path).name - elif not experiment_dir_name: - experiment_dir_name = DEFAULT_EXPERIMENT_NAME - - # Set default experiment dir name - if not local_experiment_path: - local_experiment_path = str( - Path(_get_defaults_results_dir()) / experiment_dir_name + # Sync config + self.sync_config = sync_config or SyncConfig() + + local_experiment_path, remote_experiment_path = _split_remote_local_path( + experiment_path, None ) - os.makedirs(local_experiment_path, exist_ok=True) - # Set remote experiment path if upload_dir is set - if self.sync_config.upload_dir: - if remote_experiment_path: - if not remote_experiment_path.startswith(self.sync_config.upload_dir): + # Backwards compatibility for `local_dir` + if local_dir: + if local_experiment_path: raise ValueError( - f"Both a `SyncConfig.upload_dir` and an `experiment_path` " - f"pointing to remote storage were passed, but they do not " - f"point to the same location. Got: " - f"`experiment_path={experiment_path}` and " - f"`SyncConfig.upload_dir={self.sync_config.upload_dir}`. " + "Only one of `local_dir` or `experiment_path` " + "can be passed to `Trial()`." ) - warnings.warn( - "If `experiment_path` points to a remote storage location, " - "do not set `SyncConfig.upload_dir`. ", - DeprecationWarning, - ) - else: - remote_experiment_path = str( - URI(self.sync_config.upload_dir) / experiment_dir_name + local_experiment_path = local_dir + + # Derive experiment dir name from local path + if not experiment_dir_name and local_experiment_path: + # Maybe derive experiment dir name from local storage dir + experiment_dir_name = Path(local_experiment_path).name + elif not experiment_dir_name: + experiment_dir_name = DEFAULT_EXPERIMENT_NAME + + # Set default experiment dir name + if not local_experiment_path: + local_experiment_path = str( + Path(_get_defaults_results_dir()) / experiment_dir_name ) + os.makedirs(local_experiment_path, exist_ok=True) + + # Set remote experiment path if upload_dir is set + if self.sync_config.upload_dir: + if remote_experiment_path: + if not remote_experiment_path.startswith( + self.sync_config.upload_dir + ): + raise ValueError( + f"Both a `SyncConfig.upload_dir` and an `experiment_path` " + f"pointing to remote storage were passed, but they do not " + f"point to the same location. Got: " + f"`experiment_path={experiment_path}` and " + f"`SyncConfig.upload_dir={self.sync_config.upload_dir}`. " + ) + warnings.warn( + "If `experiment_path` points to a remote storage location, " + "do not set `SyncConfig.upload_dir`. ", + DeprecationWarning, + ) + else: + remote_experiment_path = str( + URI(self.sync_config.upload_dir) / experiment_dir_name + ) - # Finally, set properties - self._local_experiment_path = local_experiment_path - self._remote_experiment_path = remote_experiment_path + # Finally, set properties + self._local_experiment_path = local_experiment_path + self._remote_experiment_path = remote_experiment_path self.config = config or {} # Save a copy of the original unresolved config so that we can swap @@ -505,14 +516,9 @@ def __init__( self.custom_trial_name = None self.custom_dirname = None - self.experiment_dir_name = experiment_dir_name - # Checkpointing fields self.saving_to = None - # Checkpoint syncing - self.sync_config = sync_config or SyncConfig() - # Checkpoint config checkpoint_config = checkpoint_config or CheckpointConfig() if not _use_storage_context(): @@ -674,18 +680,30 @@ def local_dir(self): @property def remote_experiment_path(self) -> str: + if _use_storage_context(): + return self.storage.experiment_path + return str(self._remote_experiment_path) @remote_experiment_path.setter def remote_experiment_path(self, remote_path: str): + if _use_storage_context(): + raise RuntimeError("Set storage.experiment_dir_name instead.") + self._remote_experiment_path = remote_path @property def local_experiment_path(self) -> str: + if _use_storage_context(): + return self.storage.experiment_local_path + return str(self._local_experiment_path) @local_experiment_path.setter def local_experiment_path(self, local_path: str): + if _use_storage_context(): + raise RuntimeError("Set storage.experiment_dir_name instead.") + relative_checkpoint_dirs = [] if self.local_path: # Save the relative paths of persistent trial checkpoints, which are saved @@ -726,12 +744,18 @@ def logdir(self) -> Optional[str]: @property def local_path(self) -> Optional[str]: + if _use_storage_context(): + return self.storage.trial_local_path + if not self.local_experiment_path or not self.relative_logdir: return None return str(Path(self.local_experiment_path).joinpath(self.relative_logdir)) @local_path.setter def local_path(self, logdir): + if _use_storage_context(): + raise RuntimeError("Set storage.trial_dir_name instead.") + relative_logdir = Path(logdir).relative_to(self.local_experiment_path) if ".." in str(relative_logdir): raise ValueError( @@ -755,6 +779,10 @@ def remote_checkpoint_dir(self) -> Optional[str]: @property def remote_path(self) -> Optional[str]: + # TODO(justinvyu): Remove remote_path. It's just path vs local_path now. + if _use_storage_context(): + return self.path + if not self._remote_experiment_path or not self.relative_logdir: return None uri = URI(self._remote_experiment_path) @@ -762,6 +790,9 @@ def remote_path(self) -> Optional[str]: @property def path(self) -> Optional[str]: + if _use_storage_context(): + return self.storage.trial_path + return self.remote_path or self.local_path @property @@ -774,6 +805,9 @@ def node_ip(self): @property def sync_on_checkpoint(self): + if _use_storage_context(): + return self.storage.sync_config.sync_on_checkpoint + return self.sync_config.sync_on_checkpoint @property @@ -811,6 +845,11 @@ def generate_id(cls): @property def uses_cloud_checkpointing(self): + # TODO(justinvyu): This is entangled in the old restore codepaths. + # Remove this once those are gone. + if _use_storage_context(): + return False + return bool(self.remote_path) def reset(self): @@ -858,6 +897,12 @@ def init_local_path(self): self.relative_logdir = _create_unique_logdir_name( str(self.local_experiment_path), self._generate_dirname() ) + + if _use_storage_context(): + # Populate the storage context with the trial dir name we just generated. + assert self.storage + self.storage.trial_dir_name = self.relative_logdir + assert self.local_path logdir_path = Path(self.local_path) max_path_length = _get_max_path_length() @@ -870,11 +915,6 @@ def init_local_path(self): ) logdir_path.mkdir(parents=True, exist_ok=True) - if _use_storage_context(): - # Populate the storage context with the trial dir name we just generated. - assert self.storage - self.storage.trial_dir_name = self.relative_logdir - self.invalidate_json_state() def update_resources(self, resources: Union[dict, PlacementGroupFactory]): diff --git a/python/ray/tune/result_grid.py b/python/ray/tune/result_grid.py index f72315518812..e267ddf11be0 100644 --- a/python/ray/tune/result_grid.py +++ b/python/ray/tune/result_grid.py @@ -271,16 +271,6 @@ def _populate_exception(trial: Trial) -> Optional[Union[TuneError, RayTaskError] return None def _trial_to_result(self, trial: Trial) -> Result: - local_to_remote_path_fn = ( - partial( - TrainableUtil.get_remote_storage_path, - local_path_prefix=trial.local_path, - remote_path_prefix=trial.remote_path, - ) - if trial.uses_cloud_checkpointing - else None - ) - if _use_storage_context(): from ray.train._internal.checkpoint_manager import ( _CheckpointManager as _NewCheckpointManager, @@ -294,6 +284,16 @@ def _trial_to_result(self, trial: Trial) -> Result: for checkpoint_result in best_checkpoint_results ] else: + local_to_remote_path_fn = ( + partial( + TrainableUtil.get_remote_storage_path, + local_path_prefix=trial.local_path, + remote_path_prefix=trial.remote_path, + ) + if trial.uses_cloud_checkpointing + else None + ) + checkpoint = trial.checkpoint.to_air_checkpoint( local_to_remote_path_fn, ) From 5d2ca07705ee6edff4d45beac909a0f3bf2d0d2e Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 00:08:23 -0700 Subject: [PATCH 052/108] Don't skip trainer test cases for custom_fs Signed-off-by: Justin Yu --- .../ray/train/tests/test_new_persistence.py | 23 ++++++++----------- 1 file changed, 9 insertions(+), 14 deletions(-) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index 9f95eece7a7c..d3489a0f35a9 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -278,15 +278,14 @@ def test_trainer( local_inspect_dir = LOCAL_CACHE_DIR # First, inspect that the result object returns the correct paths. - # TODO(justinvyu): [custom_fs_path_expansion] - # This doesn't work for the `custom_fs` case right now - # because Result.path <- Trial.remote_path/local_path <- Experiment.path, - # which expands the storage path to an absolute path. - # We shouldn't expand the storage path to an absolute path if a custom fs is passed. - if not storage_filesystem: + if storage_filesystem: + trial_fs_path = result.path + else: _, trial_fs_path = pyarrow.fs.FileSystem.from_uri(result.path) - assert trial_fs_path.startswith(storage_fs_path) - assert result.checkpoint.path.startswith(trial_fs_path) + + assert trial_fs_path.startswith(storage_fs_path) + for checkpoint, _ in result.best_checkpoints: + assert checkpoint.path.startswith(trial_fs_path) # Second, inspect the contents of the storage path assert len(list(local_inspect_dir.glob("*"))) == 1 # Only expect 1 experiment dir @@ -313,12 +312,8 @@ def test_trainer( # NOTE: These next 2 are technically synced by the driver. # TODO(justinvyu): In a follow-up PR, artifacts will be synced by the workers. - # TODO(justinvyu): [custom_fs_path_expansion] Same issue as above. - if not storage_filesystem: - assert ( - len(list(trial_dir.glob("artifact-*"))) == NUM_ITERATIONS * NUM_WORKERS - ) - assert len(list(trial_dir.glob(EXPR_RESULT_FILE))) == 1 + assert len(list(trial_dir.glob("artifact-*"))) == NUM_ITERATIONS * NUM_WORKERS + assert len(list(trial_dir.glob(EXPR_RESULT_FILE))) == 1 if __name__ == "__main__": From 1fcfb3f58d580a22089dd96e3583814f2d11f1a1 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 00:27:03 -0700 Subject: [PATCH 053/108] Split some utilities into helper methods + test for ResultGrid paths Signed-off-by: Justin Yu --- .../ray/train/tests/test_new_persistence.py | 109 +++++++++++++----- 1 file changed, 78 insertions(+), 31 deletions(-) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index d3489a0f35a9..0e22fe3dcaa9 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -68,6 +68,53 @@ def _resolve_storage_type( yield storage_path, storage_filesystem +def _get_local_inspect_dir( + root_local_path: Path, + storage_path: str, + storage_local_path: Path, + storage_filesystem: Optional[pyarrow.fs.FileSystem], +) -> Tuple[Path, str]: + """Downloads the storage path -> local dir for inspecting contents. + + Returns: + Tuple: (local_inspect_dir, storage_fs_path), where storage_fs_path + is the path to the storage path on the filesystem (e.g., prefix stripped). + This is used to check the correctness of paths returned from `Result`'s, + since URIs are hard to do comparisons with. + """ + local_inspect_dir = root_local_path / "inspect" + if storage_path: + if storage_filesystem: + fs, storage_fs_path = storage_filesystem, storage_path + else: + fs, storage_fs_path = pyarrow.fs.FileSystem.from_uri(storage_path) + _download_from_fs_path( + fs=fs, fs_path=storage_fs_path, local_path=str(local_inspect_dir) + ) + else: + fs, storage_fs_path = pyarrow.fs.LocalFileSystem(), str(storage_local_path) + local_inspect_dir = storage_local_path + + return local_inspect_dir, storage_fs_path + + +def _convert_path_to_fs_path( + path: str, storage_filesystem: Optional[pyarrow.fs.FileSystem] +) -> str: + """Converts a path to a (prefix-stripped) filesystem path. + + Ex: "s3://bucket/path/to/file" -> "bucket/path/to/file" + Ex: "/mnt/nfs/path/to/file" -> "/mnt/nfs/bucket/path/to/file" + """ + if not storage_filesystem: + _, fs_path = pyarrow.fs.FileSystem.from_uri(path) + return fs_path + + # Otherwise, we're using a custom filesystem, + # and the provided path is already the fs path. + return path + + def train_fn(config): in_trainer = config.get("in_trainer", False) if in_trainer: @@ -165,7 +212,7 @@ def test_tuner(monkeypatch, storage_path_type, tmp_path): run_config=train.RunConfig( storage_path=storage_path, storage_filesystem=storage_filesystem, - name="simple_persistence_test", + name=exp_name, verbose=0, failure_config=train.FailureConfig(max_failures=1), ), @@ -174,20 +221,30 @@ def test_tuner(monkeypatch, storage_path_type, tmp_path): num_samples=NUM_TRIALS, max_concurrent_trials=1 ), ) - tuner.fit() - - local_inspect_dir = tmp_path / "inspect" - if storage_path: - if storage_filesystem: - fs, fs_path = storage_filesystem, storage_path - else: - fs, fs_path = pyarrow.fs.FileSystem.from_uri(storage_path) - _download_from_fs_path( - fs=fs, fs_path=fs_path, local_path=str(local_inspect_dir) - ) - else: - local_inspect_dir = LOCAL_CACHE_DIR + result_grid = tuner.fit() + local_inspect_dir, storage_fs_path = _get_local_inspect_dir( + root_local_path=tmp_path, + storage_path=storage_path, + storage_local_path=LOCAL_CACHE_DIR, + storage_filesystem=storage_filesystem, + ) + + # First, check that the ResultGrid returns the correct paths. + experiment_fs_path = _convert_path_to_fs_path( + result_grid.experiment_path, storage_filesystem + ) + assert experiment_fs_path == os.path.join(storage_fs_path, exp_name) + assert len(result_grid) == NUM_TRIALS + for result in result_grid: + trial_fs_path = _convert_path_to_fs_path(result.path, storage_filesystem) + assert trial_fs_path.startswith(experiment_fs_path) + # TODO(justinvyu): Trainable syncing of artifacts and checkpoints + # is not yet implemented for the new persistence path. + # for checkpoint, _ in result.best_checkpoints: + # assert checkpoint.path.startswith(trial_fs_path) + + # Next, inspect the storage path contents. assert len(list(local_inspect_dir.glob("*"))) == 1 # Only expect 1 experiment dir exp_dir = local_inspect_dir / exp_name @@ -264,25 +321,15 @@ def test_trainer( ) result = trainer.fit() - local_inspect_dir = tmp_path / "inspect" - if storage_path: - if storage_filesystem: - fs, storage_fs_path = storage_filesystem, storage_path - else: - fs, storage_fs_path = pyarrow.fs.FileSystem.from_uri(storage_path) - _download_from_fs_path( - fs=fs, fs_path=storage_fs_path, local_path=str(local_inspect_dir) - ) - else: - fs, storage_fs_path = pyarrow.fs.LocalFileSystem(), str(LOCAL_CACHE_DIR) - local_inspect_dir = LOCAL_CACHE_DIR + local_inspect_dir, storage_fs_path = _get_local_inspect_dir( + root_local_path=tmp_path, + storage_path=storage_path, + storage_local_path=LOCAL_CACHE_DIR, + storage_filesystem=storage_filesystem, + ) # First, inspect that the result object returns the correct paths. - if storage_filesystem: - trial_fs_path = result.path - else: - _, trial_fs_path = pyarrow.fs.FileSystem.from_uri(result.path) - + trial_fs_path = _convert_path_to_fs_path(result.path, storage_filesystem) assert trial_fs_path.startswith(storage_fs_path) for checkpoint, _ in result.best_checkpoints: assert checkpoint.path.startswith(trial_fs_path) From 61fdadf48e868fb67fb37b067e9e6161274b3a3a Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 00:47:21 -0700 Subject: [PATCH 054/108] Prepend legacy to old path attributes in trial Signed-off-by: Justin Yu --- python/ray/tune/execution/trial_runner.py | 2 +- python/ray/tune/experiment/trial.py | 71 ++++++++++++++--------- python/ray/tune/tests/test_api.py | 2 +- 3 files changed, 45 insertions(+), 30 deletions(-) diff --git a/python/ray/tune/execution/trial_runner.py b/python/ray/tune/execution/trial_runner.py index 906a9766e3d7..081b92f7134a 100644 --- a/python/ray/tune/execution/trial_runner.py +++ b/python/ray/tune/execution/trial_runner.py @@ -474,7 +474,7 @@ def restore_from_dir(self, experiment_dir: Optional[str] = None) -> List[Trial]: # ATTN: Set `local_experiment_path` to update trial checkpoints! trial.local_experiment_path = self._legacy_local_experiment_path trial.remote_experiment_path = self._legacy_remote_experiment_path - trial.sync_config = self._legacy_sync_config + trial.legacy_sync_config = self._legacy_sync_config trial.experiment_dir_name = self._legacy_experiment_dir_name # Avoid creating logdir in client mode for returned trial results, diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index 1886a3958d4c..7af7d36f25e7 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -258,7 +258,7 @@ def _get_trainable_kwargs( # We keep these kwargs separate for backwards compatibility # with trainables that don't provide these keyword arguments kwargs["remote_checkpoint_dir"] = trial.remote_path - kwargs["sync_config"] = trial.sync_config + kwargs["sync_config"] = trial.legacy_sync_config return kwargs @@ -384,21 +384,21 @@ def __init__( assert self.storage # TODO(justinvyu): Rename these to legacy. - self._orig_experiment_path = None - self._orig_experiment_dir_name = None - self._local_experiment_path = None - self._remote_experiment_path = None - self.experiment_dir_name = None - self.sync_config = None + self._legacy_orig_experiment_path = None + self._legacy_orig_experiment_dir_name = None + self._legacy_local_experiment_path = None + self._legacy_remote_experiment_path = None + self._legacy_experiment_dir_name = None + self.legacy_sync_config = None else: # Set to pass through on `Trial.reset()` - self._orig_experiment_path = experiment_path - self._orig_experiment_dir_name = experiment_dir_name + self._legacy_orig_experiment_path = experiment_path + self._legacy_orig_experiment_dir_name = experiment_dir_name - self.experiment_dir_name = experiment_dir_name + self._legacy_experiment_dir_name = experiment_dir_name # Sync config - self.sync_config = sync_config or SyncConfig() + self.legacy_sync_config = sync_config or SyncConfig() local_experiment_path, remote_experiment_path = _split_remote_local_path( experiment_path, None @@ -428,17 +428,18 @@ def __init__( os.makedirs(local_experiment_path, exist_ok=True) # Set remote experiment path if upload_dir is set - if self.sync_config.upload_dir: + if self.legacy_sync_config.upload_dir: if remote_experiment_path: if not remote_experiment_path.startswith( - self.sync_config.upload_dir + self.legacy_sync_config.upload_dir ): raise ValueError( f"Both a `SyncConfig.upload_dir` and an `experiment_path` " f"pointing to remote storage were passed, but they do not " f"point to the same location. Got: " f"`experiment_path={experiment_path}` and " - f"`SyncConfig.upload_dir={self.sync_config.upload_dir}`. " + "`SyncConfig.upload_dir=" + f"{self.legacy_sync_config.upload_dir}`. " ) warnings.warn( "If `experiment_path` points to a remote storage location, " @@ -447,12 +448,12 @@ def __init__( ) else: remote_experiment_path = str( - URI(self.sync_config.upload_dir) / experiment_dir_name + URI(self.legacy_sync_config.upload_dir) / experiment_dir_name ) # Finally, set properties - self._local_experiment_path = local_experiment_path - self._remote_experiment_path = remote_experiment_path + self._legacy_local_experiment_path = local_experiment_path + self._legacy_remote_experiment_path = remote_experiment_path self.config = config or {} # Save a copy of the original unresolved config so that we can swap @@ -678,26 +679,40 @@ def get_runner_ip(self) -> Optional[str]: def local_dir(self): return self.local_experiment_path + @property + def experiment_dir_name(self): + if _use_storage_context(): + return self.storage.experiment_dir_name + + return self._legacy_experiment_dir_name + + @experiment_dir_name.setter + def experiment_dir_name(self, name: str): + if _use_storage_context(): + raise RuntimeError("Set storage.experiment_dir_name instead.") + + self._legacy_experiment_dir_name = name + @property def remote_experiment_path(self) -> str: if _use_storage_context(): return self.storage.experiment_path - return str(self._remote_experiment_path) + return str(self._legacy_remote_experiment_path) @remote_experiment_path.setter def remote_experiment_path(self, remote_path: str): if _use_storage_context(): raise RuntimeError("Set storage.experiment_dir_name instead.") - self._remote_experiment_path = remote_path + self._legacy_remote_experiment_path = remote_path @property def local_experiment_path(self) -> str: if _use_storage_context(): return self.storage.experiment_local_path - return str(self._local_experiment_path) + return str(self._legacy_local_experiment_path) @local_experiment_path.setter def local_experiment_path(self, local_path: str): @@ -722,9 +737,9 @@ def local_experiment_path(self, local_path: str): os.path.relpath(checkpoint_dir, self.local_path) ) - # Update the underlying `_local_experiment_path`, + # Update the underlying `_legacy_local_experiment_path`, # which also updates the trial `local_path` - self._local_experiment_path = local_path + self._legacy_local_experiment_path = local_path if self.local_path: for checkpoint, relative_checkpoint_dir in zip( @@ -783,9 +798,9 @@ def remote_path(self) -> Optional[str]: if _use_storage_context(): return self.path - if not self._remote_experiment_path or not self.relative_logdir: + if not self._legacy_remote_experiment_path or not self.relative_logdir: return None - uri = URI(self._remote_experiment_path) + uri = URI(self._legacy_remote_experiment_path) return str(uri / self.relative_logdir) @property @@ -808,7 +823,7 @@ def sync_on_checkpoint(self): if _use_storage_context(): return self.storage.sync_config.sync_on_checkpoint - return self.sync_config.sync_on_checkpoint + return self.legacy_sync_config.sync_on_checkpoint @property def checkpoint_at_end(self): @@ -870,13 +885,13 @@ def reset(self): self.trainable_name, config=self.config, trial_id=None, - experiment_path=self._orig_experiment_path, - experiment_dir_name=self._orig_experiment_dir_name, + experiment_path=self._legacy_orig_experiment_path, + experiment_dir_name=self._legacy_orig_experiment_dir_name, evaluated_params=self.evaluated_params, experiment_tag=self.experiment_tag, placement_group_factory=placement_group_factory, stopping_criterion=self.stopping_criterion, - sync_config=self.sync_config, + sync_config=self.legacy_sync_config, checkpoint_config=self.checkpoint_config, export_formats=self.export_formats, restore_path=self.restore_path, diff --git a/python/ray/tune/tests/test_api.py b/python/ray/tune/tests/test_api.py index c555aeb856b5..e13a4f5ba067 100644 --- a/python/ray/tune/tests/test_api.py +++ b/python/ray/tune/tests/test_api.py @@ -1075,7 +1075,7 @@ def _create_remote_actor(trainable_cls, sync_to_cloud): cls = trial.get_trainable_cls() actor = ray.remote(cls).remote( remote_checkpoint_dir=upload_dir, - sync_config=trial.sync_config, + sync_config=trial.legacy_sync_config, ) return actor From d38cd87f874e77d621deda4c49dba941036b0480 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 00:48:50 -0700 Subject: [PATCH 055/108] Remove todo Signed-off-by: Justin Yu --- python/ray/tune/experiment/trial.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index 7af7d36f25e7..8fb4c62fcba6 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -383,7 +383,6 @@ def __init__( if _use_storage_context(): assert self.storage - # TODO(justinvyu): Rename these to legacy. self._legacy_orig_experiment_path = None self._legacy_orig_experiment_dir_name = None self._legacy_local_experiment_path = None From 3ba944a23acafd73de9f514385075147cc7a2068 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 10:14:28 -0700 Subject: [PATCH 056/108] Bump the test size Signed-off-by: Justin Yu --- python/ray/train/BUILD | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/train/BUILD b/python/ray/train/BUILD index 1e1ced85cf05..ccdf2851d962 100644 --- a/python/ray/train/BUILD +++ b/python/ray/train/BUILD @@ -464,7 +464,7 @@ py_test( py_test( name = "test_new_persistence", - size = "small", + size = "medium", srcs = ["tests/test_new_persistence.py"], tags = ["team:ml", "exclusive"], deps = [":train_lib", ":conftest"] From 9c16120c72bcdd8042c1357649b221ec7b3a1d58 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 12:17:34 -0700 Subject: [PATCH 057/108] Clean up experiment path handling Signed-off-by: Justin Yu --- python/ray/tune/experiment/experiment.py | 164 +++++++++++++---------- python/ray/tune/impl/tuner_internal.py | 48 ++++--- 2 files changed, 121 insertions(+), 91 deletions(-) diff --git a/python/ray/tune/experiment/experiment.py b/python/ray/tune/experiment/experiment.py index ccea38c94c70..016a0fb6882b 100644 --- a/python/ray/tune/experiment/experiment.py +++ b/python/ray/tune/experiment/experiment.py @@ -158,65 +158,6 @@ def __init__( # Deprecated local_dir: Optional[str] = None, ): - if isinstance(sync_config, dict): - sync_config = SyncConfig(**sync_config) - else: - sync_config = sync_config or SyncConfig() - - self.sync_config = sync_config - - # Resolve storage_path - local_storage_path, remote_storage_path = _resolve_storage_path( - storage_path, local_dir, sync_config.upload_dir, error_location="Experiment" - ) - - if local_dir: - if log_once("tune_experiment_local_dir"): - warnings.warn( - "The `local_dir` argument of `Experiment is deprecated. " - "Use `storage_path` or set the `TUNE_RESULT_DIR` " - "environment variable instead." - ) - - local_storage_path = local_dir - - full_local_storage_path = _get_local_dir_with_expand_user(local_storage_path) - - # `_experiment_checkpoint_dir` is for internal use only for better - # support of Tuner API. - # If set, it should be a subpath under `local_dir`. Also deduce `dir_name`. - if _experiment_checkpoint_dir: - experiment_checkpoint_dir_path = Path(_experiment_checkpoint_dir) - if _use_storage_context(): - # TODO(justinvyu): This is a temporary hack for the - # custom storage_filesystem case. - # With a custom storage_filesystem, the storage path is possibly - # a "local path" (ex: [fs, path] = CustomS3FileSystem, "bucket/subdir") - # Our current path resolution treats a local storage_path as - # the cache dir, so the assertion in the else case would fail. - # This will be re-worked in a follow-up. - self.dir_name = experiment_checkpoint_dir_path.name - else: - local_dir_path = Path(full_local_storage_path) - assert local_dir_path in experiment_checkpoint_dir_path.parents, ( - local_dir_path, - str(list(experiment_checkpoint_dir_path.parents)), - ) - # `dir_name` is set by `_experiment_checkpoint_dir` indirectly. - self.dir_name = os.path.relpath( - _experiment_checkpoint_dir, full_local_storage_path - ) - - self._local_storage_path = full_local_storage_path - self._remote_storage_path = remote_storage_path - - config = config or {} - - if isinstance(checkpoint_config, dict): - checkpoint_config = CheckpointConfig(**checkpoint_config) - else: - checkpoint_config = checkpoint_config or CheckpointConfig() - if is_function_trainable(run): if checkpoint_config.checkpoint_at_end: raise ValueError( @@ -248,12 +189,84 @@ def __init__( else: raise e - self.name = name or self._run_identifier + self.storage = None + if _use_storage_context(): + assert name is not None + + self.storage = StorageContext( + storage_path=storage_path, + storage_filesystem=storage_filesystem, + sync_config=sync_config, + experiment_dir_name=name, + ) + logger.debug(f"StorageContext on the DRIVER:\n{self.storage}") + + # TODO(justinvyu): Rename these to legacy. + self._local_storage_path = None + self._remote_storage_path = None + self.sync_config = None + self.dir_name = None + else: + if isinstance(sync_config, dict): + sync_config = SyncConfig(**sync_config) + else: + sync_config = sync_config or SyncConfig() + + self.sync_config = sync_config + + # Resolve storage_path + local_storage_path, remote_storage_path = _resolve_storage_path( + storage_path, + local_dir, + sync_config.upload_dir, + error_location="Experiment", + ) - if not _experiment_checkpoint_dir: - self.dir_name = _get_dir_name(run, name, self.name) + if local_dir: + if log_once("tune_experiment_local_dir"): + warnings.warn( + "The `local_dir` argument of `Experiment is deprecated. " + "Use `storage_path` or set the `TUNE_RESULT_DIR` " + "environment variable instead." + ) - assert self.dir_name + local_storage_path = local_dir + + full_local_storage_path = _get_local_dir_with_expand_user( + local_storage_path + ) + + # `_experiment_checkpoint_dir` is for internal use only for better + # support of Tuner API. + # If set, it should be a subpath under `local_dir`. Also deduce `dir_name`. + if _experiment_checkpoint_dir: + experiment_checkpoint_dir_path = Path(_experiment_checkpoint_dir) + local_dir_path = Path(full_local_storage_path) + assert local_dir_path in experiment_checkpoint_dir_path.parents, ( + local_dir_path, + str(list(experiment_checkpoint_dir_path.parents)), + ) + # `dir_name` is set by `_experiment_checkpoint_dir` indirectly. + self.dir_name = os.path.relpath( + _experiment_checkpoint_dir, full_local_storage_path + ) + + self._local_storage_path = full_local_storage_path + self._remote_storage_path = remote_storage_path + + self.name = name or self._run_identifier + + if not _experiment_checkpoint_dir: + self.dir_name = _get_dir_name(run, name, self.name) + + assert self.dir_name + + config = config or {} + + if isinstance(checkpoint_config, dict): + checkpoint_config = CheckpointConfig(**checkpoint_config) + else: + checkpoint_config = checkpoint_config or CheckpointConfig() self._stopper = None stopping_criteria = {} @@ -298,17 +311,6 @@ def __init__( stdout_file, stderr_file = _validate_log_to_file(log_to_file) - self.storage = None - if _use_storage_context(): - storage = StorageContext( - storage_path=storage_path, - storage_filesystem=storage_filesystem, - sync_config=sync_config, - experiment_dir_name=self.dir_name, - ) - self.storage = storage - logger.debug(f"StorageContext on the DRIVER:\n{storage}") - spec = { "run": self._run_identifier, "stop": stopping_criteria, @@ -450,6 +452,20 @@ def register_if_needed(cls, run_object: Union[str, Callable, Type]): raise type(e)(str(e) + " " + extra_msg) from None return name + @classmethod + def get_experiment_dir_name( + cls, + run_obj: Union[str, Callable, Type], + ) -> str: + assert run_obj + run_identifier = cls.get_trainable_name(run_obj) + + if bool(int(os.environ.get("TUNE_DISABLE_DATED_SUBDIR", 0))): + dir_name = run_identifier + else: + dir_name = "{}_{}".format(run_identifier, date_str()) + return dir_name + @classmethod def get_experiment_checkpoint_dir( cls, diff --git a/python/ray/tune/impl/tuner_internal.py b/python/ray/tune/impl/tuner_internal.py index 08a6e1776ec0..d68d7362a8da 100644 --- a/python/ray/tune/impl/tuner_internal.py +++ b/python/ray/tune/impl/tuner_internal.py @@ -131,7 +131,10 @@ def __init__( self._resume_config = None self._is_restored = False self._tuner_kwargs = copy.deepcopy(_tuner_kwargs) or {} - self._experiment_checkpoint_dir = self.setup_create_experiment_checkpoint_dir( + ( + self._experiment_checkpoint_dir, + self._experiment_dir_name, + ) = self.setup_create_experiment_checkpoint_dir( self.converted_trainable, self._run_config ) self._experiment_analysis = None @@ -424,7 +427,7 @@ def _restore_from_path_or_uri( # If we synced, `experiment_checkpoint_dir` will contain a temporary # directory. Create an experiment checkpoint dir instead and move # our data there. - new_exp_path = Path( + new_exp_path, new_exp_name = Path( self.setup_create_experiment_checkpoint_dir( self.converted_trainable, self._run_config ) @@ -433,6 +436,7 @@ def _restore_from_path_or_uri( file_dir.replace(new_exp_path / file_dir.name) shutil.rmtree(experiment_checkpoint_path) self._experiment_checkpoint_dir = str(new_exp_path) + self._experiment_dir_name = str(new_exp_name) # Load the experiment results at the point where it left off. try: @@ -521,20 +525,30 @@ def _process_scaling_config(self) -> None: @classmethod def setup_create_experiment_checkpoint_dir( cls, trainable: TrainableType, run_config: Optional[RunConfig] - ) -> str: - """Sets up experiment checkpoint dir before actually running the experiment.""" - path = Experiment.get_experiment_checkpoint_dir( - trainable, - # TODO(justinvyu): This is a result of the old behavior of parsing - # local vs. remote storage paths and should be reworked in a follow-up. - _get_defaults_results_dir() - if _use_storage_context() - else run_config.storage_path, - run_config.name, - ) - if not os.path.exists(path): - os.makedirs(path, exist_ok=True) - return path + ) -> Tuple[str, str]: + """Sets up and creates the local experiment checkpoint dir. + This is so that the `tuner.pkl` file gets stored in the same directory + and gets synced with other experiment results. + + Returns: + Tuple: (experiment_path, experiment_dir_name) + """ + if _use_storage_context(): + experiment_dir_name = run_config.name or Experiment.get_experiment_dir_name( + trainable + ) + storage_local_path = _get_defaults_results_dir() + experiment_path = os.path.join(storage_local_path, experiment_dir_name) + else: + experiment_path = Experiment.get_experiment_checkpoint_dir( + trainable, + run_config.storage_path, + run_config.name, + ) + experiment_dir_name = os.path.basename(experiment_path) + + os.makedirs(experiment_path, exist_ok=True) + return experiment_path, experiment_dir_name # This has to be done through a function signature (@property won't do). def get_experiment_checkpoint_dir(self) -> str: @@ -669,6 +683,7 @@ def _get_tune_run_arguments(self, trainable: TrainableType) -> Dict[str, Any]: return dict( storage_path=self._run_config.storage_path, storage_filesystem=self._run_config.storage_filesystem, + name=self._experiment_dir_name, mode=self._tune_config.mode, metric=self._tune_config.metric, callbacks=self._run_config.callbacks, @@ -702,7 +717,6 @@ def _fit_internal( num_samples=self._tune_config.num_samples, search_alg=self._tune_config.search_alg, scheduler=self._tune_config.scheduler, - name=self._run_config.name, log_to_file=self._run_config.log_to_file, ), **self._tuner_kwargs, From 76468d9f69d44d2f018abb5332f46fb7dc1994cb Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 12:17:54 -0700 Subject: [PATCH 058/108] Fix for base trainer Signed-off-by: Justin Yu --- python/ray/train/base_trainer.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/train/base_trainer.py b/python/ray/train/base_trainer.py index 4702e342ea2b..263fc92ba2d0 100644 --- a/python/ray/train/base_trainer.py +++ b/python/ray/train/base_trainer.py @@ -598,7 +598,7 @@ def fit(self) -> Result: _entrypoint=AirEntrypoint.TRAINER, ) - experiment_path = Path( + experiment_path, _ = Path( TunerInternal.setup_create_experiment_checkpoint_dir( trainable, self.run_config ) From b17c17e0d26fd4368e3f44af6f41cf09232c21d1 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 12:20:09 -0700 Subject: [PATCH 059/108] Fix for base trainer pt 2 Signed-off-by: Justin Yu --- python/ray/train/base_trainer.py | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/python/ray/train/base_trainer.py b/python/ray/train/base_trainer.py index 263fc92ba2d0..3e4d002452a0 100644 --- a/python/ray/train/base_trainer.py +++ b/python/ray/train/base_trainer.py @@ -598,16 +598,19 @@ def fit(self) -> Result: _entrypoint=AirEntrypoint.TRAINER, ) - experiment_path, _ = Path( - TunerInternal.setup_create_experiment_checkpoint_dir( - trainable, self.run_config - ) + ( + experiment_local_path, + expeirment_dir_name, + ) = TunerInternal.setup_create_experiment_checkpoint_dir( + trainable, self.run_config ) - self._save(experiment_path) + + experiment_local_path = Path(experiment_local_path) + self._save(experiment_local_path) restore_msg = TrainingFailedError._RESTORE_MSG.format( trainer_cls_name=self.__class__.__name__, - path=str(experiment_path), + path=str(experiment_local_path), ) try: From 30e33289a55f00bd68ef636d12c951ea0b35e5da Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 12:22:53 -0700 Subject: [PATCH 060/108] Add in missing legacy property Signed-off-by: Justin Yu --- python/ray/tune/experiment/experiment.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/ray/tune/experiment/experiment.py b/python/ray/tune/experiment/experiment.py index 016a0fb6882b..2b2c372ee15e 100644 --- a/python/ray/tune/experiment/experiment.py +++ b/python/ray/tune/experiment/experiment.py @@ -206,6 +206,7 @@ def __init__( self._remote_storage_path = None self.sync_config = None self.dir_name = None + self.name = None else: if isinstance(sync_config, dict): sync_config = SyncConfig(**sync_config) From f25ad397a51647e989fedab9904bd283f0748c4c Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 13:08:03 -0700 Subject: [PATCH 061/108] Prepend legacy to old path attributes in experiment Signed-off-by: Justin Yu --- python/ray/tune/experiment/config_parser.py | 14 -------- python/ray/tune/experiment/experiment.py | 36 ++++++++++----------- python/ray/tune/search/basic_variant.py | 2 +- python/ray/tune/search/search_generator.py | 2 +- python/ray/tune/tune.py | 2 +- 5 files changed, 21 insertions(+), 35 deletions(-) diff --git a/python/ray/tune/experiment/config_parser.py b/python/ray/tune/experiment/config_parser.py index 659104adc153..482fc1e3e1c3 100644 --- a/python/ray/tune/experiment/config_parser.py +++ b/python/ray/tune/experiment/config_parser.py @@ -199,21 +199,7 @@ def _create_trial_from_spec( trial_kwargs["placement_group_factory"] = resources experiment_dir_name = spec.get("experiment_dir_name") or output_path - sync_config = spec.get("sync_config", SyncConfig()) - if ( - sync_config.syncer is not None - and sync_config.syncer != "auto" - and not isinstance(sync_config.syncer, Syncer) - ): - raise ValueError( - f"Unknown syncer type passed in SyncConfig: {type(sync_config.syncer)}. " - f"Note that custom sync functions and templates have been deprecated. " - f"Instead you can implement you own `Syncer` class. " - f"Please leave a comment on GitHub if you run into any issues with this: " - f"https://github.com/ray-project/ray/issues" - ) - checkpoint_config = spec.get("checkpoint_config", CheckpointConfig()) return Trial( diff --git a/python/ray/tune/experiment/experiment.py b/python/ray/tune/experiment/experiment.py index 2b2c372ee15e..e23b05fc21ec 100644 --- a/python/ray/tune/experiment/experiment.py +++ b/python/ray/tune/experiment/experiment.py @@ -202,18 +202,18 @@ def __init__( logger.debug(f"StorageContext on the DRIVER:\n{self.storage}") # TODO(justinvyu): Rename these to legacy. - self._local_storage_path = None - self._remote_storage_path = None - self.sync_config = None - self.dir_name = None - self.name = None + self._legacy_local_storage_path = None + self._legacy_remote_storage_path = None + self.legacy_sync_config = None + self.legacy_dir_name = None + self.legacy_name = None else: if isinstance(sync_config, dict): sync_config = SyncConfig(**sync_config) else: sync_config = sync_config or SyncConfig() - self.sync_config = sync_config + self.legacy_sync_config = sync_config # Resolve storage_path local_storage_path, remote_storage_path = _resolve_storage_path( @@ -248,19 +248,19 @@ def __init__( str(list(experiment_checkpoint_dir_path.parents)), ) # `dir_name` is set by `_experiment_checkpoint_dir` indirectly. - self.dir_name = os.path.relpath( + self.legacy_dir_name = os.path.relpath( _experiment_checkpoint_dir, full_local_storage_path ) - self._local_storage_path = full_local_storage_path - self._remote_storage_path = remote_storage_path + self._legacy_local_storage_path = full_local_storage_path + self._legacy_remote_storage_path = remote_storage_path - self.name = name or self._run_identifier + self.legacy_name = name or self._run_identifier if not _experiment_checkpoint_dir: - self.dir_name = _get_dir_name(run, name, self.name) + self.legacy_dir_name = _get_dir_name(run, name, self.legacy_name) - assert self.dir_name + assert self.legacy_dir_name config = config or {} @@ -320,8 +320,8 @@ def __init__( "resources_per_trial": resources_per_trial, "num_samples": num_samples, "experiment_path": self.path, - "experiment_dir_name": self.dir_name, - "sync_config": sync_config, + "experiment_dir_name": self.legacy_dir_name, + "sync_config": self.legacy_sync_config, "checkpoint_config": checkpoint_config, "trial_name_creator": trial_name_creator, "trial_dirname_creator": trial_dirname_creator, @@ -507,9 +507,9 @@ def local_path(self) -> Optional[str]: if _use_storage_context(): return self.storage.experiment_local_path - if not self._local_storage_path: + if not self._legacy_local_storage_path: return None - return str(Path(self._local_storage_path) / self.dir_name) + return str(Path(self._legacy_local_storage_path) / self.legacy_dir_name) @property @Deprecated("Replaced by `local_path`") @@ -522,9 +522,9 @@ def remote_path(self) -> Optional[str]: if _use_storage_context(): return self.storage.experiment_path - if not self._remote_storage_path: + if not self._legacy_remote_storage_path: return None - return str(URI(self._remote_storage_path) / self.dir_name) + return str(URI(self._legacy_remote_storage_path) / self.legacy_dir_name) @property def path(self) -> Optional[str]: diff --git a/python/ray/tune/search/basic_variant.py b/python/ray/tune/search/basic_variant.py index 6c8acafa2f11..c29188e1c1db 100644 --- a/python/ray/tune/search/basic_variant.py +++ b/python/ray/tune/search/basic_variant.py @@ -353,7 +353,7 @@ def add_configurations( num_samples=experiment.spec.get("num_samples", 1), unresolved_spec=experiment.spec, constant_grid_search=self._constant_grid_search, - output_path=experiment.dir_name, + output_path=experiment.legacy_dir_name, points_to_evaluate=points_to_evaluate, lazy_eval=lazy_eval, start=previous_samples, diff --git a/python/ray/tune/search/search_generator.py b/python/ray/tune/search/search_generator.py index a287dd52c27a..ab21d5124fb1 100644 --- a/python/ray/tune/search/search_generator.py +++ b/python/ray/tune/search/search_generator.py @@ -98,7 +98,7 @@ def next_trial(self): """ if not self.is_finished(): return self.create_trial_if_possible( - self._experiment.spec, self._experiment.dir_name + self._experiment.spec, self._experiment.legacy_dir_name ) return None diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index d4e097f7fcc2..123164492d08 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -1008,7 +1008,7 @@ class and registered trainables. placeholder_resolvers=placeholder_resolvers, scheduler=scheduler, experiment_path=experiments[0].path, - experiment_dir_name=experiments[0].dir_name, + experiment_dir_name=experiments[0].legacy_dir_name, sync_config=sync_config, stopper=experiments[0].stopper, resume=resume, From c99d30c28c40a634c352f910c8bf122dfe3d4d88 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 13:49:31 -0700 Subject: [PATCH 062/108] too much space Signed-off-by: Justin Yu --- python/ray/train/tests/test_new_persistence.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index 0e22fe3dcaa9..b01a73d550ed 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -287,7 +287,7 @@ def test_trainer( ├── progress.csv ├── result.json ├── checkpoint_000000 - │ ├── checkpoint_shard-rank=0.pkl <- Worker checkpoint shards + │ ├── checkpoint_shard-rank=0.pkl <- Worker checkpoint shards │ └── checkpoint_shard-rank=1.pkl ├── ... ├── artifact-rank=0-iter=0.txt <- Worker artifacts From 9d11d2d6fcfcc46aeaaae35c0c3456dbe92b7050 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 13:50:21 -0700 Subject: [PATCH 063/108] remove unused var Signed-off-by: Justin Yu --- python/ray/train/base_trainer.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/python/ray/train/base_trainer.py b/python/ray/train/base_trainer.py index 3e4d002452a0..f510dc1a49f4 100644 --- a/python/ray/train/base_trainer.py +++ b/python/ray/train/base_trainer.py @@ -598,10 +598,7 @@ def fit(self) -> Result: _entrypoint=AirEntrypoint.TRAINER, ) - ( - experiment_local_path, - expeirment_dir_name, - ) = TunerInternal.setup_create_experiment_checkpoint_dir( + experiment_local_path, _ = TunerInternal.setup_create_experiment_checkpoint_dir( trainable, self.run_config ) From 950b9910788cf0d7f23b4c5b7387fa84b4ed889d Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 13:50:54 -0700 Subject: [PATCH 064/108] Fix lint Signed-off-by: Justin Yu --- python/ray/tune/experiment/config_parser.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/tune/experiment/config_parser.py b/python/ray/tune/experiment/config_parser.py index 482fc1e3e1c3..84bdc49f3bdf 100644 --- a/python/ray/tune/experiment/config_parser.py +++ b/python/ray/tune/experiment/config_parser.py @@ -8,7 +8,7 @@ from ray.tune import TuneError from ray.tune.experiment import Trial from ray.tune.resources import json_to_resources -from ray.tune.syncer import SyncConfig, Syncer +from ray.tune.syncer import SyncConfig from ray.tune.utils.util import SafeFallbackEncoder From ed862553161c1709de6dd9b2a78b92795cfa0ef2 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 18:44:20 -0700 Subject: [PATCH 065/108] restore mostly works Signed-off-by: Justin Yu --- python/ray/train/base_trainer.py | 13 ++-- python/ray/train/trainer.py | 5 +- python/ray/tune/execution/tune_controller.py | 26 ++++++-- python/ray/tune/experiment/trial.py | 6 ++ .../ray/tune/trainable/function_trainable.py | 65 +++++++++++++++---- python/ray/tune/trainable/trainable.py | 21 ++++++ 6 files changed, 112 insertions(+), 24 deletions(-) diff --git a/python/ray/train/base_trainer.py b/python/ray/train/base_trainer.py index f510dc1a49f4..fb37d42f3f62 100644 --- a/python/ray/train/base_trainer.py +++ b/python/ray/train/base_trainer.py @@ -22,6 +22,7 @@ from ray.air.config import RunConfig, ScalingConfig from ray.air.result import Result from ray.train._internal import session +from ray.train._internal.storage import _use_storage_context from ray.train.constants import TRAIN_DATASET_KEY from ray.util import PublicAPI from ray.util.annotations import DeveloperAPI @@ -707,11 +708,13 @@ def train_func(config): checkpoint = session.get_checkpoint() if checkpoint: trainer.resume_from_checkpoint = checkpoint - # Always load the preprocessor from an available checkpoint - # Unless we are restoring the experiment and have explicitly - # passed in a new preprocessor - if not (restored and trainer.preprocessor): - trainer.preprocessor = checkpoint.get_preprocessor() + # TODO(justinvyu): Remove this when Preprocessor is removed from Trainer + if not _use_storage_context(): + # Always load the preprocessor from an available checkpoint + # Unless we are restoring the experiment and have explicitly + # passed in a new preprocessor + if not (restored and trainer.preprocessor): + trainer.preprocessor = checkpoint.get_preprocessor() trainer.setup() trainer.preprocess_datasets() diff --git a/python/ray/train/trainer.py b/python/ray/train/trainer.py index 833a9537acc5..c033067373cb 100644 --- a/python/ray/train/trainer.py +++ b/python/ray/train/trainer.py @@ -103,7 +103,10 @@ def _start_training( run_dir=run_dir, latest_checkpoint_id=latest_checkpoint_id, ) - checkpoint = self._checkpoint_manager._load_checkpoint(checkpoint) + + if not _use_storage_context(): + checkpoint = self._checkpoint_manager._load_checkpoint(checkpoint) + self._run_with_error_handling( lambda: self._backend_executor.start_training( train_func=train_func, diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index fcd803debbbd..e1dcc1a25a61 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -1955,11 +1955,29 @@ def _checkpoint_trial_if_needed(self, trial, force=False): ### # RESTORE def _schedule_trial_restore(self, trial: Trial) -> bool: - checkpoint = trial.checkpoint - if _use_storage_context(): - # TODO(justinvyu): Skipping restoration altogether for now. - return False + checkpoint_result = trial.checkpoint_manager.latest_checkpoint_result + + if not checkpoint_result: + logger.debug(f"Not restoring trial {trial}: No checkpoint found.") + return False + + # TODO(justinvyu): Is this really needed? + trial.restoring_from = checkpoint_result + + method_name = "restore" + args = (checkpoint_result,) + self._schedule_trial_task( + trial=trial, + method_name=method_name, + args=args, + kwargs={}, + on_result=self._on_restoring_result, + on_error=self._trial_task_failure, + ) + return True + + checkpoint = trial.checkpoint if checkpoint.dir_or_data is None: logger.debug(f"Not restoring trial {trial}: No checkpoint found.") diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index 966cf470ce69..31e6fbf5cf67 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -1088,6 +1088,12 @@ def on_checkpoint(self, checkpoint: _TrackedCheckpoint): def on_restore(self): """Handles restoration completion.""" assert self.is_restoring + + if _use_storage_context(): + from ray.train._internal.checkpoint_manager import _TrainingResult + + assert isinstance(self.restoring_from, _TrainingResult) + self.last_result = self.restoring_from.metrics self.last_result.setdefault("config", self.config) self.restoring_from = None diff --git a/python/ray/tune/trainable/function_trainable.py b/python/ray/tune/trainable/function_trainable.py index 569f2f9731d1..6da193cf82e2 100644 --- a/python/ray/tune/trainable/function_trainable.py +++ b/python/ray/tune/trainable/function_trainable.py @@ -9,7 +9,7 @@ import warnings from functools import partial from numbers import Number -from typing import Any, Callable, Dict, Optional, Type +from typing import Any, Callable, Dict, Optional, Type, TYPE_CHECKING from ray.air._internal.util import StartTraceback, RunnerThread import queue @@ -37,6 +37,10 @@ from ray.util.annotations import DeveloperAPI from ray.util.debug import log_once +if TYPE_CHECKING: + from ray.train._internal.checkpoint_manager import _TrainingResult + + logger = logging.getLogger(__name__) # Time between FunctionTrainable checks when fetching @@ -149,6 +153,7 @@ def __init__( self._trial_id = trial_id self._logdir = logdir self._last_checkpoint = None + self._latest_checkpoint_result: Optional["_TrainingResult"] = None self._fresh_checkpoint = False self._trial_resources = trial_resources # Mark whether the `ray.air.session.report()` API is being used, @@ -160,6 +165,7 @@ def reset(self, trial_name=None, trial_id=None, logdir=None, trial_resources=Non self._trial_id = trial_id self._logdir = logdir self._last_checkpoint = None + self._latest_checkpoint_result = None self._fresh_checkpoint = False self._trial_resources = trial_resources self._air_session_has_reported = False @@ -242,6 +248,14 @@ def set_checkpoint(self, checkpoint, is_new=True): def has_new_checkpoint(self): return self._fresh_checkpoint + def get_checkpoint_result(self) -> Optional["_TrainingResult"]: + from ray.train._internal.storage import _use_storage_context + + assert _use_storage_context() + # The checkpoint is no longer fresh after it's been handed off to Tune. + self._fresh_checkpoint = False + return self._latest_checkpoint_result + def get_checkpoint(self): # NOTE: This is not the same as `train.get_checkpoint`. # This is used internally by `FunctionTrainable.save_checkpoint`. @@ -263,15 +277,18 @@ def report(self, metrics: Dict, *, checkpoint: Optional[Checkpoint] = None) -> N # TODO(justinvyu): With a unified session, we'll still run into this doubled # report problem. This should be fixed by checking if the checkpoint has been # uploaded already (via some marker), then skipping the repeat upload. - if _use_storage_context() and isinstance(checkpoint, NewCheckpoint): + if _use_storage_context(): + assert isinstance(checkpoint, NewCheckpoint) logger.debug(f"Checkpoint received by the Tune session: {checkpoint}") self._fresh_checkpoint = True # TODO(justinvyu): `metrics` doesn't include the autofilled metrics # like `training_iteration` and `time_total_s`. # Should the session be the source of truth for these metrics? - self._last_checkpoint = _TrainingResult( + self._latest_checkpoint_result = _TrainingResult( checkpoint=checkpoint, metrics=metrics ) + + self._last_checkpoint = None else: if checkpoint: training_iteration = self._get_training_iteration() @@ -284,15 +301,17 @@ def report(self, metrics: Dict, *, checkpoint: Optional[Checkpoint] = None) -> N @property def loaded_checkpoint(self) -> Optional[Checkpoint]: - if self._last_checkpoint: - from ray.train._internal.storage import _use_storage_context - from ray.train._internal.checkpoint_manager import _TrainingResult + from ray.train._internal.storage import _use_storage_context + from ray.train._internal.checkpoint_manager import _TrainingResult - if _use_storage_context() and isinstance( - self._last_checkpoint, _TrainingResult - ): - return self._last_checkpoint.checkpoint + if _use_storage_context(): + if not self._latest_checkpoint_result: + return None + assert isinstance(self._latest_checkpoint_result, _TrainingResult) + return self._latest_checkpoint_result.checkpoint + + if self._last_checkpoint: assert isinstance(self._last_checkpoint, str) return Checkpoint.from_directory(self._last_checkpoint) return None @@ -483,6 +502,12 @@ def execute(self, fn): def get_state(self): state = super().get_state() + from ray.train._internal.storage import _use_storage_context + + if _use_storage_context(): + # TODO(justinvyu): Not too sure about this. + return state + checkpoint = self._status_reporter.get_checkpoint() if not checkpoint: state.update(iteration=0, timesteps_total=0, episodes_total=0) @@ -492,13 +517,15 @@ def save_checkpoint(self, checkpoint_dir: str = ""): if checkpoint_dir: raise ValueError("Checkpoint dir should not be used with function API.") - checkpoint = self._status_reporter.get_checkpoint() - from ray.train._internal.storage import _use_storage_context from ray.train._internal.checkpoint_manager import _TrainingResult - if _use_storage_context() and isinstance(checkpoint, _TrainingResult): - return checkpoint + if _use_storage_context(): + checkpoint_result = self._status_reporter.get_checkpoint_result() + assert isinstance(checkpoint_result, _TrainingResult) + return checkpoint_result + + checkpoint = self._status_reporter.get_checkpoint() if not checkpoint: # We drop a marker here to indicate that the checkpoint is empty @@ -537,6 +564,16 @@ def save_to_object(self): return checkpoint.to_bytes() def load_checkpoint(self, checkpoint): + from ray.train._internal.storage import _use_storage_context + from ray.train._internal.checkpoint_manager import _TrainingResult + + if _use_storage_context(): + checkpoint_result = checkpoint + assert isinstance(checkpoint_result, _TrainingResult) + self._status_reporter._latest_checkpoint_result = checkpoint_result + self._status_reporter._fresh_checkpoint = False + return + # This should be removed once Trainables are refactored. if "tune_checkpoint_path" in checkpoint: del checkpoint["tune_checkpoint_path"] diff --git a/python/ray/tune/trainable/trainable.py b/python/ray/tune/trainable/trainable.py index d5488b52a8e9..75b17be49a86 100644 --- a/python/ray/tune/trainable/trainable.py +++ b/python/ray/tune/trainable/trainable.py @@ -864,6 +864,27 @@ def restore( could not be found. """ + if _use_storage_context(): + from ray.train._internal.checkpoint_manager import _TrainingResult + + checkpoint_result = checkpoint_path + assert isinstance(checkpoint_result, _TrainingResult) + + # TODO(justinvyu): The Trainable `load_checkpoint` interface + # should be updated to take in a `_TrainingResult` / Checkpoint + self.load_checkpoint(checkpoint_result) + + # TODO(justinvyu): What should we do about Trainable metadata? + self._time_since_restore = 0.0 + self._timesteps_since_restore = 0 + self._iterations_since_restore = 0 + self._restored = True + + logger.info( + f"Restored on {self._local_ip} from checkpoint: {checkpoint_result}" + ) + return True + # Ensure Checkpoints are converted if isinstance(checkpoint_path, Checkpoint): return self._restore_from_checkpoint_obj(checkpoint_path) From de4b92400c66d3935e87b63c963c72b20ac5b23d Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 23:18:02 -0700 Subject: [PATCH 066/108] hacky way of getting checkpoint folders to increment correctly Signed-off-by: Justin Yu --- python/ray/train/_internal/storage.py | 22 ++++++++++++++--- python/ray/train/base_trainer.py | 30 ++++++++++++++++++++--- python/ray/train/data_parallel_trainer.py | 19 ++++++++++++-- python/ray/train/trainer.py | 18 ++++---------- python/ray/tune/experiment/trial.py | 5 ++-- 5 files changed, 70 insertions(+), 24 deletions(-) diff --git a/python/ray/train/_internal/storage.py b/python/ray/train/_internal/storage.py index ff1d29f00f32..92efd664a162 100644 --- a/python/ray/train/_internal/storage.py +++ b/python/ray/train/_internal/storage.py @@ -601,18 +601,34 @@ def checkpoint_fs_path(self) -> str: Raises a ValueError if `current_checkpoint_index` is not set beforehand. """ - from ray.tune.trainable.util import TrainableUtil - if self.current_checkpoint_index is None: raise RuntimeError( "Should not access `checkpoint_fs_path` without setting " "`current_checkpoint_index`" ) - checkpoint_dir_name = TrainableUtil._make_checkpoint_dir_name( + checkpoint_dir_name = StorageContext._make_checkpoint_dir_name( self.current_checkpoint_index ) return os.path.join(self.trial_fs_path, checkpoint_dir_name) + @staticmethod + def _make_checkpoint_dir_name(index: int): + """Get the name of the checkpoint directory, given an index.""" + return f"checkpoint_{index:06d}" + + @staticmethod + def _parse_checkpoint_index(checkpoint_dir_name: str): + """Parse the index from a checkpoint directory name + created by _make_checkpoint_dir_name. + + >>> StorageContext._parse_checkpoint_index("checkpoint_000001") + 1 + >>> StorageContext._parse_checkpoint_index("checkpoint_11111111") + 11111111 + """ + assert checkpoint_dir_name.startswith("checkpoint_") + return int(checkpoint_dir_name.split("_")[1]) + _storage_context: Optional[StorageContext] = None diff --git a/python/ray/train/base_trainer.py b/python/ray/train/base_trainer.py index fb37d42f3f62..f96ee8cd03a6 100644 --- a/python/ray/train/base_trainer.py +++ b/python/ray/train/base_trainer.py @@ -21,6 +21,7 @@ from ray.air.checkpoint import Checkpoint from ray.air.config import RunConfig, ScalingConfig from ray.air.result import Result +from ray.train._checkpoint import Checkpoint as NewCheckpoint from ray.train._internal import session from ray.train._internal.storage import _use_storage_context from ray.train.constants import TRAIN_DATASET_KEY @@ -196,6 +197,9 @@ def __init__( # This path should only be set through restore self._restore_path = None + # This checkpoint should only be populated internally + # by auto-recovery fault tolerance/manual Trainer.restore + self._checkpoint_for_restoration: Optional[NewCheckpoint] = None self._validate_attributes() @@ -682,6 +686,22 @@ def _extract_fields_for_tuner_param_space(self) -> Dict: result[key] = copy.deepcopy(self._param_dict[key]) return result + def _get_initial_checkpoint(self) -> Optional[NewCheckpoint]: + """If we need to set an initial checkpoint accessible by train.get_checkpoint, + we are in one of 3 cases: + 1. We are auto-recovering from a training failure (FailureConfig). + -> In this case, _checkpoint_for_restoration points to the *latest* + checkpoint, which is the one we want to restore from. + 2. We are manually restoring an existing experiment (Trainer.restore). + -> Same as 1. + 3. The user passed in a checkpoint to start a *new* training run from. + -> `resume_from_checkpoint` points to the user-specified checkpoint. + + When populating `train.get_checkpoint`, `_checkpoint_for_restoration` + should take precedence over `resume_from_checkpoint`. + """ + return self._checkpoint_for_restoration or self.resume_from_checkpoint + def _generate_trainable_cls(self) -> Type["Trainable"]: """Generate the base Trainable class. @@ -701,13 +721,15 @@ def train_func(config): # Instantiate new Trainer in Trainable. trainer = trainer_cls(**config) - # Get the checkpoint from the train context, and use it to initialize + # Get the checkpoint from Tune, and use it to initialize # the restored trainer. - # This handles both worker-level and cluster-level restoration - # of the Train experiment. checkpoint = session.get_checkpoint() if checkpoint: - trainer.resume_from_checkpoint = checkpoint + # Set `_checkpoint_for_restoration` for auto-recovery fault-tolerance + # as well as manual restoration. + # `trainer.resume_from_checkpoint` is only ever set by the user. + trainer._checkpoint_for_restoration = checkpoint + # TODO(justinvyu): Remove this when Preprocessor is removed from Trainer if not _use_storage_context(): # Always load the preprocessor from an available checkpoint diff --git a/python/ray/train/data_parallel_trainer.py b/python/ray/train/data_parallel_trainer.py index 8b37b7f0e3e9..66ab610b20e7 100644 --- a/python/ray/train/data_parallel_trainer.py +++ b/python/ray/train/data_parallel_trainer.py @@ -1,6 +1,7 @@ import copy import inspect import logging +import os from pathlib import Path from typing import TYPE_CHECKING, Any, Callable, Dict, Optional, Tuple, Type, Union from ray._private.thirdparty.tabulate.tabulate import tabulate @@ -17,7 +18,7 @@ from ray.train._internal.backend_executor import BackendExecutor, TrialInfo from ray.train._internal.checkpoint import TuneCheckpointManager from ray.train._internal.data_config import DataConfig, _LegacyDataConfigWrapper -from ray.train._internal.storage import _use_storage_context +from ray.train._internal.storage import _use_storage_context, StorageContext from ray.train._internal.utils import construct_train_func from ray.train.constants import TRAIN_DATASET_KEY, WILDCARD_KEY from ray.train.trainer import BaseTrainer, GenDataset @@ -520,6 +521,19 @@ def clear_lazy_checkpoint_marker(): checkpoint_strategy.num_to_keep = None checkpoint_strategy.checkpoint_score_attribute = None + initial_checkpoint = self._get_initial_checkpoint() + + latest_checkpoint_index = 0 + if _use_storage_context() and self._checkpoint_for_restoration: + checkpoint_dir_name = os.path.basename( + self._checkpoint_for_restoration.path + ) + # Need to add 1 because latest_checkpoint_index is the index of the + # next checkpoint to be saved. + latest_checkpoint_index = ( + StorageContext._parse_checkpoint_index(checkpoint_dir_name) + 1 + ) + training_iterator = self._training_iterator_cls( backend_executor=backend_executor, backend_config=self._backend_config, @@ -527,9 +541,10 @@ def clear_lazy_checkpoint_marker(): datasets=self.datasets, data_config=self._data_config, checkpoint_manager=checkpoint_manager, - checkpoint=self.resume_from_checkpoint, + checkpoint=initial_checkpoint, checkpoint_strategy=checkpoint_strategy, storage_path=self.run_config.storage_path, + latest_checkpoint_index=latest_checkpoint_index, ) self._report(training_iterator) diff --git a/python/ray/train/trainer.py b/python/ray/train/trainer.py index c033067373cb..46a42c83d661 100644 --- a/python/ray/train/trainer.py +++ b/python/ray/train/trainer.py @@ -56,6 +56,7 @@ def __init__( checkpoint_strategy: Optional[CheckpointConfig], run_dir: Optional[Path] = None, storage_path: Optional[str] = None, + latest_checkpoint_index: int = 0, ): self._backend_executor = backend_executor self._backend = backend_config.backend_cls() @@ -71,9 +72,8 @@ def __init__( # TrainingResult event. There's no need to do these one at a time. self._checkpoint_to_report = None - # TODO(justinvyu): Is this the best way to do this? Need to save this - # as part of checkpoint metadata and load it back on restore. - self._latest_checkpoint_index = 0 + self._storage = get_storage_context() if _use_storage_context() else None + self._storage.current_checkpoint_index = latest_checkpoint_index self._start_training( train_func=train_func, @@ -122,18 +122,10 @@ def _send_next_checkpoint_path_to_workers(self): # NOTE: Always upload to storage from workers in the new persistence path # (no need to check for the `checkpoint_upload_from_workers` flag) if _use_storage_context(): - storage = get_storage_context() - - # NOTE: Idea: this checkpoint dir name should be customizable - # and created on the fly when the checkpoint is reported with metrics. - # Ex: lambda metrics: f"checkpoint_iter={metrics['training_iteration']}" - storage.current_checkpoint_index = self._latest_checkpoint_index - self._backend_executor._set_checkpoint_index( - storage.current_checkpoint_index + self._storage.current_checkpoint_index ) - - self._latest_checkpoint_index += 1 + self._storage.current_checkpoint_index += 1 elif self._checkpoint_strategy._checkpoint_upload_from_workers: self._backend_executor._set_legacy_checkpoint_uri( diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index 31e6fbf5cf67..9afb00126387 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -1079,8 +1079,9 @@ def on_checkpoint(self, checkpoint: _TrackedCheckpoint): if _use_storage_context(): from ray.train._internal.checkpoint_manager import _TrainingResult - assert isinstance(checkpoint, _TrainingResult) - self.checkpoint_manager.register_checkpoint(checkpoint) + checkpoint_result = checkpoint + assert isinstance(checkpoint_result, _TrainingResult) + self.checkpoint_manager.register_checkpoint(checkpoint_result) else: self.checkpoint_manager.on_checkpoint(checkpoint) self.invalidate_json_state() From e060476e55d7572e854e4f65d057a02511859630 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Thu, 3 Aug 2023 23:19:31 -0700 Subject: [PATCH 067/108] Fix for xgboost trainer Signed-off-by: Justin Yu --- python/ray/train/gbdt_trainer.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/train/gbdt_trainer.py b/python/ray/train/gbdt_trainer.py index 997be4230008..9a8b941e76a2 100644 --- a/python/ray/train/gbdt_trainer.py +++ b/python/ray/train/gbdt_trainer.py @@ -280,8 +280,8 @@ def training_loop(self) -> None: evals_result = {} init_model = None - if self.resume_from_checkpoint: - init_model, _ = self._load_checkpoint(self.resume_from_checkpoint) + if self._get_initial_checkpoint(): + init_model, _ = self._load_checkpoint(self._get_initial_checkpoint()) config.setdefault("verbose_eval", False) config.setdefault("callbacks", []) From bd5c846cd24ca4911d08496519ff140cfe5fcb5c Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 4 Aug 2023 09:45:03 -0700 Subject: [PATCH 068/108] Fix race as_directory / download file lock race condition Signed-off-by: Justin Yu --- python/ray/train/_checkpoint.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/python/ray/train/_checkpoint.py b/python/ray/train/_checkpoint.py index 6505c70f877e..a5b65f1de9c2 100644 --- a/python/ray/train/_checkpoint.py +++ b/python/ray/train/_checkpoint.py @@ -211,10 +211,11 @@ def as_directory(self) -> Iterator[str]: if isinstance(self.filesystem, pyarrow.fs.LocalFileSystem): yield self.path else: - temp_dir = self.to_directory() - del_lock_path = _get_del_lock_path(temp_dir) + del_lock_path = _get_del_lock_path(self._get_temporary_checkpoint_dir()) open(del_lock_path, "a").close() + temp_dir = self.to_directory() + yield temp_dir # Cleanup @@ -236,7 +237,7 @@ def as_directory(self) -> Iterator[str]: # Timeout 0 means there will be only one attempt to acquire # the file lock. If it cannot be acquired, a TimeoutError # will be thrown. - with TempFileLock(f"{temp_dir}.lock", timeout=0): + with TempFileLock(temp_dir, timeout=0): shutil.rmtree(temp_dir, ignore_errors=True) except TimeoutError: pass From e51fb1782ac35c320b759b4611dd22908f0568bd Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 4 Aug 2023 09:45:30 -0700 Subject: [PATCH 069/108] Update test with auto-recovery fault tolerance Signed-off-by: Justin Yu --- .../ray/train/tests/test_new_persistence.py | 26 ++++++++++++++----- 1 file changed, 19 insertions(+), 7 deletions(-) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index b01a73d550ed..8de2a3641eff 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -143,24 +143,28 @@ def train_fn(config): for i in range(start, config.get("num_iterations", 5)): time.sleep(0.25) - checkpoint_file_name = "checkpoint.pkl" + temp_dir = tempfile.mkdtemp() + with open(os.path.join(temp_dir, "checkpoint.pkl"), "wb") as f: + pickle.dump({"iter": i}, f) + artifact_file_name = f"artifact-iter={i}.txt" if in_trainer: rank = train.get_context().get_world_rank() - checkpoint_file_name = f"checkpoint_shard-rank={rank}.pkl" artifact_file_name = f"artifact-rank={rank}-iter={i}.txt" + checkpoint_file_name = f"checkpoint_shard-rank={rank}.pkl" + with open(os.path.join(temp_dir, checkpoint_file_name), "wb") as f: + pickle.dump({"iter": i}, f) + with open(artifact_file_name, "w") as f: f.write(f"{i}") - temp_dir = tempfile.mkdtemp() - with open(os.path.join(temp_dir, checkpoint_file_name), "wb") as f: - pickle.dump({"iter": i}, f) - train.report( {"iter": i, _SCORE_KEY: i}, checkpoint=NewCheckpoint.from_directory(temp_dir), ) + if i in config.get("fail_iters", []): + raise RuntimeError(f"Failing on iter={i}!!") @pytest.mark.parametrize("storage_path_type", [None, "nfs", "cloud", "custom_fs"]) @@ -287,6 +291,7 @@ def test_trainer( ├── progress.csv ├── result.json ├── checkpoint_000000 + │ ├── checkpoint.pkl <- Shared checkpoint file │ ├── checkpoint_shard-rank=0.pkl <- Worker checkpoint shards │ └── checkpoint_shard-rank=1.pkl ├── ... @@ -309,7 +314,11 @@ def test_trainer( NUM_WORKERS = 2 trainer = DataParallelTrainer( train_fn, - train_loop_config={"in_trainer": True, "num_iterations": NUM_ITERATIONS}, + train_loop_config={ + "in_trainer": True, + "num_iterations": NUM_ITERATIONS, + "fail_iters": [2, 4], + }, scaling_config=train.ScalingConfig(num_workers=2), run_config=train.RunConfig( storage_path=storage_path, @@ -317,6 +326,7 @@ def test_trainer( name=exp_name, verbose=0, checkpoint_config=checkpoint_config, + failure_config=train.FailureConfig(max_failures=2), ), ) result = trainer.fit() @@ -352,6 +362,8 @@ def test_trainer( assert len(list(trial_dir.glob("checkpoint_*"))) == expected_num_checkpoints for checkpoint_dir in trial_dir.glob("checkpoint_*"): + # 1 shared checkpoint.pkl file, written by all workers. + assert len(list(checkpoint_dir.glob("checkpoint.pkl"))) == 1 # 1 checkpoint shard per worker. assert ( len(list(checkpoint_dir.glob("checkpoint_shard-*.pkl"))) == NUM_WORKERS From 0c3c5c8fc95bce4ddea8749949c1421945890441 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 4 Aug 2023 12:15:35 -0700 Subject: [PATCH 070/108] compute storage_prefix Signed-off-by: Justin Yu --- python/ray/air/_internal/uri_utils.py | 27 ++++++++++++++---- python/ray/train/_internal/storage.py | 40 ++++++++++++++++++--------- 2 files changed, 49 insertions(+), 18 deletions(-) diff --git a/python/ray/air/_internal/uri_utils.py b/python/ray/air/_internal/uri_utils.py index 54684ad01601..99648c283a41 100644 --- a/python/ray/air/_internal/uri_utils.py +++ b/python/ray/air/_internal/uri_utils.py @@ -9,15 +9,15 @@ class URI: Example Usage: - >>> s3_uri = URI("s3://bucket/a?scheme=http&endpoint_override=localhost%3A900") + >>> s3_uri = URI("s3://bucket/a?scheme=http¶m=1") >>> s3_uri - URI + URI >>> str(s3_uri / "b" / "c") - 's3://bucket/a/b/c?scheme=http&endpoint_override=localhost%3A900' + 's3://bucket/a/b/c?scheme=http¶m=1' >>> str(s3_uri.parent) - 's3://bucket?scheme=http&endpoint_override=localhost%3A900' + 's3://bucket?scheme=http¶m=1' >>> str(s3_uri) - 's3://bucket/a?scheme=http&endpoint_override=localhost%3A900' + 's3://bucket/a?scheme=http¶m=1' >>> s3_uri.parent.name, s3_uri.name ('bucket', 'a') >>> local_path = URI("/tmp/local") @@ -42,6 +42,23 @@ def __init__(self, uri: str): else: self._path = Path(os.path.normpath(self._parsed.netloc + self._parsed.path)) + def rstrip_subpath(self, subpath: Path) -> "URI": + """Returns a new URI that strips the given subpath from the end of this URI. + + Example: + >>> uri = URI("s3://bucket/a/b/c/?param=1") + >>> str(uri.rstrip_subpath(Path("b/c"))) + 's3://bucket/a?param=1' + + >>> uri = URI("/tmp/a/b/c/") + >>> str(uri.rstrip_subpath(Path("/b/c/.//"))) + '/tmp/a' + + """ + assert str(self._path).endswith(str(subpath)), (self._path, subpath) + stripped_path = str(self._path).replace(str(subpath), "") + return URI(self._get_str_representation(self._parsed, stripped_path)) + @property def name(self) -> str: return self._path.name diff --git a/python/ray/train/_internal/storage.py b/python/ray/train/_internal/storage.py index ff1d29f00f32..469ed64c6604 100644 --- a/python/ray/train/_internal/storage.py +++ b/python/ray/train/_internal/storage.py @@ -320,28 +320,25 @@ class StorageContext: For example, on the driver, the storage context is initialized, only knowing the experiment path. On the Trainable actor, the trial_dir_name is accessible. - There are 3 path concepts: - 1. *_path: A path or URI. This is the same format as the `storage_path` specified - by the user. This may need to be URI-joined. - 2. *_fs_path: A path relative to the storage filesystem. This is a regular path - that can be joined with `os.path.join`. - 3. *_local_path: The path on the local filesystem where results are saved to + There are 2 types of paths: + 1. *_fs_path: A path on the `storage_filesystem`. This is a regular path + which has been prefix-stripped by pyarrow.fs.FileSystem.from_uri and + can be joined with `os.path.join`. + 2. *_local_path: The path on the local filesystem where results are saved to before persisting to storage. - Example with storage_path="mock:///bucket/path": + Example with storage_path="mock:///bucket/path?param=1": >>> from ray.train._internal.storage import StorageContext >>> import os >>> os.environ["RAY_AIR_LOCAL_CACHE_DIR"] = "/tmp/ray_results" >>> storage = StorageContext( - ... storage_path="mock:///bucket/path", + ... storage_path="mock://netloc/bucket/path?param=1", ... sync_config=SyncConfig(), ... experiment_dir_name="exp_name", ... ) >>> storage.storage_filesystem # Auto-resolved # doctest: +ELLIPSIS >> storage.experiment_path - 'mock:///bucket/path/exp_name' >>> storage.experiment_fs_path 'bucket/path/exp_name' >>> storage.experiment_local_path @@ -354,6 +351,10 @@ class StorageContext: >>> storage.current_checkpoint_index = 1 >>> storage.checkpoint_fs_path 'bucket/path/exp_name/trial_dir/checkpoint_000001' + >>> storage.storage_prefix + URI + >>> str(storage.storage_prefix / storage.experiment_fs_path) + 'mock://netloc/bucket/path/exp_name?param=1' Example with storage_path=None: @@ -369,8 +370,6 @@ class StorageContext: '/tmp/ray_results' >>> storage.storage_local_path '/tmp/ray_results' - >>> storage.experiment_path - '/tmp/ray_results/exp_name' >>> storage.experiment_local_path '/tmp/ray_results/exp_name' >>> storage.experiment_fs_path @@ -379,7 +378,10 @@ class StorageContext: True >>> storage.storage_filesystem # Auto-resolved # doctest: +ELLIPSIS >> storage.storage_prefix + URI<.> + >>> str(storage.storage_prefix / storage.experiment_fs_path) + '/tmp/ray_results/exp_name' Internal Usage Examples: - To copy files to the trial directory on the storage filesystem: @@ -436,6 +438,18 @@ def __init__( self.storage_fs_path, ) = pyarrow.fs.FileSystem.from_uri(self.storage_path) + # The storage prefix is the URI that remains after stripping the + # URI prefix away from the user-provided `storage_path` (using `from_uri`). + # Ex: `storage_path="s3://bucket/path?param=1` + # -> `storage_prefix=URI` + # See the doctests for more examples. + # This is used to construct URI's of the same format as `storage_path`. + # However, we don't track these URI's internally, because pyarrow only + # needs to interact with the prefix-stripped fs_path. + self.storage_prefix: URI = URI(self.storage_path).rstrip_subpath( + Path(self.storage_fs_path) + ) + # Only initialize a syncer if a `storage_path` was provided. self.syncer: Optional[Syncer] = ( _FilesystemSyncer( From 217af77baf76be14af93b60dacf2d630f06a0aff Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 4 Aug 2023 12:30:25 -0700 Subject: [PATCH 071/108] Remove '_path' properties from storage Signed-off-by: Justin Yu --- python/ray/train/_internal/storage.py | 20 -------------------- python/ray/tune/execution/tune_controller.py | 3 ++- python/ray/tune/experiment/experiment.py | 2 +- python/ray/tune/experiment/trial.py | 4 ++-- 4 files changed, 5 insertions(+), 24 deletions(-) diff --git a/python/ray/train/_internal/storage.py b/python/ray/train/_internal/storage.py index 469ed64c6604..1a4acb3816d5 100644 --- a/python/ray/train/_internal/storage.py +++ b/python/ray/train/_internal/storage.py @@ -546,16 +546,6 @@ def persist_current_checkpoint(self, checkpoint: "Checkpoint") -> "Checkpoint": logger.debug(f"Checkpoint successfully created at: {uploaded_checkpoint}") return uploaded_checkpoint - @property - def experiment_path(self) -> str: - """The path the experiment directory, where the format matches the - original `storage_path` format specified by the user. - - Ex: If the user passed in storage_path="s3://bucket/path?param=1", then - this property returns "s3://bucket/path/exp_name?param=1". - """ - return str(URI(self.storage_path) / self.experiment_dir_name) - @property def experiment_fs_path(self) -> str: """The path on the `storage_filesystem` to the experiment directory. @@ -575,16 +565,6 @@ def experiment_local_path(self) -> str: """ return os.path.join(self.storage_local_path, self.experiment_dir_name) - @property - def trial_path(self) -> str: - """The path the experiment directory, where the format matches the - original `storage_path` format specified by the user. - - Ex: If the user passed in storage_path="s3://bucket/path?param=1", then - this property returns "s3://bucket/path/exp_name?param=1". - """ - return str(URI(self.experiment_path) / self.trial_dir_name) - @property def trial_local_path(self) -> str: """The local filesystem path to the trial directory. diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index fcd803debbbd..b880257bab69 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -406,7 +406,8 @@ def experiment_state_path(self) -> str: @property def experiment_path(self) -> str: if _use_storage_context(): - return self._storage.experiment_path + return str(self._storage.storage_prefix / self._storage.experiment_fs_path) + return self._legacy_remote_experiment_path or self._legacy_local_experiment_path def _create_checkpoint_manager(self): diff --git a/python/ray/tune/experiment/experiment.py b/python/ray/tune/experiment/experiment.py index e23b05fc21ec..7cfcf0fd1ecb 100644 --- a/python/ray/tune/experiment/experiment.py +++ b/python/ray/tune/experiment/experiment.py @@ -520,7 +520,7 @@ def local_dir(self): @property def remote_path(self) -> Optional[str]: if _use_storage_context(): - return self.storage.experiment_path + return str(self.storage.storage_prefix / self.storage.experiment_fs_path) if not self._legacy_remote_storage_path: return None diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index 966cf470ce69..267918050f9f 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -698,7 +698,7 @@ def experiment_dir_name(self, name: str): @property def remote_experiment_path(self) -> str: if _use_storage_context(): - return self.storage.experiment_path + return str(self.storage.storage_prefix / self.storage.experiment_fs_path) return str(self._legacy_remote_experiment_path) @@ -808,7 +808,7 @@ def remote_path(self) -> Optional[str]: @property def path(self) -> Optional[str]: if _use_storage_context(): - return self.storage.trial_path + return str(self.storage.storage_prefix / self.storage.trial_fs_path) return self.remote_path or self.local_path From 8e2330c77f7bdd97bf480cfcb1a1ac3c8bc34b62 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 4 Aug 2023 12:41:29 -0700 Subject: [PATCH 072/108] Move exp dir name helper to storage ctx Signed-off-by: Justin Yu --- python/ray/train/_internal/storage.py | 15 ++++++++++++++- python/ray/tune/experiment/experiment.py | 14 -------------- python/ray/tune/impl/tuner_internal.py | 6 +++--- 3 files changed, 17 insertions(+), 18 deletions(-) diff --git a/python/ray/train/_internal/storage.py b/python/ray/train/_internal/storage.py index 1a4acb3816d5..eddb142bb504 100644 --- a/python/ray/train/_internal/storage.py +++ b/python/ray/train/_internal/storage.py @@ -4,7 +4,7 @@ import os from pathlib import Path import shutil -from typing import Callable, Dict, List, Optional, Tuple, TYPE_CHECKING +from typing import Callable, Dict, List, Optional, Tuple, Type, Union, TYPE_CHECKING try: import fsspec @@ -607,6 +607,19 @@ def checkpoint_fs_path(self) -> str: ) return os.path.join(self.trial_fs_path, checkpoint_dir_name) + @staticmethod + def get_experiment_dir_name(run_obj: Union[str, Callable, Type]) -> str: + from ray.tune.experiment import Experiment + from ray.tune.utils import date_str + + run_identifier = Experiment.get_trainable_name(run_obj) + + if bool(int(os.environ.get("TUNE_DISABLE_DATED_SUBDIR", 0))): + dir_name = run_identifier + else: + dir_name = "{}_{}".format(run_identifier, date_str()) + return dir_name + _storage_context: Optional[StorageContext] = None diff --git a/python/ray/tune/experiment/experiment.py b/python/ray/tune/experiment/experiment.py index 7cfcf0fd1ecb..db2f7d4963eb 100644 --- a/python/ray/tune/experiment/experiment.py +++ b/python/ray/tune/experiment/experiment.py @@ -453,20 +453,6 @@ def register_if_needed(cls, run_object: Union[str, Callable, Type]): raise type(e)(str(e) + " " + extra_msg) from None return name - @classmethod - def get_experiment_dir_name( - cls, - run_obj: Union[str, Callable, Type], - ) -> str: - assert run_obj - run_identifier = cls.get_trainable_name(run_obj) - - if bool(int(os.environ.get("TUNE_DISABLE_DATED_SUBDIR", 0))): - dir_name = run_identifier - else: - dir_name = "{}_{}".format(run_identifier, date_str()) - return dir_name - @classmethod def get_experiment_checkpoint_dir( cls, diff --git a/python/ray/tune/impl/tuner_internal.py b/python/ray/tune/impl/tuner_internal.py index d68d7362a8da..c995d639789d 100644 --- a/python/ray/tune/impl/tuner_internal.py +++ b/python/ray/tune/impl/tuner_internal.py @@ -26,7 +26,7 @@ from ray.air._internal.uri_utils import URI from ray.air._internal.usage import AirEntrypoint from ray.air.config import RunConfig, ScalingConfig -from ray.train._internal.storage import _use_storage_context +from ray.train._internal.storage import _use_storage_context, StorageContext from ray.tune import Experiment, TuneError, ExperimentAnalysis from ray.tune.execution.experiment_state import _ResumeConfig from ray.tune.tune import _Config @@ -534,8 +534,8 @@ def setup_create_experiment_checkpoint_dir( Tuple: (experiment_path, experiment_dir_name) """ if _use_storage_context(): - experiment_dir_name = run_config.name or Experiment.get_experiment_dir_name( - trainable + experiment_dir_name = ( + run_config.name or StorageContext.get_experiment_dir_name(trainable) ) storage_local_path = _get_defaults_results_dir() experiment_path = os.path.join(storage_local_path, experiment_dir_name) From f3f22fd3cb1ceba7a7806ee41063cb2bfc05b0e1 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 4 Aug 2023 12:54:43 -0700 Subject: [PATCH 073/108] Fix bugs causing broken CI Signed-off-by: Justin Yu --- python/ray/tune/experiment/experiment.py | 11 +++++------ python/ray/tune/impl/tuner_internal.py | 7 +++---- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/python/ray/tune/experiment/experiment.py b/python/ray/tune/experiment/experiment.py index db2f7d4963eb..4c8f51573f90 100644 --- a/python/ray/tune/experiment/experiment.py +++ b/python/ray/tune/experiment/experiment.py @@ -158,6 +158,11 @@ def __init__( # Deprecated local_dir: Optional[str] = None, ): + if isinstance(checkpoint_config, dict): + checkpoint_config = CheckpointConfig(**checkpoint_config) + else: + checkpoint_config = checkpoint_config or CheckpointConfig() + if is_function_trainable(run): if checkpoint_config.checkpoint_at_end: raise ValueError( @@ -201,7 +206,6 @@ def __init__( ) logger.debug(f"StorageContext on the DRIVER:\n{self.storage}") - # TODO(justinvyu): Rename these to legacy. self._legacy_local_storage_path = None self._legacy_remote_storage_path = None self.legacy_sync_config = None @@ -264,11 +268,6 @@ def __init__( config = config or {} - if isinstance(checkpoint_config, dict): - checkpoint_config = CheckpointConfig(**checkpoint_config) - else: - checkpoint_config = checkpoint_config or CheckpointConfig() - self._stopper = None stopping_criteria = {} if not stop: diff --git a/python/ray/tune/impl/tuner_internal.py b/python/ray/tune/impl/tuner_internal.py index c995d639789d..c9740b4c4ae4 100644 --- a/python/ray/tune/impl/tuner_internal.py +++ b/python/ray/tune/impl/tuner_internal.py @@ -427,11 +427,10 @@ def _restore_from_path_or_uri( # If we synced, `experiment_checkpoint_dir` will contain a temporary # directory. Create an experiment checkpoint dir instead and move # our data there. - new_exp_path, new_exp_name = Path( - self.setup_create_experiment_checkpoint_dir( - self.converted_trainable, self._run_config - ) + new_exp_path, new_exp_name = self.setup_create_experiment_checkpoint_dir( + self.converted_trainable, self._run_config ) + new_exp_path = Path(new_exp_path) for file_dir in experiment_checkpoint_path.glob("*"): file_dir.replace(new_exp_path / file_dir.name) shutil.rmtree(experiment_checkpoint_path) From dc1c7c3b021d0d678190174471f4ce1b3eae8c88 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 4 Aug 2023 16:28:34 -0700 Subject: [PATCH 074/108] Fix syncing needed logic to handle storage path == local path case Signed-off-by: Justin Yu --- python/ray/train/_internal/storage.py | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/python/ray/train/_internal/storage.py b/python/ray/train/_internal/storage.py index 4bdd3daabc5e..25a429b178b9 100644 --- a/python/ray/train/_internal/storage.py +++ b/python/ray/train/_internal/storage.py @@ -402,7 +402,7 @@ def __init__( trial_dir_name: Optional[str] = None, current_checkpoint_index: Optional[int] = None, ): - storage_path_provided = storage_path is not None + custom_fs_provided = storage_filesystem is not None self.storage_local_path = _get_defaults_results_dir() # If `storage_path=None`, then set it to the local path. @@ -450,14 +450,19 @@ def __init__( Path(self.storage_fs_path) ) - # Only initialize a syncer if a `storage_path` was provided. + # Syncing is always needed if a custom `storage_filesystem` is provided. + # Otherwise, syncing is only needed if storage_local_path + # and storage_fs_path point to different locations. + syncing_needed = ( + custom_fs_provided or self.storage_fs_path != self.storage_local_path + ) self.syncer: Optional[Syncer] = ( _FilesystemSyncer( storage_filesystem=self.storage_filesystem, sync_period=self.sync_config.sync_period, sync_timeout=self.sync_config.sync_timeout, ) - if storage_path_provided + if syncing_needed else None ) From a262cb3c4891885f538274daee27efcfcbf3fb62 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 4 Aug 2023 16:29:56 -0700 Subject: [PATCH 075/108] working for manual Trainer.restore Signed-off-by: Justin Yu --- python/ray/train/base_trainer.py | 20 ++++++++++++++++ .../ray/tune/analysis/experiment_analysis.py | 6 ++++- python/ray/tune/execution/tune_controller.py | 23 +++++++++++-------- python/ray/tune/experiment/trial.py | 2 -- python/ray/tune/impl/tuner_internal.py | 13 +++++++++++ python/ray/tune/tuner.py | 8 +++++++ 6 files changed, 60 insertions(+), 12 deletions(-) diff --git a/python/ray/train/base_trainer.py b/python/ray/train/base_trainer.py index 5949933c7c30..908eebd97842 100644 --- a/python/ray/train/base_trainer.py +++ b/python/ray/train/base_trainer.py @@ -373,6 +373,13 @@ def can_restore(cls: Type["BaseTrainer"], path: Union[str, Path]) -> bool: Returns: bool: Whether this path exists and contains the trainer state to resume from """ + if _use_storage_context(): + from ray.train._internal.storage import _list_at_fs_path + import pyarrow.fs + + fs, fs_path = pyarrow.fs.FileSystem.from_uri(str(path)) + return _TRAINER_PKL in _list_at_fs_path(fs, fs_path) + return _TRAINER_PKL in list_at_uri(str(path)) def __repr__(self): @@ -482,6 +489,19 @@ def _maybe_sync_down_trainer_state(cls, restore_path: str) -> Path: Returns: str: Local directory containing the trainer state """ + if _use_storage_context(): + from ray.train._internal.storage import _download_from_fs_path + import pyarrow.fs + + tempdir = tempfile.mkdtemp("tmp_experiment_dir") + fs, fs_path = pyarrow.fs.FileSystem.from_uri(restore_path) + _download_from_fs_path( + fs=fs, + fs_path=os.path.join(fs_path, _TRAINER_PKL), + local_path=os.path.join(tempdir, _TRAINER_PKL), + ) + return Path(tempdir) / _TRAINER_PKL + if not is_non_local_path_uri(restore_path): return Path(os.path.expanduser(restore_path)) / _TRAINER_PKL diff --git a/python/ray/tune/analysis/experiment_analysis.py b/python/ray/tune/analysis/experiment_analysis.py index 9d4530aa8908..2b16ded8520d 100644 --- a/python/ray/tune/analysis/experiment_analysis.py +++ b/python/ray/tune/analysis/experiment_analysis.py @@ -978,9 +978,13 @@ def _get_trial_paths(self) -> List[str]: ) self.trials = [] for trial_json_state, path in self._checkpoints_and_paths: + from ray.train._internal.storage import _use_storage_context + try: trial = Trial.from_json_state(trial_json_state, stub=True) - trial.local_experiment_path = str(path) + # TODO(justinvyu): [handle_moved_storage_path] + if not _use_storage_context(): + trial.local_experiment_path = str(path) except Exception: logger.warning( f"Could not load trials from experiment checkpoint. " diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 61e675baef7b..dc53e0eaeea2 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -482,9 +482,6 @@ def restore_from_dir(self, experiment_dir: Optional[str] = None) -> List[Trial]: and return them as a list of Trial objects. """ if _use_storage_context(): - raise NotImplementedError( - "Restoration is not fully working. TODO for a follow-up PR." - ) assert not experiment_dir, "Remove the `experiment_dir` argument." experiment_dir = self._storage.experiment_local_path else: @@ -534,11 +531,17 @@ def restore_from_dir(self, experiment_dir: Optional[str] = None) -> List[Trial]: # The following properties may be updated on restoration # Ex: moved local/cloud experiment directory - # ATTN: Set `local_experiment_path` to update trial checkpoints! - trial.local_experiment_path = self._legacy_local_experiment_path - trial.remote_experiment_path = self._legacy_remote_experiment_path - trial.sync_config = self._legacy_sync_config - trial.experiment_dir_name = self._legacy_experiment_dir_name + if not _use_storage_context(): + # ATTN: Set `local_experiment_path` to update trial checkpoints! + trial.local_experiment_path = self._legacy_local_experiment_path + trial.remote_experiment_path = self._legacy_remote_experiment_path + trial.sync_config = self._legacy_sync_config + trial.experiment_dir_name = self._legacy_experiment_dir_name + + # Propagate updated storage ctx properties to the trial's restored copy. + # TODO(justinvyu): [handle_moved_storage_path] + trial.storage.storage_path = self._storage.storage_path + trial.storage.experiment_dir_name = self._storage.experiment_dir_name # Avoid creating logdir in client mode for returned trial results, # since the dir might not be creatable locally. @@ -603,7 +606,9 @@ def resume( trial_to_add.error_filename = None trial_to_add.pickled_error_filename = None trial_to_add.set_status(Trial.PENDING) - trial_to_add.restore_path = trial.checkpoint.dir_or_data + if not _use_storage_context(): + # TODO(justinvyu): Mark this as legacy. + trial_to_add.restore_path = trial.checkpoint.dir_or_data elif restart_errored: trial_to_add = trial.reset() trial_to_add.restore_path = None diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index 9d25ef0b86d9..c55f6837b951 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -384,8 +384,6 @@ def __init__( self.storage = copy.copy(storage) if _use_storage_context(): - assert self.storage - self._legacy_orig_experiment_path = None self._legacy_orig_experiment_dir_name = None self._legacy_local_experiment_path = None diff --git a/python/ray/tune/impl/tuner_internal.py b/python/ray/tune/impl/tuner_internal.py index c9740b4c4ae4..e621c246564d 100644 --- a/python/ray/tune/impl/tuner_internal.py +++ b/python/ray/tune/impl/tuner_internal.py @@ -456,6 +456,19 @@ def _maybe_sync_down_tuner_state(self, restore_path: str) -> Tuple[bool, str]: Returns: Tuple of (downloaded from remote, local_dir) """ + if _use_storage_context(): + from ray.train._internal.storage import _download_from_fs_path + import pyarrow.fs + + tempdir = tempfile.mkdtemp("tmp_experiment_dir") + fs, fs_path = pyarrow.fs.FileSystem.from_uri(restore_path) + _download_from_fs_path( + fs=fs, + fs_path=os.path.join(fs_path, _TUNER_PKL), + local_path=os.path.join(tempdir, _TUNER_PKL), + ) + return True, tempdir + if not is_non_local_path_uri(restore_path): return False, os.path.abspath(os.path.expanduser(restore_path)) diff --git a/python/ray/tune/tuner.py b/python/ray/tune/tuner.py index 792fb2af4933..2baae0f4d0b7 100644 --- a/python/ray/tune/tuner.py +++ b/python/ray/tune/tuner.py @@ -301,6 +301,14 @@ def train_fn(config): Returns: bool: True if this path exists and contains the Tuner state to resume from """ + from ray.train._internal.storage import _list_at_fs_path, _use_storage_context + + if _use_storage_context(): + import pyarrow.fs + + fs, fs_path = pyarrow.fs.FileSystem.from_uri(str(path)) + return _TUNER_PKL in _list_at_fs_path(fs, fs_path) + return _TUNER_PKL in list_at_uri(str(path)) def _prepare_remote_tuner_for_jupyter_progress_reporting(self): From e59b40821e008fc33734b1a832f5983e25466dbc Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 4 Aug 2023 16:30:14 -0700 Subject: [PATCH 076/108] Add manual restore to e2e test Signed-off-by: Justin Yu --- .../ray/train/tests/test_new_persistence.py | 35 ++++++++++++++++--- 1 file changed, 30 insertions(+), 5 deletions(-) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index 8de2a3641eff..3d9ae174a336 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -10,9 +10,11 @@ import pyarrow.fs from ray import train, tune +from ray.air._internal.uri_utils import URI from ray.air.constants import EXPR_RESULT_FILE from ray.train._internal.storage import _download_from_fs_path from ray.train._checkpoint import Checkpoint as NewCheckpoint +from ray.train.base_trainer import TrainingFailedError from ray.train.data_parallel_trainer import DataParallelTrainer from ray.air.tests.test_checkpoints import mock_s3_bucket_uri @@ -280,8 +282,10 @@ def test_trainer( - restoration, train.get_checkpoint {storage_path}/{exp_name} - ├── experiment_state-2023-07-28_10-00-38.json + ├── experiment_state-2023-07-28_10-00-38.json <- Initial exp state ├── basic-variant-state-2023-07-28_10-00-38.json + ├── experiment_state-2023-07-28_10-01-38.json <- Restored exp state + ├── basic-variant-state-2023-07-28_10-01-38.json ├── trainer.pkl ├── tuner.pkl └── DataParallelTrainer_46367_00000_0_... @@ -306,12 +310,18 @@ def test_trainer( monkeypatch.setenv("RAY_AIR_LOCAL_CACHE_DIR", str(LOCAL_CACHE_DIR)) exp_name = "trainer_new_persistence" + # TODO(justinvyu): Manual restore doesn't work yet for custom fs. + # Need to introduce restore(storage_filesystem) API in a follow-up. + TEST_MANUAL_RESTORE = storage_path_type != "custom_fs" + with _resolve_storage_type(storage_path_type, tmp_path) as ( storage_path, storage_filesystem, ): NUM_ITERATIONS = 6 NUM_WORKERS = 2 + + max_failures = 1 if TEST_MANUAL_RESTORE else 2 trainer = DataParallelTrainer( train_fn, train_loop_config={ @@ -326,10 +336,19 @@ def test_trainer( name=exp_name, verbose=0, checkpoint_config=checkpoint_config, - failure_config=train.FailureConfig(max_failures=2), + failure_config=train.FailureConfig(max_failures=max_failures), ), ) - result = trainer.fit() + if TEST_MANUAL_RESTORE: + with pytest.raises(TrainingFailedError): + result = trainer.fit() + + restored_trainer = DataParallelTrainer.restore( + path=str(URI(storage_path or str(LOCAL_CACHE_DIR)) / exp_name) + ) + result = restored_trainer.fit() + else: + result = trainer.fit() local_inspect_dir, storage_fs_path = _get_local_inspect_dir( root_local_path=tmp_path, @@ -349,10 +368,16 @@ def test_trainer( exp_dir = local_inspect_dir / exp_name # Files synced by the driver - assert len(list(exp_dir.glob("basic-variant-state-*"))) == 1 - assert len(list(exp_dir.glob("experiment_state-*"))) == 1 assert len(list(exp_dir.glob("tuner.pkl"))) == 1 assert len(list(exp_dir.glob("trainer.pkl"))) == 1 + if TEST_MANUAL_RESTORE: + # 2 copies of these files: + # 1 for the initial run, and 1 for the manually restored run. + assert len(list(exp_dir.glob("basic-variant-state-*"))) == 2 + assert len(list(exp_dir.glob("experiment_state-*"))) == 2 + else: + assert len(list(exp_dir.glob("basic-variant-state-*"))) == 1 + assert len(list(exp_dir.glob("experiment_state-*"))) == 1 # Files synced by the worker assert len(list(exp_dir.glob("DataParallelTrainer_*"))) == 1 From c6c3dfe09a43383a141a7eb6796c38c75e7af600 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 4 Aug 2023 16:37:07 -0700 Subject: [PATCH 077/108] Fix renamed attribute in mock test class Signed-off-by: Justin Yu --- python/ray/tune/tests/test_trial_scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/tune/tests/test_trial_scheduler.py b/python/ray/tune/tests/test_trial_scheduler.py index c2f42c3c8585..ec4b556f7c6f 100644 --- a/python/ray/tune/tests/test_trial_scheduler.py +++ b/python/ray/tune/tests/test_trial_scheduler.py @@ -872,7 +872,7 @@ def __init__(self, i, config): self.placement_group_factory = PlacementGroupFactory([{"CPU": 1}]) self.custom_trial_name = None self.custom_dirname = None - self._local_experiment_path = None + self._legacy_local_experiment_path = None self.relative_logdir = None self._default_result_or_future = None self.checkpoint_manager = _CheckpointManager( From 36464af6c93b88ddc2fff59385097757b0842468 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 4 Aug 2023 17:17:26 -0700 Subject: [PATCH 078/108] fix storage attr setting to only happen if ff enabled Signed-off-by: Justin Yu --- python/ray/train/trainer.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/python/ray/train/trainer.py b/python/ray/train/trainer.py index 46a42c83d661..5da6639ac6cf 100644 --- a/python/ray/train/trainer.py +++ b/python/ray/train/trainer.py @@ -72,8 +72,10 @@ def __init__( # TrainingResult event. There's no need to do these one at a time. self._checkpoint_to_report = None - self._storage = get_storage_context() if _use_storage_context() else None - self._storage.current_checkpoint_index = latest_checkpoint_index + self._storage = None + if _use_storage_context(): + self._storage = get_storage_context() + self._storage.current_checkpoint_index = latest_checkpoint_index self._start_training( train_func=train_func, From 6e73f6e731848373a65ecb9b99ce08e218f9c740 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 4 Aug 2023 17:38:17 -0700 Subject: [PATCH 079/108] cleanup on errors in as_directory Signed-off-by: Justin Yu --- python/ray/train/_checkpoint.py | 52 ++++++++++++----------- python/ray/train/tests/test_checkpoint.py | 15 +++++++ 2 files changed, 42 insertions(+), 25 deletions(-) diff --git a/python/ray/train/_checkpoint.py b/python/ray/train/_checkpoint.py index a5b65f1de9c2..d7214fb7fb77 100644 --- a/python/ray/train/_checkpoint.py +++ b/python/ray/train/_checkpoint.py @@ -214,33 +214,35 @@ def as_directory(self) -> Iterator[str]: del_lock_path = _get_del_lock_path(self._get_temporary_checkpoint_dir()) open(del_lock_path, "a").close() - temp_dir = self.to_directory() - - yield temp_dir - - # Cleanup try: - os.remove(del_lock_path) - except Exception: - logger.warning( - f"Could not remove {del_lock_path} deletion file lock. " - f"Traceback:\n{traceback.format_exc()}" - ) - - # In the edge case (process crash before del lock file is removed), - # we do not remove the directory at all. - # Since it's in /tmp, this is not that big of a deal. - # check if any lock files are remaining - remaining_locks = _list_existing_del_locks(temp_dir) - if not remaining_locks: + temp_dir = self.to_directory() + yield temp_dir + finally: + # Always cleanup the del lock after we're done with the directory. + # This avoids leaving a lock file behind in the case of an exception + # in the user code. try: - # Timeout 0 means there will be only one attempt to acquire - # the file lock. If it cannot be acquired, a TimeoutError - # will be thrown. - with TempFileLock(temp_dir, timeout=0): - shutil.rmtree(temp_dir, ignore_errors=True) - except TimeoutError: - pass + os.remove(del_lock_path) + except Exception: + logger.warning( + f"Could not remove {del_lock_path} deletion file lock. " + f"Traceback:\n{traceback.format_exc()}" + ) + + # In the edge case (process crash before del lock file is removed), + # we do not remove the directory at all. + # Since it's in /tmp, this is not that big of a deal. + # check if any lock files are remaining + remaining_locks = _list_existing_del_locks(temp_dir) + if not remaining_locks: + try: + # Timeout 0 means there will be only one attempt to acquire + # the file lock. If it cannot be acquired, a TimeoutError + # will be thrown. + with TempFileLock(temp_dir, timeout=0): + shutil.rmtree(temp_dir, ignore_errors=True) + except TimeoutError: + pass def _get_temporary_checkpoint_dir(self) -> str: """Return the name for the temporary checkpoint dir that this checkpoint diff --git a/python/ray/train/tests/test_checkpoint.py b/python/ray/train/tests/test_checkpoint.py index 91f192e3cf23..f01352829e0f 100644 --- a/python/ray/train/tests/test_checkpoint.py +++ b/python/ray/train/tests/test_checkpoint.py @@ -156,6 +156,21 @@ def test_multiprocess_as_directory(checkpoint: Checkpoint, monkeypatch): assert not Path(checkpoint_dir_1).exists() +def test_as_directory_lock_cleanup(checkpoint: Checkpoint): + """Errors when accessing a checkpoint with `as_directory` + shouldn't leave behind lock files. + """ + with pytest.raises(RuntimeError): + with checkpoint.as_directory() as checkpoint_dir: + raise RuntimeError + + assert not _list_existing_del_locks(checkpoint_dir) + + is_local_checkpoint = isinstance(checkpoint.filesystem, pyarrow.fs.LocalFileSystem) + if not is_local_checkpoint: + assert not Path(checkpoint_dir).exists() + + def test_metadata(checkpoint: Checkpoint): assert checkpoint.get_metadata() == {} From c7de72aac4e204f96d2853e6bdaff8a8878583f2 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 4 Aug 2023 18:10:13 -0700 Subject: [PATCH 080/108] Support + test resume_from_checkpoint Signed-off-by: Justin Yu --- python/ray/train/BUILD | 2 +- python/ray/train/base_trainer.py | 5 +++- .../ray/train/tests/test_new_persistence.py | 24 +++++++++++++++++++ 3 files changed, 29 insertions(+), 2 deletions(-) diff --git a/python/ray/train/BUILD b/python/ray/train/BUILD index ccdf2851d962..472532c63a7d 100644 --- a/python/ray/train/BUILD +++ b/python/ray/train/BUILD @@ -464,7 +464,7 @@ py_test( py_test( name = "test_new_persistence", - size = "medium", + size = "large", srcs = ["tests/test_new_persistence.py"], tags = ["team:ml", "exclusive"], deps = [":train_lib", ":conftest"] diff --git a/python/ray/train/base_trainer.py b/python/ray/train/base_trainer.py index 5949933c7c30..1753f371c074 100644 --- a/python/ray/train/base_trainer.py +++ b/python/ray/train/base_trainer.py @@ -457,8 +457,11 @@ def _validate_attributes(self): f"found {type(self.preprocessor)} with value `{self.preprocessor}`." ) + expected_checkpoint_type = ( + NewCheckpoint if _use_storage_context() else ray.air.Checkpoint + ) if self.resume_from_checkpoint is not None and not isinstance( - self.resume_from_checkpoint, ray.air.Checkpoint + self.resume_from_checkpoint, expected_checkpoint_type ): raise ValueError( f"`resume_from_checkpoint` should be an instance of " diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index 8de2a3641eff..95f59d7a3873 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -167,6 +167,26 @@ def train_fn(config): raise RuntimeError(f"Failing on iter={i}!!") +def _resume_from_checkpoint(checkpoint: NewCheckpoint, expected: dict): + def assert_fn(config): + checkpoint_to_check = train.get_checkpoint() + with checkpoint_to_check.as_directory() as checkpoint_dir: + with open(os.path.join(checkpoint_dir, "checkpoint.pkl"), "rb") as f: + state = pickle.load(f) + + print("Loaded state from `resume_from_checkpoint`:", state) + print("Expected state:", expected) + assert state == expected, (state, expected) + + trainer = DataParallelTrainer( + assert_fn, + scaling_config=train.ScalingConfig(num_workers=2), + run_config=train.RunConfig(name="test_resume_from_checkpoint"), + resume_from_checkpoint=checkpoint, + ) + trainer.fit() + + @pytest.mark.parametrize("storage_path_type", [None, "nfs", "cloud", "custom_fs"]) def test_tuner(monkeypatch, storage_path_type, tmp_path): """End-to-end test that the new persistence mode works with the Tuner API. @@ -331,6 +351,10 @@ def test_trainer( ) result = trainer.fit() + _resume_from_checkpoint( + result.checkpoint, expected={"iter": NUM_ITERATIONS - 1} + ) + local_inspect_dir, storage_fs_path = _get_local_inspect_dir( root_local_path=tmp_path, storage_path=storage_path, From db016da13bce8fea484c5cf71b06d2a456ed541d Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Fri, 4 Aug 2023 18:10:35 -0700 Subject: [PATCH 081/108] Fix result grid bug when no checkpoints saved Signed-off-by: Justin Yu --- python/ray/tune/result_grid.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/result_grid.py b/python/ray/tune/result_grid.py index e5b891844c9a..55a851b145aa 100644 --- a/python/ray/tune/result_grid.py +++ b/python/ray/tune/result_grid.py @@ -277,7 +277,11 @@ def _trial_to_result(self, trial: Trial) -> Result: ) assert isinstance(trial.checkpoint_manager, _NewCheckpointManager) - checkpoint = trial.checkpoint_manager.latest_checkpoint_result.checkpoint + checkpoint = None + if trial.checkpoint_manager.latest_checkpoint_result: + checkpoint = ( + trial.checkpoint_manager.latest_checkpoint_result.checkpoint + ) best_checkpoint_results = trial.checkpoint_manager.best_checkpoint_results best_checkpoints = [ (checkpoint_result.checkpoint, checkpoint_result.metrics) From d7497e1b52996b7700db74045ec6973aac968993 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Sat, 5 Aug 2023 11:39:56 -0700 Subject: [PATCH 082/108] fix merge conflict remainder Signed-off-by: Justin Yu --- .../ray/train/tests/test_new_persistence.py | 55 +++++++++---------- 1 file changed, 26 insertions(+), 29 deletions(-) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index 61ab976cd377..8de2a3641eff 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -275,35 +275,32 @@ def test_trainer( tmp_path, monkeypatch, storage_path_type, checkpoint_config: train.CheckpointConfig ): """ - TODO(justinvyu): Test for these once implemented: - - artifacts - - restoration, train.get_checkpoint - - {storage_path}/{exp_name} - ├── experiment_state-2023-07-28_10-00-38.json - ├── basic-variant-state-2023-07-28_10-00-38.json - ├── trainer.pkl - ├── tuner.pkl - └── DataParallelTrainer_46367_00000_0_... - ├── events.out.tfevents... - ├── params.json - ├── params.pkl - ├── progress.csv - ├── result.json - ├── checkpoint_000000 - <<<<<<< HEAD - │ ├── checkpoint.pkl <- Shared checkpoint file - ======= - >>>>>>> 470c10bd0d46d5db90b1cf83a0ed423af720ba16 - │ ├── checkpoint_shard-rank=0.pkl <- Worker checkpoint shards - │ └── checkpoint_shard-rank=1.pkl - ├── ... - ├── artifact-rank=0-iter=0.txt <- Worker artifacts - ├── artifact-rank=1-iter=0.txt - ├── ... - ├── artifact-rank=0-iter=1.txt - ├── artifact-rank=1-iter=1.txt - └── ... + TODO(justinvyu): Test for these once implemented: + - artifacts + - restoration, train.get_checkpoint + + {storage_path}/{exp_name} + ├── experiment_state-2023-07-28_10-00-38.json + ├── basic-variant-state-2023-07-28_10-00-38.json + ├── trainer.pkl + ├── tuner.pkl + └── DataParallelTrainer_46367_00000_0_... + ├── events.out.tfevents... + ├── params.json + ├── params.pkl + ├── progress.csv + ├── result.json + ├── checkpoint_000000 + │ ├── checkpoint.pkl <- Shared checkpoint file + │ ├── checkpoint_shard-rank=0.pkl <- Worker checkpoint shards + │ └── checkpoint_shard-rank=1.pkl + ├── ... + ├── artifact-rank=0-iter=0.txt <- Worker artifacts + ├── artifact-rank=1-iter=0.txt + ├── ... + ├── artifact-rank=0-iter=1.txt + ├── artifact-rank=1-iter=1.txt + └── ... """ LOCAL_CACHE_DIR = tmp_path / "ray_results" monkeypatch.setenv("RAY_AIR_LOCAL_CACHE_DIR", str(LOCAL_CACHE_DIR)) From aeb89baf96e965458a5b19b05b3b53f4bc0b91b8 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 7 Aug 2023 13:33:18 -0700 Subject: [PATCH 083/108] Recover trainable metadata from last_result rather than .tune_metadata Signed-off-by: Justin Yu --- python/ray/tune/trainable/trainable.py | 21 ++++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/python/ray/tune/trainable/trainable.py b/python/ray/tune/trainable/trainable.py index 75b17be49a86..dbd7669e88a1 100644 --- a/python/ray/tune/trainable/trainable.py +++ b/python/ray/tune/trainable/trainable.py @@ -506,7 +506,11 @@ def save( if _use_storage_context() and isinstance( checkpoint_dict_or_path, _TrainingResult ): - return checkpoint_dict_or_path + checkpoint_result = checkpoint_dict_or_path + assert self._last_result + # Update the checkpoint result to include auto-filled metrics. + checkpoint_result.metrics.update(self._last_result) + return checkpoint_result if checkpoint_dict_or_path is None: # checkpoint_dict_or_path can only be None in class trainables. @@ -870,14 +874,21 @@ def restore( checkpoint_result = checkpoint_path assert isinstance(checkpoint_result, _TrainingResult) + checkpoint_metrics = checkpoint_result.metrics + self._iteration = checkpoint_metrics[TRAINING_ITERATION] + self._time_total = checkpoint_metrics[TIME_TOTAL_S] + self._time_since_restore = 0.0 + self._iterations_since_restore = 0 + + # TODO(justinvyu): This stuff should be moved to rllib. + self._timesteps_total = checkpoint_metrics.get(TIMESTEPS_TOTAL) + self._timesteps_since_restore = 0 + self._episodes_total = checkpoint_metrics.get(EPISODES_TOTAL) + # TODO(justinvyu): The Trainable `load_checkpoint` interface # should be updated to take in a `_TrainingResult` / Checkpoint self.load_checkpoint(checkpoint_result) - # TODO(justinvyu): What should we do about Trainable metadata? - self._time_since_restore = 0.0 - self._timesteps_since_restore = 0 - self._iterations_since_restore = 0 self._restored = True logger.info( From 955637181c81a04c4f2ed85d240ba804933a7810 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 7 Aug 2023 13:44:04 -0700 Subject: [PATCH 084/108] Fix restore info log Signed-off-by: Justin Yu --- python/ray/tune/trainable/trainable.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/trainable/trainable.py b/python/ray/tune/trainable/trainable.py index dbd7669e88a1..8d65618def26 100644 --- a/python/ray/tune/trainable/trainable.py +++ b/python/ray/tune/trainable/trainable.py @@ -892,7 +892,8 @@ def restore( self._restored = True logger.info( - f"Restored on {self._local_ip} from checkpoint: {checkpoint_result}" + f"Restored on {self._local_ip} from checkpoint: " + f"{checkpoint_result.checkpoint}" ) return True From e897eaad37fc4200a38e3c8fdf38e9a4784f94ed Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 7 Aug 2023 14:03:16 -0700 Subject: [PATCH 085/108] Keep current checkpoint index synchronized on the driver Signed-off-by: Justin Yu --- python/ray/train/_internal/storage.py | 13 +++++-------- python/ray/train/data_parallel_trainer.py | 12 ------------ python/ray/train/trainer.py | 2 -- python/ray/tune/experiment/trial.py | 4 ++++ 4 files changed, 9 insertions(+), 22 deletions(-) diff --git a/python/ray/train/_internal/storage.py b/python/ray/train/_internal/storage.py index 4bdd3daabc5e..9aa5f9a2d166 100644 --- a/python/ray/train/_internal/storage.py +++ b/python/ray/train/_internal/storage.py @@ -400,7 +400,7 @@ def __init__( experiment_dir_name: str, storage_filesystem: Optional[pyarrow.fs.FileSystem] = None, trial_dir_name: Optional[str] = None, - current_checkpoint_index: Optional[int] = None, + current_checkpoint_index: int = 0, ): storage_path_provided = storage_path is not None @@ -591,15 +591,12 @@ def trial_fs_path(self) -> str: @property def checkpoint_fs_path(self) -> str: - """The trial directory path on the `storage_filesystem`. + """The current checkpoint directory path on the `storage_filesystem`. - Raises a ValueError if `current_checkpoint_index` is not set beforehand. + "Current" refers to the checkpoint that is currently being created/persisted. + The user of this class is responsible for setting the `current_checkpoint_index` + (e.g., incrementing when needed). """ - if self.current_checkpoint_index is None: - raise RuntimeError( - "Should not access `checkpoint_fs_path` without setting " - "`current_checkpoint_index`" - ) checkpoint_dir_name = StorageContext._make_checkpoint_dir_name( self.current_checkpoint_index ) diff --git a/python/ray/train/data_parallel_trainer.py b/python/ray/train/data_parallel_trainer.py index c11cba5a9b49..2c1acb6e610a 100644 --- a/python/ray/train/data_parallel_trainer.py +++ b/python/ray/train/data_parallel_trainer.py @@ -523,17 +523,6 @@ def clear_lazy_checkpoint_marker(): initial_checkpoint = self._get_initial_checkpoint() - latest_checkpoint_index = 0 - if _use_storage_context() and self._checkpoint_for_restoration: - checkpoint_dir_name = os.path.basename( - self._checkpoint_for_restoration.path - ) - # Need to add 1 because latest_checkpoint_index is the index of the - # next checkpoint to be saved. - latest_checkpoint_index = ( - StorageContext._parse_checkpoint_index(checkpoint_dir_name) + 1 - ) - training_iterator = self._training_iterator_cls( backend_executor=backend_executor, backend_config=self._backend_config, @@ -544,7 +533,6 @@ def clear_lazy_checkpoint_marker(): checkpoint=initial_checkpoint, checkpoint_strategy=checkpoint_strategy, storage_path=self.run_config.storage_path, - latest_checkpoint_index=latest_checkpoint_index, ) self._report(training_iterator) diff --git a/python/ray/train/trainer.py b/python/ray/train/trainer.py index 5da6639ac6cf..d64c66b7e33d 100644 --- a/python/ray/train/trainer.py +++ b/python/ray/train/trainer.py @@ -56,7 +56,6 @@ def __init__( checkpoint_strategy: Optional[CheckpointConfig], run_dir: Optional[Path] = None, storage_path: Optional[str] = None, - latest_checkpoint_index: int = 0, ): self._backend_executor = backend_executor self._backend = backend_config.backend_cls() @@ -75,7 +74,6 @@ def __init__( self._storage = None if _use_storage_context(): self._storage = get_storage_context() - self._storage.current_checkpoint_index = latest_checkpoint_index self._start_training( train_func=train_func, diff --git a/python/ray/tune/experiment/trial.py b/python/ray/tune/experiment/trial.py index c55f6837b951..b0f2713934ae 100644 --- a/python/ray/tune/experiment/trial.py +++ b/python/ray/tune/experiment/trial.py @@ -1080,6 +1080,10 @@ def on_checkpoint(self, checkpoint: _TrackedCheckpoint): checkpoint_result = checkpoint assert isinstance(checkpoint_result, _TrainingResult) self.checkpoint_manager.register_checkpoint(checkpoint_result) + # Increment the checkpoint index to keep the checkpoint index in sync. + # This index will get restored when the trial is restored and will + # be passed to the Trainable as the starting checkpoint index. + self.storage.current_checkpoint_index += 1 else: self.checkpoint_manager.on_checkpoint(checkpoint) self.invalidate_json_state() From 3eef4176a5fb556e178fe6065eff40ab93fc82d4 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 7 Aug 2023 14:05:11 -0700 Subject: [PATCH 086/108] Remove checkpoint dirname parsing Signed-off-by: Justin Yu --- python/ray/train/_internal/storage.py | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/python/ray/train/_internal/storage.py b/python/ray/train/_internal/storage.py index 9aa5f9a2d166..7e92ecd01af0 100644 --- a/python/ray/train/_internal/storage.py +++ b/python/ray/train/_internal/storage.py @@ -620,19 +620,6 @@ def _make_checkpoint_dir_name(index: int): """Get the name of the checkpoint directory, given an index.""" return f"checkpoint_{index:06d}" - @staticmethod - def _parse_checkpoint_index(checkpoint_dir_name: str): - """Parse the index from a checkpoint directory name - created by _make_checkpoint_dir_name. - - >>> StorageContext._parse_checkpoint_index("checkpoint_000001") - 1 - >>> StorageContext._parse_checkpoint_index("checkpoint_11111111") - 11111111 - """ - assert checkpoint_dir_name.startswith("checkpoint_") - return int(checkpoint_dir_name.split("_")[1]) - _storage_context: Optional[StorageContext] = None From 89631abfa51efdfc1f1bd31028a6376255ac5d02 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 7 Aug 2023 14:15:59 -0700 Subject: [PATCH 087/108] Update todo comment Signed-off-by: Justin Yu --- python/ray/tune/trainable/function_trainable.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/ray/tune/trainable/function_trainable.py b/python/ray/tune/trainable/function_trainable.py index 6da193cf82e2..d5dd7b0aff59 100644 --- a/python/ray/tune/trainable/function_trainable.py +++ b/python/ray/tune/trainable/function_trainable.py @@ -505,7 +505,9 @@ def get_state(self): from ray.train._internal.storage import _use_storage_context if _use_storage_context(): - # TODO(justinvyu): Not too sure about this. + # TODO(justinvyu): This is only used to populate the tune metadata + # file within the checkpoint, so can be removed after if remove + # the metadata file. return state checkpoint = self._status_reporter.get_checkpoint() From d4e20f27518c98861811c6d023cc06c9f1edac7c Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 7 Aug 2023 14:16:35 -0700 Subject: [PATCH 088/108] Fix lint Signed-off-by: Justin Yu --- python/ray/train/data_parallel_trainer.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/ray/train/data_parallel_trainer.py b/python/ray/train/data_parallel_trainer.py index 2c1acb6e610a..319105967d15 100644 --- a/python/ray/train/data_parallel_trainer.py +++ b/python/ray/train/data_parallel_trainer.py @@ -1,7 +1,6 @@ import copy import inspect import logging -import os from pathlib import Path from typing import TYPE_CHECKING, Any, Callable, Dict, Optional, Tuple, Type, Union from ray._private.thirdparty.tabulate.tabulate import tabulate @@ -18,7 +17,7 @@ from ray.train._internal.backend_executor import BackendExecutor, TrialInfo from ray.train._internal.checkpoint import TuneCheckpointManager from ray.train._internal.data_config import DataConfig, _LegacyDataConfigWrapper -from ray.train._internal.storage import _use_storage_context, StorageContext +from ray.train._internal.storage import _use_storage_context from ray.train._internal.utils import construct_train_func from ray.train.constants import TRAIN_DATASET_KEY, WILDCARD_KEY from ray.train.trainer import BaseTrainer, GenDataset From 5c50dd44b655c8e38ad9fced052f51e4db3965e6 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Mon, 7 Aug 2023 16:22:54 -0700 Subject: [PATCH 089/108] Some small imports cleanup Signed-off-by: Justin Yu --- python/ray/train/base_trainer.py | 14 +++++++------- python/ray/tune/impl/tuner_internal.py | 3 ++- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/python/ray/train/base_trainer.py b/python/ray/train/base_trainer.py index 908eebd97842..12c3746e6b89 100644 --- a/python/ray/train/base_trainer.py +++ b/python/ray/train/base_trainer.py @@ -8,6 +8,8 @@ from typing import TYPE_CHECKING, Any, Callable, Dict, List, Optional, Type, Union import warnings +import pyarrow.fs + import ray import ray.cloudpickle as pickle from ray.air._internal.config import ensure_only_allowed_dataclass_keys_updated @@ -23,7 +25,11 @@ from ray.air.result import Result from ray.train._checkpoint import Checkpoint as NewCheckpoint from ray.train._internal import session -from ray.train._internal.storage import _use_storage_context +from ray.train._internal.storage import ( + _download_from_fs_path, + _list_at_fs_path, + _use_storage_context, +) from ray.train.constants import TRAIN_DATASET_KEY from ray.util import PublicAPI from ray.util.annotations import DeveloperAPI @@ -374,9 +380,6 @@ def can_restore(cls: Type["BaseTrainer"], path: Union[str, Path]) -> bool: bool: Whether this path exists and contains the trainer state to resume from """ if _use_storage_context(): - from ray.train._internal.storage import _list_at_fs_path - import pyarrow.fs - fs, fs_path = pyarrow.fs.FileSystem.from_uri(str(path)) return _TRAINER_PKL in _list_at_fs_path(fs, fs_path) @@ -490,9 +493,6 @@ def _maybe_sync_down_trainer_state(cls, restore_path: str) -> Path: str: Local directory containing the trainer state """ if _use_storage_context(): - from ray.train._internal.storage import _download_from_fs_path - import pyarrow.fs - tempdir = tempfile.mkdtemp("tmp_experiment_dir") fs, fs_path = pyarrow.fs.FileSystem.from_uri(restore_path) _download_from_fs_path( diff --git a/python/ray/tune/impl/tuner_internal.py b/python/ray/tune/impl/tuner_internal.py index e621c246564d..f851946ae618 100644 --- a/python/ray/tune/impl/tuner_internal.py +++ b/python/ray/tune/impl/tuner_internal.py @@ -19,6 +19,8 @@ Tuple, ) +import pyarrow.fs + import ray import ray.cloudpickle as pickle from ray.util import inspect_serializability @@ -458,7 +460,6 @@ def _maybe_sync_down_tuner_state(self, restore_path: str) -> Tuple[bool, str]: """ if _use_storage_context(): from ray.train._internal.storage import _download_from_fs_path - import pyarrow.fs tempdir = tempfile.mkdtemp("tmp_experiment_dir") fs, fs_path = pyarrow.fs.FileSystem.from_uri(restore_path) From d329e1bc63c860c757cbdddcf193ca7b47347ebb Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 8 Aug 2023 13:09:33 -0700 Subject: [PATCH 090/108] Fix e2e test for storage_path=None case Signed-off-by: Justin Yu --- .../ray/train/tests/test_new_persistence.py | 33 ++++++++++++++----- 1 file changed, 25 insertions(+), 8 deletions(-) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index 95f59d7a3873..e7e810717146 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -11,7 +11,7 @@ from ray import train, tune from ray.air.constants import EXPR_RESULT_FILE -from ray.train._internal.storage import _download_from_fs_path +from ray.train._internal.storage import _download_from_fs_path, StorageContext from ray.train._checkpoint import Checkpoint as NewCheckpoint from ray.train.data_parallel_trainer import DataParallelTrainer @@ -167,7 +167,9 @@ def train_fn(config): raise RuntimeError(f"Failing on iter={i}!!") -def _resume_from_checkpoint(checkpoint: NewCheckpoint, expected: dict): +def _resume_from_checkpoint(checkpoint: NewCheckpoint, expected_state: dict): + print(f"\nStarting run with `resume_from_checkpoint`: {checkpoint}\n") + def assert_fn(config): checkpoint_to_check = train.get_checkpoint() with checkpoint_to_check.as_directory() as checkpoint_dir: @@ -175,8 +177,13 @@ def assert_fn(config): state = pickle.load(f) print("Loaded state from `resume_from_checkpoint`:", state) - print("Expected state:", expected) - assert state == expected, (state, expected) + print("Expected state:", expected_state) + assert state == expected_state, (state, expected_state) + + dummy_ckpt = tempfile.mkdtemp() + with open(os.path.join(dummy_ckpt, "dummy.txt"), "w") as f: + f.write("data") + train.report({"dummy": 1}, checkpoint=NewCheckpoint.from_directory(dummy_ckpt)) trainer = DataParallelTrainer( assert_fn, @@ -184,7 +191,12 @@ def assert_fn(config): run_config=train.RunConfig(name="test_resume_from_checkpoint"), resume_from_checkpoint=checkpoint, ) - trainer.fit() + result = trainer.fit() + + # Make sure that the checkpoint indexing starts from scratch. + assert Path( + result.checkpoint.path + ).name == StorageContext._make_checkpoint_dir_name(0) @pytest.mark.parametrize("storage_path_type", [None, "nfs", "cloud", "custom_fs"]) @@ -349,11 +361,16 @@ def test_trainer( failure_config=train.FailureConfig(max_failures=2), ), ) + print("\nStarting initial run.\n") result = trainer.fit() - _resume_from_checkpoint( - result.checkpoint, expected={"iter": NUM_ITERATIONS - 1} - ) + with monkeypatch.context() as m: + # This is so that the `resume_from_checkpoint` run doesn't mess up the + # assertions later for the `storage_path=None` case. + m.setenv("RAY_AIR_LOCAL_CACHE_DIR", tmp_path / "resume_from_checkpoint") + _resume_from_checkpoint( + result.checkpoint, expected_state={"iter": NUM_ITERATIONS - 1} + ) local_inspect_dir, storage_fs_path = _get_local_inspect_dir( root_local_path=tmp_path, From e382e29dd5b409639f0fa86d14070df8b01e1ecc Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 8 Aug 2023 13:12:02 -0700 Subject: [PATCH 091/108] Remove unused code Signed-off-by: Justin Yu --- python/ray/train/base_trainer.py | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/python/ray/train/base_trainer.py b/python/ray/train/base_trainer.py index 4b5e16749f9f..a3f7860675c6 100644 --- a/python/ray/train/base_trainer.py +++ b/python/ray/train/base_trainer.py @@ -197,9 +197,6 @@ def __init__( # This path should only be set through restore self._restore_path = None - # This checkpoint should only be populated internally - # by auto-recovery fault tolerance/manual Trainer.restore - self._checkpoint_for_restoration: Optional[NewCheckpoint] = None self._validate_attributes() @@ -689,22 +686,6 @@ def _extract_fields_for_tuner_param_space(self) -> Dict: result[key] = copy.deepcopy(self._param_dict[key]) return result - def _get_initial_checkpoint(self) -> Optional[NewCheckpoint]: - """If we need to set an initial checkpoint accessible by train.get_checkpoint, - we are in one of 3 cases: - 1. We are auto-recovering from a training failure (FailureConfig). - -> In this case, _checkpoint_for_restoration points to the *latest* - checkpoint, which is the one we want to restore from. - 2. We are manually restoring an existing experiment (Trainer.restore). - -> Same as 1. - 3. The user passed in a checkpoint to start a *new* training run from. - -> `resume_from_checkpoint` points to the user-specified checkpoint. - - When populating `train.get_checkpoint`, `_checkpoint_for_restoration` - should take precedence over `resume_from_checkpoint`. - """ - return self._checkpoint_for_restoration or self.resume_from_checkpoint - def _generate_trainable_cls(self) -> Type["Trainable"]: """Generates the base Trainable class. From 86367ef6d82b1227ac091ad602e96c86c28a6015 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 8 Aug 2023 13:26:46 -0700 Subject: [PATCH 092/108] Guard new codepath correctly Signed-off-by: Justin Yu --- python/ray/tune/execution/tune_controller.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index dc53e0eaeea2..9136731ebf67 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -531,18 +531,18 @@ def restore_from_dir(self, experiment_dir: Optional[str] = None) -> List[Trial]: # The following properties may be updated on restoration # Ex: moved local/cloud experiment directory - if not _use_storage_context(): + if _use_storage_context(): + # Propagate updated storage ctx properties to the trial's restored copy. + # TODO(justinvyu): [handle_moved_storage_path] + trial.storage.storage_path = self._storage.storage_path + trial.storage.experiment_dir_name = self._storage.experiment_dir_name + else: # ATTN: Set `local_experiment_path` to update trial checkpoints! trial.local_experiment_path = self._legacy_local_experiment_path trial.remote_experiment_path = self._legacy_remote_experiment_path trial.sync_config = self._legacy_sync_config trial.experiment_dir_name = self._legacy_experiment_dir_name - # Propagate updated storage ctx properties to the trial's restored copy. - # TODO(justinvyu): [handle_moved_storage_path] - trial.storage.storage_path = self._storage.storage_path - trial.storage.experiment_dir_name = self._storage.experiment_dir_name - # Avoid creating logdir in client mode for returned trial results, # since the dir might not be creatable locally. # TODO(ekl) this is kind of a hack. From 3317e3b46dffdb9a135526947a6df45b879219df Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 8 Aug 2023 13:36:48 -0700 Subject: [PATCH 093/108] Separate out fs resolution into a helper Signed-off-by: Justin Yu Support pathlike Signed-off-by: Justin Yu --- python/ray/train/_internal/storage.py | 68 +++++++++++++++++---------- 1 file changed, 43 insertions(+), 25 deletions(-) diff --git a/python/ray/train/_internal/storage.py b/python/ray/train/_internal/storage.py index 043c5e8b04f4..c5ce59158b6c 100644 --- a/python/ray/train/_internal/storage.py +++ b/python/ray/train/_internal/storage.py @@ -272,6 +272,44 @@ def _create_directory(fs: pyarrow.fs.FileSystem, fs_path: str) -> None: ) +def get_fs_and_path( + storage_path: Union[str, os.PathLike], + storage_filesystem: Optional[pyarrow.fs.FileSystem] = None, +) -> Tuple[pyarrow.fs.FileSystem, str]: + """Returns the fs and path from a storage path and an optional custom fs. + + Args: + storage_path: A storage path or URI. (ex: s3://bucket/path or /tmp/ray_results) + storage_filesystem: A custom filesystem to use. If not provided, + this will be auto-resolved by pyarrow. If provided, the storage_path + is assumed to be prefix-stripped already, and must be a valid path + on the filesystem. + + Raises: + ValueError: if the storage path is a URI and a custom filesystem is given. + """ + storage_path = str(storage_path) + + if storage_filesystem: + if is_uri(storage_path): + raise ValueError( + "If you specify a custom `storage_filesystem`, the corresponding " + "`storage_path` must be a *path* on that filesystem, not a URI.\n" + "For example: " + "(storage_filesystem=CustomS3FileSystem(), " + "storage_path='s3://bucket/path') should be changed to " + "(storage_filesystem=CustomS3FileSystem(), " + "storage_path='bucket/path')\n" + "This is what you provided: " + f"(storage_filesystem={storage_filesystem}, " + f"storage_path={storage_path})\n" + "Note that this may depend on the custom filesystem you use." + ) + return storage_filesystem, storage_path + + return pyarrow.fs.FileSystem.from_uri(storage_path) + + class _FilesystemSyncer(_BackgroundSyncer): """Syncer between local filesystem and a `storage_filesystem`.""" @@ -414,32 +452,12 @@ def __init__( self.current_checkpoint_index = current_checkpoint_index self.sync_config = dataclasses.replace(sync_config) - if storage_filesystem: - # Custom pyarrow filesystem - self.storage_filesystem = storage_filesystem - if is_uri(self.storage_path): - raise ValueError( - "If you specify a custom `storage_filesystem`, the corresponding " - "`storage_path` must be a *path* on that filesystem, not a URI.\n" - "For example: " - "(storage_filesystem=CustomS3FileSystem(), " - "storage_path='s3://bucket/path') should be changed to " - "(storage_filesystem=CustomS3FileSystem(), " - "storage_path='bucket/path')\n" - "This is what you provided: " - f"(storage_filesystem={storage_filesystem}, " - f"storage_path={storage_path})\n" - "Note that this may depend on the custom filesystem you use." - ) - self.storage_fs_path = self.storage_path - else: - ( - self.storage_filesystem, - self.storage_fs_path, - ) = pyarrow.fs.FileSystem.from_uri(self.storage_path) + self.storage_filesystem, self.storage_fs_path = get_fs_and_path( + self.storage_path, storage_filesystem + ) - # The storage prefix is the URI that remains after stripping the - # URI prefix away from the user-provided `storage_path` (using `from_uri`). + # The storage prefix is part of the URI that is stripped away + # from the user-provided `storage_path` by pyarrow's `from_uri`. # Ex: `storage_path="s3://bucket/path?param=1` # -> `storage_prefix=URI` # See the doctests for more examples. From 6fb90645407e29fc033250df2e2a81cc3bc9f019 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 8 Aug 2023 14:49:13 -0700 Subject: [PATCH 094/108] Add custom filesystem arg on restore Signed-off-by: Justin Yu --- python/ray/train/base_trainer.py | 33 ++++++++++++++----- python/ray/train/data_parallel_trainer.py | 8 ++--- .../ray/train/lightning/lightning_trainer.py | 26 --------------- python/ray/tune/impl/tuner_internal.py | 16 ++++++--- python/ray/tune/tuner.py | 24 ++++++++++---- 5 files changed, 55 insertions(+), 52 deletions(-) diff --git a/python/ray/train/base_trainer.py b/python/ray/train/base_trainer.py index 4440747482ee..634208dcb747 100644 --- a/python/ray/train/base_trainer.py +++ b/python/ray/train/base_trainer.py @@ -29,6 +29,7 @@ _download_from_fs_path, _list_at_fs_path, _use_storage_context, + get_fs_and_path, ) from ray.train.constants import TRAIN_DATASET_KEY from ray.util import PublicAPI @@ -201,8 +202,9 @@ def __init__( self.preprocessor = preprocessor self.starting_checkpoint = resume_from_checkpoint - # This path should only be set through restore + # These attributes should only be set through `BaseTrainer.restore` self._restore_path = None + self._restore_storage_filesystem = None self._validate_attributes() @@ -220,7 +222,8 @@ def __init__( @classmethod def restore( cls: Type["BaseTrainer"], - path: str, + path: Union[str, os.PathLike], + storage_filesystem: Optional[pyarrow.fs.FileSystem] = None, datasets: Optional[Dict[str, GenDataset]] = None, preprocessor: Optional["Preprocessor"] = None, scaling_config: Optional[ScalingConfig] = None, @@ -304,13 +307,15 @@ def training_loop(self): Returns: BaseTrainer: A restored instance of the class that is calling this method. """ - if not cls.can_restore(path): + if not cls.can_restore(path, storage_filesystem): raise ValueError( f"Invalid restore path: {path}. Make sure that this path exists and " "is the experiment directory that results from a call to " "`trainer.fit()`." ) - trainer_state_path = cls._maybe_sync_down_trainer_state(path) + trainer_state_path = cls._maybe_sync_down_trainer_state( + path, storage_filesystem + ) assert trainer_state_path.exists() with open(trainer_state_path, "rb") as fp: @@ -361,11 +366,16 @@ def training_loop(self): f"`{cls.__name__}.restore`\n" ) from e trainer._restore_path = path + trainer._restore_storage_filesystem = storage_filesystem return trainer @PublicAPI(stability="alpha") @classmethod - def can_restore(cls: Type["BaseTrainer"], path: Union[str, Path]) -> bool: + def can_restore( + cls: Type["BaseTrainer"], + path: Union[str, os.PathLike], + storage_filesystem: Optional[pyarrow.fs.FileSystem], + ) -> bool: """Checks whether a given directory contains a restorable Train experiment. Args: @@ -377,7 +387,7 @@ def can_restore(cls: Type["BaseTrainer"], path: Union[str, Path]) -> bool: bool: Whether this path exists and contains the trainer state to resume from """ if _use_storage_context(): - fs, fs_path = pyarrow.fs.FileSystem.from_uri(str(path)) + fs, fs_path = get_fs_and_path(path, storage_filesystem) return _TRAINER_PKL in _list_at_fs_path(fs, fs_path) return _TRAINER_PKL in list_at_uri(str(path)) @@ -486,7 +496,11 @@ def _validate_scaling_config(cls, scaling_config: ScalingConfig) -> ScalingConfi return scaling_config @classmethod - def _maybe_sync_down_trainer_state(cls, restore_path: str) -> Path: + def _maybe_sync_down_trainer_state( + cls, + restore_path: Union[str, os.PathLike], + storage_filesystem: Optional[pyarrow.fs.FileSystem], + ) -> Path: """Syncs down trainer state from remote storage. Returns: @@ -494,7 +508,7 @@ def _maybe_sync_down_trainer_state(cls, restore_path: str) -> Path: """ if _use_storage_context(): tempdir = tempfile.mkdtemp("tmp_experiment_dir") - fs, fs_path = pyarrow.fs.FileSystem.from_uri(restore_path) + fs, fs_path = get_fs_and_path(restore_path, storage_filesystem) _download_from_fs_path( fs=fs, fs_path=os.path.join(fs_path, _TRAINER_PKL), @@ -609,11 +623,12 @@ def fit(self) -> Result: if self._restore_path: tuner = Tuner.restore( - self._restore_path, + path=self._restore_path, trainable=trainable, param_space=param_space, resume_unfinished=True, resume_errored=True, + storage_filesystem=self._restore_storage_filesystem, ) else: tuner = Tuner( diff --git a/python/ray/train/data_parallel_trainer.py b/python/ray/train/data_parallel_trainer.py index b17b02c5054e..e1a413ef5bf4 100644 --- a/python/ray/train/data_parallel_trainer.py +++ b/python/ray/train/data_parallel_trainer.py @@ -338,9 +338,7 @@ def restore( Union[Callable[[], None], Callable[[Dict], None]] ] = None, train_loop_config: Optional[Dict] = None, - datasets: Optional[Dict[str, GenDataset]] = None, - preprocessor: Optional["Preprocessor"] = None, - scaling_config: Optional[ScalingConfig] = None, + **kwargs, ) -> "DataParallelTrainer": """Restores a DataParallelTrainer from a previously interrupted/failed run. @@ -366,9 +364,7 @@ def restore( path=path, train_loop_per_worker=train_loop_per_worker, train_loop_config=train_loop_config, - datasets=datasets, - preprocessor=preprocessor, - scaling_config=scaling_config, + **kwargs, ) def _validate_attributes(self): diff --git a/python/ray/train/lightning/lightning_trainer.py b/python/ray/train/lightning/lightning_trainer.py index 9ace32cff1a6..7bd0fc1fec0d 100644 --- a/python/ray/train/lightning/lightning_trainer.py +++ b/python/ray/train/lightning/lightning_trainer.py @@ -488,32 +488,6 @@ def _unify_checkpoint_configs( else: return air_ckpt_config - @PublicAPI(stability="alpha") - @classmethod - def restore( - cls: Type["LightningTrainer"], - path: str, - datasets: Optional[Dict[str, GenDataset]] = None, - preprocessor: Optional["Preprocessor"] = None, - scaling_config: Optional[ScalingConfig] = None, - **kwargs, - ) -> "LightningTrainer": - """Restores a LightningTrainer from a previously interrupted/failed run. - - See :meth:`BaseTrainer.restore() ` - for descriptions of the arguments. - - Returns: - LightningTrainer: A restored instance of `LightningTrainer` - """ - return super(LightningTrainer, cls).restore( - path=path, - datasets=datasets, - preprocessor=preprocessor, - scaling_config=scaling_config, - **kwargs, - ) - def _lightning_train_loop_per_worker(config): """Per-worker training loop for a Lightning Trainer.""" diff --git a/python/ray/tune/impl/tuner_internal.py b/python/ray/tune/impl/tuner_internal.py index f851946ae618..c4603680d55d 100644 --- a/python/ray/tune/impl/tuner_internal.py +++ b/python/ray/tune/impl/tuner_internal.py @@ -89,6 +89,7 @@ class TunerInternal: def __init__( self, restore_path: str = None, + storage_filesystem: Optional[pyarrow.fs.FileSystem] = None, resume_config: Optional[_ResumeConfig] = None, trainable: Optional[TrainableTypeOrTrainer] = None, param_space: Optional[Dict[str, Any]] = None, @@ -117,6 +118,7 @@ def __init__( trainable=trainable, overwrite_param_space=param_space, resume_config=resume_config, + storage_filesystem=storage_filesystem, ) return @@ -390,12 +392,13 @@ def _restore_from_path_or_uri( trainable: TrainableTypeOrTrainer, overwrite_param_space: Optional[Dict[str, Any]], resume_config: _ResumeConfig, + storage_filesystem: Optional[pyarrow.fs.FileSystem], ): # Sync down from cloud storage if needed ( restoring_from_cloud, local_experiment_checkpoint_dir, - ) = self._maybe_sync_down_tuner_state(path_or_uri) + ) = self._maybe_sync_down_tuner_state(path_or_uri, storage_filesystem) experiment_checkpoint_path = Path(local_experiment_checkpoint_dir) old_trainable_name, flattened_param_space_keys = self._load_tuner_state( @@ -452,17 +455,22 @@ def _restore_from_path_or_uri( self._resume_config = resume_config self._is_restored = True - def _maybe_sync_down_tuner_state(self, restore_path: str) -> Tuple[bool, str]: + def _maybe_sync_down_tuner_state( + self, restore_path: str, storage_filesystem: Optional[pyarrow.fs.FileSystem] + ) -> Tuple[bool, str]: """Sync down trainable state from remote storage. Returns: Tuple of (downloaded from remote, local_dir) """ if _use_storage_context(): - from ray.train._internal.storage import _download_from_fs_path + from ray.train._internal.storage import ( + _download_from_fs_path, + get_fs_and_path, + ) tempdir = tempfile.mkdtemp("tmp_experiment_dir") - fs, fs_path = pyarrow.fs.FileSystem.from_uri(restore_path) + fs, fs_path = get_fs_and_path(restore_path, storage_filesystem) _download_from_fs_path( fs=fs, fs_path=os.path.join(fs_path, _TUNER_PKL), diff --git a/python/ray/tune/tuner.py b/python/ray/tune/tuner.py index 2baae0f4d0b7..99b9b325ba35 100644 --- a/python/ray/tune/tuner.py +++ b/python/ray/tune/tuner.py @@ -1,9 +1,10 @@ import logging -from pathlib import Path +import os from typing import Any, Callable, Dict, Optional, Type, Union, TYPE_CHECKING -import ray +import pyarrow.fs +import ray from ray.air.config import RunConfig from ray.air._internal.remote_storage import list_at_uri from ray.air._internal.usage import AirEntrypoint @@ -185,6 +186,7 @@ def restore( resume_errored: bool = False, restart_errored: bool = False, param_space: Optional[Dict[str, Any]] = None, + storage_filesystem: Optional[pyarrow.fs.FileSystem] = None, ) -> "Tuner": """Restores Tuner after a previously failed run. @@ -246,6 +248,7 @@ def restore( resume_config=resume_config, trainable=trainable, param_space=param_space, + storage_filesystem=storage_filesystem, ) return Tuner(_tuner_internal=tuner_internal) else: @@ -256,11 +259,16 @@ def restore( resume_config=resume_config, trainable=trainable, param_space=param_space, + storage_filesystem=storage_filesystem, ) return Tuner(_tuner_internal=tuner_internal) @classmethod - def can_restore(cls, path: Union[str, Path]) -> bool: + def can_restore( + cls, + path: Union[str, os.PathLike], + storage_filesystem: Optional[pyarrow.fs.FileSystem] = None, + ) -> bool: """Checks whether a given directory contains a restorable Tune experiment. Usage Pattern: @@ -301,12 +309,14 @@ def train_fn(config): Returns: bool: True if this path exists and contains the Tuner state to resume from """ - from ray.train._internal.storage import _list_at_fs_path, _use_storage_context + from ray.train._internal.storage import ( + _list_at_fs_path, + _use_storage_context, + get_fs_and_path, + ) if _use_storage_context(): - import pyarrow.fs - - fs, fs_path = pyarrow.fs.FileSystem.from_uri(str(path)) + fs, fs_path = get_fs_and_path(path, storage_filesystem) return _TUNER_PKL in _list_at_fs_path(fs, fs_path) return _TUNER_PKL in list_at_uri(str(path)) From f388dcca62de6e91253d40fb022a9ceec066c66c Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 8 Aug 2023 14:52:38 -0700 Subject: [PATCH 095/108] Don't skip the custom fs test case for restore Signed-off-by: Justin Yu --- .../ray/train/tests/test_new_persistence.py | 48 ++++++++----------- 1 file changed, 20 insertions(+), 28 deletions(-) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index 2a42c8f6a097..e0e19c6799ae 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -9,6 +9,7 @@ import pyarrow.fs +import ray from ray import train, tune from ray.air._internal.uri_utils import URI from ray.air.constants import EXPR_RESULT_FILE @@ -28,11 +29,13 @@ def dummy_context_manager(): yield "dummy value" -@pytest.fixture(autouse=True) -def enable_new_persistence_mode(monkeypatch): - monkeypatch.setenv("RAY_AIR_NEW_PERSISTENCE_MODE", "1") +@pytest.fixture(autouse=True, scope="module") +def ray_start_4_cpus(): + ray.init( + num_cpus=4, runtime_env={"env_vars": {"RAY_AIR_NEW_PERSISTENCE_MODE": "1"}} + ) yield - monkeypatch.setenv("RAY_AIR_NEW_PERSISTENCE_MODE", "0") + ray.shutdown() def _create_mock_custom_fs(custom_fs_root_dir: Path) -> pyarrow.fs.FileSystem: @@ -311,7 +314,6 @@ def test_trainer( """ TODO(justinvyu): Test for these once implemented: - artifacts - - restoration, train.get_checkpoint {storage_path}/{exp_name} ├── experiment_state-2023-07-28_10-00-38.json <- Initial exp state @@ -344,16 +346,12 @@ def test_trainer( # TODO(justinvyu): Manual restore doesn't work yet for custom fs. # Need to introduce restore(storage_filesystem) API in a follow-up. - TEST_MANUAL_RESTORE = storage_path_type != "custom_fs" - with _resolve_storage_type(storage_path_type, tmp_path) as ( storage_path, storage_filesystem, ): NUM_ITERATIONS = 6 NUM_WORKERS = 2 - - max_failures = 1 if TEST_MANUAL_RESTORE else 2 trainer = DataParallelTrainer( train_fn, train_loop_config={ @@ -368,22 +366,20 @@ def test_trainer( name=exp_name, verbose=0, checkpoint_config=checkpoint_config, - failure_config=train.FailureConfig(max_failures=max_failures), + failure_config=train.FailureConfig(max_failures=1), ), ) print("\nStarting initial run.\n") - if TEST_MANUAL_RESTORE: - with pytest.raises(TrainingFailedError): - result = trainer.fit() - - print("\nStarting manually restored run.\n") - restored_trainer = DataParallelTrainer.restore( - path=str(URI(storage_path or str(LOCAL_CACHE_DIR)) / exp_name) - ) - result = restored_trainer.fit() - else: + with pytest.raises(TrainingFailedError): result = trainer.fit() + print("\nStarting manually restored run.\n") + restored_trainer = DataParallelTrainer.restore( + path=str(URI(storage_path or str(LOCAL_CACHE_DIR)) / exp_name), + storage_filesystem=storage_filesystem, + ) + result = restored_trainer.fit() + with monkeypatch.context() as m: # This is so that the `resume_from_checkpoint` run doesn't mess up the # assertions later for the `storage_path=None` case. @@ -412,14 +408,10 @@ def test_trainer( # Files synced by the driver assert len(list(exp_dir.glob("tuner.pkl"))) == 1 assert len(list(exp_dir.glob("trainer.pkl"))) == 1 - if TEST_MANUAL_RESTORE: - # 2 copies of these files: - # 1 for the initial run, and 1 for the manually restored run. - assert len(list(exp_dir.glob("basic-variant-state-*"))) == 2 - assert len(list(exp_dir.glob("experiment_state-*"))) == 2 - else: - assert len(list(exp_dir.glob("basic-variant-state-*"))) == 1 - assert len(list(exp_dir.glob("experiment_state-*"))) == 1 + # 2 copies of these files: + # 1 for the initial run, and 1 for the manually restored run. + assert len(list(exp_dir.glob("basic-variant-state-*"))) == 2 + assert len(list(exp_dir.glob("experiment_state-*"))) == 2 # Files synced by the worker assert len(list(exp_dir.glob("DataParallelTrainer_*"))) == 1 From ade1078f44a089764b801b98e0d763c8e41761d3 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 8 Aug 2023 15:07:01 -0700 Subject: [PATCH 096/108] clean up some imports Signed-off-by: Justin Yu --- python/ray/tune/analysis/experiment_analysis.py | 3 +-- python/ray/tune/impl/tuner_internal.py | 12 ++++++------ python/ray/tune/tuner.py | 11 +++++------ 3 files changed, 12 insertions(+), 14 deletions(-) diff --git a/python/ray/tune/analysis/experiment_analysis.py b/python/ray/tune/analysis/experiment_analysis.py index 2b16ded8520d..867b0f5baf8f 100644 --- a/python/ray/tune/analysis/experiment_analysis.py +++ b/python/ray/tune/analysis/experiment_analysis.py @@ -21,6 +21,7 @@ EXPR_PARAM_FILE, TRAINING_ITERATION, ) +from ray.train._internal.storage import _use_storage_context from ray.tune.syncer import SyncConfig from ray.tune.utils import flatten_dict from ray.tune.utils.serialization import TuneFunctionDecoder @@ -978,8 +979,6 @@ def _get_trial_paths(self) -> List[str]: ) self.trials = [] for trial_json_state, path in self._checkpoints_and_paths: - from ray.train._internal.storage import _use_storage_context - try: trial = Trial.from_json_state(trial_json_state, stub=True) # TODO(justinvyu): [handle_moved_storage_path] diff --git a/python/ray/tune/impl/tuner_internal.py b/python/ray/tune/impl/tuner_internal.py index c4603680d55d..7a423f9057a0 100644 --- a/python/ray/tune/impl/tuner_internal.py +++ b/python/ray/tune/impl/tuner_internal.py @@ -28,7 +28,12 @@ from ray.air._internal.uri_utils import URI from ray.air._internal.usage import AirEntrypoint from ray.air.config import RunConfig, ScalingConfig -from ray.train._internal.storage import _use_storage_context, StorageContext +from ray.train._internal.storage import ( + _download_from_fs_path, + _use_storage_context, + StorageContext, + get_fs_and_path, +) from ray.tune import Experiment, TuneError, ExperimentAnalysis from ray.tune.execution.experiment_state import _ResumeConfig from ray.tune.tune import _Config @@ -464,11 +469,6 @@ def _maybe_sync_down_tuner_state( Tuple of (downloaded from remote, local_dir) """ if _use_storage_context(): - from ray.train._internal.storage import ( - _download_from_fs_path, - get_fs_and_path, - ) - tempdir = tempfile.mkdtemp("tmp_experiment_dir") fs, fs_path = get_fs_and_path(restore_path, storage_filesystem) _download_from_fs_path( diff --git a/python/ray/tune/tuner.py b/python/ray/tune/tuner.py index 99b9b325ba35..fe4ae24bda74 100644 --- a/python/ray/tune/tuner.py +++ b/python/ray/tune/tuner.py @@ -9,6 +9,11 @@ from ray.air._internal.remote_storage import list_at_uri from ray.air._internal.usage import AirEntrypoint from ray.air.util.node import _force_on_current_node +from ray.train._internal.storage import ( + _list_at_fs_path, + _use_storage_context, + get_fs_and_path, +) from ray.tune import TuneError from ray.tune.execution.experiment_state import _ResumeConfig from ray.tune.experimental.output import ( @@ -309,12 +314,6 @@ def train_fn(config): Returns: bool: True if this path exists and contains the Tuner state to resume from """ - from ray.train._internal.storage import ( - _list_at_fs_path, - _use_storage_context, - get_fs_and_path, - ) - if _use_storage_context(): fs, fs_path = get_fs_and_path(path, storage_filesystem) return _TUNER_PKL in _list_at_fs_path(fs, fs_path) From 8b78dee1869f4786bacad244fe23e14a55955a61 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 8 Aug 2023 15:08:29 -0700 Subject: [PATCH 097/108] Fix the test fixtures Signed-off-by: Justin Yu --- python/ray/train/tests/test_new_persistence.py | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index e0e19c6799ae..493ce994f49e 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -29,11 +29,18 @@ def dummy_context_manager(): yield "dummy value" +@pytest.fixture(scope="module") +def enable_new_persistence_mode(): + with pytest.MonkeyPatch.context() as mp: + mp.setenv("RAY_AIR_NEW_PERSISTENCE_MODE", "1") + yield + mp.setenv("RAY_AIR_NEW_PERSISTENCE_MODE", "0") + + @pytest.fixture(autouse=True, scope="module") -def ray_start_4_cpus(): - ray.init( - num_cpus=4, runtime_env={"env_vars": {"RAY_AIR_NEW_PERSISTENCE_MODE": "1"}} - ) +def ray_start_4_cpus(enable_new_persistence_mode): + # Make sure to set the env var before calling ray.init() + ray.init(num_cpus=4) yield ray.shutdown() From cf963fd1ff2991a99a064595c763110060b08435 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 8 Aug 2023 15:11:15 -0700 Subject: [PATCH 098/108] Remove done todo Signed-off-by: Justin Yu --- python/ray/train/tests/test_new_persistence.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index 493ce994f49e..849d9cb0feae 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -351,8 +351,6 @@ def test_trainer( monkeypatch.setenv("RAY_AIR_LOCAL_CACHE_DIR", str(LOCAL_CACHE_DIR)) exp_name = "trainer_new_persistence" - # TODO(justinvyu): Manual restore doesn't work yet for custom fs. - # Need to introduce restore(storage_filesystem) API in a follow-up. with _resolve_storage_type(storage_path_type, tmp_path) as ( storage_path, storage_filesystem, From ca5f5bf355e154abe83dbf44db7188752bcb721e Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 8 Aug 2023 18:20:08 -0700 Subject: [PATCH 099/108] Fix optional can_restore argument Signed-off-by: Justin Yu --- python/ray/train/base_trainer.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/train/base_trainer.py b/python/ray/train/base_trainer.py index 634208dcb747..ccfbc76e0a14 100644 --- a/python/ray/train/base_trainer.py +++ b/python/ray/train/base_trainer.py @@ -374,7 +374,7 @@ def training_loop(self): def can_restore( cls: Type["BaseTrainer"], path: Union[str, os.PathLike], - storage_filesystem: Optional[pyarrow.fs.FileSystem], + storage_filesystem: Optional[pyarrow.fs.FileSystem] = None, ) -> bool: """Checks whether a given directory contains a restorable Train experiment. From 75e947c42cc4cbba2acafd77d4047bae5f8c00d7 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Tue, 8 Aug 2023 18:37:48 -0700 Subject: [PATCH 100/108] Remove duplicate in test Signed-off-by: Justin Yu --- python/ray/train/tests/test_new_persistence.py | 8 -------- 1 file changed, 8 deletions(-) diff --git a/python/ray/train/tests/test_new_persistence.py b/python/ray/train/tests/test_new_persistence.py index 2ff45ccecdcd..849d9cb0feae 100644 --- a/python/ray/train/tests/test_new_persistence.py +++ b/python/ray/train/tests/test_new_persistence.py @@ -385,14 +385,6 @@ def test_trainer( ) result = restored_trainer.fit() - with monkeypatch.context() as m: - # This is so that the `resume_from_checkpoint` run doesn't mess up the - # assertions later for the `storage_path=None` case. - m.setenv("RAY_AIR_LOCAL_CACHE_DIR", tmp_path / "resume_from_checkpoint") - _resume_from_checkpoint( - result.checkpoint, expected_state={"iter": NUM_ITERATIONS - 1} - ) - with monkeypatch.context() as m: # This is so that the `resume_from_checkpoint` run doesn't mess up the # assertions later for the `storage_path=None` case. From a13d88d0743afa4d4d341915d5538bfb267251e6 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 9 Aug 2023 09:54:05 -0700 Subject: [PATCH 101/108] read file directly from fs for trainer restore Signed-off-by: Justin Yu --- python/ray/train/base_trainer.py | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/python/ray/train/base_trainer.py b/python/ray/train/base_trainer.py index ccfbc76e0a14..89d0aece75be 100644 --- a/python/ray/train/base_trainer.py +++ b/python/ray/train/base_trainer.py @@ -313,13 +313,19 @@ def training_loop(self): "is the experiment directory that results from a call to " "`trainer.fit()`." ) - trainer_state_path = cls._maybe_sync_down_trainer_state( - path, storage_filesystem - ) - assert trainer_state_path.exists() + if _use_storage_context(): + fs, fs_path = get_fs_and_path(path, storage_filesystem) + with fs.open_input_file(os.path.join(fs_path, _TRAINER_PKL)) as f: + trainer_cls, param_dict = pickle.loads(f.readall()) + else: + trainer_state_path = cls._maybe_sync_down_trainer_state( + path, storage_filesystem + ) + assert trainer_state_path.exists() + + with open(trainer_state_path, "rb") as fp: + trainer_cls, param_dict = pickle.load(fp) - with open(trainer_state_path, "rb") as fp: - trainer_cls, param_dict = pickle.load(fp) if trainer_cls is not cls: warnings.warn( f"Invalid trainer type. You are attempting to restore a trainer of type" From 7e468c682c8bc3a4a0fd282e5dac729bc942714e Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 9 Aug 2023 09:55:09 -0700 Subject: [PATCH 102/108] check for existence rather than list Signed-off-by: Justin Yu --- python/ray/train/base_trainer.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/train/base_trainer.py b/python/ray/train/base_trainer.py index 89d0aece75be..a02b9bb9085b 100644 --- a/python/ray/train/base_trainer.py +++ b/python/ray/train/base_trainer.py @@ -27,7 +27,7 @@ from ray.train._internal import session from ray.train._internal.storage import ( _download_from_fs_path, - _list_at_fs_path, + _exists_at_fs_path, _use_storage_context, get_fs_and_path, ) @@ -394,7 +394,7 @@ def can_restore( """ if _use_storage_context(): fs, fs_path = get_fs_and_path(path, storage_filesystem) - return _TRAINER_PKL in _list_at_fs_path(fs, fs_path) + return _exists_at_fs_path(fs, os.path.join(fs_path, _TRAINER_PKL)) return _TRAINER_PKL in list_at_uri(str(path)) From 13c224f72bda9fdd4fef9461d4ccbcca9d1dea3a Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 9 Aug 2023 10:39:22 -0700 Subject: [PATCH 103/108] Update tuner restore Signed-off-by: Justin Yu --- python/ray/tune/impl/tuner_internal.py | 123 +++++++++++-------------- python/ray/tune/tuner.py | 4 +- 2 files changed, 56 insertions(+), 71 deletions(-) diff --git a/python/ray/tune/impl/tuner_internal.py b/python/ray/tune/impl/tuner_internal.py index 7a423f9057a0..ad8a94e23264 100644 --- a/python/ray/tune/impl/tuner_internal.py +++ b/python/ray/tune/impl/tuner_internal.py @@ -344,7 +344,7 @@ def _set_param_space_on_restore( ) def _load_tuner_state( - self, tuner_pkl_path: Path + self, tuner_state: Dict[str, Any] ) -> Tuple[Optional[str], Optional[List[str]]]: """Loads Tuner state from the previously saved `tuner.pkl`. @@ -356,38 +356,13 @@ def _load_tuner_state( tuple: of `(old_trainable_name, flattened_param_space_keys)` used for validating the re-specified `trainable` and `param_space`. """ - if not tuner_pkl_path.exists(): - raise RuntimeError( - f"Could not find Tuner state in restore directory. Did you pass" - f"the correct path (the top-level experiment directory?) Got: " - f"{tuner_pkl_path.parent}" - ) - - with open(tuner_pkl_path, "rb") as fp: - tuner_state = pickle.load(fp) - - if isinstance(tuner_state, TunerInternal): - # TODO(ml-team): Remove in 2.7. - # Backwards compatibility: ray<=2.4 pickles the full Tuner object - # within `tuner.pkl`. ray>=2.5 pickles the object state as a dict. - tuner: TunerInternal = tuner_state - self.__setstate__(tuner.__getstate__()) - - logger.warning( - "You are restoring a Tune experiment that was run with an older " - "version of Ray. Note that backwards compatibility of restoring " - "this experiment will only be guaranteed until Ray 2.7." - ) - - old_trainable_name, flattened_param_space_keys = None, None - else: - # NOTE: These are magic keys used for validating restore args. - old_trainable_name = tuner_state.pop("__trainable_name", None) - flattened_param_space_keys = tuner_state.pop( - "__flattened_param_space_keys", None - ) + # NOTE: These are magic keys used for validating restore args. + old_trainable_name = tuner_state.pop("__trainable_name", None) + flattened_param_space_keys = tuner_state.pop( + "__flattened_param_space_keys", None + ) - self.__setstate__(tuner_state) + self.__setstate__(tuner_state) return old_trainable_name, flattened_param_space_keys @@ -399,15 +374,27 @@ def _restore_from_path_or_uri( resume_config: _ResumeConfig, storage_filesystem: Optional[pyarrow.fs.FileSystem], ): - # Sync down from cloud storage if needed - ( - restoring_from_cloud, - local_experiment_checkpoint_dir, - ) = self._maybe_sync_down_tuner_state(path_or_uri, storage_filesystem) - experiment_checkpoint_path = Path(local_experiment_checkpoint_dir) + if _use_storage_context(): + fs, fs_path = get_fs_and_path(path_or_uri, storage_filesystem) + with fs.open_input_file(os.path.join(fs_path, _TUNER_PKL)) as f: + tuner_state = pickle.loads(f.readall()) + + restoring_from_cloud = None + local_experiment_checkpoint_dir = None + experiment_checkpoint_path = None + else: + # Sync down from cloud storage if needed + ( + restoring_from_cloud, + local_experiment_checkpoint_dir, + ) = self._maybe_sync_down_tuner_state(path_or_uri) + experiment_checkpoint_path = Path(local_experiment_checkpoint_dir) + + with open(experiment_checkpoint_path / _TUNER_PKL, "rb") as fp: + tuner_state = pickle.load(fp) old_trainable_name, flattened_param_space_keys = self._load_tuner_state( - experiment_checkpoint_path / _TUNER_PKL + tuner_state ) # Perform validation and set the re-specified `trainable` and `param_space` @@ -420,9 +407,8 @@ def _restore_from_path_or_uri( ) # Update RunConfig to reflect changes in the experiment directory - path_or_uri_obj: Union[Path, URI] = ( - URI(path_or_uri) if restoring_from_cloud else experiment_checkpoint_path - ) + path_or_uri_obj = URI(path_or_uri) + # Infer the `storage_path` and run `name` of the restored run using the # experiment directory. # Ex: ~/ray_results/exp_name -> ~/ray_results, exp_name @@ -430,22 +416,33 @@ def _restore_from_path_or_uri( self._run_config.name = path_or_uri_obj.name self._run_config.storage_path = str(path_or_uri_obj.parent) - # Set the experiment directory - if not restoring_from_cloud: - self._experiment_checkpoint_dir = local_experiment_checkpoint_dir - else: - # If we synced, `experiment_checkpoint_dir` will contain a temporary - # directory. Create an experiment checkpoint dir instead and move - # our data there. - new_exp_path, new_exp_name = self.setup_create_experiment_checkpoint_dir( + if _use_storage_context(): + ( + self._experiment_checkpoint_dir, + self._experiment_dir_name, + ) = self.setup_create_experiment_checkpoint_dir( self.converted_trainable, self._run_config ) - new_exp_path = Path(new_exp_path) - for file_dir in experiment_checkpoint_path.glob("*"): - file_dir.replace(new_exp_path / file_dir.name) - shutil.rmtree(experiment_checkpoint_path) - self._experiment_checkpoint_dir = str(new_exp_path) - self._experiment_dir_name = str(new_exp_name) + else: + # Set the experiment directory + if not restoring_from_cloud: + self._experiment_checkpoint_dir = local_experiment_checkpoint_dir + else: + # If we synced, `experiment_checkpoint_dir` will contain a temporary + # directory. Create an experiment checkpoint dir instead and move + # our data there. + ( + new_exp_path, + new_exp_name, + ) = self.setup_create_experiment_checkpoint_dir( + self.converted_trainable, self._run_config + ) + new_exp_path = Path(new_exp_path) + for file_dir in experiment_checkpoint_path.glob("*"): + file_dir.replace(new_exp_path / file_dir.name) + shutil.rmtree(experiment_checkpoint_path) + self._experiment_checkpoint_dir = str(new_exp_path) + self._experiment_dir_name = str(new_exp_name) # Load the experiment results at the point where it left off. try: @@ -460,24 +457,12 @@ def _restore_from_path_or_uri( self._resume_config = resume_config self._is_restored = True - def _maybe_sync_down_tuner_state( - self, restore_path: str, storage_filesystem: Optional[pyarrow.fs.FileSystem] - ) -> Tuple[bool, str]: + def _maybe_sync_down_tuner_state(self, restore_path: str) -> Tuple[bool, str]: """Sync down trainable state from remote storage. Returns: Tuple of (downloaded from remote, local_dir) """ - if _use_storage_context(): - tempdir = tempfile.mkdtemp("tmp_experiment_dir") - fs, fs_path = get_fs_and_path(restore_path, storage_filesystem) - _download_from_fs_path( - fs=fs, - fs_path=os.path.join(fs_path, _TUNER_PKL), - local_path=os.path.join(tempdir, _TUNER_PKL), - ) - return True, tempdir - if not is_non_local_path_uri(restore_path): return False, os.path.abspath(os.path.expanduser(restore_path)) diff --git a/python/ray/tune/tuner.py b/python/ray/tune/tuner.py index fe4ae24bda74..88ff6c2416df 100644 --- a/python/ray/tune/tuner.py +++ b/python/ray/tune/tuner.py @@ -10,7 +10,7 @@ from ray.air._internal.usage import AirEntrypoint from ray.air.util.node import _force_on_current_node from ray.train._internal.storage import ( - _list_at_fs_path, + _exists_at_fs_path, _use_storage_context, get_fs_and_path, ) @@ -316,7 +316,7 @@ def train_fn(config): """ if _use_storage_context(): fs, fs_path = get_fs_and_path(path, storage_filesystem) - return _TUNER_PKL in _list_at_fs_path(fs, fs_path) + return _exists_at_fs_path(fs, os.path.join(fs_path, _TUNER_PKL)) return _TUNER_PKL in list_at_uri(str(path)) From 1085666045445eba1af359bb0b1fd8d84c359eaa Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 9 Aug 2023 10:42:00 -0700 Subject: [PATCH 104/108] Mark experiment_checkpoint_dir as legacy Signed-off-by: Justin Yu --- python/ray/tune/impl/tuner_internal.py | 20 +++++++++----------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/python/ray/tune/impl/tuner_internal.py b/python/ray/tune/impl/tuner_internal.py index ad8a94e23264..48233a2efdc7 100644 --- a/python/ray/tune/impl/tuner_internal.py +++ b/python/ray/tune/impl/tuner_internal.py @@ -141,7 +141,7 @@ def __init__( self._is_restored = False self._tuner_kwargs = copy.deepcopy(_tuner_kwargs) or {} ( - self._experiment_checkpoint_dir, + self._legacy_experiment_checkpoint_dir, self._experiment_dir_name, ) = self.setup_create_experiment_checkpoint_dir( self.converted_trainable, self._run_config @@ -154,7 +154,7 @@ def __init__( # without allowing for checkpointing tuner and trainable. # Thus this has to happen before tune.run() so that we can have something # to restore from. - experiment_checkpoint_path = Path(self._experiment_checkpoint_dir) + experiment_checkpoint_path = Path(self._legacy_experiment_checkpoint_dir) with open(experiment_checkpoint_path / _TUNER_PKL, "wb") as fp: pickle.dump(self.__getstate__(), fp) @@ -417,16 +417,15 @@ def _restore_from_path_or_uri( self._run_config.storage_path = str(path_or_uri_obj.parent) if _use_storage_context(): - ( - self._experiment_checkpoint_dir, - self._experiment_dir_name, - ) = self.setup_create_experiment_checkpoint_dir( + _, self._experiment_dir_name = self.setup_create_experiment_checkpoint_dir( self.converted_trainable, self._run_config ) + + self._legacy_experiment_checkpoint_dir = None else: # Set the experiment directory if not restoring_from_cloud: - self._experiment_checkpoint_dir = local_experiment_checkpoint_dir + self._legacy_experiment_checkpoint_dir = local_experiment_checkpoint_dir else: # If we synced, `experiment_checkpoint_dir` will contain a temporary # directory. Create an experiment checkpoint dir instead and move @@ -441,7 +440,7 @@ def _restore_from_path_or_uri( for file_dir in experiment_checkpoint_path.glob("*"): file_dir.replace(new_exp_path / file_dir.name) shutil.rmtree(experiment_checkpoint_path) - self._experiment_checkpoint_dir = str(new_exp_path) + self._legacy_experiment_checkpoint_dir = str(new_exp_path) self._experiment_dir_name = str(new_exp_name) # Load the experiment results at the point where it left off. @@ -558,7 +557,7 @@ def setup_create_experiment_checkpoint_dir( # This has to be done through a function signature (@property won't do). def get_experiment_checkpoint_dir(self) -> str: - return self._experiment_checkpoint_dir + return self._legacy_experiment_checkpoint_dir @property def trainable(self) -> TrainableTypeOrTrainer: @@ -608,7 +607,6 @@ def _convert_trainable(self, trainable: TrainableTypeOrTrainer) -> TrainableType def fit(self) -> ResultGrid: trainable = self.converted_trainable - assert self._experiment_checkpoint_dir param_space = copy.deepcopy(self.param_space) if not self._is_restored: analysis = self._fit_internal(trainable, param_space) @@ -697,7 +695,7 @@ def _get_tune_run_arguments(self, trainable: TrainableType) -> Dict[str, Any]: stop=self._run_config.stop, max_failures=self._run_config.failure_config.max_failures, checkpoint_config=checkpoint_config, - _experiment_checkpoint_dir=self._experiment_checkpoint_dir, + _experiment_checkpoint_dir=self._legacy_experiment_checkpoint_dir, raise_on_failed_trial=False, fail_fast=(self._run_config.failure_config.fail_fast), progress_reporter=self._run_config.progress_reporter, From 0957d68e9bde4d6620276dd3d0dfd41840036c13 Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 9 Aug 2023 10:43:16 -0700 Subject: [PATCH 105/108] Revert changes to sync down logic in trainer Signed-off-by: Justin Yu --- python/ray/train/base_trainer.py | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/python/ray/train/base_trainer.py b/python/ray/train/base_trainer.py index a02b9bb9085b..e5ec5cecd2aa 100644 --- a/python/ray/train/base_trainer.py +++ b/python/ray/train/base_trainer.py @@ -318,9 +318,7 @@ def training_loop(self): with fs.open_input_file(os.path.join(fs_path, _TRAINER_PKL)) as f: trainer_cls, param_dict = pickle.loads(f.readall()) else: - trainer_state_path = cls._maybe_sync_down_trainer_state( - path, storage_filesystem - ) + trainer_state_path = cls._maybe_sync_down_trainer_state(path) assert trainer_state_path.exists() with open(trainer_state_path, "rb") as fp: @@ -503,25 +501,13 @@ def _validate_scaling_config(cls, scaling_config: ScalingConfig) -> ScalingConfi @classmethod def _maybe_sync_down_trainer_state( - cls, - restore_path: Union[str, os.PathLike], - storage_filesystem: Optional[pyarrow.fs.FileSystem], + cls, restore_path: Union[str, os.PathLike] ) -> Path: """Syncs down trainer state from remote storage. Returns: str: Local directory containing the trainer state """ - if _use_storage_context(): - tempdir = tempfile.mkdtemp("tmp_experiment_dir") - fs, fs_path = get_fs_and_path(restore_path, storage_filesystem) - _download_from_fs_path( - fs=fs, - fs_path=os.path.join(fs_path, _TRAINER_PKL), - local_path=os.path.join(tempdir, _TRAINER_PKL), - ) - return Path(tempdir) / _TRAINER_PKL - if not is_non_local_path_uri(restore_path): return Path(os.path.expanduser(restore_path)) / _TRAINER_PKL From 1716fa763e9b52550a6594daa0bf165bac7145bd Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 9 Aug 2023 10:43:51 -0700 Subject: [PATCH 106/108] Fix lint Signed-off-by: Justin Yu --- python/ray/train/base_trainer.py | 1 - python/ray/tune/impl/tuner_internal.py | 1 - 2 files changed, 2 deletions(-) diff --git a/python/ray/train/base_trainer.py b/python/ray/train/base_trainer.py index e5ec5cecd2aa..052b422b14bf 100644 --- a/python/ray/train/base_trainer.py +++ b/python/ray/train/base_trainer.py @@ -26,7 +26,6 @@ from ray.train._checkpoint import Checkpoint as NewCheckpoint from ray.train._internal import session from ray.train._internal.storage import ( - _download_from_fs_path, _exists_at_fs_path, _use_storage_context, get_fs_and_path, diff --git a/python/ray/tune/impl/tuner_internal.py b/python/ray/tune/impl/tuner_internal.py index 48233a2efdc7..7f95c0541745 100644 --- a/python/ray/tune/impl/tuner_internal.py +++ b/python/ray/tune/impl/tuner_internal.py @@ -29,7 +29,6 @@ from ray.air._internal.usage import AirEntrypoint from ray.air.config import RunConfig, ScalingConfig from ray.train._internal.storage import ( - _download_from_fs_path, _use_storage_context, StorageContext, get_fs_and_path, From 5b6be43573b841b141460ee1a513416e63b0eb8e Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 9 Aug 2023 10:51:10 -0700 Subject: [PATCH 107/108] minor fixes Signed-off-by: Justin Yu --- python/ray/train/base_trainer.py | 4 +--- python/ray/tune/execution/tune_controller.py | 3 ++- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/python/ray/train/base_trainer.py b/python/ray/train/base_trainer.py index 052b422b14bf..883709167e57 100644 --- a/python/ray/train/base_trainer.py +++ b/python/ray/train/base_trainer.py @@ -499,9 +499,7 @@ def _validate_scaling_config(cls, scaling_config: ScalingConfig) -> ScalingConfi return scaling_config @classmethod - def _maybe_sync_down_trainer_state( - cls, restore_path: Union[str, os.PathLike] - ) -> Path: + def _maybe_sync_down_trainer_state(cls, restore_path: str) -> Path: """Syncs down trainer state from remote storage. Returns: diff --git a/python/ray/tune/execution/tune_controller.py b/python/ray/tune/execution/tune_controller.py index 9136731ebf67..7aad4373a975 100644 --- a/python/ray/tune/execution/tune_controller.py +++ b/python/ray/tune/execution/tune_controller.py @@ -607,7 +607,8 @@ def resume( trial_to_add.pickled_error_filename = None trial_to_add.set_status(Trial.PENDING) if not _use_storage_context(): - # TODO(justinvyu): Mark this as legacy. + # TODO(justinvyu): Remove this. + # Not needed since trial.checkpoint will be used anyways. trial_to_add.restore_path = trial.checkpoint.dir_or_data elif restart_errored: trial_to_add = trial.reset() From ca0a3a03f95428747e34228daba3ff2233313cfd Mon Sep 17 00:00:00 2001 From: Justin Yu Date: Wed, 9 Aug 2023 14:35:38 -0700 Subject: [PATCH 108/108] Remove backwards compatibility test Signed-off-by: Justin Yu --- python/ray/tune/tests/test_tuner_restore.py | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/python/ray/tune/tests/test_tuner_restore.py b/python/ray/tune/tests/test_tuner_restore.py index f0ce90a9c36b..ea846359412b 100644 --- a/python/ray/tune/tests/test_tuner_restore.py +++ b/python/ray/tune/tests/test_tuner_restore.py @@ -9,7 +9,6 @@ import pytest import ray -import ray.cloudpickle as ray_pickle from ray import train, tune from ray.train import ( Checkpoint, @@ -1176,18 +1175,6 @@ def train_fn(config): assert r.config["test2"].name in ["11", "12", "13", "14"] -def test_tuner_pkl_backwards_compatibility(tmp_path, propagate_logs, caplog): - tuner_internal = Tuner( - _train_fn_sometimes_failing, param_space={"a": 1} - )._local_tuner - with open(tmp_path / "tuner.pkl", "wb") as f: - ray_pickle.dump(tuner_internal, f) - - with caplog.at_level(logging.WARNING, "ray.tune.impl.tuner_internal"): - tuner_internal._load_tuner_state(tmp_path / "tuner.pkl") - assert "run with an older version of Ray" in caplog.text - - if __name__ == "__main__": import sys