Skip to content

Commit 22f1f70

Browse files
authored
Fix Ray Client when 'uv run' runtime environment is used (#51683)
<!-- Thank you for your contribution! Please review https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before opening a pull request. --> <!-- Please add a reviewer to the assignee section when you create a PR. If you don't have the access to it, we will shortly find a reviewer and assign them to your PR. --> ## Why are these changes needed? Fixes #51368 The problem was happening because we check the command line of the Ray Client server, but when used with `py_executable` equal to `uv run`, the command line would be `uv run -m ray.util.client.server ...` and not `python -m ray.util.client.server`, so the check needed to be adapted. ## Related issue number <!-- For example: "Closes #1234" --> ## Checks - [ ] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [ ] I've run `scripts/format.sh` to lint the changes in this PR. - [ ] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [ ] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [ ] Unit tests - [ ] Release tests - [ ] This PR is not tested :(
1 parent 4ddaa8a commit 22f1f70

File tree

3 files changed

+21
-9
lines changed

3 files changed

+21
-9
lines changed

python/ray/tests/test_client.py

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@
2929
from ray.tests.conftest import call_ray_start_context
3030
from ray.util.client.common import OBJECT_TRANSFER_CHUNK_SIZE, ClientObjectRef
3131
from ray.util.client.ray_client_helpers import (
32+
ray_start_client_server,
3233
ray_start_client_server_for_address,
3334
)
3435

@@ -547,6 +548,21 @@ def f(x):
547548
assert ray.get(a.doit.remote()) == "foo"
548549

549550

551+
# Regression test for https://github.com/ray-project/ray/pull/51683
552+
def test_runtime_env_py_executable(ray_start_regular):
553+
"""Test that Ray Client works with a custom py_executable."""
554+
555+
with ray_start_client_server(
556+
ray_init_kwargs={"runtime_env": {"py_executable": sys.executable + " -q"}}
557+
) as ray:
558+
559+
@ray.remote
560+
def f():
561+
return "hi"
562+
563+
assert ray.get(f.remote()) == "hi"
564+
565+
550566
def test_basic_named_actor(call_ray_start_shared):
551567
"""Test that ray.get_actor() can create and return a detached actor."""
552568
with ray_start_client_server_for_address(call_ray_start_shared) as ray:

python/ray/tests/test_client_proxy.py

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -321,9 +321,8 @@ def modify_namespace(job_config: JobConfig):
321321
(["ipython", "-m", "ray.util.client.server"], True),
322322
(["ipython -m ray.util.client.server"], True),
323323
(["ipython -m", "ray.util.client.server"], True),
324-
(["bash", "ipython", "-m", "ray.util.client.server"], False),
325-
(["bash", "ipython -m ray.util.client.server"], False),
326-
(["python", "-m", "bash", "ipython -m ray.util.client.server"], False),
324+
(["bash", "-c", "ipython -m ray.util.client.server"], True),
325+
(["python", "-m", "bash", "ipython"], False),
327326
],
328327
)
329328
def test_match_running_client_server(test_case):

python/ray/util/client/server/proxier.py

Lines changed: 3 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -100,14 +100,11 @@ def set_result(self, proc: Optional[ProcessInfo]) -> None:
100100
def _match_running_client_server(command: List[str]) -> bool:
101101
"""
102102
Detects if the main process in the given command is the RayClient Server.
103-
This works by ensuring that the the first three arguments are similar to:
104-
<python> -m ray.util.client.server
103+
This works by ensuring that the command is of the form:
104+
<py_executable> -m ray.util.client.server <args>
105105
"""
106106
flattened = " ".join(command)
107-
rejoined = flattened.split()
108-
if len(rejoined) < 3:
109-
return False
110-
return rejoined[1:3] == ["-m", "ray.util.client.server"]
107+
return "-m ray.util.client.server" in flattened
111108

112109

113110
class ProxyManager:

0 commit comments

Comments
 (0)