diff --git a/doc/source/ray-core/handling-dependencies.rst b/doc/source/ray-core/handling-dependencies.rst index 844793ef64f2..1e7d5d9522fc 100644 --- a/doc/source/ray-core/handling-dependencies.rst +++ b/doc/source/ray-core/handling-dependencies.rst @@ -507,6 +507,8 @@ The ``runtime_env`` is a Python dictionary or a Python class :class:`ray.runtime Note: By default, if the local directory contains a ``.gitignore`` and/or ``.rayignore`` file, the specified files are not uploaded to the cluster. To disable the ``.gitignore`` from being considered, set ``RAY_RUNTIME_ENV_IGNORE_GITIGNORE=1`` on the machine doing the uploading. + Note: By default, common directories (``.git``, ``.venv``, ``venv``, ``__pycache__``) are automatically excluded from the ``working_dir`` upload. You can override these defaults by setting the ``RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES`` environment variable to a comma-separated list of patterns, or set it to an empty string to disable default excludes entirely. + Note: If the local directory contains symbolic links, Ray follows the links and the files they point to are uploaded to the cluster. - ``py_modules`` (List[str|module]): Specifies Python modules to be available for import in the Ray workers. (For more ways to specify packages, see also the ``pip`` and ``conda`` fields below.) @@ -534,7 +536,6 @@ The ``runtime_env`` is a Python dictionary or a Python class :class:`ray.runtime Note: For option (1), by default, if the local directory contains a ``.gitignore`` and/or ``.rayignore`` file, the specified files are not uploaded to the cluster. To disable the ``.gitignore`` from being considered, set ``RAY_RUNTIME_ENV_IGNORE_GITIGNORE=1`` on the machine doing the uploading. - - ``py_executable`` (str): Specifies the executable used for running the Ray workers. It can include arguments as well. The executable can be located in the `working_dir`. This runtime environment is useful to run workers in a custom debugger or profiler as well as to run workers in an environment set up by a package manager like `UV` (see :ref:`here `). diff --git a/python/ray/_private/ray_constants.py b/python/ray/_private/ray_constants.py index 6a70a6396388..aa904d3b6341 100644 --- a/python/ray/_private/ray_constants.py +++ b/python/ray/_private/ray_constants.py @@ -155,6 +155,21 @@ def env_set_by_user(key): # If set to 1, then `.gitignore` files will not be parsed and loaded into "excludes" # when using a local working_dir or py_modules. RAY_RUNTIME_ENV_IGNORE_GITIGNORE = "RAY_RUNTIME_ENV_IGNORE_GITIGNORE" +# Default directories to exclude when packaging working_dir. +# Override by setting the RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES +# (comma-separated) environment variable. Set to an empty string to disable. +# `.git` is necessary since it is never in .gitignore. +RAY_RUNTIME_ENV_DEFAULT_EXCLUDES = ".git,.venv,venv,__pycache__" + + +def get_runtime_env_default_excludes() -> list[str]: + """Get default excludes for working_dir, overridable via RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES environment variable.""" + val = os.environ.get( + "RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES", RAY_RUNTIME_ENV_DEFAULT_EXCLUDES + ) + return [x.strip() for x in val.split(",") if x.strip()] + + # Hook for running a user-specified runtime-env hook. This hook will be called # unconditionally given the runtime_env dict passed for ray.init. It must return # a rewritten runtime_env dict. Example: "your.module.runtime_env_hook". diff --git a/python/ray/_private/runtime_env/packaging.py b/python/ray/_private/runtime_env/packaging.py index 2c4ea9a18b46..5691c6796f30 100644 --- a/python/ray/_private/runtime_env/packaging.py +++ b/python/ray/_private/runtime_env/packaging.py @@ -453,10 +453,10 @@ def _store_package_in_gcs( if len(data) >= GCS_STORAGE_MAX_SIZE: raise ValueError( f"Package size ({size_str}) exceeds the maximum size of " - f"{_mib_string(GCS_STORAGE_MAX_SIZE)}. You can exclude large " - "files using the 'excludes' option to the runtime_env or provide " - "a remote URI of a zip file using protocols such as 's3://', " - "'https://' and so on, refer to " + f"{_mib_string(GCS_STORAGE_MAX_SIZE)}. To exclude large files, " + "add them to '.gitignore' or '.rayignore' files, or " + "use the 'excludes' option in the runtime_env, or provide a " + "URI of a remote zip file. For more information, refer to " "https://docs.ray.io/en/latest/ray-core/handling-dependencies.html#api-reference." # noqa ) diff --git a/python/ray/_private/runtime_env/working_dir.py b/python/ray/_private/runtime_env/working_dir.py index 5c9ff11e20fe..27579e550e3a 100644 --- a/python/ray/_private/runtime_env/working_dir.py +++ b/python/ray/_private/runtime_env/working_dir.py @@ -22,10 +22,12 @@ from ray._private.utils import get_directory_size_bytes from ray._raylet import GcsClient from ray.exceptions import RuntimeEnvSetupError +from ray.util.debug import log_once default_logger = logging.getLogger(__name__) _WIN32 = os.name == "nt" +_LOG_ONCE_DEFAULT_EXCLUDE_PREFIX = "runtime_env_default_exclude:" def upload_working_dir_if_needed( @@ -38,6 +40,12 @@ def upload_working_dir_if_needed( """Uploads the working_dir and replaces it with a URI. If the working_dir is already a URI, this is a no-op. + + Excludes are combined from: + - .gitignore and .rayignore files in the working_dir + - runtime_env["excludes"] field + - RAY_RUNTIME_ENV_DEFAULT_EXCLUDES constant, overridable via + RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES environment variable """ working_dir = runtime_env.get("working_dir") if working_dir is None: @@ -63,7 +71,25 @@ def upload_working_dir_if_needed( raise ValueError("Only .zip files supported for remote URIs.") return runtime_env - excludes = runtime_env.get("excludes", None) + default_excludes = ray_constants.get_runtime_env_default_excludes() + user_excludes = runtime_env.get("excludes") or [] + excludes = default_excludes + list(user_excludes) + # TODO(ricardo): 2026-01-07 Remove these warnings in a few releases. Added in + # case users rely on these directories being uploaded with their working_dir + # since this change would be difficult to debug. + logger = logger or default_logger + working_dir_path = Path(working_dir) + for d in default_excludes: + if (working_dir_path / d).exists() and log_once( + f"{_LOG_ONCE_DEFAULT_EXCLUDE_PREFIX}{d}" + ): + logger.warning( + "Directory %r is now ignored by default when packaging the working " + "directory. To disable this behavior, set " + "the `RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES=''` environment " + "variable.", + d, + ) try: working_dir_uri = get_uri_for_directory( working_dir, diff --git a/python/ray/tests/test_runtime_env_working_dir.py b/python/ray/tests/test_runtime_env_working_dir.py index 7662cc6ef899..c7de34464538 100644 --- a/python/ray/tests/test_runtime_env_working_dir.py +++ b/python/ray/tests/test_runtime_env_working_dir.py @@ -612,5 +612,54 @@ class B: B.options(runtime_env={"working_dir": "."}) +def test_default_excludes(start_cluster, monkeypatch): + """Tests that default excludes (.git, .venv, etc.) are applied.""" + cluster, address = start_cluster + monkeypatch.delenv("RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES", raising=False) + + excluded_dirs = [".git", ".venv", "venv", "__pycache__"] + + with tempfile.TemporaryDirectory() as tmp_working_dir: + # Create excluded directories with a marker file + for d in excluded_dirs: + os.makedirs(os.path.join(tmp_working_dir, d)) + Path(tmp_working_dir, d, "to_exclude").write_text("x") + + # Create a file that should be included + Path(tmp_working_dir, "included.txt").write_text("x") + + ray.init(address, runtime_env={"working_dir": tmp_working_dir}) + + @ray.remote + def check_dirs(dirs): + return {d: os.path.exists(d) for d in dirs + ["included.txt"]} + + result = ray.get(check_dirs.remote(excluded_dirs)) + + assert result["included.txt"], "included.txt should be present" + for d in excluded_dirs: + assert not result[d], f"{d} should be excluded by default" + + +def test_default_excludes_disabled_via_env_var(start_cluster, monkeypatch): + """Tests that RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES='' disables defaults.""" + cluster, address = start_cluster + monkeypatch.setenv("RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES", "") + + with tempfile.TemporaryDirectory() as tmp_working_dir: + os.makedirs(os.path.join(tmp_working_dir, ".git")) + Path(tmp_working_dir, ".git", "to_exclude").write_text("x") + + ray.init(address, runtime_env={"working_dir": tmp_working_dir}) + + @ray.remote + def check_git(): + return os.path.exists(".git") + + assert ray.get( + check_git.remote() + ), ".git should be included when defaults disabled" + + if __name__ == "__main__": sys.exit(pytest.main(["-sv", __file__])) diff --git a/python/ray/tests/unit/test_runtime_env_working_dir.py b/python/ray/tests/unit/test_runtime_env_working_dir.py new file mode 100644 index 000000000000..a25fcec9521d --- /dev/null +++ b/python/ray/tests/unit/test_runtime_env_working_dir.py @@ -0,0 +1,30 @@ +"""Unit tests for working_dir runtime environment functionality.""" + +import sys + +import pytest + +from ray._private.ray_constants import get_runtime_env_default_excludes + +ENV_VAR = "RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES" + + +class TestGetRuntimeEnvDefaultExcludes: + """Tests for get_runtime_env_default_excludes().""" + + def test_returns_defaults_when_env_var_not_set(self, monkeypatch): + monkeypatch.delenv(ENV_VAR, raising=False) + result = get_runtime_env_default_excludes() + assert ".git" in result and ".venv" in result + + def test_empty_env_var_disables_defaults(self, monkeypatch): + monkeypatch.setenv(ENV_VAR, "") + assert get_runtime_env_default_excludes() == [] + + def test_custom_env_var_overrides_defaults(self, monkeypatch): + monkeypatch.setenv(ENV_VAR, "foo, bar ,baz") + assert get_runtime_env_default_excludes() == ["foo", "bar", "baz"] + + +if __name__ == "__main__": + sys.exit(pytest.main(["-vv", __file__]))