Skip to content

Commit 3a306ef

Browse files
authored
[Core] Shell escape worker process command in RuntimeEnvContext.exec_worker (ray-project#42332)
RuntimeEnvContext.exec_worker used " ".join(cmds) to construct the worker process command but it didn't do any shell escape. This will cause the worker process fail to start if there is any special character (e.g. ?) in the command. Instead, we should use shlex.join. Signed-off-by: Jiajun Yao <jeromeyjj@gmail.com>
1 parent 943490a commit 3a306ef

File tree

3 files changed

+43
-24
lines changed

3 files changed

+43
-24
lines changed

python/ray/_private/runtime_env/context.py

Lines changed: 24 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -2,6 +2,7 @@
22
import logging
33
import os
44
import subprocess
5+
import shlex
56
import sys
67
from typing import Any, Dict, List, Optional
78

@@ -48,11 +49,11 @@ def exec_worker(self, passthrough_args: List[str], language: Language):
4849
update_envs(self.env_vars)
4950

5051
if language == Language.PYTHON and sys.platform == "win32":
51-
executable = self.py_executable
52+
executable = [self.py_executable]
5253
elif language == Language.PYTHON:
53-
executable = f"exec {self.py_executable}"
54+
executable = ["exec", self.py_executable]
5455
elif language == Language.JAVA:
55-
executable = "java"
56+
executable = ["java"]
5657
ray_jars = os.path.join(get_ray_jars_dir(), "*")
5758

5859
local_java_jars = []
@@ -63,9 +64,9 @@ def exec_worker(self, passthrough_args: List[str], language: Language):
6364
class_path_args = ["-cp", ray_jars + ":" + str(":".join(local_java_jars))]
6465
passthrough_args = class_path_args + passthrough_args
6566
elif sys.platform == "win32":
66-
executable = ""
67+
executable = []
6768
else:
68-
executable = "exec "
69+
executable = ["exec"]
6970

7071
# By default, raylet uses the path to default_worker.py on host.
7172
# However, the path to default_worker.py inside the container
@@ -79,29 +80,29 @@ def exec_worker(self, passthrough_args: List[str], language: Language):
7980
)
8081
passthrough_args[0] = default_worker_path
8182

82-
passthrough_args = [s.replace(" ", r"\ ") for s in passthrough_args]
83-
exec_command = " ".join([f"{executable}"] + passthrough_args)
84-
command_str = " ".join(self.command_prefix + [exec_command])
85-
# TODO(SongGuyang): We add this env to command for macOS because it doesn't
86-
# work for the C++ process of `os.execvp`. We should find a better way to
87-
# fix it.
88-
MACOS_LIBRARY_PATH_ENV_NAME = "DYLD_LIBRARY_PATH"
89-
if MACOS_LIBRARY_PATH_ENV_NAME in os.environ:
90-
command_str = (
91-
MACOS_LIBRARY_PATH_ENV_NAME
92-
+ "="
93-
+ os.environ.get(MACOS_LIBRARY_PATH_ENV_NAME)
94-
+ " "
95-
+ command_str
96-
)
97-
logger.debug(f"Exec'ing worker with command: {command_str}")
9883
if sys.platform == "win32":
99-
cmd = [*self.command_prefix, executable, *passthrough_args]
84+
cmd = [*self.command_prefix, *executable, *passthrough_args]
85+
logger.debug(f"Exec'ing worker with command: {cmd}")
10086
subprocess.Popen(cmd, shell=True).wait()
10187
else:
88+
# We use shlex to do the necessary shell escape
89+
# of special characters in passthrough_args.
90+
passthrough_args = [shlex.quote(s) for s in passthrough_args]
91+
cmd = [*self.command_prefix, *executable, *passthrough_args]
92+
# TODO(SongGuyang): We add this env to command for macOS because it doesn't
93+
# work for the C++ process of `os.execvp`. We should find a better way to
94+
# fix it.
95+
MACOS_LIBRARY_PATH_ENV_NAME = "DYLD_LIBRARY_PATH"
96+
if MACOS_LIBRARY_PATH_ENV_NAME in os.environ:
97+
cmd.insert(
98+
0,
99+
f"{MACOS_LIBRARY_PATH_ENV_NAME}="
100+
f"{os.environ[MACOS_LIBRARY_PATH_ENV_NAME]}",
101+
)
102+
logger.debug(f"Exec'ing worker with command: {cmd}")
102103
# PyCharm will monkey patch the os.execvp at
103104
# .pycharm_helpers/pydev/_pydev_bundle/pydev_monkey.py
104105
# The monkey patched os.execvp function has a different
105106
# signature. So, we use os.execvp("executable", args=[])
106107
# instead of os.execvp(file="executable", args=[])
107-
os.execvp("bash", args=["bash", "-c", command_str])
108+
os.execvp("bash", args=["bash", "-c", " ".join(cmd)])

python/ray/_private/services.py

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1661,7 +1661,6 @@ def start_raylet(
16611661
f"--object-store-name={plasma_store_name}",
16621662
f"--raylet-name={raylet_name}",
16631663
f"--redis-address={redis_address}",
1664-
f"--temp-dir={temp_dir}",
16651664
f"--metrics-agent-port={metrics_agent_port}",
16661665
f"--runtime-env-agent-port={runtime_env_agent_port}",
16671666
f"--logging-rotate-bytes={max_bytes}",

python/ray/tests/test_storage.py

Lines changed: 19 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,5 @@
11
import os
2+
import sys
23
import subprocess
34
import urllib
45
from pathlib import Path
@@ -64,6 +65,24 @@ def test_get_filesystem_s3(shutdown_only):
6465
assert isinstance(fs, pyarrow.fs.S3FileSystem), fs
6566

6667

68+
@pytest.mark.skipif(
69+
sys.platform == "win32", reason="The issue is not fixed for windows yet"
70+
)
71+
def test_escape_storage_uri_with_runtime_env(shutdown_only):
72+
# https://github.com/ray-project/ray/issues/41568
73+
# Test to make sure we can successfully start worker process
74+
# when storage uri contains ? and we use runtime env.
75+
with simulate_storage("s3") as s3_uri:
76+
assert "?" in s3_uri
77+
ray.init(storage=s3_uri, runtime_env={"env_vars": {"TEST_ENV": "1"}})
78+
79+
@ray.remote
80+
def f():
81+
return 1
82+
83+
assert ray.get(f.remote()) == 1
84+
85+
6786
def test_get_filesystem_invalid(shutdown_only, tmp_path):
6887
with pytest.raises(pyarrow.lib.ArrowInvalid):
6988
ray.init(storage="blahblah://bad")

0 commit comments

Comments
 (0)