From 1a68ebfeef2c5ba052970e3da18d3918fa1bee34 Mon Sep 17 00:00:00 2001 From: Jiajun Yao Date: Thu, 11 Jan 2024 14:57:55 +0800 Subject: [PATCH 1/4] Shell escape worker process command in RuntimeEnvContext.exec_worker Signed-off-by: Jiajun Yao --- python/ray/_private/runtime_env/context.py | 31 +++++++++++----------- python/ray/_private/services.py | 1 - python/ray/tests/test_storage.py | 15 +++++++++++ 3 files changed, 30 insertions(+), 17 deletions(-) diff --git a/python/ray/_private/runtime_env/context.py b/python/ray/_private/runtime_env/context.py index aef28dfd4ae4..2e57f20f08d8 100644 --- a/python/ray/_private/runtime_env/context.py +++ b/python/ray/_private/runtime_env/context.py @@ -2,6 +2,7 @@ import logging import os import subprocess +import shlex import sys from typing import Any, Dict, List, Optional @@ -48,11 +49,11 @@ def exec_worker(self, passthrough_args: List[str], language: Language): update_envs(self.env_vars) if language == Language.PYTHON and sys.platform == "win32": - executable = self.py_executable + executable = [self.py_executable] elif language == Language.PYTHON: - executable = f"exec {self.py_executable}" + executable = ["exec", self.py_executable] elif language == Language.JAVA: - executable = "java" + executable = ["java"] ray_jars = os.path.join(get_ray_jars_dir(), "*") local_java_jars = [] @@ -63,9 +64,9 @@ def exec_worker(self, passthrough_args: List[str], language: Language): class_path_args = ["-cp", ray_jars + ":" + str(":".join(local_java_jars))] passthrough_args = class_path_args + passthrough_args elif sys.platform == "win32": - executable = "" + executable = [] else: - executable = "exec " + executable = ["exec"] # By default, raylet uses the path to default_worker.py on host. # However, the path to default_worker.py inside the container @@ -80,23 +81,19 @@ def exec_worker(self, passthrough_args: List[str], language: Language): passthrough_args[0] = default_worker_path passthrough_args = [s.replace(" ", r"\ ") for s in passthrough_args] - exec_command = " ".join([f"{executable}"] + passthrough_args) - command_str = " ".join(self.command_prefix + [exec_command]) + cmd = [*self.command_prefix, *executable, *passthrough_args] # TODO(SongGuyang): We add this env to command for macOS because it doesn't # work for the C++ process of `os.execvp`. We should find a better way to # fix it. MACOS_LIBRARY_PATH_ENV_NAME = "DYLD_LIBRARY_PATH" if MACOS_LIBRARY_PATH_ENV_NAME in os.environ: - command_str = ( - MACOS_LIBRARY_PATH_ENV_NAME - + "=" - + os.environ.get(MACOS_LIBRARY_PATH_ENV_NAME) - + " " - + command_str + cmd.insert( + 0, + f"{MACOS_LIBRARY_PATH_ENV_NAME}=" + f"{os.environ[MACOS_LIBRARY_PATH_ENV_NAME]}", ) - logger.debug(f"Exec'ing worker with command: {command_str}") + logger.debug(f"Exec'ing worker with command: {cmd}") if sys.platform == "win32": - cmd = [*self.command_prefix, executable, *passthrough_args] subprocess.Popen(cmd, shell=True).wait() else: # PyCharm will monkey patch the os.execvp at @@ -104,4 +101,6 @@ def exec_worker(self, passthrough_args: List[str], language: Language): # The monkey patched os.execvp function has a different # signature. So, we use os.execvp("executable", args=[]) # instead of os.execvp(file="executable", args=[]) - os.execvp("bash", args=["bash", "-c", command_str]) + # We use shlex to do the necessary shell escape + # of special characters in cmd. + os.execvp("bash", args=["bash", "-c", shlex.join(cmd)])" " diff --git a/python/ray/_private/services.py b/python/ray/_private/services.py index 867b748a0e8d..c09b5c0c9fc4 100644 --- a/python/ray/_private/services.py +++ b/python/ray/_private/services.py @@ -1661,7 +1661,6 @@ def start_raylet( f"--object-store-name={plasma_store_name}", f"--raylet-name={raylet_name}", f"--redis-address={redis_address}", - f"--temp-dir={temp_dir}", f"--metrics-agent-port={metrics_agent_port}", f"--runtime-env-agent-port={runtime_env_agent_port}", f"--logging-rotate-bytes={max_bytes}", diff --git a/python/ray/tests/test_storage.py b/python/ray/tests/test_storage.py index f1d7659cf17c..d6121a658456 100644 --- a/python/ray/tests/test_storage.py +++ b/python/ray/tests/test_storage.py @@ -64,6 +64,21 @@ def test_get_filesystem_s3(shutdown_only): assert isinstance(fs, pyarrow.fs.S3FileSystem), fs +def test_escape_storage_uri_with_runtime_env(shutdown_only): + # https://github.com/ray-project/ray/issues/41568 + # Test to make sure we can successfully start worker process + # when storage uri contains ? and we use runtime env. + with simulate_storage("s3") as s3_uri: + assert "?" in s3_uri + ray.init(storage=s3_uri, runtime_env={"env_vars": {"TEST_ENV": "1"}}) + + @ray.remote + def f(): + return 1 + + assert ray.get(f.remote()) == 1 + + def test_get_filesystem_invalid(shutdown_only, tmp_path): with pytest.raises(pyarrow.lib.ArrowInvalid): ray.init(storage="blahblah://bad") From 98ac860550720d8a010688132cb4bac5615bce15 Mon Sep 17 00:00:00 2001 From: Jiajun Yao Date: Thu, 11 Jan 2024 18:40:24 +0800 Subject: [PATCH 2/4] fix Signed-off-by: Jiajun Yao --- python/ray/_private/runtime_env/context.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/_private/runtime_env/context.py b/python/ray/_private/runtime_env/context.py index 2e57f20f08d8..b435dd10c685 100644 --- a/python/ray/_private/runtime_env/context.py +++ b/python/ray/_private/runtime_env/context.py @@ -103,4 +103,4 @@ def exec_worker(self, passthrough_args: List[str], language: Language): # instead of os.execvp(file="executable", args=[]) # We use shlex to do the necessary shell escape # of special characters in cmd. - os.execvp("bash", args=["bash", "-c", shlex.join(cmd)])" " + os.execvp("bash", args=["bash", "-c", shlex.join(cmd)]) From d18543df77bcb4166b296a0f456ac3842557c8f1 Mon Sep 17 00:00:00 2001 From: Jiajun Yao Date: Fri, 12 Jan 2024 06:51:25 +0800 Subject: [PATCH 3/4] up Signed-off-by: Jiajun Yao --- python/ray/_private/runtime_env/context.py | 34 ++++++++++++---------- 1 file changed, 18 insertions(+), 16 deletions(-) diff --git a/python/ray/_private/runtime_env/context.py b/python/ray/_private/runtime_env/context.py index b435dd10c685..733860da475b 100644 --- a/python/ray/_private/runtime_env/context.py +++ b/python/ray/_private/runtime_env/context.py @@ -80,27 +80,29 @@ def exec_worker(self, passthrough_args: List[str], language: Language): ) passthrough_args[0] = default_worker_path - passthrough_args = [s.replace(" ", r"\ ") for s in passthrough_args] - cmd = [*self.command_prefix, *executable, *passthrough_args] - # TODO(SongGuyang): We add this env to command for macOS because it doesn't - # work for the C++ process of `os.execvp`. We should find a better way to - # fix it. - MACOS_LIBRARY_PATH_ENV_NAME = "DYLD_LIBRARY_PATH" - if MACOS_LIBRARY_PATH_ENV_NAME in os.environ: - cmd.insert( - 0, - f"{MACOS_LIBRARY_PATH_ENV_NAME}=" - f"{os.environ[MACOS_LIBRARY_PATH_ENV_NAME]}", - ) - logger.debug(f"Exec'ing worker with command: {cmd}") if sys.platform == "win32": + cmd = [*self.command_prefix, *executable, *passthrough_args] + logger.debug(f"Exec'ing worker with command: {cmd}") subprocess.Popen(cmd, shell=True).wait() else: + # We use shlex to do the necessary shell escape + # of special characters in passthrough_args. + passthrough_args = [shlex.quote(s) for s in passthrough_args] + cmd = [*self.command_prefix, *executable, *passthrough_args] + # TODO(SongGuyang): We add this env to command for macOS because it doesn't + # work for the C++ process of `os.execvp`. We should find a better way to + # fix it. + MACOS_LIBRARY_PATH_ENV_NAME = "DYLD_LIBRARY_PATH" + if MACOS_LIBRARY_PATH_ENV_NAME in os.environ: + cmd.insert( + 0, + f"{MACOS_LIBRARY_PATH_ENV_NAME}=" + f"{os.environ[MACOS_LIBRARY_PATH_ENV_NAME]}", + ) + logger.debug(f"Exec'ing worker with command: {cmd}") # PyCharm will monkey patch the os.execvp at # .pycharm_helpers/pydev/_pydev_bundle/pydev_monkey.py # The monkey patched os.execvp function has a different # signature. So, we use os.execvp("executable", args=[]) # instead of os.execvp(file="executable", args=[]) - # We use shlex to do the necessary shell escape - # of special characters in cmd. - os.execvp("bash", args=["bash", "-c", shlex.join(cmd)]) + os.execvp("bash", args=["bash", "-c", " ".join(cmd)]) From 9cf467b5945ddaaeb702bdd7b6a0b6bd1afd86dd Mon Sep 17 00:00:00 2001 From: Jiajun Yao Date: Fri, 12 Jan 2024 14:41:34 +0800 Subject: [PATCH 4/4] up Signed-off-by: Jiajun Yao --- python/ray/tests/test_storage.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/ray/tests/test_storage.py b/python/ray/tests/test_storage.py index d6121a658456..952a86f94068 100644 --- a/python/ray/tests/test_storage.py +++ b/python/ray/tests/test_storage.py @@ -1,4 +1,5 @@ import os +import sys import subprocess import urllib from pathlib import Path @@ -64,6 +65,9 @@ def test_get_filesystem_s3(shutdown_only): assert isinstance(fs, pyarrow.fs.S3FileSystem), fs +@pytest.mark.skipif( + sys.platform == "win32", reason="The issue is not fixed for windows yet" +) def test_escape_storage_uri_with_runtime_env(shutdown_only): # https://github.com/ray-project/ray/issues/41568 # Test to make sure we can successfully start worker process