Skip to content

Commit

Permalink
[runtime env] runtime env inheritance refactor (#24538)
Browse files Browse the repository at this point in the history
* [runtime env] runtime env inheritance refactor (#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.
  • Loading branch information
SongGuyang authored May 20, 2022
1 parent d89c8aa commit eb2692c
Show file tree
Hide file tree
Showing 12 changed files with 214 additions and 264 deletions.
48 changes: 33 additions & 15 deletions doc/source/ray-core/handling-dependencies.rst
Original file line number Diff line number Diff line change
Expand Up @@ -378,29 +378,47 @@ To disable all deletion behavior (for example, for debugging purposes) you may s
Inheritance
"""""""""""

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.
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.

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:
1. By default, all actors and tasks inherit the current runtime env.

* 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.
.. code-block:: python
# Current `runtime_env`
ray.init(runtime_env={"pip": ["requests", "chess"]})
# Create child actor
ChildActor.remote()
# ChildActor's actual `runtime_env` (inherit from current runtime env)
{"pip": ["requests", "chess"]}
Example:
2. However, if you specify runtime_env for task/actor, it will override current runtime env.

.. code-block:: python
# Parent's `runtime_env`
{"pip": ["requests", "chess"],
"env_vars": {"A": "a", "B": "b"}}
# Current `runtime_env`
ray.init(runtime_env={"pip": ["requests", "chess"]})
# Create child actor
ChildActor.options(runtime_env={"env_vars": {"A": "a", "B": "b"}}).remote()
# Child's specified `runtime_env`
{"pip": ["torch", "ray[serve]"],
"env_vars": {"B": "new", "C", "c"}}
# ChildActor's actual `runtime_env` (specify runtime_env overrides)
{"env_vars": {"A": "a", "B": "b"}}
# Child's actual `runtime_env` (merged with parent's)
{"pip": ["torch", "ray[serve]"],
"env_vars": {"A": "a", "B": "new", "C", "c"}}
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.

.. code-block:: python
# Current `runtime_env`
ray.init(runtime_env={"pip": ["requests", "chess"]})
# Child updates `runtime_env`
Actor.options(runtime_env=ray.get_runtime_context().runtime_env.update({"env_vars": {"A": "a", "B": "b"}}))
# Child's actual `runtime_env` (merged with current runtime env)
{"pip": ["requests", "chess"],
"env_vars": {"A": "a", "B": "b"}}
.. _runtime-env-faq:

Expand Down
65 changes: 64 additions & 1 deletion java/test/src/main/java/io/ray/test/RuntimeEnvTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,69 @@ public void testPerJobEnvVars() {
}
}

public void testPerActorEnvVars() {
try {
Ray.init();
{
RuntimeEnv runtimeEnv =
new RuntimeEnv.Builder()
.addEnvVar("KEY1", "A")
.addEnvVar("KEY2", "B")
.addEnvVar("KEY1", "C")
.build();

ActorHandle<A> actor1 = Ray.actor(A::new).setRuntimeEnv(runtimeEnv).remote();
String val = actor1.task(A::getEnv, "KEY1").remote().get();
Assert.assertEquals(val, "C");
val = actor1.task(A::getEnv, "KEY2").remote().get();
Assert.assertEquals(val, "B");
}

{
/// Because we didn't set them for actor2 , all should be null.
ActorHandle<A> actor2 = Ray.actor(A::new).remote();
String val = actor2.task(A::getEnv, "KEY1").remote().get();
Assert.assertNull(val);
val = actor2.task(A::getEnv, "KEY2").remote().get();
Assert.assertNull(val);
}

} finally {
Ray.shutdown();
}
}

public void testPerActorEnvVarsOverwritePerJobEnvVars() {
System.setProperty("ray.job.runtime-env.env-vars.KEY1", "A");
System.setProperty("ray.job.runtime-env.env-vars.KEY2", "B");

try {
Ray.init();
{
RuntimeEnv runtimeEnv = new RuntimeEnv.Builder().addEnvVar("KEY1", "C").build();

ActorHandle<A> actor1 = Ray.actor(A::new).setRuntimeEnv(runtimeEnv).remote();
String val = actor1.task(A::getEnv, "KEY1").remote().get();
Assert.assertEquals(val, "C");
val = actor1.task(A::getEnv, "KEY2").remote().get();
Assert.assertNull(val);
}

{
/// Because we didn't set them for actor2 explicitly, it should use the per job
/// runtime env.
ActorHandle<A> actor2 = Ray.actor(A::new).remote();
String val = actor2.task(A::getEnv, "KEY1").remote().get();
Assert.assertEquals(val, "A");
val = actor2.task(A::getEnv, "KEY2").remote().get();
Assert.assertEquals(val, "B");
}

} finally {
Ray.shutdown();
}
}

private static String getEnvVar(String key) {
return System.getenv(key);
}
Expand Down Expand Up @@ -92,7 +155,7 @@ public void testPerTaskEnvVarsOverwritePerJobEnvVars() {
Ray.task(RuntimeEnvTest::getEnvVar, "KEY1").setRuntimeEnv(runtimeEnv).remote().get();
Assert.assertEquals(val, "C");
val = Ray.task(RuntimeEnvTest::getEnvVar, "KEY2").setRuntimeEnv(runtimeEnv).remote().get();
Assert.assertEquals(val, "B");
Assert.assertNull(val);
} finally {
Ray.shutdown();
}
Expand Down
23 changes: 18 additions & 5 deletions python/ray/runtime_context.py
Original file line number Diff line number Diff line change
Expand Up @@ -159,7 +159,7 @@ def should_capture_child_tasks_in_placement_group(self):
"""
return self.worker.should_capture_child_tasks_in_placement_group

def get_runtime_env_string(self):
def _get_runtime_env_string(self):
"""Get the runtime env string used for the current driver or worker.
Returns:
Expand All @@ -169,14 +169,27 @@ def get_runtime_env_string(self):

@property
def runtime_env(self):
"""Get the runtime env used for the current driver or worker.
"""Get the runtime env of the current job/worker.
If this API is called in driver or ray client, returns the job level runtime
env.
If this API is called in workers/actors, returns the worker level runtime env.
Returns:
The runtime env currently using by this worker. The type of
return value is ray.runtime_env.RuntimeEnv.
A new ray.runtime_env.RuntimeEnv instance.
To merge from the current runtime env in some specific cases, you can get the
current runtime env by this API and modify it by yourself.
Example:
>>> # Inherit current runtime env, except `env_vars`
>>> Actor.options( # doctest: +SKIP
... runtime_env=ray.get_runtime_context().runtime_env.update(
... {"env_vars": {"A": "a", "B": "b"}})
... ) # doctest: +SKIP
"""

return RuntimeEnv.deserialize(self.get_runtime_env_string())
return RuntimeEnv.deserialize(self._get_runtime_env_string())

@property
def current_actor(self):
Expand Down
2 changes: 1 addition & 1 deletion python/ray/tests/test_client.py
Original file line number Diff line number Diff line change
Expand Up @@ -742,7 +742,7 @@ def decorate_actor():
def test_init_requires_no_resources(call_ray_start, use_client):
import ray

if use_client:
if not use_client:
address = call_ray_start
ray.init(address)
else:
Expand Down
35 changes: 35 additions & 0 deletions python/ray/tests/test_runtime_env_2.py
Original file line number Diff line number Diff line change
Expand Up @@ -124,6 +124,41 @@ def run(runtime_env):
run(runtime_env)


@pytest.mark.parametrize(
"call_ray_start",
["ray start --head --ray-client-server-port 25553"],
indirect=True,
)
@pytest.mark.parametrize("use_client", [False, True])
def test_get_current_runtime_env(call_ray_start, use_client):
job_runtime_env = {"env_vars": {"a": "b"}}

if not use_client:
address = call_ray_start
ray.init(address, runtime_env=job_runtime_env)
else:
ray.init("ray://localhost:25553", runtime_env=job_runtime_env)

current_runtime_env = ray.get_runtime_context().runtime_env
current_runtime_env_2 = ray.get_runtime_context().runtime_env
# Ensure we can get a new instance for update.
assert current_runtime_env is not current_runtime_env_2
assert isinstance(current_runtime_env, dict)
assert current_runtime_env == job_runtime_env

@ray.remote
def get_runtime_env():
return ray.get_runtime_context().runtime_env

assert ray.get(get_runtime_env.remote()) == job_runtime_env

task_runtime_env = {"env_vars": {"a": "c"}}
assert (
ray.get(get_runtime_env.options(runtime_env=task_runtime_env).remote())
== task_runtime_env
)


if __name__ == "__main__":
import sys

Expand Down
30 changes: 26 additions & 4 deletions python/ray/tests/test_runtime_env_complicated.py
Original file line number Diff line number Diff line change
Expand Up @@ -887,16 +887,16 @@ def test(self):

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

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

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

return Path("./test").read_text()
return "Hello"

assert (
ray.get(
Expand All @@ -905,22 +905,44 @@ def test_pip():
== "Hello"
)

@ray.remote
def test_working_dir():
import pip_install_test # noqa

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

with pytest.raises(ray.exceptions.RayTaskError) as excinfo:
ray.get(
test_working_dir.options(
runtime_env={"pip": ["pip-install-test"]}
).remote()
)
assert "FileNotFoundError" in str(excinfo.value)

# Check that pip_install_test is not in the job's pip requirements.
with pytest.raises(ray.exceptions.RayTaskError) as excinfo:
ray.get(test_pip.remote())
assert "ModuleNotFoundError" in str(excinfo.value)

# Check that per-actor pip specification works and that the job's
# working_dir is still inherited.
# working_dir is not inherited.
@ray.remote
class TestActor:
def test(self):
import pip_install_test # noqa

return "Hello"

def test_working_dir(self):
import pip_install_test # noqa

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

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


@pytest.mark.skipif(_WIN32, reason="Fails on windows")
Expand Down
12 changes: 6 additions & 6 deletions python/ray/tests/test_runtime_env_env_vars.py
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ def get_env(key):
}
).remote("foo2")
)
== "bar2"
is None
)


Expand Down Expand Up @@ -164,7 +164,7 @@ def nested_get(self, key):

assert ray.get(a.get.remote("a")) == "b"
assert ray.get(a.get_task.remote("a")) == "b"
assert ray.get(a.nested_get.remote("a")) == "b"
assert ray.get(a.nested_get.remote("a")) is None
assert ray.get(a.nested_get.remote("c")) == "e"
assert ray.get(a.nested_get.remote("d")) == "dd"
assert (
Expand All @@ -180,9 +180,9 @@ def nested_get(self, key):
== "b"
)

assert ray.get(a.get.remote("z")) == "job_z"
assert ray.get(a.get_task.remote("z")) == "job_z"
assert ray.get(a.nested_get.remote("z")) == "job_z"
assert ray.get(a.get.remote("z")) is None
assert ray.get(a.get_task.remote("z")) is None
assert ray.get(a.nested_get.remote("z")) is None
assert (
ray.get(
get_env.options(
Expand All @@ -193,7 +193,7 @@ def nested_get(self, key):
}
).remote("z")
)
== "job_z"
is None
)


Expand Down
2 changes: 1 addition & 1 deletion python/ray/util/client/server/server.py
Original file line number Diff line number Diff line change
Expand Up @@ -236,7 +236,7 @@ def ClusterInfo(self, request, context=None) -> ray_client_pb2.ClusterInfoRespon
ctx.capture_client_tasks = (
rtc.should_capture_child_tasks_in_placement_group
)
ctx.runtime_env = rtc.get_runtime_env_string()
ctx.runtime_env = rtc._get_runtime_env_string()
resp.runtime_context.CopyFrom(ctx)
else:
with disable_client_hook():
Expand Down
2 changes: 1 addition & 1 deletion src/ray/core_worker/context.cc
Original file line number Diff line number Diff line change
Expand Up @@ -215,7 +215,7 @@ const std::string &WorkerContext::GetCurrentSerializedRuntimeEnv() const {
return runtime_env_info_.serialized_runtime_env();
}

std::shared_ptr<rpc::RuntimeEnv> WorkerContext::GetCurrentRuntimeEnv() const {
std::shared_ptr<const rpc::RuntimeEnv> WorkerContext::GetCurrentRuntimeEnv() const {
absl::ReaderMutexLock lock(&mutex_);
return runtime_env_;
}
Expand Down
3 changes: 2 additions & 1 deletion src/ray/core_worker/context.h
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,8 @@ class WorkerContext {

const std::string &GetCurrentSerializedRuntimeEnv() const LOCKS_EXCLUDED(mutex_);

std::shared_ptr<rpc::RuntimeEnv> GetCurrentRuntimeEnv() const LOCKS_EXCLUDED(mutex_);
std::shared_ptr<const rpc::RuntimeEnv> GetCurrentRuntimeEnv() const
LOCKS_EXCLUDED(mutex_);

// TODO(edoakes): remove this once Python core worker uses the task interfaces.
void SetCurrentTaskId(const TaskID &task_id, uint64_t attempt_number);
Expand Down
Loading

0 comments on commit eb2692c

Please sign in to comment.