[Dashboard] Disable unnecessary event messages. (#19490)

* Disable unnecessary event messages.

* use warning

* Fix tests
This commit is contained in:
SangBin Cho 2021-10-21 09:40:25 +09:00 committed by GitHub
parent bcf584294f
commit 7fb681a35d
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
3 changed files with 47 additions and 19 deletions

View file

@ -352,6 +352,8 @@ if __name__ == "__main__":
object_store_name=args.object_store_name,
raylet_name=args.raylet_name,
logging_params=logging_params)
if os.environ.get("_RAY_AGENT_FAILING"):
raise Exception("Failure injection failure.")
loop = asyncio.get_event_loop()
loop.run_until_complete(agent.run())

View file

@ -1,3 +1,4 @@
import json
import os
import re
import socket
@ -140,16 +141,24 @@ def test_dashboard(shutdown_only):
f"Dashboard output log: {out_log}\n")
@pytest.fixture
def set_agent_failure_env_var():
os.environ["_RAY_AGENT_FAILING"] = "1"
yield
del os.environ["_RAY_AGENT_FAILING"]
@pytest.mark.parametrize(
"ray_start_cluster_head", [{
"metrics_export_port": 6379,
"_system_config": {
"agent_restart_interval_ms": 10,
"agent_max_restart_count": 5
}
}],
indirect=True)
def test_dashboard_agent_restart(ray_start_cluster_head, error_pubsub):
def test_dashboard_agent_restart(set_agent_failure_env_var,
ray_start_cluster_head, error_pubsub,
log_pubsub):
"""Test that when the agent fails to start many times in a row
if the error message is suppressed correctly without spamming
the driver.
@ -158,12 +167,28 @@ def test_dashboard_agent_restart(ray_start_cluster_head, error_pubsub):
p = error_pubsub
errors = get_error_message(
p, 1, ray_constants.DASHBOARD_AGENT_DIED_ERROR, timeout=10)
assert len(errors) == 1
for e in errors:
assert ("There are 2 possible problems "
assert ("There are 3 possible problems "
"if you see this error." in e.error_message)
# Make sure the agent process is not started anymore.
cluster = ray_start_cluster_head
wait_for_condition(lambda: search_agents(cluster) is None)
# Make sure there's no spammy message for 5 seconds.
timeout = 5
start_time = time.time()
while time.time() - start_time < timeout:
msg = log_pubsub.get_message()
if msg is None:
time.sleep(0.01)
continue
log_data = json.loads(ray._private.utils.decode(msg["data"]))
# Only autoscaler logs should be printed from the log channel.
is_autoscaler_log = log_data["pid"] == "autoscaler"
if not is_autoscaler_log:
raise AssertionError(
"There are spammy logs during Ray agent restart process."
f"Logs: {log_data['lines']}")
if __name__ == "__main__":

View file

@ -88,11 +88,9 @@ void AgentManager::StartAgent() {
auto timer = delay_executor_(
[this, child]() mutable {
if (agent_pid_ != child.GetId()) {
RAY_EVENT(ERROR, EL_RAY_AGENT_NOT_REGISTERED)
.WithField("ip", agent_ip_address_)
.WithField("pid", agent_pid_)
<< "Agent process with pid " << child.GetId()
<< " has not registered, restart it.";
RAY_LOG(WARNING) << "Agent process with pid " << child.GetId()
<< " has not registered, restart it. ip "
<< agent_ip_address_ << ". pid " << agent_pid_;
child.Kill();
}
},
@ -100,12 +98,9 @@ void AgentManager::StartAgent() {
int exit_code = child.Wait();
timer->cancel();
RAY_EVENT(ERROR, EL_RAY_AGENT_EXIT)
.WithField("ip", agent_ip_address_)
.WithField("pid", agent_pid_)
<< "Agent process with pid " << child.GetId() << " exit, return value "
<< exit_code;
RAY_LOG(WARNING) << "Agent process with pid " << child.GetId()
<< " exit, return value " << exit_code << ". ip "
<< agent_ip_address_ << ". pid " << agent_pid_;
if (agent_restart_count_ < RayConfig::instance().agent_max_restart_count()) {
RAY_UNUSED(delay_executor_(
[this] {
@ -116,11 +111,17 @@ void AgentManager::StartAgent() {
RayConfig::instance().agent_restart_interval_ms() *
std::pow(2, (agent_restart_count_ + 1))));
} else {
RAY_LOG(INFO) << "Agent has failed "
<< RayConfig::instance().agent_max_restart_count()
<< " times in a row without registering the agent. This is highly "
"likely there's a bug in the dashboard agent. Please check out "
"the dashboard_agent.log file.";
RAY_LOG(WARNING) << "Agent has failed "
<< RayConfig::instance().agent_max_restart_count()
<< " times in a row without registering the agent. This is highly "
"likely there's a bug in the dashboard agent. Please check out "
"the dashboard_agent.log file.";
RAY_EVENT(WARNING, EL_RAY_AGENT_EXIT)
.WithField("ip", agent_ip_address_)
.WithField("pid", agent_pid_)
<< "Agent failed to be restarted "
<< RayConfig::instance().agent_max_restart_count()
<< " times. Agent won't be restarted.";
}
});
monitor_thread.detach();