-
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 15 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 |
---|---|---|
@@ -0,0 +1,144 @@ | ||
// 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" | ||
|
||
#include "ray/stats/metric.h" | ||
DEFINE_stats(health_check_rpc_latency_ms, | ||
"Latency of rpc request for health check.", | ||
(), | ||
({1, 10, 100, 1000, 10000}, ), | ||
ray::stats::HISTOGRAM); | ||
|
||
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) { | ||
RAY_CHECK(on_node_death_callback != nullptr); | ||
RAY_CHECK(initial_delay_ms >= 0); | ||
RAY_CHECK(timeout_ms >= 0); | ||
RAY_CHECK(period_ms >= 0); | ||
RAY_CHECK(failure_threshold >= 0); | ||
} | ||
|
||
GcsHealthCheckManager::~GcsHealthCheckManager() {} | ||
|
||
void GcsHealthCheckManager::RemoveNode(const NodeID &node_id) { | ||
io_service_.dispatch( | ||
[this, node_id]() { | ||
auto iter = health_check_contexts_.find(node_id); | ||
if (iter == health_check_contexts_.end()) { | ||
return; | ||
} | ||
health_check_contexts_.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. |
||
health_check_contexts_.erase(node_id); | ||
} | ||
|
||
std::vector<NodeID> GcsHealthCheckManager::GetAllNodes() const { | ||
std::vector<NodeID> nodes; | ||
for (const auto &[node_id, _] : health_check_contexts_) { | ||
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( | ||
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. why do we not include this to existing RPC paths? Like raylet_client.cc. Seems more complicated to have a separate code path like this. 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 think the exiting path is way more complicated than this one. We shouldn't do it just because we are doing it. |
||
context_.get(), | ||
&request_, | ||
&response_, | ||
[this, stopped = this->stopped_, context = this->context_, now = absl::Now()]( | ||
::grpc::Status status) { | ||
// This callback is done in gRPC's thread pool. | ||
STATS_health_check_rpc_latency_ms.Record( | ||
absl::ToInt64Milliseconds(absl::Now() - now)); | ||
if (status.error_code() == ::grpc::StatusCode::CANCELLED) { | ||
fishbone marked this conversation as resolved.
Show resolved
Hide resolved
|
||
return; | ||
} | ||
manager_->io_service_.post( | ||
[this, stopped, status]() { | ||
// Stopped has to be read in the same thread where it's updated. | ||
if (*stopped) { | ||
return; | ||
} | ||
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_ | ||
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. Maybe DEBUG? Feel like this may be too spammy? 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'm not sure. This should only happen when error happened (scale down won't trigger this). If it got printed a lot maybe it means we should increase the interval? |
||
<< ", remaining checks " << health_check_remaining_; | ||
} | ||
|
||
if (health_check_remaining_ == 0) { | ||
manager_->io_service_.post([this]() { manager_->FailNode(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. maybe we don't need to post again? Since we are already in the same 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. Remove the node will make 'this' not valid. So run it in another place after this. I can put some comments there. |
||
""); | ||
} else { | ||
// Do another health check. | ||
timer_.expires_from_now( | ||
boost::posix_time::milliseconds(manager_->period_ms_)); | ||
timer_.async_wait([this, stopped](auto ec) { | ||
// We need to check stopped here as well since cancel | ||
// won't impact the queued tasks. | ||
if (ec != boost::asio::error::operation_aborted && !*stopped) { | ||
StartHealthCheck(); | ||
} | ||
}); | ||
} | ||
}, | ||
"HealthCheck"); | ||
}); | ||
} | ||
|
||
void GcsHealthCheckManager::AddNode(const NodeID &node_id, | ||
std::shared_ptr<grpc::Channel> channel) { | ||
io_service_.dispatch( | ||
[this, channel, node_id]() { | ||
RAY_CHECK(health_check_contexts_.count(node_id) == 0); | ||
auto context = std::make_unique<HealthCheckContext>(this, channel, node_id); | ||
health_check_contexts_.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.
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 comment
The 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.