mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
[Core] Disable runtime env logs (#22198)
Disable runtime env logs streamed to the driver by default and improve the documentation.
This commit is contained in:
parent
fc88b0895e
commit
e5cab878b8
5 changed files with 56 additions and 39 deletions
|
@ -557,3 +557,37 @@ Here is a list of different use cases and corresponding URLs:
|
||||||
Once you have specified the URL in your ``runtime_env`` dictionary, you can pass the dictionary
|
Once you have specified the URL in your ``runtime_env`` dictionary, you can pass the dictionary
|
||||||
into a ``ray.init()`` or ``.options()`` call. Congratulations! You have now hosted a ``runtime_env`` dependency
|
into a ``ray.init()`` or ``.options()`` call. Congratulations! You have now hosted a ``runtime_env`` dependency
|
||||||
remotely on GitHub!
|
remotely on GitHub!
|
||||||
|
|
||||||
|
|
||||||
|
Debugging
|
||||||
|
---------
|
||||||
|
If runtime_env cannot be set up (e.g., network issues, download failures, etc.), Ray will fail to schedule tasks/actors
|
||||||
|
that require the runtime_env. If you call ``ray.get``, it will raise ``RuntimeEnvSetupError`` with
|
||||||
|
the error message in detail.
|
||||||
|
|
||||||
|
.. code-block:: python
|
||||||
|
|
||||||
|
@ray.remote
|
||||||
|
def f():
|
||||||
|
pass
|
||||||
|
|
||||||
|
@ray.remote
|
||||||
|
class A:
|
||||||
|
def f(self):
|
||||||
|
pass
|
||||||
|
|
||||||
|
start = time.time()
|
||||||
|
bad_env = {"conda": {"dependencies": ["this_doesnt_exist"]}}
|
||||||
|
|
||||||
|
# [Tasks] will raise `RuntimeEnvSetupError`.
|
||||||
|
ray.get(f.options(runtime_env=bad_env).remote())
|
||||||
|
|
||||||
|
# [Actors] will raise `RuntimeEnvSetupError`.
|
||||||
|
a = A.options(runtime_env=bad_env).remote()
|
||||||
|
ray.get(a.f.remote())
|
||||||
|
|
||||||
|
You can also enable runtime_env debugging logs by setting an environment variable ``RAY_RUNTIME_ENV_LOG_TO_DRIVER_ENABLED=1``.
|
||||||
|
It will print the full runtime_env setup log messages to the driver.
|
||||||
|
The same information can be found from the log file ``runtime_env*.log`` from the log directory.
|
||||||
|
|
||||||
|
Look :ref:`Logging Directory Structure <logging-directory-structure>` for more details.
|
||||||
|
|
|
@ -38,7 +38,7 @@ RAY_LOG_MONITOR_MANY_FILES_THRESHOLD = int(
|
||||||
os.getenv("RAY_LOG_MONITOR_MANY_FILES_THRESHOLD", 1000)
|
os.getenv("RAY_LOG_MONITOR_MANY_FILES_THRESHOLD", 1000)
|
||||||
)
|
)
|
||||||
RAY_RUNTIME_ENV_LOG_TO_DRIVER_ENABLED = int(
|
RAY_RUNTIME_ENV_LOG_TO_DRIVER_ENABLED = int(
|
||||||
os.getenv("RAY_RUNTIME_ENV_LOG_TO_DRIVER_ENABLED", 1)
|
os.getenv("RAY_RUNTIME_ENV_LOG_TO_DRIVER_ENABLED", 0)
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -186,7 +186,7 @@ def exec_cmd(
|
||||||
|
|
||||||
|
|
||||||
def exec_cmd_stream_to_logger(
|
def exec_cmd_stream_to_logger(
|
||||||
cmd: List[str], logger: logging.Logger, n_lines: int = 10
|
cmd: List[str], logger: logging.Logger, n_lines: int = 50
|
||||||
) -> Tuple[int, str]:
|
) -> Tuple[int, str]:
|
||||||
"""Runs a command as a child process, streaming output to the logger.
|
"""Runs a command as a child process, streaming output to the logger.
|
||||||
|
|
||||||
|
|
|
@ -22,7 +22,6 @@ from ray._private.test_utils import (
|
||||||
run_string_as_driver,
|
run_string_as_driver,
|
||||||
run_string_as_driver_nonblocking,
|
run_string_as_driver_nonblocking,
|
||||||
wait_for_condition,
|
wait_for_condition,
|
||||||
get_log_batch,
|
|
||||||
chdir,
|
chdir,
|
||||||
)
|
)
|
||||||
from ray._private.utils import get_conda_env_dir, get_conda_bin_executable
|
from ray._private.utils import get_conda_env_dir, get_conda_bin_executable
|
||||||
|
@ -935,25 +934,6 @@ def test_runtime_env_override(call_ray_start):
|
||||||
ray.shutdown()
|
ray.shutdown()
|
||||||
|
|
||||||
|
|
||||||
@pytest.mark.skipif(
|
|
||||||
os.environ.get("CI") and sys.platform != "linux",
|
|
||||||
reason="This test is only run on linux CI machines.",
|
|
||||||
)
|
|
||||||
def test_runtime_env_logging_to_driver(ray_start_regular_shared, log_pubsub):
|
|
||||||
@ray.remote(runtime_env={"pip": [f"requests=={REQUEST_VERSIONS[0]}"]})
|
|
||||||
def func():
|
|
||||||
pass
|
|
||||||
|
|
||||||
ray.get(func.remote())
|
|
||||||
|
|
||||||
# Check the stderr from the worker.
|
|
||||||
def matcher(log_batch):
|
|
||||||
return log_batch["pid"] == "runtime_env"
|
|
||||||
|
|
||||||
match = get_log_batch(log_pubsub, 1, timeout=5, matcher=matcher)
|
|
||||||
assert len(match) > 0
|
|
||||||
|
|
||||||
|
|
||||||
if __name__ == "__main__":
|
if __name__ == "__main__":
|
||||||
import sys
|
import sys
|
||||||
|
|
||||||
|
|
|
@ -3,7 +3,7 @@ from typing import Dict
|
||||||
import pytest
|
import pytest
|
||||||
import sys
|
import sys
|
||||||
from ray.exceptions import RuntimeEnvSetupError
|
from ray.exceptions import RuntimeEnvSetupError
|
||||||
from ray._private.test_utils import wait_for_condition, generate_runtime_env_dict
|
from ray._private.test_utils import generate_runtime_env_dict
|
||||||
import ray
|
import ray
|
||||||
|
|
||||||
if not os.environ.get("CI"):
|
if not os.environ.get("CI"):
|
||||||
|
@ -13,10 +13,14 @@ if not os.environ.get("CI"):
|
||||||
|
|
||||||
|
|
||||||
@pytest.mark.parametrize("field", ["conda", "pip"])
|
@pytest.mark.parametrize("field", ["conda", "pip"])
|
||||||
@pytest.mark.parametrize("specify_env_in_init", [False, True])
|
@pytest.mark.parametrize("specify_env_in_init", [True, False])
|
||||||
@pytest.mark.parametrize("spec_format", ["file", "python_object"])
|
@pytest.mark.parametrize("spec_format", ["file", "python_object"])
|
||||||
def test_install_failure_logging(
|
def test_install_failure_logging(
|
||||||
start_cluster, specify_env_in_init, field, spec_format, tmp_path, capsys
|
start_cluster,
|
||||||
|
specify_env_in_init,
|
||||||
|
field,
|
||||||
|
spec_format,
|
||||||
|
tmp_path,
|
||||||
):
|
):
|
||||||
cluster, address = start_cluster
|
cluster, address = start_cluster
|
||||||
using_ray_client = address.startswith("ray://")
|
using_ray_client = address.startswith("ray://")
|
||||||
|
@ -36,34 +40,33 @@ def test_install_failure_logging(
|
||||||
assert bad_packages["init"] in str(excinfo.value)
|
assert bad_packages["init"] in str(excinfo.value)
|
||||||
else:
|
else:
|
||||||
ray.init(address, runtime_env=bad_envs["init"])
|
ray.init(address, runtime_env=bad_envs["init"])
|
||||||
wait_for_condition(
|
|
||||||
lambda: bad_packages["init"] in capsys.readouterr().out, timeout=30
|
@ray.remote
|
||||||
)
|
def g():
|
||||||
|
pass
|
||||||
|
|
||||||
|
with pytest.raises(RuntimeEnvSetupError, match=bad_packages["init"]):
|
||||||
|
ray.get(g.remote())
|
||||||
return
|
return
|
||||||
|
|
||||||
ray.init(address)
|
ray.init(address)
|
||||||
|
|
||||||
@ray.remote(runtime_env=bad_envs["actor"])
|
@ray.remote(runtime_env=bad_envs["actor"])
|
||||||
class A:
|
class A:
|
||||||
|
def f(self):
|
||||||
pass
|
pass
|
||||||
|
|
||||||
a = A.remote() # noqa
|
a = A.remote() # noqa
|
||||||
|
with pytest.raises(RuntimeEnvSetupError, match=bad_packages["actor"]):
|
||||||
wait_for_condition(
|
ray.get(a.f.remote())
|
||||||
lambda: bad_packages["actor"] in capsys.readouterr().out, timeout=30
|
|
||||||
)
|
|
||||||
|
|
||||||
@ray.remote(runtime_env=bad_envs["task"])
|
@ray.remote(runtime_env=bad_envs["task"])
|
||||||
def f():
|
def f():
|
||||||
pass
|
pass
|
||||||
|
|
||||||
with pytest.raises(RuntimeEnvSetupError):
|
with pytest.raises(RuntimeEnvSetupError, match=bad_packages["task"]):
|
||||||
ray.get(f.remote())
|
ray.get(f.remote())
|
||||||
|
|
||||||
wait_for_condition(
|
|
||||||
lambda: bad_packages["task"] in capsys.readouterr().out, timeout=30
|
|
||||||
)
|
|
||||||
|
|
||||||
|
|
||||||
if __name__ == "__main__":
|
if __name__ == "__main__":
|
||||||
sys.exit(pytest.main(["-sv", __file__]))
|
sys.exit(pytest.main(["-sv", __file__]))
|
||||||
|
|
Loading…
Add table
Reference in a new issue