Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 2 additions & 1 deletion doc/source/ray-core/handling-dependencies.rst
Original file line number Diff line number Diff line change
Expand Up @@ -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.)
Expand Down Expand Up @@ -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 <use-uv-for-package-management>`).
Expand Down
15 changes: 15 additions & 0 deletions python/ray/_private/ray_constants.py
Original file line number Diff line number Diff line change
Expand Up @@ -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".
Expand Down
8 changes: 4 additions & 4 deletions python/ray/_private/runtime_env/packaging.py
Original file line number Diff line number Diff line change
Expand Up @@ -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
)

Expand Down
28 changes: 27 additions & 1 deletion python/ray/_private/runtime_env/working_dir.py
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand All @@ -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:
Expand All @@ -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,
Expand Down
49 changes: 49 additions & 0 deletions python/ray/tests/test_runtime_env_working_dir.py
Original file line number Diff line number Diff line change
Expand Up @@ -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__]))
30 changes: 30 additions & 0 deletions python/ray/tests/unit/test_runtime_env_working_dir.py
Original file line number Diff line number Diff line change
@@ -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__]))