mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
[runtime_env] Allow working_dir and py_module to be Path type (#20810)
This commit is contained in:
parent
765e8d8d53
commit
fb3d57fb71
3 changed files with 23 additions and 4 deletions
|
@ -2,6 +2,7 @@ import logging
|
|||
import os
|
||||
from types import ModuleType
|
||||
from typing import Any, Dict, Optional
|
||||
from pathlib import Path
|
||||
|
||||
from ray.experimental.internal_kv import _internal_kv_initialized
|
||||
from ray._private.runtime_env.context import RuntimeEnvContext
|
||||
|
@ -47,6 +48,8 @@ def upload_py_modules_if_needed(
|
|||
if isinstance(module, str):
|
||||
# module_path is a local path or a URI.
|
||||
module_path = module
|
||||
elif isinstance(module, Path):
|
||||
module_path = str(module)
|
||||
elif isinstance(module, ModuleType):
|
||||
# NOTE(edoakes): Python allows some installed Python packages to
|
||||
# be split into multiple directories. We could probably handle
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
import logging
|
||||
import os
|
||||
from typing import Any, Dict, Optional
|
||||
from pathlib import Path
|
||||
|
||||
from ray._private.runtime_env.utils import RuntimeEnv
|
||||
from ray.experimental.internal_kv import _internal_kv_initialized
|
||||
|
@ -24,10 +25,13 @@ def upload_working_dir_if_needed(
|
|||
if working_dir is None:
|
||||
return runtime_env
|
||||
|
||||
if not isinstance(working_dir, str):
|
||||
if not isinstance(working_dir, str) and not isinstance(working_dir, Path):
|
||||
raise TypeError(
|
||||
"working_dir must be a string (either a local path or remote "
|
||||
f"URI), got {type(working_dir)}.")
|
||||
"working_dir must be a string or Path (either a local path "
|
||||
f"or remote URI), got {type(working_dir)}.")
|
||||
|
||||
if isinstance(working_dir, Path):
|
||||
working_dir = str(working_dir)
|
||||
|
||||
# working_dir is already a URI -- just pass it through.
|
||||
try:
|
||||
|
|
|
@ -46,7 +46,10 @@ def tmp_working_dir():
|
|||
yield tmp_dir
|
||||
|
||||
|
||||
@pytest.mark.parametrize("option", ["failure", "working_dir", "py_modules"])
|
||||
@pytest.mark.parametrize("option", [
|
||||
"failure", "working_dir", "py_modules", "py_modules_path",
|
||||
"working_dir_path"
|
||||
])
|
||||
@pytest.mark.skipif(sys.platform == "win32", reason="Fail to create temp dir.")
|
||||
def test_lazy_reads(start_cluster, tmp_working_dir, option: str):
|
||||
"""Tests the case where we lazily read files or import inside a task/actor.
|
||||
|
@ -62,12 +65,21 @@ def test_lazy_reads(start_cluster, tmp_working_dir, option: str):
|
|||
ray.init(address)
|
||||
elif option == "working_dir":
|
||||
ray.init(address, runtime_env={"working_dir": tmp_working_dir})
|
||||
elif option == "working_dir_path":
|
||||
ray.init(
|
||||
address, runtime_env={"working_dir": Path(tmp_working_dir)})
|
||||
elif option == "py_modules":
|
||||
ray.init(
|
||||
address,
|
||||
runtime_env={
|
||||
"py_modules": [str(Path(tmp_working_dir) / "test_module")]
|
||||
})
|
||||
elif option == "py_modules_path":
|
||||
ray.init(
|
||||
address,
|
||||
runtime_env={
|
||||
"py_modules": [Path(tmp_working_dir) / "test_module"]
|
||||
})
|
||||
|
||||
call_ray_init()
|
||||
|
||||
|
|
Loading…
Add table
Reference in a new issue