-
Notifications
You must be signed in to change notification settings - Fork 5.8k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[core] Introduce pull based health check to GCS. #29442
Changes from 8 commits
b3136dd
8691d16
9203adb
5313f65
9ddd08b
de6eea8
f16d691
a1ac923
067c6af
131f471
6100241
702abab
f779d4e
341569f
cc79f6a
386bac3
f9508bc
761b9bd
e4882d2
02b3469
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -699,3 +699,11 @@ RAY_CONFIG(std::string, REDIS_SERVER_NAME, "") | |
// The delay is a random number between the interval. If method equals '*', | ||
// it will apply to all methods. | ||
RAY_CONFIG(std::string, testing_asio_delay_us, "") | ||
|
||
/// A feature flag to enable pull based health check. | ||
/// TODO: Turn it on by default | ||
RAY_CONFIG(bool, pull_based_healthcheck, false) | ||
RAY_CONFIG(int64_t, health_check_initial_delay_ms, 1000) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Feel like it is unnecessary? It'd take 30 seconds until the node is marked as dead anyway? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. No. 30s is old protocol. This is the new protocol. So here, if a node is dead, it only need to take 3s to mark the node dead. |
||
RAY_CONFIG(int64_t, health_check_period_ms, 1000) | ||
RAY_CONFIG(int64_t, health_check_timeout_ms, 5000) | ||
RAY_CONFIG(int64_t, health_check_failure_threshold, 5) |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,121 @@ | ||
// Copyright 2022 The Ray Authors. | ||
// | ||
// Licensed under the Apache License, Version 2.0 (the "License"); | ||
// you may not use this file except in compliance with the License. | ||
// You may obtain a copy of the License at | ||
// | ||
// http://www.apache.org/licenses/LICENSE-2.0 | ||
// | ||
// Unless required by applicable law or agreed to in writing, software | ||
// distributed under the License is distributed on an "AS IS" BASIS, | ||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
// See the License for the specific language governing permissions and | ||
// limitations under the License. | ||
|
||
#include "ray/gcs/gcs_server/gcs_health_check_manager.h" | ||
|
||
namespace ray { | ||
namespace gcs { | ||
|
||
GcsHealthCheckManager::GcsHealthCheckManager( | ||
instrumented_io_context &io_service, | ||
std::function<void(const NodeID &)> on_node_death_callback, | ||
int64_t initial_delay_ms, | ||
int64_t timeout_ms, | ||
int64_t period_ms, | ||
int64_t failure_threshold) | ||
: io_service_(io_service), | ||
on_node_death_callback_(on_node_death_callback), | ||
initial_delay_ms_(initial_delay_ms), | ||
timeout_ms_(timeout_ms), | ||
period_ms_(period_ms), | ||
failure_threshold_(failure_threshold) {} | ||
fishbone marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
GcsHealthCheckManager::~GcsHealthCheckManager() {} | ||
|
||
void GcsHealthCheckManager::RemoveNode(const NodeID &node_id) { | ||
io_service_.dispatch( | ||
[this, node_id]() { | ||
auto iter = inflight_health_checks_.find(node_id); | ||
if (iter == inflight_health_checks_.end()) { | ||
return; | ||
} | ||
inflight_health_checks_.erase(iter); | ||
}, | ||
"GcsHealthCheckManager::RemoveNode"); | ||
} | ||
|
||
void GcsHealthCheckManager::FailNode(const NodeID &node_id) { | ||
RAY_LOG(WARNING) << "Node " << node_id << " is dead because the health check failed."; | ||
on_node_death_callback_(node_id); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is there any way to check if this is running inside io_service? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't think it's easy to do. |
||
inflight_health_checks_.erase(node_id); | ||
} | ||
|
||
std::vector<NodeID> GcsHealthCheckManager::GetAllNodes() const { | ||
std::vector<NodeID> nodes; | ||
for (const auto &[node_id, _] : inflight_health_checks_) { | ||
nodes.emplace_back(node_id); | ||
} | ||
return nodes; | ||
} | ||
|
||
void GcsHealthCheckManager::HealthCheckContext::StartHealthCheck() { | ||
using ::grpc::health::v1::HealthCheckResponse; | ||
fishbone marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
context = std::make_shared<grpc::ClientContext>(); | ||
|
||
auto deadline = | ||
std::chrono::system_clock::now() + std::chrono::milliseconds(manager->timeout_ms_); | ||
context->set_deadline(deadline); | ||
stub->async()->Check( | ||
context.get(), | ||
&request, | ||
&response, | ||
[this, context = this->context](::grpc::Status status) { | ||
if (status.error_code() == ::grpc::StatusCode::CANCELLED) { | ||
fishbone marked this conversation as resolved.
Show resolved
Hide resolved
|
||
return; | ||
} | ||
|
||
manager->io_service_.post( | ||
[this, status]() { | ||
RAY_LOG(DEBUG) << "Health check status: " << int(response.status()); | ||
|
||
if (status.ok() && response.status() == HealthCheckResponse::SERVING) { | ||
// Health check passed | ||
health_check_remaining = manager->failure_threshold_; | ||
} else { | ||
--health_check_remaining; | ||
RAY_LOG(WARNING) << "Health check failed for node " << node_id | ||
<< ", remaining checks " << health_check_remaining; | ||
} | ||
|
||
if (health_check_remaining == 0) { | ||
manager->io_service_.post([this]() { manager->FailNode(node_id); }, ""); | ||
} else { | ||
// Do another health check. | ||
timer.expires_from_now( | ||
boost::posix_time::milliseconds(manager->period_ms_)); | ||
timer.async_wait([this](auto ec) { | ||
if (ec != boost::asio::error::operation_aborted) { | ||
StartHealthCheck(); | ||
} | ||
}); | ||
} | ||
}, | ||
"HealthCheck"); | ||
}); | ||
} | ||
|
||
void GcsHealthCheckManager::AddNode(const NodeID &node_id, | ||
std::shared_ptr<grpc::Channel> channel) { | ||
io_service_.dispatch( | ||
[this, channel, node_id]() { | ||
RAY_CHECK(inflight_health_checks_.count(node_id) == 0); | ||
auto context = std::make_unique<HealthCheckContext>(this, channel, node_id); | ||
inflight_health_checks_.emplace(std::make_pair(node_id, std::move(context))); | ||
}, | ||
"GcsHealthCheckManager::AddNode"); | ||
} | ||
|
||
} // namespace gcs | ||
} // namespace ray |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Consider adding metrics for pull based heartbeat too?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
could be in other PR
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I could add it in this PR
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@rkooo567 I think we should do a better job to monitor the RPC requests.
The cpp -> python -> opencensus is not a good practice I believe.
I can add gRPC metrics here. But in long term, we should do cpp -> opencensus agent directly.
We can use tools builtin gRPC to do this (https://github.com/grpc/grpc/blob/5f6c357e741207f4af7e3b642a486bdda12c93df/include/grpcpp/opencensus.h#L35)