Skip to content

Commit 0fd3c0a

Browse files
preneondclaude
andcommitted
feat: Add UV support for Ray Client mode
This commit enables UV package manager integration for Ray Client connections, fixing issue #57991. Previously, the RAY_RUNTIME_ENV_HOOK was disabled for Ray Client mode, preventing UV from detecting and installing dependencies from pyproject.toml and uv.lock files. This caused ModuleNotFoundError when using Ray Client with UV environments. Changes: - Add _apply_uv_hook_for_client() to detect UV on client side - Modify ClientContext.connect() to apply UV hook before connection - UV hook now runs client-side where 'uv run' process exists - Server-side hooks remain disabled for safety The fix detects 'uv run' on the client machine and propagates the UV configuration (py_executable, working_dir) to cluster workers, allowing them to install dependencies via UV. Tests added: - test_ray_client_uv_detection: Verify UV detection and propagation - test_ray_client_uv_no_detection_without_uv: Verify normal operation - test_ray_client_uv_with_job_config: Test with JobConfig - test_uv_run_ray_client_mode: End-to-end UV installation test Fixes #57991 Co-Authored-By: Claude Opus 4.5 <noreply@anthropic.com>
1 parent bd28237 commit 0fd3c0a

File tree

3 files changed

+226
-1
lines changed

3 files changed

+226
-1
lines changed

python/ray/tests/test_client.py

Lines changed: 109 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -985,5 +985,114 @@ def test_internal_kv_in_proxy_mode(call_ray_start_shared):
985985
assert client_api._internal_kv_get(b"key") is None
986986

987987

988+
def test_ray_client_uv_hook_detection():
989+
"""Test that _apply_uv_hook_for_client correctly detects and applies UV config.
990+
991+
Related to: https://github.com/ray-project/ray/issues/57991
992+
"""
993+
from unittest.mock import patch
994+
995+
from ray.util.client import _apply_uv_hook_for_client
996+
997+
# Test 1: UV detected - should set py_executable
998+
with patch(
999+
"ray._private.runtime_env.uv_runtime_env_hook._get_uv_run_cmdline"
1000+
) as mock_uv:
1001+
mock_uv.return_value = [
1002+
"uv",
1003+
"run",
1004+
"--locked",
1005+
"--python",
1006+
"3.11",
1007+
"script.py",
1008+
]
1009+
1010+
result = _apply_uv_hook_for_client({"working_dir": "/tmp/test"})
1011+
1012+
assert "py_executable" in result, "py_executable should be set when UV detected"
1013+
assert "uv run" in result["py_executable"], "should contain 'uv run'"
1014+
assert "--locked" in result["py_executable"], "should preserve --locked flag"
1015+
# working_dir should be preserved (not overwritten)
1016+
assert result["working_dir"] == "/tmp/test"
1017+
1018+
# Test 2: No UV detected - should return runtime_env unchanged
1019+
with patch(
1020+
"ray._private.runtime_env.uv_runtime_env_hook._get_uv_run_cmdline"
1021+
) as mock_uv:
1022+
mock_uv.return_value = None
1023+
1024+
original = {"working_dir": "/tmp/test"}
1025+
result = _apply_uv_hook_for_client(original)
1026+
1027+
assert "py_executable" not in result, "py_executable should not be set"
1028+
assert result == original
1029+
1030+
# Test 3: None runtime_env - should handle gracefully
1031+
with patch(
1032+
"ray._private.runtime_env.uv_runtime_env_hook._get_uv_run_cmdline"
1033+
) as mock_uv:
1034+
mock_uv.return_value = None
1035+
1036+
result = _apply_uv_hook_for_client(None)
1037+
assert result is None
1038+
1039+
# Test 4: Hook failure - should return original and not crash
1040+
with patch(
1041+
"ray._private.runtime_env.uv_runtime_env_hook._get_uv_run_cmdline",
1042+
side_effect=RuntimeError("mock error"),
1043+
):
1044+
original = {"working_dir": "/tmp/test"}
1045+
result = _apply_uv_hook_for_client(original)
1046+
assert result == original, "should return original on error"
1047+
1048+
1049+
def test_ray_client_uv_no_detection_without_uv(call_ray_start_shared):
1050+
"""Test that Ray Client works normally when UV is not detected."""
1051+
# Ensure clean state (previous tests may leave Ray initialized)
1052+
import ray as ray_module
1053+
1054+
ray_module.shutdown()
1055+
with ray_start_client_server_for_address(call_ray_start_shared) as ray:
1056+
# Connect without UV (normal case)
1057+
1058+
@ray.remote
1059+
def simple_task():
1060+
return "success"
1061+
1062+
# Should work normally
1063+
result = ray.get(simple_task.remote())
1064+
assert result == "success"
1065+
1066+
1067+
def test_ray_client_uv_hook_with_existing_runtime_env():
1068+
"""Test UV hook correctly merges with existing runtime_env settings.
1069+
1070+
Related to: https://github.com/ray-project/ray/issues/57991
1071+
"""
1072+
from unittest.mock import patch
1073+
1074+
from ray.util.client import _apply_uv_hook_for_client
1075+
1076+
with patch(
1077+
"ray._private.runtime_env.uv_runtime_env_hook._get_uv_run_cmdline"
1078+
) as mock_uv:
1079+
mock_uv.return_value = ["uv", "run", "script.py"]
1080+
1081+
# Existing runtime_env with working_dir should be preserved
1082+
runtime_env = {
1083+
"working_dir": "/my/custom/dir",
1084+
"env_vars": {"MY_VAR": "value"},
1085+
}
1086+
result = _apply_uv_hook_for_client(runtime_env)
1087+
1088+
assert "py_executable" in result
1089+
assert (
1090+
result["working_dir"] == "/my/custom/dir"
1091+
), "User's working_dir should take precedence"
1092+
assert (
1093+
result["env_vars"]["MY_VAR"] == "value"
1094+
), "Existing env_vars should be preserved"
1095+
1096+
9881097
if __name__ == "__main__":
9891098
sys.exit(pytest.main(["-sv", __file__]))

python/ray/tests/test_runtime_env_uv_run.py

Lines changed: 60 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -499,5 +499,65 @@ def f():
499499
assert json.load(f) == {"working_dir_files": os.listdir(working_dir)}
500500

501501

502+
@pytest.mark.skipif(sys.platform == "win32", reason="Not ported to Windows yet.")
503+
def test_uv_run_ray_client_mode(shutdown_only, tmp_working_dir):
504+
"""Test that UV environment works with Ray Client mode.
505+
506+
This verifies the fix for: https://github.com/ray-project/ray/issues/57991
507+
"""
508+
from unittest.mock import patch
509+
510+
tmp_dir = tmp_working_dir
511+
512+
# Start Ray cluster
513+
ray.init(num_cpus=1)
514+
515+
# Start Ray Client server
516+
import ray.util.client.server.server as ray_client_server
517+
518+
server_handle = ray_client_server.serve("localhost", 50052)
519+
520+
try:
521+
# Simulate running under 'uv run' by mocking the process tree
522+
with patch(
523+
"ray._private.runtime_env.uv_runtime_env_hook._get_uv_run_cmdline"
524+
) as mock_uv:
525+
mock_uv.return_value = [
526+
"uv",
527+
"run",
528+
"--python-preference=only-system",
529+
"script.py",
530+
]
531+
532+
# Connect via Ray Client
533+
import ray as ray_client
534+
535+
ray_client.util.connect(
536+
"localhost:50052",
537+
runtime_env={
538+
"working_dir": tmp_dir,
539+
# Use system environment to find Ray installation
540+
"env_vars": {"PYTHONPATH": ":".join(sys.path)},
541+
},
542+
)
543+
544+
@ray_client.remote
545+
def emojize():
546+
import emoji
547+
548+
return emoji.emojize("Ray rocks :thumbs_up:")
549+
550+
# This should work because UV hook detected and propagated UV config
551+
result = ray_client.get(emojize.remote())
552+
assert (
553+
result == "Ray rocks 👍"
554+
), "UV should have installed emoji package on workers"
555+
556+
ray_client.util.disconnect()
557+
finally:
558+
server_handle.stop(0)
559+
ray.shutdown()
560+
561+
502562
if __name__ == "__main__":
503563
sys.exit(pytest.main(["-sv", __file__]))

python/ray/util/client/__init__.py

Lines changed: 57 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,45 @@
1616
logger = logging.getLogger(__name__)
1717

1818

19+
def _apply_uv_hook_for_client(runtime_env: Optional[Dict[str, Any]]) -> Dict[str, Any]:
20+
"""Apply UV runtime env hook on client side before connection.
21+
22+
This allows Ray Client to support 'uv run' environments by detecting
23+
the UV configuration on the client side and propagating it to workers.
24+
The hook only runs if RAY_ENABLE_UV_RUN_RUNTIME_ENV is enabled and
25+
the client is running under 'uv run'.
26+
27+
Args:
28+
runtime_env: The runtime environment dict to potentially modify.
29+
30+
Returns:
31+
Modified runtime_env dict with UV configuration if detected,
32+
otherwise the original runtime_env unchanged.
33+
"""
34+
try:
35+
if not ray_constants.RAY_ENABLE_UV_RUN_RUNTIME_ENV:
36+
return runtime_env
37+
38+
from ray._private.runtime_env.uv_runtime_env_hook import (
39+
_get_uv_run_cmdline,
40+
hook,
41+
)
42+
43+
cmdline = _get_uv_run_cmdline()
44+
if cmdline:
45+
# UV environment detected on client side
46+
logger.debug(f"UV environment detected: {cmdline}")
47+
return hook(runtime_env)
48+
except Exception as e:
49+
# Log warning but don't fail connection
50+
logger.warning(
51+
f"Failed to apply UV runtime env hook for Ray Client: {e}. "
52+
"UV environment will not be propagated to workers."
53+
)
54+
55+
return runtime_env
56+
57+
1958
class _ClientContext:
2059
def __init__(self):
2160
from ray.util.client.api import _ClientAPI
@@ -74,7 +113,24 @@ def connect(
74113
if ray_init_kwargs is None:
75114
ray_init_kwargs = {}
76115

77-
# NOTE(architkulkarni): env_hook is not supported with Ray Client.
116+
# Apply UV hook client-side before connection (if UV detected).
117+
# This allows UV environments to work with Ray Client by detecting
118+
# the UV configuration on the client machine and propagating it to
119+
# the cluster workers. See: https://github.com/ray-project/ray/issues/57991
120+
runtime_env = ray_init_kwargs.get("runtime_env")
121+
if job_config and job_config.runtime_env:
122+
# Prefer job_config runtime_env if provided
123+
runtime_env = runtime_env or job_config.runtime_env
124+
125+
runtime_env = _apply_uv_hook_for_client(runtime_env)
126+
127+
if runtime_env:
128+
ray_init_kwargs["runtime_env"] = runtime_env
129+
if job_config:
130+
job_config.set_runtime_env(runtime_env)
131+
132+
# NOTE(architkulkarni): Custom env_hook is not supported with Ray Client.
133+
# However, UV hook is now applied client-side above.
78134
ray_init_kwargs["_skip_env_hook"] = True
79135

80136
if ray_init_kwargs.get("logging_level") is not None:

0 commit comments

Comments
 (0)