Skip to content

Commit 7befc02

Browse files
committed
[runtime env] runtime env inheritance refactor (ray-project#22244)
Runtime Environments is already GA in Ray 1.6.0. The latest doc is [here](https://docs.ray.io/en/master/ray-core/handling-dependencies.html#runtime-environments). And now, we already supported a [inheritance](https://docs.ray.io/en/master/ray-core/handling-dependencies.html#inheritance) behavior as follows (copied from the doc): - The runtime_env["env_vars"] field will be merged with the runtime_env["env_vars"] field of the parent. This allows for environment variables set in the parent’s runtime environment to be automatically propagated to the child, even if new environment variables are set in the child’s runtime environment. - Every other field in the runtime_env will be overridden by the child, not merged. For example, if runtime_env["py_modules"] is specified, it will replace the runtime_env["py_modules"] field of the parent. We think this runtime env merging logic is so complex and confusing to users because users can't know the final runtime env before the jobs are run. Current PR tries to do a refactor and change the behavior of Runtime Environments inheritance. Here is the new behavior: - **If there is no runtime env option when we create actor, inherit the parent runtime env.** - **Otherwise, use the optional runtime env directly and don't do the merging.** Add a new API named `ray.runtime_env.get_current_runtime_env()` to get the parent runtime env and modify this dict by yourself. Like: ```Actor.options(runtime_env=ray.runtime_env.get_current_runtime_env().update({"X": "Y"}))``` This new API also can be used in ray client.
1 parent 5a7c5ab commit 7befc02

File tree

13 files changed

+166
-258
lines changed

13 files changed

+166
-258
lines changed

doc/source/ray-core/handling-dependencies.rst

Lines changed: 33 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -378,29 +378,47 @@ To disable all deletion behavior (for example, for debugging purposes) you may s
378378
Inheritance
379379
"""""""""""
380380

381-
The runtime environment is inheritable, so it will apply to all tasks/actors within a job and all child tasks/actors of a task or actor once set, unless it is overridden.
381+
The runtime environment is inheritable, so it will apply to all tasks/actors within a job and all child tasks/actors of a task or actor once set, unless it is overridden by explicitly specifying a runtime environment for the child task/actor.
382382

383-
If an actor or task specifies a new ``runtime_env``, it will override the parent’s ``runtime_env`` (i.e., the parent actor/task's ``runtime_env``, or the job's ``runtime_env`` if there is no parent actor or task) as follows:
383+
1. By default, all actors and tasks inherit the current runtime env.
384384

385-
* The ``runtime_env["env_vars"]`` field will be merged with the ``runtime_env["env_vars"]`` field of the parent.
386-
This allows for environment variables set in the parent's runtime environment to be automatically propagated to the child, even if new environment variables are set in the child's runtime environment.
387-
* Every other field in the ``runtime_env`` will be *overridden* by the child, not merged. For example, if ``runtime_env["py_modules"]`` is specified, it will replace the ``runtime_env["py_modules"]`` field of the parent.
385+
.. code-block:: python
386+
387+
# Current `runtime_env`
388+
ray.init(runtime_env={"pip": ["requests", "chess"]})
389+
390+
# Create child actor
391+
ChildActor.remote()
392+
393+
# ChildActor's actual `runtime_env` (inherit from current runtime env)
394+
{"pip": ["requests", "chess"]}
388395
389-
Example:
396+
2. However, if you specify runtime_env for task/actor, it will override current runtime env.
390397

391398
.. code-block:: python
392399
393-
# Parent's `runtime_env`
394-
{"pip": ["requests", "chess"],
395-
"env_vars": {"A": "a", "B": "b"}}
400+
# Current `runtime_env`
401+
ray.init(runtime_env={"pip": ["requests", "chess"]})
402+
403+
# Create child actor
404+
ChildActor.options(runtime_env={"env_vars": {"A": "a", "B": "b"}}).remote()
396405
397-
# Child's specified `runtime_env`
398-
{"pip": ["torch", "ray[serve]"],
399-
"env_vars": {"B": "new", "C", "c"}}
406+
# ChildActor's actual `runtime_env` (specify runtime_env overrides)
407+
{"env_vars": {"A": "a", "B": "b"}}
400408
401-
# Child's actual `runtime_env` (merged with parent's)
402-
{"pip": ["torch", "ray[serve]"],
403-
"env_vars": {"A": "a", "B": "new", "C", "c"}}
409+
3. If you'd like to still use current runtime env, you can use the API :ref:`ray.get_current_runtime_env() <runtime-env-apis>` to get the current runtime env and modify it by yourself.
410+
411+
.. code-block:: python
412+
413+
# Current `runtime_env`
414+
ray.init(runtime_env={"pip": ["requests", "chess"]})
415+
416+
# Child updates `runtime_env`
417+
Actor.options(runtime_env=ray.get_current_runtime_env().update({"env_vars": {"A": "a", "B": "b"}}))
418+
419+
# Child's actual `runtime_env` (merged with current runtime env)
420+
{"pip": ["requests", "chess"],
421+
"env_vars": {"A": "a", "B": "b"}}
404422
405423
406424
.. _remote-uris:

doc/source/ray-core/package-ref.rst

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -211,14 +211,15 @@ Runtime Context APIs
211211
.. _runtime-env-apis:
212212

213213
Runtime Env APIs
214-
----------------
215214

216215
.. autoclass:: ray.runtime_env.RuntimeEnvConfig
217216
:members:
218217

219218
.. autoclass:: ray.runtime_env.RuntimeEnv
220219
:members:
221220

221+
.. autofunction:: ray.runtime_env.get_current_runtime_env
222+
222223
.. _package-ref-debugging-apis:
223224

224225
Debugging APIs

python/ray/__init__.py

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -171,6 +171,7 @@ def _configure_system():
171171
from ray import util # noqa: E402
172172
from ray import _private # noqa: E402,F401
173173
from ray import workflow # noqa: E402,F401
174+
from ray import runtime_env # noqa: E402,F401
174175

175176
# We import ClientBuilder so that modules can inherit from `ray.ClientBuilder`.
176177
from ray.client_builder import client, ClientBuilder # noqa: E402
@@ -210,6 +211,7 @@ def _configure_system():
210211
"LOCAL_MODE",
211212
"SCRIPT_MODE",
212213
"WORKER_MODE",
214+
"runtime_env",
213215
]
214216

215217
# ID types

python/ray/runtime_env.py

Lines changed: 24 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -710,3 +710,27 @@ def _build_proto_plugin_runtime_env(self, runtime_env: ProtoRuntimeEnv):
710710
plugin = runtime_env.python_runtime_env.plugin_runtime_env.plugins.add()
711711
plugin.class_path = class_path
712712
plugin.config = plugin_field
713+
714+
715+
@PublicAPI(stability="beta")
716+
@client_mode_hook(auto_init=False)
717+
def get_current_runtime_env():
718+
"""Get the runtime env of the current job/worker.
719+
720+
If this API is called in driver or ray client, returns the job level runtime env.
721+
If this API is called in workers/actors, returns the worker level runtime env.
722+
723+
Returns:
724+
A dict of the current runtime env
725+
726+
To merge from the parent runtime env in some specific cases, you can get the parent
727+
runtime env by this API and modify it by yourself.
728+
729+
Example:
730+
731+
>>> # Inherit parent runtime env, except `env_vars`
732+
>>> Actor.options(runtime_env=ray.get_current_runtime_env().update(
733+
{"env_vars": {"A": "a", "B": "b"}}))
734+
"""
735+
736+
return dict(ray.get_runtime_context().runtime_env)

python/ray/tests/test_client.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -741,7 +741,7 @@ def decorate_actor():
741741
def test_init_requires_no_resources(call_ray_start, use_client):
742742
import ray
743743

744-
if use_client:
744+
if not use_client:
745745
address = call_ray_start
746746
ray.init(address)
747747
else:

python/ray/tests/test_runtime_env.py

Lines changed: 32 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -549,6 +549,38 @@ def f(self):
549549
ray.get(a2.f.remote())
550550

551551

552+
@pytest.mark.parametrize(
553+
"call_ray_start",
554+
["ray start --head --ray-client-server-port 25553"],
555+
indirect=True,
556+
)
557+
@pytest.mark.parametrize("use_client", [False, True])
558+
def test_get_current_runtime_env(call_ray_start, use_client):
559+
job_runtime_env = {"env_vars": {"a": "b"}}
560+
561+
if not use_client:
562+
address = call_ray_start
563+
ray.init(address, runtime_env=job_runtime_env)
564+
else:
565+
ray.init("ray://localhost:25553", runtime_env=job_runtime_env)
566+
567+
current_runtime_env = ray.runtime_env.get_current_runtime_env()
568+
assert type(current_runtime_env) is dict
569+
assert current_runtime_env == job_runtime_env
570+
571+
@ray.remote
572+
def get_runtime_env():
573+
return ray.runtime_env.get_current_runtime_env()
574+
575+
assert ray.get(get_runtime_env.remote()) == job_runtime_env
576+
577+
task_runtime_env = {"env_vars": {"a": "c"}}
578+
assert (
579+
ray.get(get_runtime_env.options(runtime_env=task_runtime_env).remote())
580+
== task_runtime_env
581+
)
582+
583+
552584
MY_PLUGIN_CLASS_PATH = "ray.tests.test_runtime_env.MyPlugin"
553585
success_retry_number = 3
554586
runtime_env_retry_times = 0

python/ray/tests/test_runtime_env_complicated.py

Lines changed: 26 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -887,16 +887,16 @@ def test(self):
887887

888888
return Path("./test").read_text()
889889

890-
a = TestActor.options(runtime_env={"pip": str(requirement_path)}).remote()
890+
a = TestActor.remote()
891891
assert ray.get(a.test.remote()) == "Hello"
892892

893893
# Check that per-task pip specification works and that the job's
894-
# working_dir is still inherited.
894+
# working_dir is not inherited.
895895
@ray.remote
896896
def test_pip():
897897
import pip_install_test # noqa
898898

899-
return Path("./test").read_text()
899+
return "Hello"
900900

901901
assert (
902902
ray.get(
@@ -905,22 +905,44 @@ def test_pip():
905905
== "Hello"
906906
)
907907

908+
@ray.remote
909+
def test_working_dir():
910+
import pip_install_test # noqa
911+
912+
return Path("./test").read_text()
913+
914+
with pytest.raises(ray.exceptions.RayTaskError) as excinfo:
915+
ray.get(
916+
test_working_dir.options(
917+
runtime_env={"pip": ["pip-install-test"]}
918+
).remote()
919+
)
920+
assert "FileNotFoundError" in str(excinfo.value)
921+
908922
# Check that pip_install_test is not in the job's pip requirements.
909923
with pytest.raises(ray.exceptions.RayTaskError) as excinfo:
910924
ray.get(test_pip.remote())
911925
assert "ModuleNotFoundError" in str(excinfo.value)
912926

913927
# Check that per-actor pip specification works and that the job's
914-
# working_dir is still inherited.
928+
# working_dir is not inherited.
915929
@ray.remote
916930
class TestActor:
917931
def test(self):
918932
import pip_install_test # noqa
919933

934+
return "Hello"
935+
936+
def test_working_dir(self):
937+
import pip_install_test # noqa
938+
920939
return Path("./test").read_text()
921940

922941
a = TestActor.options(runtime_env={"pip": ["pip-install-test"]}).remote()
923942
assert ray.get(a.test.remote()) == "Hello"
943+
with pytest.raises(ray.exceptions.RayTaskError) as excinfo:
944+
ray.get(a.test_working_dir.remote())
945+
assert "FileNotFoundError" in str(excinfo.value)
924946

925947

926948
@pytest.mark.skipif(_WIN32, reason="Fails on windows")

python/ray/tests/test_runtime_env_env_vars.py

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -107,7 +107,7 @@ def get_env(key):
107107
}
108108
).remote("foo2")
109109
)
110-
== "bar2"
110+
is None
111111
)
112112

113113

@@ -164,7 +164,7 @@ def nested_get(self, key):
164164

165165
assert ray.get(a.get.remote("a")) == "b"
166166
assert ray.get(a.get_task.remote("a")) == "b"
167-
assert ray.get(a.nested_get.remote("a")) == "b"
167+
assert ray.get(a.nested_get.remote("a")) is None
168168
assert ray.get(a.nested_get.remote("c")) == "e"
169169
assert ray.get(a.nested_get.remote("d")) == "dd"
170170
assert (
@@ -180,9 +180,9 @@ def nested_get(self, key):
180180
== "b"
181181
)
182182

183-
assert ray.get(a.get.remote("z")) == "job_z"
184-
assert ray.get(a.get_task.remote("z")) == "job_z"
185-
assert ray.get(a.nested_get.remote("z")) == "job_z"
183+
assert ray.get(a.get.remote("z")) is None
184+
assert ray.get(a.get_task.remote("z")) is None
185+
assert ray.get(a.nested_get.remote("z")) is None
186186
assert (
187187
ray.get(
188188
get_env.options(
@@ -193,7 +193,7 @@ def nested_get(self, key):
193193
}
194194
).remote("z")
195195
)
196-
== "job_z"
196+
is None
197197
)
198198

199199

python/ray/util/client/api.py

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -287,6 +287,14 @@ def get_runtime_context(self):
287287
"""
288288
return ClientWorkerPropertyAPI(self.worker).build_runtime_context()
289289

290+
def get_current_runtime_env(self):
291+
"""Get the runtime env of the current client/driver.
292+
293+
Returns:
294+
A dict of current runtime env.
295+
"""
296+
return dict(self.get_runtime_context().runtime_env)
297+
290298
# Client process isn't assigned any GPUs.
291299
def get_gpu_ids(self) -> list:
292300
return []

src/ray/core_worker/context.cc

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -215,7 +215,7 @@ const std::string &WorkerContext::GetCurrentSerializedRuntimeEnv() const {
215215
return runtime_env_info_.serialized_runtime_env();
216216
}
217217

218-
std::shared_ptr<rpc::RuntimeEnv> WorkerContext::GetCurrentRuntimeEnv() const {
218+
std::shared_ptr<const rpc::RuntimeEnv> WorkerContext::GetCurrentRuntimeEnv() const {
219219
absl::ReaderMutexLock lock(&mutex_);
220220
return runtime_env_;
221221
}

0 commit comments

Comments
 (0)