Skip to content

Commit ddf8a92

Browse files
crypdickRicardo Decalgemini-code-assist[bot]edoakes
authored andcommitted
[Core] Add default excludes for working_dir uploads (#59566)
## Description Automatically exclude common directories (.git, .venv, venv, __pycache__) when uploading working_dir in runtime environment packages. At a minimum we need to exclude `.git/` because unlike the others, nobody includes .git/ in `.gitignore`. This causes Ray to throw a `ray.exceptions.RuntimeEnvSetupError` if your `.git` dir is larger than 512 MiB. I also updated the documentation in handling-dependencies.rst and improved the error message if the env exceeds the GCS_STORAGE_MAX_SIZE limit. ## Related issues N/A ## Additional information This PR pytorch/tutorials#3709 was failing to run because the PyTorch tutorials .git/ folder is huge. --------- Signed-off-by: Ricardo Decal <public@ricardodecal.com> Signed-off-by: Ricardo Decal <crypdick@users.noreply.github.com> Signed-off-by: Ricardo Decal <rdecal@anyscale.com> Co-authored-by: Ricardo Decal <public@ricardodecal.com> Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com> Co-authored-by: Edward Oakes <ed.nmi.oakes@gmail.com> Signed-off-by: elliot-barn <elliot.barnwell@anyscale.com>
1 parent 4c6492c commit ddf8a92

File tree

6 files changed

+127
-6
lines changed

6 files changed

+127
-6
lines changed

doc/source/ray-core/handling-dependencies.rst

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -507,6 +507,8 @@ The ``runtime_env`` is a Python dictionary or a Python class :class:`ray.runtime
507507

508508
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.
509509

510+
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.
511+
510512
Note: If the local directory contains symbolic links, Ray follows the links and the files they point to are uploaded to the cluster.
511513

512514
- ``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
534536

535537
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.
536538

537-
538539
- ``py_executable`` (str): Specifies the executable used for running the Ray workers. It can include arguments as well. The executable can be
539540
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
540541
in an environment set up by a package manager like `UV` (see :ref:`here <use-uv-for-package-management>`).

python/ray/_private/ray_constants.py

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -155,6 +155,21 @@ def env_set_by_user(key):
155155
# If set to 1, then `.gitignore` files will not be parsed and loaded into "excludes"
156156
# when using a local working_dir or py_modules.
157157
RAY_RUNTIME_ENV_IGNORE_GITIGNORE = "RAY_RUNTIME_ENV_IGNORE_GITIGNORE"
158+
# Default directories to exclude when packaging working_dir.
159+
# Override by setting the RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES
160+
# (comma-separated) environment variable. Set to an empty string to disable.
161+
# `.git` is necessary since it is never in .gitignore.
162+
RAY_RUNTIME_ENV_DEFAULT_EXCLUDES = ".git,.venv,venv,__pycache__"
163+
164+
165+
def get_runtime_env_default_excludes() -> list[str]:
166+
"""Get default excludes for working_dir, overridable via RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES environment variable."""
167+
val = os.environ.get(
168+
"RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES", RAY_RUNTIME_ENV_DEFAULT_EXCLUDES
169+
)
170+
return [x.strip() for x in val.split(",") if x.strip()]
171+
172+
158173
# Hook for running a user-specified runtime-env hook. This hook will be called
159174
# unconditionally given the runtime_env dict passed for ray.init. It must return
160175
# a rewritten runtime_env dict. Example: "your.module.runtime_env_hook".

python/ray/_private/runtime_env/packaging.py

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -453,10 +453,10 @@ def _store_package_in_gcs(
453453
if len(data) >= GCS_STORAGE_MAX_SIZE:
454454
raise ValueError(
455455
f"Package size ({size_str}) exceeds the maximum size of "
456-
f"{_mib_string(GCS_STORAGE_MAX_SIZE)}. You can exclude large "
457-
"files using the 'excludes' option to the runtime_env or provide "
458-
"a remote URI of a zip file using protocols such as 's3://', "
459-
"'https://' and so on, refer to "
456+
f"{_mib_string(GCS_STORAGE_MAX_SIZE)}. To exclude large files, "
457+
"add them to '.gitignore' or '.rayignore' files, or "
458+
"use the 'excludes' option in the runtime_env, or provide a "
459+
"URI of a remote zip file. For more information, refer to "
460460
"https://docs.ray.io/en/latest/ray-core/handling-dependencies.html#api-reference." # noqa
461461
)
462462

python/ray/_private/runtime_env/working_dir.py

Lines changed: 27 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,10 +22,12 @@
2222
from ray._private.utils import get_directory_size_bytes
2323
from ray._raylet import GcsClient
2424
from ray.exceptions import RuntimeEnvSetupError
25+
from ray.util.debug import log_once
2526

2627
default_logger = logging.getLogger(__name__)
2728

2829
_WIN32 = os.name == "nt"
30+
_LOG_ONCE_DEFAULT_EXCLUDE_PREFIX = "runtime_env_default_exclude:"
2931

3032

3133
def upload_working_dir_if_needed(
@@ -38,6 +40,12 @@ def upload_working_dir_if_needed(
3840
"""Uploads the working_dir and replaces it with a URI.
3941
4042
If the working_dir is already a URI, this is a no-op.
43+
44+
Excludes are combined from:
45+
- .gitignore and .rayignore files in the working_dir
46+
- runtime_env["excludes"] field
47+
- RAY_RUNTIME_ENV_DEFAULT_EXCLUDES constant, overridable via
48+
RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES environment variable
4149
"""
4250
working_dir = runtime_env.get("working_dir")
4351
if working_dir is None:
@@ -63,7 +71,25 @@ def upload_working_dir_if_needed(
6371
raise ValueError("Only .zip files supported for remote URIs.")
6472
return runtime_env
6573

66-
excludes = runtime_env.get("excludes", None)
74+
default_excludes = ray_constants.get_runtime_env_default_excludes()
75+
user_excludes = runtime_env.get("excludes") or []
76+
excludes = default_excludes + list(user_excludes)
77+
# TODO(ricardo): 2026-01-07 Remove these warnings in a few releases. Added in
78+
# case users rely on these directories being uploaded with their working_dir
79+
# since this change would be difficult to debug.
80+
logger = logger or default_logger
81+
working_dir_path = Path(working_dir)
82+
for d in default_excludes:
83+
if (working_dir_path / d).exists() and log_once(
84+
f"{_LOG_ONCE_DEFAULT_EXCLUDE_PREFIX}{d}"
85+
):
86+
logger.warning(
87+
"Directory %r is now ignored by default when packaging the working "
88+
"directory. To disable this behavior, set "
89+
"the `RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES=''` environment "
90+
"variable.",
91+
d,
92+
)
6793
try:
6894
working_dir_uri = get_uri_for_directory(
6995
working_dir,

python/ray/tests/test_runtime_env_working_dir.py

Lines changed: 49 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -612,5 +612,54 @@ class B:
612612
B.options(runtime_env={"working_dir": "."})
613613

614614

615+
def test_default_excludes(start_cluster, monkeypatch):
616+
"""Tests that default excludes (.git, .venv, etc.) are applied."""
617+
cluster, address = start_cluster
618+
monkeypatch.delenv("RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES", raising=False)
619+
620+
excluded_dirs = [".git", ".venv", "venv", "__pycache__"]
621+
622+
with tempfile.TemporaryDirectory() as tmp_working_dir:
623+
# Create excluded directories with a marker file
624+
for d in excluded_dirs:
625+
os.makedirs(os.path.join(tmp_working_dir, d))
626+
Path(tmp_working_dir, d, "to_exclude").write_text("x")
627+
628+
# Create a file that should be included
629+
Path(tmp_working_dir, "included.txt").write_text("x")
630+
631+
ray.init(address, runtime_env={"working_dir": tmp_working_dir})
632+
633+
@ray.remote
634+
def check_dirs(dirs):
635+
return {d: os.path.exists(d) for d in dirs + ["included.txt"]}
636+
637+
result = ray.get(check_dirs.remote(excluded_dirs))
638+
639+
assert result["included.txt"], "included.txt should be present"
640+
for d in excluded_dirs:
641+
assert not result[d], f"{d} should be excluded by default"
642+
643+
644+
def test_default_excludes_disabled_via_env_var(start_cluster, monkeypatch):
645+
"""Tests that RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES='' disables defaults."""
646+
cluster, address = start_cluster
647+
monkeypatch.setenv("RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES", "")
648+
649+
with tempfile.TemporaryDirectory() as tmp_working_dir:
650+
os.makedirs(os.path.join(tmp_working_dir, ".git"))
651+
Path(tmp_working_dir, ".git", "to_exclude").write_text("x")
652+
653+
ray.init(address, runtime_env={"working_dir": tmp_working_dir})
654+
655+
@ray.remote
656+
def check_git():
657+
return os.path.exists(".git")
658+
659+
assert ray.get(
660+
check_git.remote()
661+
), ".git should be included when defaults disabled"
662+
663+
615664
if __name__ == "__main__":
616665
sys.exit(pytest.main(["-sv", __file__]))
Lines changed: 30 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,30 @@
1+
"""Unit tests for working_dir runtime environment functionality."""
2+
3+
import sys
4+
5+
import pytest
6+
7+
from ray._private.ray_constants import get_runtime_env_default_excludes
8+
9+
ENV_VAR = "RAY_OVERRIDE_RUNTIME_ENV_DEFAULT_EXCLUDES"
10+
11+
12+
class TestGetRuntimeEnvDefaultExcludes:
13+
"""Tests for get_runtime_env_default_excludes()."""
14+
15+
def test_returns_defaults_when_env_var_not_set(self, monkeypatch):
16+
monkeypatch.delenv(ENV_VAR, raising=False)
17+
result = get_runtime_env_default_excludes()
18+
assert ".git" in result and ".venv" in result
19+
20+
def test_empty_env_var_disables_defaults(self, monkeypatch):
21+
monkeypatch.setenv(ENV_VAR, "")
22+
assert get_runtime_env_default_excludes() == []
23+
24+
def test_custom_env_var_overrides_defaults(self, monkeypatch):
25+
monkeypatch.setenv(ENV_VAR, "foo, bar ,baz")
26+
assert get_runtime_env_default_excludes() == ["foo", "bar", "baz"]
27+
28+
29+
if __name__ == "__main__":
30+
sys.exit(pytest.main(["-vv", __file__]))

0 commit comments

Comments
 (0)