From ab6b4e9ea794ca127436fc46ce112b1642ef12df Mon Sep 17 00:00:00 2001 From: dentiny Date: Wed, 6 Nov 2024 20:52:09 +0000 Subject: [PATCH 01/14] uv installation Signed-off-by: dentiny --- python/ray/_private/runtime_env/BUILD | 3 + .../runtime_env/agent/runtime_env_agent.py | 3 + python/ray/_private/runtime_env/packaging.py | 1 + python/ray/_private/runtime_env/uv.py | 160 +++++++++++++++++- python/ray/_private/runtime_env/validation.py | 3 +- python/ray/runtime_env/runtime_env.py | 46 +++-- python/ray/tests/BUILD | 1 + python/ray/tests/test_runtime_uv.py | 64 +++++++ 8 files changed, 266 insertions(+), 15 deletions(-) create mode 100644 python/ray/tests/test_runtime_uv.py diff --git a/python/ray/_private/runtime_env/BUILD b/python/ray/_private/runtime_env/BUILD index 4ba4f80d5b9e..34edfab1f290 100644 --- a/python/ray/_private/runtime_env/BUILD +++ b/python/ray/_private/runtime_env/BUILD @@ -1,3 +1,6 @@ +# TODO(hjiang): All existing pythons are not using bazel as build system, which leads to missing BUILD file and targets. +# Revisit if we decide to support bazel build in the future. + load("@rules_python//python:defs.bzl", "py_library", "py_test") package(default_visibility = ["//visibility:public"]) diff --git a/python/ray/_private/runtime_env/agent/runtime_env_agent.py b/python/ray/_private/runtime_env/agent/runtime_env_agent.py index 58c332fb3077..9118aa33d266 100644 --- a/python/ray/_private/runtime_env/agent/runtime_env_agent.py +++ b/python/ray/_private/runtime_env/agent/runtime_env_agent.py @@ -18,6 +18,7 @@ from ray._private.runtime_env.java_jars import JavaJarsPlugin from ray._private.runtime_env.image_uri import ContainerPlugin from ray._private.runtime_env.pip import PipPlugin +from ray._private.runtime_env.uv import UvPlugin from ray._private.gcs_utils import GcsAioClient from ray._private.runtime_env.plugin import ( RuntimeEnvPlugin, @@ -206,6 +207,7 @@ def __init__( ) self._pip_plugin = PipPlugin(self._runtime_env_dir) + self._uv_plugin = UvPlugin(self._runtime_env_dir) self._conda_plugin = CondaPlugin(self._runtime_env_dir) self._py_modules_plugin = PyModulesPlugin( self._runtime_env_dir, self._gcs_aio_client @@ -228,6 +230,7 @@ def __init__( # self._xxx_plugin, we should just iterate through self._plugins. self._base_plugins: List[RuntimeEnvPlugin] = [ self._working_dir_plugin, + self._uv_plugin, self._pip_plugin, self._conda_plugin, self._py_modules_plugin, diff --git a/python/ray/_private/runtime_env/packaging.py b/python/ray/_private/runtime_env/packaging.py index dd9212556c47..279c40139484 100644 --- a/python/ray/_private/runtime_env/packaging.py +++ b/python/ray/_private/runtime_env/packaging.py @@ -87,6 +87,7 @@ def __new__(cls, value, doc=None): GCS = "gcs", "For packages dynamically uploaded and managed by the GCS." CONDA = "conda", "For conda environments installed locally on each node." PIP = "pip", "For pip environments installed locally on each node." + UV = "uv", "For uv environments install locally on each node." HTTPS = "https", "Remote https path, assumes everything packed in one zip file." S3 = "s3", "Remote s3 path, assumes everything packed in one zip file." GS = "gs", "Remote google storage path, assumes everything packed in one zip file." diff --git a/python/ray/_private/runtime_env/uv.py b/python/ray/_private/runtime_env/uv.py index 5d0c5db980c9..3a934e3d69ad 100644 --- a/python/ray/_private/runtime_env/uv.py +++ b/python/ray/_private/runtime_env/uv.py @@ -1,21 +1,50 @@ """Util class to install packages via uv. """ -# TODO(hjiang): Implement `UvPlugin`, which is the counterpart for `PipPlugin`. - from typing import Dict, List, Optional -from asyncio import get_running_loop import os +import hashlib from ray._private.runtime_env import virtualenv_utils from ray._private.runtime_env import dependency_utils from ray._private.runtime_env.utils import check_output_cmd +from ray._private.runtime_env.plugin import RuntimeEnvPlugin +from ray._private.runtime_env.packaging import Protocol, parse_uri +from asyncio import create_task, get_running_loop import shutil import logging +import json +import asyncio import sys +from ray._private.utils import try_to_create_directory, get_directory_size_bytes default_logger = logging.getLogger(__name__) +def _get_uv_hash(uv_dict: Dict) -> str: + """Get a deterministic hash value for `uv` related runtime envs.""" + serialized_uv_spec = json.dumps(uv_dict, sort_keys=True) + hash_val = hashlib.sha1(serialized_uv_spec.encode("utf-8")).hexdigest() + return hash_val + + +def get_uri(runtime_env: Dict) -> Optional[str]: + """Return `"uv://"`, or None if no GC required.""" + uv = runtime_env.get("uv") + if uv is not None: + if isinstance(uv, dict): + uri = "uv://" + _get_uv_hash(uv_dict=uv) + elif isinstance(uv, list): + uri = "uv://" + _get_uv_hash(uv_dict=dict(packages=uv)) + else: + raise TypeError( + "uv field received by RuntimeEnvAgent must be " + f"list or dict, not {type(uv).__name__}." + ) + else: + uri = None + return uri + + class UvProcessor: def __init__( self, @@ -131,3 +160,128 @@ async def _run(self): def __await__(self): return self._run().__await__() + + +class UvPlugin(RuntimeEnvPlugin): + name = "uv" + + def __init__(self, resources_dir: str): + self._uv_resource_dir = os.path.join(resources_dir, "uv") + self._creating_task = {} + # Maps a URI to a lock that is used to prevent multiple concurrent + # installs of the same virtualenv, see #24513 + self._create_locks: Dict[str, asyncio.Lock] = {} + # Key: created hashes. Value: size of the uv dir. + self._created_hash_bytes: Dict[str, int] = {} + try_to_create_directory(self._uv_resource_dir) + + def _get_path_from_hash(self, hash_val: str) -> str: + """Generate a path from the hash of a uv spec. + + Example output: + /tmp/ray/session_2021-11-03_16-33-59_356303_41018/runtime_resources + /uv/ray-9a7972c3a75f55e976e620484f58410c920db091 + """ + return os.path.join(self._uv_resource_dir, hash_val) + + def get_uris(self, runtime_env: "RuntimeEnv") -> List[str]: # noqa: F821 + """Return the uv URI from the RuntimeEnv if it exists, else return [].""" + uv_uri = runtime_env.uv_uri() + if uv_uri: + return [uv_uri] + return [] + + def delete_uri( + self, uri: str, logger: Optional[logging.Logger] = default_logger + ) -> int: + """Delete URI and return the number of bytes deleted.""" + logger.info("Got request to delete uv URI %s", uri) + protocol, hash_val = parse_uri(uri) + if protocol != Protocol.UV: + raise ValueError( + "UvPlugin can only delete URIs with protocol " + f"uv. Received protocol {protocol}, URI {uri}" + ) + + # Cancel running create task. + task = self._creating_task.pop(hash_val, None) + if task is not None: + task.cancel() + + del self._created_hash_bytes[hash_val] + + uv_env_path = self._get_path_from_hash(hash_val) + local_dir_size = get_directory_size_bytes(uv_env_path) + del self._create_locks[uri] + try: + shutil.rmtree(uv_env_path) + except OSError as e: + logger.warning(f"Error when deleting uv env {uv_env_path}: {str(e)}") + return 0 + + return local_dir_size + + async def create( + self, + uri: str, + runtime_env: "RuntimeEnv", # noqa: F821 + context: "RuntimeEnvContext", # noqa: F821 + logger: Optional[logging.Logger] = default_logger, + ) -> int: + if not runtime_env.has_uv(): + return 0 + + protocol, hash_val = parse_uri(uri) + target_dir = self._get_path_from_hash(hash_val) + + async def _create_for_hash(): + await UvProcessor( + target_dir, + runtime_env, + logger, + ) + + loop = get_running_loop() + return await loop.run_in_executor( + None, get_directory_size_bytes, target_dir + ) + + if uri not in self._create_locks: + # async lock to prevent the same virtualenv being concurrently installed + self._create_locks[uri] = asyncio.Lock() + + async with self._create_locks[uri]: + if hash_val in self._created_hash_bytes: + return self._created_hash_bytes[hash_val] + self._creating_task[hash_val] = task = create_task(_create_for_hash()) + task.add_done_callback(lambda _: self._creating_task.pop(hash_val, None)) + uv_dir_bytes = await task + self._created_hash_bytes[hash_val] = uv_dir_bytes + return uv_dir_bytes + + def modify_context( + self, + uris: List[str], + runtime_env: "RuntimeEnv", # noqa: F821 + context: "RuntimeEnvContext", # noqa: F821 + logger: logging.Logger = default_logger, + ): + if not runtime_env.has_uv(): + return + # UvPlugin only uses a single URI. + uri = uris[0] + # Update py_executable. + protocol, hash_val = parse_uri(uri) + target_dir = self._get_path_from_hash(hash_val) + virtualenv_python = virtualenv_utils.get_virtualenv_python(target_dir) + + if not os.path.exists(virtualenv_python): + raise ValueError( + f"Local directory {target_dir} for URI {uri} does " + "not exist on the cluster. Something may have gone wrong while " + "installing the runtime_env `uv` packages." + ) + context.py_executable = virtualenv_python + context.command_prefix += virtualenv_utils.get_virtualenv_activate_command( + target_dir + ) diff --git a/python/ray/_private/runtime_env/validation.py b/python/ray/_private/runtime_env/validation.py index ac478df59203..86e4eee3d2e1 100644 --- a/python/ray/_private/runtime_env/validation.py +++ b/python/ray/_private/runtime_env/validation.py @@ -337,14 +337,13 @@ def parse_and_validate_env_vars(env_vars: Dict[str, str]) -> Optional[Dict[str, # Dictionary mapping runtime_env options with the function to parse and # validate them. -# -# TODO(hjiang): Expose `uv` related validation after implementation finished. OPTION_TO_VALIDATION_FN = { "py_modules": parse_and_validate_py_modules, "working_dir": parse_and_validate_working_dir, "excludes": parse_and_validate_excludes, "conda": parse_and_validate_conda, "pip": parse_and_validate_pip, + "uv": parse_and_validate_uv, "env_vars": parse_and_validate_env_vars, "container": parse_and_validate_container, } diff --git a/python/ray/runtime_env/runtime_env.py b/python/ray/runtime_env/runtime_env.py index ab91e04ef8cd..20ebc1793c4e 100644 --- a/python/ray/runtime_env/runtime_env.py +++ b/python/ray/runtime_env/runtime_env.py @@ -10,6 +10,7 @@ from ray._private.runtime_env.conda import get_uri as get_conda_uri from ray._private.runtime_env.pip import get_uri as get_pip_uri from ray._private.runtime_env.plugin_schema_manager import RuntimeEnvPluginSchemaManager +from ray._private.runtime_env.uv import get_uri as get_uv_uri from ray._private.runtime_env.validation import OPTION_TO_VALIDATION_FN from ray._private.thirdparty.dacite import from_dict from ray.core.generated.runtime_env_common_pb2 import ( @@ -148,7 +149,6 @@ def to_dict(self) -> Dict: ] = RuntimeEnvConfig.parse_and_validate_runtime_env_config -# TODO(hjiang): Expose `uv` related fields after implementation finished. @PublicAPI class RuntimeEnv(dict): """This class is used to define a runtime environment for a job, task, @@ -237,6 +237,8 @@ class MyClass: the package name "pip" in front of the ``pip_version`` to form the final requirement string, the syntax of a requirement specifier is defined in full in PEP 508. + uv: Either a list of pip packages, or a python dictionary that has one field: + 1) ``packages`` (required, List[str]). conda: Either the conda YAML config, the name of a local conda env (e.g., "pytorch_p36"), or the path to a conda environment.yaml file. @@ -272,6 +274,7 @@ class MyClass: "working_dir", "conda", "pip", + "uv", "container", "excludes", "env_vars", @@ -311,6 +314,7 @@ def __init__( _validate: bool = True, mpi: Optional[Dict] = None, image_uri: Optional[str] = None, + uv: Optional[List[str]] = None, **kwargs, ): super().__init__() @@ -322,6 +326,8 @@ def __init__( runtime_env["working_dir"] = working_dir if pip is not None: runtime_env["pip"] = pip + if uv is not None: + runtime_env["uv"] = uv if conda is not None: runtime_env["conda"] = conda if nsight is not None: @@ -349,16 +355,19 @@ def __init__( if not _validate: return - if self.get("conda") and self.get("pip"): + use_conda_install = int(self.get("conda") is not None) + use_pip_install = int(self.get("pip") is not None) + use_uv_install = int(self.get("uv") is not None) + if use_conda_install + use_pip_install + use_uv_install > 1: raise ValueError( - "The 'pip' field and 'conda' field of " - "runtime_env cannot both be specified.\n" - f"specified pip field: {self['pip']}\n" - f"specified conda field: {self['conda']}\n" - "To use pip with conda, please only set the 'conda' " - "field, and specify your pip dependencies " - "within the conda YAML config dict: see " - "https://conda.io/projects/conda/en/latest/" + "The 'pip' field, 'uv' field, and 'conda' field of " + "runtime_env cannot be specified at the same time.\n" + f"specified pip field: {self.get('pip')}\n" + f"specified conda field: {self.get('conda')}\n" + f"specified conda field: {self.get('uv')}\n" + "To use pip with conda or uv, please only set the 'conda' or 'uv'" + "field, and specify your pip dependencies within the conda YAML " + "config dict: see https://conda.io/projects/conda/en/latest/" "user-guide/tasks/manage-environments.html" "#create-env-file-manually" ) @@ -472,6 +481,11 @@ def pip_uri(self) -> Optional[str]: return get_pip_uri(self) return None + def uv_uri(self) -> Optional[str]: + if "uv" in self: + return get_uv_uri(self) + return None + def plugin_uris(self) -> List[str]: """Not implemented yet, always return a empty list""" return [] @@ -518,6 +532,11 @@ def has_pip(self) -> bool: return True return False + def has_uv(self) -> bool: + if self.get("uv"): + return True + return False + def virtualenv_name(self) -> Optional[str]: if not self.has_pip() or not isinstance(self["pip"], str): return None @@ -530,6 +549,13 @@ def pip_config(self) -> Dict: self["pip"] = self["pip"] return self["pip"] + def uv_config(self) -> Dict: + if not self.has_uv() or isinstance(self["uv"], str): + return {} + # Parse and validate field pip on method `__setitem__` + self["uv"] = self["uv"] + return self["uv"] + def get_extension(self, key) -> Optional[str]: if key not in RuntimeEnv.extensions_fields: raise ValueError( diff --git a/python/ray/tests/BUILD b/python/ray/tests/BUILD index 300279a883b8..016827389604 100644 --- a/python/ray/tests/BUILD +++ b/python/ray/tests/BUILD @@ -572,6 +572,7 @@ py_test_module_list( "test_runtime_env_conda_and_pip_3.py", "test_runtime_env_conda_and_pip_4.py", "test_runtime_env_conda_and_pip_5.py", + "test_runtime_uv.py", ], size = "large", tags = ["exclusive", "post_wheel_build", "team:core"], diff --git a/python/ray/tests/test_runtime_uv.py b/python/ray/tests/test_runtime_uv.py new file mode 100644 index 000000000000..780d952f0caf --- /dev/null +++ b/python/ray/tests/test_runtime_uv.py @@ -0,0 +1,64 @@ +# TODO(hjiang): A few unit tests to add after full functionality implemented. +# 1. Install specialized version of `uv`. +# 2. Options for `uv install`. +# 3. Use requirement files for packages. + +import os +import pytest +import sys + +from ray._private.runtime_env import virtualenv_utils +import ray + + +def test_uv_install_in_virtualenv(start_cluster): + assert ( + virtualenv_utils.is_in_virtualenv() is False + and "IN_VIRTUALENV" not in os.environ + ) or (virtualenv_utils.is_in_virtualenv() is True and "IN_VIRTUALENV" in os.environ) + cluster, address = start_cluster + runtime_env = {"pip": ["pip-install-test==0.5"]} + + ray.init(address, runtime_env=runtime_env) + + @ray.remote + def f(): + import pip_install_test # noqa: F401 + + return virtualenv_utils.is_in_virtualenv() + + # Ensure that the runtime env has been installed and virtualenv is activated. + assert ray.get(f.remote()) + + +# Package installation succeeds. +def test_package_install_with_uv(): + @ray.remote(runtime_env={"uv": {"packages": ["pip-install-test==0.5"]}}) + def f(): + import pip + + return pip.__version__ + + assert ray.get(f.remote()) == "24.1.2" + + +# Package installation fails due to conflict versions. +def test_package_install_has_conflict_with_uv(): + # moto require requests>=2.5 + conflict_packages = ["moto==3.0.5", "requests==2.4.0"] + + @ray.remote(runtime_env={"uv": {"packages": conflict_packages}}) + def f(): + import pip + + return pip.__version__ + + with pytest.raises(ray.exceptions.RuntimeEnvSetupError) as _: + ray.get(f.remote()) + + +if __name__ == "__main__": + if os.environ.get("PARALLEL_CI"): + sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) + else: + sys.exit(pytest.main(["-sv", __file__])) From c4eccea56e8f906e14796846e5df29dee9493a56 Mon Sep 17 00:00:00 2001 From: dentiny Date: Thu, 7 Nov 2024 19:51:50 +0000 Subject: [PATCH 02/14] rename test file Signed-off-by: dentiny --- python/ray/tests/BUILD | 2 +- python/ray/tests/{test_runtime_uv.py => test_runtime_env_uv.py} | 0 2 files changed, 1 insertion(+), 1 deletion(-) rename python/ray/tests/{test_runtime_uv.py => test_runtime_env_uv.py} (100%) diff --git a/python/ray/tests/BUILD b/python/ray/tests/BUILD index 016827389604..28cec5544278 100644 --- a/python/ray/tests/BUILD +++ b/python/ray/tests/BUILD @@ -572,7 +572,7 @@ py_test_module_list( "test_runtime_env_conda_and_pip_3.py", "test_runtime_env_conda_and_pip_4.py", "test_runtime_env_conda_and_pip_5.py", - "test_runtime_uv.py", + "test_runtime_env_uv.py", ], size = "large", tags = ["exclusive", "post_wheel_build", "team:core"], diff --git a/python/ray/tests/test_runtime_uv.py b/python/ray/tests/test_runtime_env_uv.py similarity index 100% rename from python/ray/tests/test_runtime_uv.py rename to python/ray/tests/test_runtime_env_uv.py From f329492bcc4f713dd9ed8ace7d786659530dcabc Mon Sep 17 00:00:00 2001 From: dentiny Date: Thu, 7 Nov 2024 20:25:28 +0000 Subject: [PATCH 03/14] simplify package manager check Signed-off-by: dentiny --- python/ray/runtime_env/runtime_env.py | 7 +++---- python/ray/tests/test_runtime_env_uv.py | 8 ++++---- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/python/ray/runtime_env/runtime_env.py b/python/ray/runtime_env/runtime_env.py index 20ebc1793c4e..9e16c751d84d 100644 --- a/python/ray/runtime_env/runtime_env.py +++ b/python/ray/runtime_env/runtime_env.py @@ -355,10 +355,9 @@ def __init__( if not _validate: return - use_conda_install = int(self.get("conda") is not None) - use_pip_install = int(self.get("pip") is not None) - use_uv_install = int(self.get("uv") is not None) - if use_conda_install + use_pip_install + use_uv_install > 1: + if (self.get("conda") is not None) + (self.get("pip") is not None) + ( + self.get("uv") is not None + ) > 1: raise ValueError( "The 'pip' field, 'uv' field, and 'conda' field of " "runtime_env cannot be specified at the same time.\n" diff --git a/python/ray/tests/test_runtime_env_uv.py b/python/ray/tests/test_runtime_env_uv.py index 780d952f0caf..6e530af112a4 100644 --- a/python/ray/tests/test_runtime_env_uv.py +++ b/python/ray/tests/test_runtime_env_uv.py @@ -33,13 +33,13 @@ def f(): # Package installation succeeds. def test_package_install_with_uv(): - @ray.remote(runtime_env={"uv": {"packages": ["pip-install-test==0.5"]}}) + @ray.remote(runtime_env={"uv": {"packages": ["requests==2.3.0"]}}) def f(): - import pip + import requests - return pip.__version__ + return requests.__version__ - assert ray.get(f.remote()) == "24.1.2" + assert ray.get(f.remote()) == "2.3.0" # Package installation fails due to conflict versions. From 481e0dac9103a08eebce2290baf58a70635ccf16 Mon Sep 17 00:00:00 2001 From: dentiny Date: Thu, 7 Nov 2024 20:30:23 +0000 Subject: [PATCH 04/14] update comments on uv and conda Signed-off-by: dentiny --- python/ray/runtime_env/runtime_env.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/runtime_env/runtime_env.py b/python/ray/runtime_env/runtime_env.py index 9e16c751d84d..f0dfe377be29 100644 --- a/python/ray/runtime_env/runtime_env.py +++ b/python/ray/runtime_env/runtime_env.py @@ -364,7 +364,7 @@ def __init__( f"specified pip field: {self.get('pip')}\n" f"specified conda field: {self.get('conda')}\n" f"specified conda field: {self.get('uv')}\n" - "To use pip with conda or uv, please only set the 'conda' or 'uv'" + "To use pip with conda, please only set the 'conda'" "field, and specify your pip dependencies within the conda YAML " "config dict: see https://conda.io/projects/conda/en/latest/" "user-guide/tasks/manage-environments.html" From 6d2092ec7f33cc2a0ac7ae4cbe21647b6cf79efe Mon Sep 17 00:00:00 2001 From: dentiny Date: Thu, 7 Nov 2024 21:12:04 +0000 Subject: [PATCH 05/14] Allow uv version Signed-off-by: dentiny --- python/ray/_private/runtime_env/uv.py | 10 +++++++++- python/ray/_private/runtime_env/validation.py | 12 ++++++++---- python/ray/tests/test_runtime_env_uv.py | 13 +++++++++++++ .../ray/tests/unit/test_runtime_env_validation.py | 6 ++++++ 4 files changed, 36 insertions(+), 5 deletions(-) diff --git a/python/ray/_private/runtime_env/uv.py b/python/ray/_private/runtime_env/uv.py index 3a934e3d69ad..97a7ce290dcf 100644 --- a/python/ray/_private/runtime_env/uv.py +++ b/python/ray/_private/runtime_env/uv.py @@ -79,6 +79,14 @@ async def _install_uv( virtualenv_path = virtualenv_utils.get_virtualenv_path(path) python = virtualenv_utils.get_virtualenv_python(path) + def _get_uv_exec_to_install(): + """Get `uv` executable with version to install.""" + uv_version = self._uv_config.get("uv_version", None) + if uv_version: + return f"uv{uv_version}" + # Use default version. + return "uv" + uv_install_cmd = [ python, "-m", @@ -86,7 +94,7 @@ async def _install_uv( "install", "--disable-pip-version-check", "--no-cache-dir", - "uv", + _get_uv_exec_to_install(), ] logger.info("Installing package uv to %s", virtualenv_path) await check_output_cmd(uv_install_cmd, logger=logger, cwd=cwd, env=pip_env) diff --git a/python/ray/_private/runtime_env/validation.py b/python/ray/_private/runtime_env/validation.py index 86e4eee3d2e1..9665b79004e4 100644 --- a/python/ray/_private/runtime_env/validation.py +++ b/python/ray/_private/runtime_env/validation.py @@ -110,8 +110,7 @@ def parse_and_validate_conda(conda: Union[str, dict]) -> Union[str, dict]: # TODO(hjiang): More package installation options to implement: # 1. Allow users to pass in a local requirements.txt file, which relates to all # packages to install; -# 2. Allow specific version of `uv` to use; as of now we only use default version. -# 3. `pip_check` has different semantics for `uv` and `pip`, see +# 2. `pip_check` has different semantics for `uv` and `pip`, see # https://github.com/astral-sh/uv/pull/2544/files, consider whether we need to support # it; or simply ignore the field when people come from `pip`. def parse_and_validate_uv(uv: Union[str, List[str], Dict]) -> Optional[Dict]: @@ -139,16 +138,21 @@ def parse_and_validate_uv(uv: Union[str, List[str], Dict]) -> Optional[Dict]: if isinstance(uv, list) and all(isinstance(dep, str) for dep in uv): result = dict(packages=uv) elif isinstance(uv, dict): - if set(uv.keys()) - {"packages"}: + if set(uv.keys()) - {"packages", "uv_version"}: raise ValueError( "runtime_env['uv'] can only have these fields: " - "packages, but got: " + "packages and uv_version, but got: " f"{list(uv.keys())}" ) if "packages" not in uv: raise ValueError( f"runtime_env['uv'] must include field 'packages', but got {uv}" ) + if "uv_version" in uv and not isinstance(uv["uv_version"], str): + raise TypeError( + "runtime_env['uv']['uv_version'] must be of type bool, " + f"got {type(uv['uv_version'])}" + ) result = uv.copy() if not isinstance(uv["packages"], list): diff --git a/python/ray/tests/test_runtime_env_uv.py b/python/ray/tests/test_runtime_env_uv.py index 6e530af112a4..ada396b97b7f 100644 --- a/python/ray/tests/test_runtime_env_uv.py +++ b/python/ray/tests/test_runtime_env_uv.py @@ -57,6 +57,19 @@ def f(): ray.get(f.remote()) +# Specify uv version. +def test_uv_with_version(): + @ray.remote( + runtime_env={"uv": {"packages": ["requests==2.3.0"], "uv_version": "==0.4.0"}} + ) + def f(): + import requests + + return requests.__version__ + + assert ray.get(f.remote()) == "2.3.0" + + if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) diff --git a/python/ray/tests/unit/test_runtime_env_validation.py b/python/ray/tests/unit/test_runtime_env_validation.py index 6676f6181106..041494aaaca3 100644 --- a/python/ray/tests/unit/test_runtime_env_validation.py +++ b/python/ray/tests/unit/test_runtime_env_validation.py @@ -54,6 +54,12 @@ def test_parse_and_validate_uv(self): with pytest.raises(ValueError): result = validation.parse_and_validate_uv({"random_key": "random_value"}) + # Valid case w/ uv version. + result = validation.parse_and_validate_uv( + {"packages": ["tensorflow"], "uv_version": "==0.4.30"} + ) + assert result == {"packages": ["tensorflow"], "uv_version": "==0.4.30"} + class TestValidatePip: def test_validate_pip_invalid_types(self): From 3e3c516890620bb200fb6933e0f0e54512fdedb8 Mon Sep 17 00:00:00 2001 From: dentiny Date: Fri, 8 Nov 2024 08:23:43 +0000 Subject: [PATCH 06/14] lint Signed-off-by: dentiny --- python/ray/tests/test_runtime_env_uv.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/python/ray/tests/test_runtime_env_uv.py b/python/ray/tests/test_runtime_env_uv.py index e3cd8b55d8ac..79bab0af375c 100644 --- a/python/ray/tests/test_runtime_env_uv.py +++ b/python/ray/tests/test_runtime_env_uv.py @@ -50,11 +50,8 @@ def f(): import pip return pip.__version__ - - with pytest.raises(ray.exceptions.RuntimeEnvSetupError): - ray.get(f.remote()) - with pytest.raises(ray.exceptions.RuntimeEnvSetupError) as _: + with pytest.raises(ray.exceptions.RuntimeEnvSetupError): ray.get(f.remote()) From 1394249f2d2b931197b09025be4bd624251f7c4a Mon Sep 17 00:00:00 2001 From: dentiny Date: Fri, 8 Nov 2024 20:28:40 +0000 Subject: [PATCH 07/14] fixture: shutdown only Signed-off-by: dentiny --- python/ray/tests/test_runtime_env_uv.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/tests/test_runtime_env_uv.py b/python/ray/tests/test_runtime_env_uv.py index ceed4c904105..fe6e217fab8a 100644 --- a/python/ray/tests/test_runtime_env_uv.py +++ b/python/ray/tests/test_runtime_env_uv.py @@ -70,7 +70,7 @@ def f(): # Specify uv version. -def test_uv_with_version(): +def test_uv_with_version(shutdown_only): @ray.remote( runtime_env={"uv": {"packages": ["requests==2.3.0"], "uv_version": "==0.4.0"}} ) From dc0daf5179a8e6b90d44e95ccfee232952927a83 Mon Sep 17 00:00:00 2001 From: dentiny Date: Sat, 9 Nov 2024 01:26:53 +0000 Subject: [PATCH 08/14] add extra check for uv version Signed-off-by: dentiny --- python/ray/tests/test_runtime_env_uv.py | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/python/ray/tests/test_runtime_env_uv.py b/python/ray/tests/test_runtime_env_uv.py index fe6e217fab8a..08616a3095b6 100644 --- a/python/ray/tests/test_runtime_env_uv.py +++ b/python/ray/tests/test_runtime_env_uv.py @@ -82,6 +82,19 @@ def f(): assert ray.get(f.remote()) == "2.3.0" +# Specify uv version and check. +def test_uv_with_version_and_check(shutdown_only): + @ray.remote( + runtime_env={"uv": {"packages": ["requests==2.3.0"], "uv_version": "==0.4.0"}} + ) + def f(): + import pkg_resources + + return pkg_resources.get_distribution("uv").version + + assert ray.get(f.remote()) == "0.4.0" + + # Package installation via requirements file. def test_package_install_with_requirements(shutdown_only, tmp_working_dir): requirements_file = tmp_working_dir From 538aa03ad1f851d25a3273e4dacf1a969fe44da8 Mon Sep 17 00:00:00 2001 From: dentiny Date: Sat, 9 Nov 2024 01:37:24 +0000 Subject: [PATCH 09/14] install specific version Signed-off-by: dentiny --- python/ray/_private/runtime_env/uv.py | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/python/ray/_private/runtime_env/uv.py b/python/ray/_private/runtime_env/uv.py index f62792e0d474..fb9730ea0d70 100644 --- a/python/ray/_private/runtime_env/uv.py +++ b/python/ray/_private/runtime_env/uv.py @@ -77,11 +77,13 @@ def __init__( async def _install_uv( self, path: str, cwd: str, pip_env: dict, logger: logging.Logger ): - """Before package install, make sure `uv` is installed.""" + """Before package install, make sure the required version `uv` (if specifieds) + is installed. + """ virtualenv_path = virtualenv_utils.get_virtualenv_path(path) python = virtualenv_utils.get_virtualenv_python(path) - def _get_uv_exec_to_install(): + def _get_uv_exec_to_install() -> str: """Get `uv` executable with version to install.""" uv_version = self._uv_config.get("uv_version", None) if uv_version: @@ -96,6 +98,7 @@ def _get_uv_exec_to_install(): "install", "--disable-pip-version-check", "--no-cache-dir", + "--force-reinstall", _get_uv_exec_to_install(), ] logger.info("Installing package uv to %s", virtualenv_path) @@ -139,7 +142,12 @@ async def _install_uv_packages( uv_exists = await self._check_uv_existence(python, cwd, pip_env, logger) # Install uv, which acts as the default package manager. - if not uv_exists: + # + # TODO(hjiang): If `uv` in virtual env perfectly matches the version users + # require, we don't need to install also. It requires a different + # implementation to execute and check existence. Here we take the simpliest + # implementation, always reinstall the required version. + if (not uv_exists) or (self._uv_config.get("uv_version", None) is not None): await self._install_uv(path, cwd, pip_env, logger) # Avoid blocking the event loop. From 49b2ecd31c8aa54cfa3f1216cb57d82fe8bc77af Mon Sep 17 00:00:00 2001 From: dentiny Date: Sat, 9 Nov 2024 01:41:31 +0000 Subject: [PATCH 10/14] remove useless TODO Signed-off-by: dentiny --- python/ray/_private/runtime_env/uv.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/python/ray/_private/runtime_env/uv.py b/python/ray/_private/runtime_env/uv.py index fb9730ea0d70..270fb1ad891e 100644 --- a/python/ray/_private/runtime_env/uv.py +++ b/python/ray/_private/runtime_env/uv.py @@ -72,8 +72,6 @@ def __init__( self._uv_env = os.environ.copy() self._uv_env.update(self._runtime_env.env_vars()) - # TODO(hjiang): Check `uv` existence before installation, so we don't blindly - # install. async def _install_uv( self, path: str, cwd: str, pip_env: dict, logger: logging.Logger ): From 7180447f66cd188781d04359dd41f84fc0a39098 Mon Sep 17 00:00:00 2001 From: dentiny Date: Sat, 9 Nov 2024 05:14:34 +0000 Subject: [PATCH 11/14] skip uv installation if version matches Signed-off-by: dentiny --- python/ray/_private/runtime_env/uv.py | 52 +++++++++++++++----- python/ray/tests/unit/test_runtime_env_uv.py | 12 ++++- 2 files changed, 51 insertions(+), 13 deletions(-) diff --git a/python/ray/_private/runtime_env/uv.py b/python/ray/_private/runtime_env/uv.py index 270fb1ad891e..c39d2d18c0d6 100644 --- a/python/ray/_private/runtime_env/uv.py +++ b/python/ray/_private/runtime_env/uv.py @@ -102,10 +102,12 @@ def _get_uv_exec_to_install() -> str: logger.info("Installing package uv to %s", virtualenv_path) await check_output_cmd(uv_install_cmd, logger=logger, cwd=cwd, env=pip_env) - async def _check_uv_existence( + async def _get_existing_uv_version( self, path: str, cwd: str, env: dict, logger: logging.Logger - ) -> bool: - """Check and return the existence of `uv` in virtual env.""" + ) -> Optional[str]: + """Get the version of `uv` in virtual env. + If not installed, return None. + """ python = virtualenv_utils.get_virtualenv_python(path) check_existence_cmd = [ @@ -117,11 +119,42 @@ async def _check_uv_existence( try: # If `uv` doesn't exist, exception will be thrown. - await check_output_cmd(check_existence_cmd, logger=logger, cwd=cwd, env=env) - return True + version_output = await check_output_cmd( + check_existence_cmd, logger=logger, cwd=cwd, env=env + ) + + # If exists, the output format would look like + # uv ( ), for example, + # uv 0.5.1 (f399a5271 2024-11-08) + version_strs = version_output.split() + if version_strs[0] == "uv": + return version_strs[1] + return None except Exception: + return None + + def _whether_to_install_uv(self, uv_version: Optional[str]) -> bool: + """Returns whether we need to re-install uv. + params: + uv_version: version for uv in virtual env; None if doesn't exist. + return: + whether need to (re)install uv. + """ + if uv_version is None: + return True + + # User doesn't specify uv version, so as long as we have uv it's fine. + required_uv = self._uv_config.get("uv_version", None) + if required_uv is None: + return False + + # Uv version in virtual environment perfectly matches user request. + if required_uv.endswith(uv_version): return False + # Version we have doesn't match with required one. + return True + async def _install_uv_packages( self, path: str, @@ -137,15 +170,10 @@ async def _install_uv_packages( requirements_file = dependency_utils.get_requirements_file(path, uv_packages) # Check existence for `uv` and see if we could skip `uv` installation. - uv_exists = await self._check_uv_existence(python, cwd, pip_env, logger) + uv_version = await self._get_existing_uv_version(python, cwd, pip_env, logger) # Install uv, which acts as the default package manager. - # - # TODO(hjiang): If `uv` in virtual env perfectly matches the version users - # require, we don't need to install also. It requires a different - # implementation to execute and check existence. Here we take the simpliest - # implementation, always reinstall the required version. - if (not uv_exists) or (self._uv_config.get("uv_version", None) is not None): + if self._whether_to_install_uv(uv_version): await self._install_uv(path, cwd, pip_env, logger) # Avoid blocking the event loop. diff --git a/python/ray/tests/unit/test_runtime_env_uv.py b/python/ray/tests/unit/test_runtime_env_uv.py index b4e210049003..600a31909050 100644 --- a/python/ray/tests/unit/test_runtime_env_uv.py +++ b/python/ray/tests/unit/test_runtime_env_uv.py @@ -7,7 +7,7 @@ class TestRuntimeEnv: def uv_config(self): - return {"packages": ["requests"]} + return {"packages": ["requests"], "uv_version": "==0.4.30"} def env_vars(self): return {} @@ -40,5 +40,15 @@ async def test_run(mock_install_uv, mock_install_uv_packages): await uv_processor._run() +def test_whether_to_install_uv(): + target_dir = "/tmp" + runtime_env = TestRuntimeEnv() + + uv_processor = uv.UvProcessor(target_dir=target_dir, runtime_env=runtime_env) + assert uv_processor._whether_to_install_uv(None) + assert uv_processor._whether_to_install_uv("0.0.1") + assert not uv_processor._whether_to_install_uv("0.4.30") + + if __name__ == "__main__": sys.exit(pytest.main(["-vv", __file__])) From 4288ae0d0713f47228bd293b3e2074a6115bb162 Mon Sep 17 00:00:00 2001 From: dentiny Date: Sat, 9 Nov 2024 20:50:39 +0000 Subject: [PATCH 12/14] fix invalid memory access Signed-off-by: dentiny --- python/ray/_private/runtime_env/uv.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/_private/runtime_env/uv.py b/python/ray/_private/runtime_env/uv.py index c39d2d18c0d6..7e1a7c363f59 100644 --- a/python/ray/_private/runtime_env/uv.py +++ b/python/ray/_private/runtime_env/uv.py @@ -127,7 +127,7 @@ async def _get_existing_uv_version( # uv ( ), for example, # uv 0.5.1 (f399a5271 2024-11-08) version_strs = version_output.split() - if version_strs[0] == "uv": + if len(version_strs) == 4 and version_strs[0] == "uv": return version_strs[1] return None except Exception: From ea430d763870f615ccb92b40099a73c806b259b0 Mon Sep 17 00:00:00 2001 From: dentiny Date: Tue, 12 Nov 2024 18:03:18 +0000 Subject: [PATCH 13/14] fix conflict Signed-off-by: dentiny --- python/ray/tests/test_runtime_env_uv.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/python/ray/tests/test_runtime_env_uv.py b/python/ray/tests/test_runtime_env_uv.py index 1488f396c950..bfcb4b57d060 100644 --- a/python/ray/tests/test_runtime_env_uv.py +++ b/python/ray/tests/test_runtime_env_uv.py @@ -85,12 +85,19 @@ def f(): # Package installation via requirements file. +def test_package_install_with_requirements(shutdown_only, tmp_working_dir): + requirements_file = tmp_working_dir + + @ray.remote(runtime_env={"uv": requirements_file}) + def f(): + import requests return requests.__version__ assert ray.get(f.remote()) == "2.3.0" +if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) else: From 0065573681bf435892438c257be5aa0d563653f3 Mon Sep 17 00:00:00 2001 From: dentiny Date: Tue, 12 Nov 2024 18:05:13 +0000 Subject: [PATCH 14/14] fix typo Signed-off-by: dentiny --- python/ray/_private/runtime_env/uv.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/_private/runtime_env/uv.py b/python/ray/_private/runtime_env/uv.py index 7e1a7c363f59..cbc4fc2a9f5a 100644 --- a/python/ray/_private/runtime_env/uv.py +++ b/python/ray/_private/runtime_env/uv.py @@ -75,7 +75,7 @@ def __init__( async def _install_uv( self, path: str, cwd: str, pip_env: dict, logger: logging.Logger ): - """Before package install, make sure the required version `uv` (if specifieds) + """Before package install, make sure the required version `uv` (if specified) is installed. """ virtualenv_path = virtualenv_utils.get_virtualenv_path(path)