mirror of
https://github.com/vale981/ray
synced 2025-03-06 02:21:39 -05:00
[runtime_env] Make pip installs incremental (#20341)
Uses a direct `pip install` instead of creating a conda env to make pip installs incremental to the cluster environment. Separates the handling of `pip` and `conda` dependencies. The new `pip` approach still works if only the base Ray is installed on the cluster and the user specifies libraries like "ray[serve]" in the `pip` field. The mechanism is as follows: - We don't actually want to reinstall ray via pip, since this could lead to version mismatch issues. Instead, we want to use the Ray that's already installed in the cluster. - So if "ray" was included by the user in the pip list, remove it - If a library "ray[serve]" or "ray[tune, rllib]" was included in the pip list, remove it and replace it by its dependencies (e.g. "uvicorn", "requests", ..) Co-authored-by: architkulkarni <arkulkar@gmail.com> Co-authored-by: architkulkarni <architkulkarni@users.noreply.github.com>
This commit is contained in:
parent
57cc76cf5e
commit
10947c83b3
17 changed files with 374 additions and 118 deletions
|
@ -183,7 +183,6 @@ test_python() {
|
|||
-python/ray/tests:test_runtime_env_plugin # runtime_env not supported on Windows
|
||||
-python/ray/tests:test_runtime_env_env_vars # runtime_env not supported on Windows
|
||||
-python/ray/tests:test_runtime_env_complicated # conda install slow leading to timeout
|
||||
-python/ray/tests:test_runtime_env_conda # conda not supported on Windows
|
||||
-python/ray/tests:test_stress # timeout
|
||||
-python/ray/tests:test_stress_sharded # timeout
|
||||
-python/ray/tests:test_k8s_operator_unit_tests
|
||||
|
|
|
@ -17,6 +17,7 @@ import ray.dashboard.modules.runtime_env.runtime_env_consts \
|
|||
from ray.experimental.internal_kv import _internal_kv_initialized, \
|
||||
_initialize_internal_kv
|
||||
from ray._private.ray_logging import setup_component_logger
|
||||
from ray._private.runtime_env.pip import PipManager
|
||||
from ray._private.runtime_env.conda import CondaManager
|
||||
from ray._private.runtime_env.context import RuntimeEnvContext
|
||||
from ray._private.runtime_env.py_modules import PyModulesManager
|
||||
|
@ -68,6 +69,7 @@ class RuntimeEnvAgent(dashboard_utils.DashboardAgentModule,
|
|||
_initialize_internal_kv(self._dashboard_agent.gcs_client)
|
||||
assert _internal_kv_initialized()
|
||||
|
||||
self._pip_manager = PipManager(self._runtime_env_dir)
|
||||
self._conda_manager = CondaManager(self._runtime_env_dir)
|
||||
self._py_modules_manager = PyModulesManager(self._runtime_env_dir)
|
||||
self._working_dir_manager = WorkingDirManager(self._runtime_env_dir)
|
||||
|
@ -100,6 +102,8 @@ class RuntimeEnvAgent(dashboard_utils.DashboardAgentModule,
|
|||
per_job_logger.debug(f"Worker has resource :"
|
||||
f"{allocated_resource}")
|
||||
context = RuntimeEnvContext(env_vars=runtime_env.env_vars())
|
||||
self._pip_manager.setup(
|
||||
runtime_env, context, logger=per_job_logger)
|
||||
self._conda_manager.setup(
|
||||
runtime_env, context, logger=per_job_logger)
|
||||
self._py_modules_manager.setup(
|
||||
|
@ -120,6 +124,9 @@ class RuntimeEnvAgent(dashboard_utils.DashboardAgentModule,
|
|||
if runtime_env.conda_uri():
|
||||
uri = runtime_env.conda_uri()
|
||||
self._uris_to_envs[uri].add(serialized_runtime_env)
|
||||
if runtime_env.pip_uri():
|
||||
uri = runtime_env.pip_uri()
|
||||
self._uris_to_envs[uri].add(serialized_runtime_env)
|
||||
if runtime_env.plugin_uris():
|
||||
for uri in runtime_env.plugin_uris():
|
||||
self._uris_to_envs[uri].add(serialized_runtime_env)
|
||||
|
@ -226,6 +233,9 @@ class RuntimeEnvAgent(dashboard_utils.DashboardAgentModule,
|
|||
elif plugin == "conda":
|
||||
if not self._conda_manager.delete_uri(uri):
|
||||
failed_uris.append(uri)
|
||||
elif plugin == "pip":
|
||||
if not self._pip_manager.delete_uri(uri):
|
||||
failed_uris.append(uri)
|
||||
else:
|
||||
raise ValueError(
|
||||
"RuntimeEnvAgent received DeleteURI request "
|
||||
|
|
|
@ -25,21 +25,33 @@ from ray._private.runtime_env.packaging import Protocol, parse_uri
|
|||
default_logger = logging.getLogger(__name__)
|
||||
|
||||
|
||||
def _resolve_current_ray_path():
|
||||
def _resolve_current_ray_path() -> str:
|
||||
# When ray is built from source with pip install -e,
|
||||
# ray.__file__ returns .../python/ray/__init__.py.
|
||||
# When ray is installed from a prebuilt binary, it returns
|
||||
# .../site-packages/ray/__init__.py
|
||||
# ray.__file__ returns .../python/ray/__init__.py and this function returns
|
||||
# ".../python".
|
||||
# When ray is installed from a prebuilt binary, ray.__file__ returns
|
||||
# .../site-packages/ray/__init__.py and this function returns
|
||||
# ".../site-packages".
|
||||
return os.path.split(os.path.split(ray.__file__)[0])[0]
|
||||
|
||||
|
||||
def _resolve_install_from_source_ray_dependencies():
|
||||
"""Find the ray dependencies when Ray is install from source"""
|
||||
def _get_ray_setup_spec():
|
||||
"""Find the Ray setup_spec from the currently running Ray.
|
||||
|
||||
This function works even when Ray is built from source with pip install -e.
|
||||
"""
|
||||
ray_source_python_path = _resolve_current_ray_path()
|
||||
setup_py_path = os.path.join(ray_source_python_path, "setup.py")
|
||||
ray_install_requires = runpy.run_path(setup_py_path)[
|
||||
"setup_spec"].install_requires
|
||||
return ray_install_requires
|
||||
return runpy.run_path(setup_py_path)["setup_spec"]
|
||||
|
||||
|
||||
def _resolve_install_from_source_ray_dependencies():
|
||||
"""Find the Ray dependencies when Ray is installed from source."""
|
||||
return _get_ray_setup_spec().install_requires
|
||||
|
||||
|
||||
def _resolve_install_from_source_ray_extras() -> Dict[str, List[str]]:
|
||||
return _get_ray_setup_spec().extras
|
||||
|
||||
|
||||
def _inject_ray_to_conda_site(
|
||||
|
@ -69,43 +81,6 @@ def _current_py_version():
|
|||
return ".".join(map(str, sys.version_info[:3])) # like 3.6.10
|
||||
|
||||
|
||||
def get_conda_dict(runtime_env, resources_dir) -> Optional[Dict[Any, Any]]:
|
||||
""" Construct a conda dependencies dict from a runtime env.
|
||||
|
||||
This function does not inject Ray or Python into the conda dict.
|
||||
If the runtime env does not specify pip or conda, or if it specifies
|
||||
the name of a preinstalled conda environment, this function returns
|
||||
None. If pip is specified, a conda dict is created containing the
|
||||
pip dependencies. If conda is already given as a dict, this function
|
||||
is the identity function.
|
||||
"""
|
||||
if runtime_env.has_conda():
|
||||
if runtime_env.conda_config():
|
||||
return json.loads(runtime_env.conda_config())
|
||||
else:
|
||||
return None
|
||||
if runtime_env.has_pip():
|
||||
requirements_txt = "\n".join(runtime_env.pip_packages()) + "\n"
|
||||
pip_hash = hashlib.sha1(requirements_txt.encode("utf-8")).hexdigest()
|
||||
pip_hash_str = f"pip-generated-{pip_hash}"
|
||||
|
||||
requirements_txt_path = os.path.join(
|
||||
resources_dir, f"requirements-{pip_hash_str}.txt")
|
||||
conda_dict = {
|
||||
"name": pip_hash_str,
|
||||
"dependencies": ["pip", {
|
||||
"pip": [f"-r {requirements_txt_path}"]
|
||||
}]
|
||||
}
|
||||
file_lock_name = f"ray-{pip_hash_str}.lock"
|
||||
with FileLock(os.path.join(resources_dir, file_lock_name)):
|
||||
try_to_create_directory(resources_dir)
|
||||
with open(requirements_txt_path, "w") as file:
|
||||
file.write(requirements_txt)
|
||||
return conda_dict
|
||||
return None
|
||||
|
||||
|
||||
def current_ray_pip_specifier(
|
||||
logger: Optional[logging.Logger] = default_logger) -> Optional[str]:
|
||||
"""The pip requirement specifier for the running version of Ray.
|
||||
|
@ -199,16 +174,9 @@ def _get_conda_env_hash(conda_dict: Dict) -> str:
|
|||
return hash
|
||||
|
||||
|
||||
def _get_pip_hash(pip_list: List[str]) -> str:
|
||||
serialized_pip_spec = json.dumps(pip_list)
|
||||
hash = hashlib.sha1(serialized_pip_spec.encode("utf-8")).hexdigest()
|
||||
return hash
|
||||
|
||||
|
||||
def get_uri(runtime_env: Dict) -> Optional[str]:
|
||||
"""Return `"conda://<hashed_dependencies>"`, or None if no GC required."""
|
||||
conda = runtime_env.get("conda")
|
||||
pip = runtime_env.get("pip")
|
||||
if conda is not None:
|
||||
if isinstance(conda, str):
|
||||
# User-preinstalled conda env. We don't garbage collect these, so
|
||||
|
@ -219,12 +187,6 @@ def get_uri(runtime_env: Dict) -> Optional[str]:
|
|||
else:
|
||||
raise TypeError("conda field received by RuntimeEnvAgent must be "
|
||||
f"str or dict, not {type(conda).__name__}.")
|
||||
elif pip is not None:
|
||||
if isinstance(pip, list):
|
||||
uri = "conda://" + _get_pip_hash(pip_list=pip)
|
||||
else:
|
||||
raise TypeError("pip field received by RuntimeEnvAgent must be "
|
||||
f"list, not {type(pip).__name__}.")
|
||||
else:
|
||||
uri = None
|
||||
return uri
|
||||
|
@ -233,8 +195,7 @@ def get_uri(runtime_env: Dict) -> Optional[str]:
|
|||
class CondaManager:
|
||||
def __init__(self, resources_dir: str):
|
||||
self._resources_dir = os.path.join(resources_dir, "conda")
|
||||
if not os.path.isdir(self._resources_dir):
|
||||
os.makedirs(self._resources_dir)
|
||||
try_to_create_directory(self._resources_dir)
|
||||
self._created_envs: Set[str] = set()
|
||||
|
||||
def _get_path_from_hash(self, hash: str) -> str:
|
||||
|
@ -252,7 +213,7 @@ class CondaManager:
|
|||
def delete_uri(self,
|
||||
uri: str,
|
||||
logger: Optional[logging.Logger] = default_logger) -> bool:
|
||||
logger.debug(f"Got request to delete URI {uri}")
|
||||
logger.info(f"Got request to delete URI {uri}")
|
||||
protocol, hash = parse_uri(uri)
|
||||
if protocol != Protocol.CONDA:
|
||||
raise ValueError(
|
||||
|
@ -263,14 +224,14 @@ class CondaManager:
|
|||
self._created_envs.remove(conda_env_path)
|
||||
successful = delete_conda_env(prefix=conda_env_path, logger=logger)
|
||||
if not successful:
|
||||
logger.debug(f"Error when deleting conda env {conda_env_path}. ")
|
||||
logger.warning(f"Error when deleting conda env {conda_env_path}. ")
|
||||
return successful
|
||||
|
||||
def setup(self,
|
||||
runtime_env: RuntimeEnv,
|
||||
context: RuntimeEnvContext,
|
||||
logger: Optional[logging.Logger] = default_logger):
|
||||
if not runtime_env.has_conda() and not runtime_env.has_pip():
|
||||
if not runtime_env.has_conda():
|
||||
return
|
||||
|
||||
logger.debug("Setting up conda or pip for runtime_env: "
|
||||
|
@ -279,7 +240,7 @@ class CondaManager:
|
|||
if runtime_env.conda_env_name():
|
||||
conda_env_name = runtime_env.conda_env_name()
|
||||
else:
|
||||
conda_dict = get_conda_dict(runtime_env, self._resources_dir)
|
||||
conda_dict = json.loads(runtime_env.conda_config())
|
||||
protocol, hash = parse_uri(runtime_env.conda_uri())
|
||||
conda_env_name = self._get_path_from_hash(hash)
|
||||
assert conda_dict is not None
|
||||
|
|
|
@ -40,6 +40,7 @@ class Protocol(Enum):
|
|||
|
||||
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."
|
||||
HTTPS = "https", ("Remote https path, "
|
||||
"assumes everything packed in one zip file.")
|
||||
S3 = "s3", "Remote s3 path, assumes everything packed in one zip file."
|
||||
|
|
118
python/ray/_private/runtime_env/pip.py
Normal file
118
python/ray/_private/runtime_env/pip.py
Normal file
|
@ -0,0 +1,118 @@
|
|||
import os
|
||||
import json
|
||||
import logging
|
||||
import hashlib
|
||||
import shutil
|
||||
|
||||
from pathlib import Path
|
||||
from typing import Optional, List, Dict
|
||||
|
||||
from ray._private.runtime_env.conda_utils import exec_cmd_stream_to_logger
|
||||
from ray._private.runtime_env.context import RuntimeEnvContext
|
||||
from ray._private.runtime_env.packaging import Protocol, parse_uri
|
||||
from ray._private.runtime_env.utils import RuntimeEnv
|
||||
from ray._private.utils import try_to_create_directory
|
||||
|
||||
default_logger = logging.getLogger(__name__)
|
||||
|
||||
RAY_RUNTIME_ENV_ALLOW_RAY_IN_PIP = "RAY_RUNTIME_ENV_ALLOW_RAY_IN_PIP"
|
||||
|
||||
|
||||
def _get_pip_hash(pip_list: List[str]) -> str:
|
||||
serialized_pip_spec = json.dumps(pip_list, sort_keys=True)
|
||||
hash = hashlib.sha1(serialized_pip_spec.encode("utf-8")).hexdigest()
|
||||
return hash
|
||||
|
||||
|
||||
def _install_pip_list_to_dir(
|
||||
pip_list: List[str],
|
||||
target_dir: str,
|
||||
logger: Optional[logging.Logger] = default_logger):
|
||||
try_to_create_directory(target_dir)
|
||||
exit_code, output = exec_cmd_stream_to_logger(
|
||||
["pip", "install", f"--target={target_dir}"] + pip_list, logger)
|
||||
if exit_code != 0:
|
||||
shutil.rmtree(target_dir)
|
||||
raise RuntimeError(f"Failed to install pip requirements:\n{output}")
|
||||
|
||||
|
||||
def get_uri(runtime_env: Dict) -> Optional[str]:
|
||||
"""Return `"pip://<hashed_dependencies>"`, or None if no GC required."""
|
||||
pip = runtime_env.get("pip")
|
||||
if pip is not None:
|
||||
if isinstance(pip, list):
|
||||
uri = "pip://" + _get_pip_hash(pip_list=pip)
|
||||
else:
|
||||
raise TypeError("pip field received by RuntimeEnvAgent must be "
|
||||
f"list, not {type(pip).__name__}.")
|
||||
else:
|
||||
uri = None
|
||||
return uri
|
||||
|
||||
|
||||
class PipManager:
|
||||
def __init__(self, resources_dir: str):
|
||||
self._resources_dir = os.path.join(resources_dir, "pip")
|
||||
try_to_create_directory(self._resources_dir)
|
||||
|
||||
def _get_path_from_hash(self, hash: str) -> str:
|
||||
"""Generate a path from the hash of a pip spec.
|
||||
|
||||
Example output:
|
||||
/tmp/ray/session_2021-11-03_16-33-59_356303_41018/runtime_resources
|
||||
/pip/ray-9a7972c3a75f55e976e620484f58410c920db091
|
||||
"""
|
||||
return os.path.join(self._resources_dir, hash)
|
||||
|
||||
def delete_uri(self,
|
||||
uri: str,
|
||||
logger: Optional[logging.Logger] = default_logger) -> bool:
|
||||
logger.info(f"Got request to delete URI {uri}")
|
||||
protocol, hash = parse_uri(uri)
|
||||
if protocol != Protocol.PIP:
|
||||
raise ValueError("PipManager can only delete URIs with protocol "
|
||||
f"pip. Received protocol {protocol}, URI {uri}")
|
||||
|
||||
pip_env_path = self._get_path_from_hash(hash)
|
||||
try:
|
||||
shutil.rmtree(pip_env_path)
|
||||
successful = True
|
||||
except OSError:
|
||||
successful = False
|
||||
logger.warning(f"Error when deleting pip env {pip_env_path}.")
|
||||
return successful
|
||||
|
||||
def setup(self,
|
||||
runtime_env: RuntimeEnv,
|
||||
context: RuntimeEnvContext,
|
||||
logger: Optional[logging.Logger] = default_logger):
|
||||
if not runtime_env.has_pip():
|
||||
return
|
||||
|
||||
logger.debug(f"Setting up pip for runtime_env: {runtime_env}")
|
||||
pip_packages: List[str] = runtime_env.pip_packages()
|
||||
target_dir = self._get_path_from_hash(_get_pip_hash(pip_packages))
|
||||
|
||||
_install_pip_list_to_dir(pip_packages, target_dir, logger=logger)
|
||||
|
||||
# Despite Ray being removed from the input pip list during validation,
|
||||
# other packages in the pip list (for example, xgboost_ray) may
|
||||
# themselves include Ray as a dependency. In this case, we will have
|
||||
# inadvertently installed the latest Ray version in the target_dir,
|
||||
# which may cause Ray version mismatch issues. Uninstall it here, if it
|
||||
# exists, to make the workers use the Ray that is already
|
||||
# installed in the cluster.
|
||||
#
|
||||
# In the case where the user explicitly wants to include Ray in their
|
||||
# pip list (and signals this by setting the environment variable below)
|
||||
# then we don't want this deletion logic, so we skip it.
|
||||
if os.environ.get(RAY_RUNTIME_ENV_ALLOW_RAY_IN_PIP) != 1:
|
||||
ray_path = Path(target_dir) / "ray"
|
||||
if ray_path.exists() and ray_path.is_dir():
|
||||
shutil.rmtree(ray_path)
|
||||
|
||||
# Insert the target directory into the PYTHONPATH.
|
||||
python_path = target_dir
|
||||
if "PYTHONPATH" in context.env_vars:
|
||||
python_path += os.pathsep + context.env_vars["PYTHONPATH"]
|
||||
context.env_vars["PYTHONPATH"] = python_path
|
|
@ -10,6 +10,7 @@ from ray._private.runtime_env.packaging import (
|
|||
download_and_unpack_package, delete_package, get_uri_for_directory,
|
||||
parse_uri, Protocol, upload_package_if_needed)
|
||||
from ray._private.runtime_env.utils import RuntimeEnv
|
||||
from ray._private.utils import try_to_create_directory
|
||||
|
||||
default_logger = logging.getLogger(__name__)
|
||||
|
||||
|
@ -89,8 +90,7 @@ def upload_py_modules_if_needed(
|
|||
class PyModulesManager:
|
||||
def __init__(self, resources_dir: str):
|
||||
self._resources_dir = os.path.join(resources_dir, "py_modules_files")
|
||||
if not os.path.isdir(self._resources_dir):
|
||||
os.makedirs(self._resources_dir)
|
||||
try_to_create_directory(self._resources_dir)
|
||||
assert _internal_kv_initialized()
|
||||
|
||||
def delete_uri(self,
|
||||
|
|
|
@ -135,6 +135,7 @@ class RuntimeEnv:
|
|||
if uris.working_dir_uri \
|
||||
or uris.py_modules_uris \
|
||||
or uris.conda_uri \
|
||||
or uris.pip_uri \
|
||||
or uris.plugin_uris:
|
||||
return True
|
||||
return False
|
||||
|
@ -148,6 +149,9 @@ class RuntimeEnv:
|
|||
def conda_uri(self) -> str:
|
||||
return self._proto_runtime_env.uris.conda_uri
|
||||
|
||||
def pip_uri(self) -> str:
|
||||
return self._proto_runtime_env.uris.pip_uri
|
||||
|
||||
def plugin_uris(self) -> List[str]:
|
||||
return list(self._proto_runtime_env.uris.plugin_uris)
|
||||
|
||||
|
@ -216,8 +220,8 @@ class RuntimeEnv:
|
|||
self._proto_runtime_env.py_container_runtime_env.run_options)
|
||||
|
||||
@classmethod
|
||||
def from_dict(cls, runtime_env_dict: Dict[str, Any],
|
||||
conda_get_uri_fn) -> "RuntimeEnv":
|
||||
def from_dict(cls, runtime_env_dict: Dict[str, Any], conda_get_uri_fn,
|
||||
pip_get_uri_fn) -> "RuntimeEnv":
|
||||
proto_runtime_env = ProtoRuntimeEnv()
|
||||
proto_runtime_env.py_modules.extend(
|
||||
runtime_env_dict.get("py_modules", []))
|
||||
|
@ -228,10 +232,14 @@ class RuntimeEnv:
|
|||
if "py_modules" in runtime_env_dict:
|
||||
for uri in runtime_env_dict["py_modules"]:
|
||||
proto_runtime_env.uris.py_modules_uris.append(uri)
|
||||
if "conda" or "pip" in runtime_env_dict:
|
||||
if "conda" in runtime_env_dict:
|
||||
uri = conda_get_uri_fn(runtime_env_dict)
|
||||
if uri is not None:
|
||||
proto_runtime_env.uris.conda_uri = uri
|
||||
if "pip" in runtime_env_dict:
|
||||
uri = pip_get_uri_fn(runtime_env_dict)
|
||||
if uri is not None:
|
||||
proto_runtime_env.uris.pip_uri = uri
|
||||
env_vars = runtime_env_dict.get("env_vars", {})
|
||||
proto_runtime_env.env_vars.update(env_vars.items())
|
||||
if "_ray_release" in runtime_env_dict:
|
||||
|
|
|
@ -3,6 +3,8 @@ import os
|
|||
from pathlib import Path
|
||||
import sys
|
||||
from typing import Any, Dict, List, Optional, Set, Union
|
||||
from pkg_resources import Requirement
|
||||
from collections import OrderedDict
|
||||
import yaml
|
||||
|
||||
import ray
|
||||
|
@ -10,7 +12,11 @@ from ray._private.runtime_env.plugin import (RuntimeEnvPlugin,
|
|||
encode_plugin_uri)
|
||||
from ray._private.runtime_env.utils import RuntimeEnv
|
||||
from ray._private.utils import import_attr
|
||||
from ray._private.runtime_env import conda
|
||||
from ray._private.runtime_env.conda import (
|
||||
_resolve_install_from_source_ray_extras, get_uri as get_conda_uri)
|
||||
from ray._private.runtime_env.pip import RAY_RUNTIME_ENV_ALLOW_RAY_IN_PIP
|
||||
|
||||
from ray._private.runtime_env.pip import get_uri as get_pip_uri
|
||||
|
||||
logger = logging.getLogger(__name__)
|
||||
|
||||
|
@ -103,17 +109,57 @@ def parse_and_validate_conda(conda: Union[str, dict]) -> Union[str, dict]:
|
|||
return result
|
||||
|
||||
|
||||
def _rewrite_pip_list_ray_libraries(pip_list: List[str]) -> List[str]:
|
||||
result = []
|
||||
for specifier in pip_list:
|
||||
requirement = Requirement.parse(specifier)
|
||||
package_name = requirement.name
|
||||
if package_name == "ray":
|
||||
libraries = requirement.extras # e.g. ("serve", "tune")
|
||||
if libraries == ():
|
||||
# Ray alone was specified (e.g. "ray" or "ray>1.4"). Remove it.
|
||||
if os.environ.get(RAY_RUNTIME_ENV_ALLOW_RAY_IN_PIP) != "1":
|
||||
logger.warning(
|
||||
"Ray was specified in the `pip` field of the "
|
||||
f"`runtime_env`: '{specifier}'. This is not needed; "
|
||||
"Ray is already installed on the cluster, so that Ray"
|
||||
"installation will be used. To prevent Ray version "
|
||||
f"incompatibility issues, '{specifier}' has been "
|
||||
"deleted from the `pip` field. To disable this "
|
||||
"deletion, set the environment variable "
|
||||
f"{RAY_RUNTIME_ENV_ALLOW_RAY_IN_PIP} to 1.")
|
||||
else:
|
||||
result.append(specifier)
|
||||
else:
|
||||
# Replace the library with its dependencies.
|
||||
extras = _resolve_install_from_source_ray_extras()
|
||||
for library in libraries:
|
||||
result += extras[library]
|
||||
else:
|
||||
# Pass through all non-Ray packages unmodified.
|
||||
result.append(specifier)
|
||||
return result
|
||||
|
||||
|
||||
def parse_and_validate_pip(pip: Union[str, List[str]]) -> Optional[List[str]]:
|
||||
"""Parses and validates a user-provided 'pip' option.
|
||||
|
||||
Conda can be one of two cases:
|
||||
The value of the input 'pip' field can be one of two cases:
|
||||
1) A List[str] describing the requirements. This is passed through.
|
||||
2) A string pointing to a local requirements file. In this case, the
|
||||
file contents will be read split into a list.
|
||||
|
||||
The returned parsed value will be a list of pip packages. If a Ray library
|
||||
(e.g. "ray[serve]") is specified, it will be deleted and replaced by its
|
||||
dependencies (e.g. "uvicorn", "requests").
|
||||
|
||||
If the base Ray package (e.g. "ray>1.4" or "ray") is specified in the
|
||||
input, it will be removed, unless the environment variable
|
||||
RAY_RUNTIME_ENV_ALLOW_RAY_IN_PIP is set to 1.
|
||||
"""
|
||||
assert pip is not None
|
||||
|
||||
result = None
|
||||
pip_list = None
|
||||
if sys.platform == "win32":
|
||||
raise NotImplementedError("The 'pip' field in runtime_env "
|
||||
"is not currently supported on "
|
||||
|
@ -123,16 +169,26 @@ def parse_and_validate_pip(pip: Union[str, List[str]]) -> Optional[List[str]]:
|
|||
pip_file = Path(pip)
|
||||
if not pip_file.is_file():
|
||||
raise ValueError(f"{pip_file} is not a valid file")
|
||||
result = pip_file.read_text().strip().split("\n")
|
||||
pip_list = pip_file.read_text().strip().split("\n")
|
||||
elif isinstance(pip, list) and all(isinstance(dep, str) for dep in pip):
|
||||
if len(pip) == 0:
|
||||
result = None
|
||||
else:
|
||||
result = pip
|
||||
pip_list = pip
|
||||
else:
|
||||
raise TypeError("runtime_env['pip'] must be of type str or "
|
||||
f"List[str], got {type(pip)}")
|
||||
|
||||
result = _rewrite_pip_list_ray_libraries(pip_list)
|
||||
|
||||
# Eliminate duplicates to prevent `pip install` from erroring. Use
|
||||
# OrderedDict to preserve the order of the list. This makes the output
|
||||
# deterministic and easier to debug, because pip install can have
|
||||
# different behavior depending on the order of the input.
|
||||
result = list(OrderedDict.fromkeys(result))
|
||||
|
||||
if len(result) == 0:
|
||||
result = None
|
||||
|
||||
logger.debug(f"Rewrote runtime_env `pip` field from {pip} to {result}.")
|
||||
|
||||
return result
|
||||
|
||||
|
||||
|
@ -345,17 +401,22 @@ class ParsedRuntimeEnv(dict):
|
|||
if "py_modules" in self:
|
||||
for uri in self["py_modules"]:
|
||||
plugin_uris.append(encode_plugin_uri("py_modules", uri))
|
||||
if "conda" or "pip" in self:
|
||||
uri = conda.get_uri(self)
|
||||
if "conda" in self:
|
||||
uri = get_conda_uri(self)
|
||||
if uri is not None:
|
||||
plugin_uris.append(encode_plugin_uri("conda", uri))
|
||||
if "pip" in self:
|
||||
uri = get_pip_uri(self)
|
||||
if uri is not None:
|
||||
plugin_uris.append(encode_plugin_uri("pip", uri))
|
||||
|
||||
return plugin_uris
|
||||
|
||||
def get_proto_runtime_env(self):
|
||||
"""Return the protobuf structure of runtime env."""
|
||||
if self._cached_pb is None:
|
||||
self._cached_pb = RuntimeEnv.from_dict(self, conda.get_uri)
|
||||
self._cached_pb = RuntimeEnv.from_dict(self, get_conda_uri,
|
||||
get_pip_uri)
|
||||
|
||||
return self._cached_pb
|
||||
|
||||
|
|
|
@ -9,6 +9,7 @@ from ray._private.runtime_env.context import RuntimeEnvContext
|
|||
from ray._private.runtime_env.packaging import (
|
||||
download_and_unpack_package, delete_package, get_uri_for_directory,
|
||||
parse_uri, Protocol, upload_package_if_needed)
|
||||
from ray._private.utils import try_to_create_directory
|
||||
|
||||
default_logger = logging.getLogger(__name__)
|
||||
|
||||
|
@ -61,8 +62,7 @@ def upload_working_dir_if_needed(
|
|||
class WorkingDirManager:
|
||||
def __init__(self, resources_dir: str):
|
||||
self._resources_dir = os.path.join(resources_dir, "working_dir_files")
|
||||
if not os.path.isdir(self._resources_dir):
|
||||
os.makedirs(self._resources_dir)
|
||||
try_to_create_directory(self._resources_dir)
|
||||
assert _internal_kv_initialized()
|
||||
|
||||
def delete_uri(self,
|
||||
|
|
|
@ -1076,3 +1076,26 @@ def chdir(d: str):
|
|||
os.chdir(d)
|
||||
yield
|
||||
os.chdir(old_dir)
|
||||
|
||||
|
||||
def generate_runtime_env_dict(field, spec_format, tmp_path, pip_list=None):
|
||||
if pip_list is None:
|
||||
pip_list = ["pip-install-test==0.5"]
|
||||
if field == "conda":
|
||||
conda_dict = {"dependencies": ["pip", {"pip": pip_list}]}
|
||||
if spec_format == "file":
|
||||
conda_file = tmp_path / f"environment-{hash(str(pip_list))}.yml"
|
||||
conda_file.write_text(yaml.dump(conda_dict))
|
||||
conda = str(conda_file)
|
||||
elif spec_format == "python_object":
|
||||
conda = conda_dict
|
||||
runtime_env = {"conda": conda}
|
||||
elif field == "pip":
|
||||
if spec_format == "file":
|
||||
pip_file = tmp_path / f"requirements-{hash(str(pip_list))}.txt"
|
||||
pip_file.write_text("\n".join(pip_list))
|
||||
pip = str(pip_file)
|
||||
elif spec_format == "python_object":
|
||||
pip = pip_list
|
||||
runtime_env = {"pip": pip}
|
||||
return runtime_env
|
||||
|
|
|
@ -209,8 +209,9 @@ py_test_module_list(
|
|||
|
||||
py_test_module_list(
|
||||
files = [
|
||||
"test_runtime_env_conda.py",
|
||||
"test_runtime_env_complicated.py",
|
||||
"test_runtime_env_conda_and_pip.py",
|
||||
"test_runtime_env_conda_and_pip_2.py",
|
||||
"test_runtime_env_complicated.py"
|
||||
],
|
||||
size = "large",
|
||||
extra_srcs = SRCS,
|
||||
|
|
|
@ -431,10 +431,17 @@ def test_pip_task(shutdown_only, pip_as_str, tmp_path):
|
|||
@pytest.mark.skipif(
|
||||
os.environ.get("CI") and sys.platform != "linux",
|
||||
reason="This test is only run on linux CI machines.")
|
||||
def test_pip_ray_serve(shutdown_only):
|
||||
"""Tests that ray[serve] can be included as a pip dependency."""
|
||||
@pytest.mark.parametrize("option", ["conda", "pip"])
|
||||
def test_conda_pip_extras_ray_serve(shutdown_only, option):
|
||||
"""Tests that ray[extras] can be included as a conda/pip dependency."""
|
||||
ray.init()
|
||||
runtime_env = {"pip": ["pip-install-test==0.5", "ray[serve]"]}
|
||||
pip = ["pip-install-test==0.5", "ray[serve]"]
|
||||
if option == "conda":
|
||||
runtime_env = {"conda": {"dependencies": ["pip", {"pip": pip}]}}
|
||||
elif option == "pip":
|
||||
runtime_env = {"pip": pip}
|
||||
else:
|
||||
assert False, f"Unknown option: {option}"
|
||||
|
||||
@ray.remote
|
||||
def f():
|
||||
|
@ -775,7 +782,7 @@ def test_e2e_complex(call_ray_start, tmp_path):
|
|||
"ray[serve, tune]",
|
||||
"texthero",
|
||||
"PyGithub",
|
||||
"xgboost_ray",
|
||||
"xgboost_ray", # has Ray as a dependency
|
||||
"pandas==1.1", # pandas 1.2.4 in the demo, but not supported on py36
|
||||
"typer",
|
||||
"aiofiles",
|
||||
|
|
|
@ -1,7 +1,9 @@
|
|||
import os
|
||||
import pytest
|
||||
import sys
|
||||
from ray._private.test_utils import wait_for_condition, chdir
|
||||
from ray._private.test_utils import (wait_for_condition, chdir,
|
||||
generate_runtime_env_dict)
|
||||
|
||||
import yaml
|
||||
import tempfile
|
||||
from pathlib import Path
|
||||
|
@ -16,7 +18,7 @@ if not os.environ.get("CI"):
|
|||
|
||||
def check_local_files_gced(cluster):
|
||||
for node in cluster.list_all_nodes():
|
||||
for subdir in ["conda"]:
|
||||
for subdir in ["conda", "pip"]:
|
||||
all_files = os.listdir(
|
||||
os.path.join(node.get_runtime_env_dir_path(), subdir))
|
||||
# Check that there are no files remaining except for .lock files
|
||||
|
@ -32,31 +34,6 @@ def check_local_files_gced(cluster):
|
|||
return True
|
||||
|
||||
|
||||
def generate_runtime_env_dict(field, spec_format, tmp_path):
|
||||
if field == "conda":
|
||||
conda_dict = {
|
||||
"dependencies": ["pip", {
|
||||
"pip": ["pip-install-test==0.5"]
|
||||
}]
|
||||
}
|
||||
if spec_format == "file":
|
||||
conda_file = tmp_path / "environment.yml"
|
||||
conda_file.write_text(yaml.dump(conda_dict))
|
||||
conda = str(conda_file)
|
||||
elif spec_format == "python_object":
|
||||
conda = conda_dict
|
||||
runtime_env = {"conda": conda}
|
||||
elif field == "pip":
|
||||
if spec_format == "file":
|
||||
pip_file = tmp_path / "requirements.txt"
|
||||
pip_file.write_text("\n".join(["pip-install-test==0.5"]))
|
||||
pip = str(pip_file)
|
||||
elif spec_format == "python_object":
|
||||
pip = ["pip-install-test==0.5"]
|
||||
runtime_env = {"pip": pip}
|
||||
return runtime_env
|
||||
|
||||
|
||||
@pytest.mark.skipif(
|
||||
os.environ.get("CI") and sys.platform != "linux",
|
||||
reason="Requires PR wheels built in CI, so only run on linux CI machines.")
|
66
python/ray/tests/test_runtime_env_conda_and_pip_2.py
Normal file
66
python/ray/tests/test_runtime_env_conda_and_pip_2.py
Normal file
|
@ -0,0 +1,66 @@
|
|||
import os
|
||||
from typing import Dict
|
||||
import pytest
|
||||
import sys
|
||||
from ray.exceptions import RuntimeEnvSetupError
|
||||
from ray._private.test_utils import (wait_for_condition,
|
||||
generate_runtime_env_dict)
|
||||
import ray
|
||||
|
||||
if not os.environ.get("CI"):
|
||||
# This flags turns on the local development that link against current ray
|
||||
# packages and fall back all the dependencies to current python's site.
|
||||
os.environ["RAY_RUNTIME_ENV_LOCAL_DEV_MODE"] = "1"
|
||||
|
||||
|
||||
@pytest.mark.parametrize("field", ["conda", "pip"])
|
||||
@pytest.mark.parametrize("specify_env_in_init", [False, True])
|
||||
@pytest.mark.parametrize("spec_format", ["file", "python_object"])
|
||||
def test_install_failure_logging(start_cluster, specify_env_in_init, field,
|
||||
spec_format, tmp_path, capsys):
|
||||
cluster, address = start_cluster
|
||||
using_ray_client = address.startswith("ray://")
|
||||
|
||||
bad_envs: Dict[str, Dict] = {}
|
||||
bad_packages: Dict[str, str] = {}
|
||||
for scope in "init", "actor", "task":
|
||||
bad_packages[scope] = "doesnotexist" + scope
|
||||
bad_envs[scope] = generate_runtime_env_dict(
|
||||
field, spec_format, tmp_path, pip_list=[bad_packages[scope]])
|
||||
|
||||
if specify_env_in_init:
|
||||
if using_ray_client:
|
||||
with pytest.raises(ConnectionAbortedError) as excinfo:
|
||||
ray.init(address, runtime_env=bad_envs["init"])
|
||||
assert bad_packages["init"] in str(excinfo.value)
|
||||
else:
|
||||
ray.init(address, runtime_env=bad_envs["init"])
|
||||
wait_for_condition(
|
||||
lambda: bad_packages["init"] in capsys.readouterr().out,
|
||||
timeout=30)
|
||||
return
|
||||
|
||||
ray.init(address)
|
||||
|
||||
@ray.remote(runtime_env=bad_envs["actor"])
|
||||
class A:
|
||||
pass
|
||||
|
||||
a = A.remote() # noqa
|
||||
|
||||
wait_for_condition(
|
||||
lambda: bad_packages["actor"] in capsys.readouterr().out, timeout=30)
|
||||
|
||||
@ray.remote(runtime_env=bad_envs["task"])
|
||||
def f():
|
||||
pass
|
||||
|
||||
with pytest.raises(RuntimeEnvSetupError):
|
||||
ray.get(f.remote())
|
||||
|
||||
wait_for_condition(
|
||||
lambda: bad_packages["task"] in capsys.readouterr().out, timeout=30)
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
sys.exit(pytest.main(["-sv", __file__]))
|
|
@ -11,6 +11,7 @@ from ray._private.runtime_env.validation import (
|
|||
parse_and_validate_conda, parse_and_validate_pip,
|
||||
parse_and_validate_env_vars, parse_and_validate_py_modules,
|
||||
ParsedRuntimeEnv)
|
||||
from ray._private.runtime_env.pip import RAY_RUNTIME_ENV_ALLOW_RAY_IN_PIP
|
||||
from ray._private.runtime_env.plugin import (decode_plugin_uri,
|
||||
encode_plugin_uri)
|
||||
|
||||
|
@ -205,6 +206,23 @@ class TestValidatePip:
|
|||
result = parse_and_validate_pip(PIP_LIST)
|
||||
assert result == PIP_LIST
|
||||
|
||||
def test_remove_ray(self):
|
||||
result = parse_and_validate_pip(["pkg1", "ray", "pkg2"])
|
||||
assert result == ["pkg1", "pkg2"]
|
||||
|
||||
def test_remove_ray_env_var(self, monkeypatch):
|
||||
monkeypatch.setenv(RAY_RUNTIME_ENV_ALLOW_RAY_IN_PIP, "1")
|
||||
result = parse_and_validate_pip(["pkg1", "ray", "pkg2"])
|
||||
assert result == ["pkg1", "ray", "pkg2"]
|
||||
|
||||
def test_replace_ray_libraries_with_dependencies(self):
|
||||
result = parse_and_validate_pip(["pkg1", "ray[serve, tune]", "pkg2"])
|
||||
assert "pkg1" in result
|
||||
assert "pkg2" in result
|
||||
assert "uvicorn" in result # from ray[serve]
|
||||
assert "pandas" in result # from ray[tune]
|
||||
assert not any(["ray" in specifier for specifier in result])
|
||||
|
||||
|
||||
class TestValidateEnvVars:
|
||||
def test_type_validation(self):
|
||||
|
|
|
@ -1425,6 +1425,10 @@ static std::vector<std::string> GetUrisFromRuntimeEnv(
|
|||
const auto &uri = runtime_env->uris().conda_uri();
|
||||
result.emplace_back(encode_plugin_uri("conda", uri));
|
||||
}
|
||||
if (!runtime_env->uris().pip_uri().empty()) {
|
||||
const auto &uri = runtime_env->uris().pip_uri();
|
||||
result.emplace_back(encode_plugin_uri("pip", uri));
|
||||
}
|
||||
for (const auto &uri : runtime_env->uris().plugin_uris()) {
|
||||
result.emplace_back(encode_plugin_uri("plugin", uri));
|
||||
}
|
||||
|
|
|
@ -83,8 +83,10 @@ message RuntimeEnvUris {
|
|||
repeated string py_modules_uris = 2;
|
||||
/// conda uri
|
||||
string conda_uri = 3;
|
||||
/// pip uri
|
||||
string pip_uri = 4;
|
||||
/// plugin uris
|
||||
repeated string plugin_uris = 4;
|
||||
repeated string plugin_uris = 5;
|
||||
}
|
||||
|
||||
/// The runtime environment describes all the runtime packages needed to
|
||||
|
|
Loading…
Add table
Reference in a new issue