mirror of
https://github.com/vale981/ray
synced 2025-03-09 12:56:46 -04:00
Fix raylet crash during cluster shutdown (#6272)
This commit is contained in:
parent
ac6aa21411
commit
fa98694dd0
5 changed files with 28 additions and 6 deletions
|
@ -16,6 +16,9 @@ constexpr int kObjectIdIndexSize = 32;
|
||||||
static_assert(kObjectIdIndexSize % CHAR_BIT == 0,
|
static_assert(kObjectIdIndexSize % CHAR_BIT == 0,
|
||||||
"ObjectID prefix not a multiple of bytes");
|
"ObjectID prefix not a multiple of bytes");
|
||||||
|
|
||||||
|
/// Raylet exit code on plasma store socket error.
|
||||||
|
constexpr int kRayletStoreErrorExitCode = 100;
|
||||||
|
|
||||||
/// Prefix for the object table keys in redis.
|
/// Prefix for the object table keys in redis.
|
||||||
constexpr char kObjectTablePrefix[] = "ObjectTable";
|
constexpr char kObjectTablePrefix[] = "ObjectTable";
|
||||||
/// Prefix for the task table keys in redis.
|
/// Prefix for the task table keys in redis.
|
||||||
|
|
|
@ -673,6 +673,8 @@ Status ClientTable::Disconnect(const DisconnectCallback &callback) {
|
||||||
return Status::OK();
|
return Status::OK();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool ClientTable::IsDisconnected() const { return disconnected_; }
|
||||||
|
|
||||||
ray::Status ClientTable::MarkDisconnected(const ClientID &dead_node_id) {
|
ray::Status ClientTable::MarkDisconnected(const ClientID &dead_node_id) {
|
||||||
auto node_info = std::make_shared<GcsNodeInfo>();
|
auto node_info = std::make_shared<GcsNodeInfo>();
|
||||||
node_info->set_node_id(dead_node_id.Binary());
|
node_info->set_node_id(dead_node_id.Binary());
|
||||||
|
|
|
@ -860,6 +860,10 @@ class ClientTable : public Log<ClientID, GcsNodeInfo> {
|
||||||
/// \return Status
|
/// \return Status
|
||||||
ray::Status Disconnect(const DisconnectCallback &callback = nullptr);
|
ray::Status Disconnect(const DisconnectCallback &callback = nullptr);
|
||||||
|
|
||||||
|
/// Whether the client is disconnected from the GCS.
|
||||||
|
/// \return Whether the client is disconnected.
|
||||||
|
bool IsDisconnected() const;
|
||||||
|
|
||||||
/// Mark a different client as disconnected. The client ID should never be
|
/// Mark a different client as disconnected. The client ID should never be
|
||||||
/// reused for a new client.
|
/// reused for a new client.
|
||||||
///
|
///
|
||||||
|
|
|
@ -43,9 +43,14 @@ void ObjectStoreNotificationManager::ProcessStoreLength(
|
||||||
const boost::system::error_code &error) {
|
const boost::system::error_code &error) {
|
||||||
notification_.resize(length_);
|
notification_.resize(length_);
|
||||||
if (error) {
|
if (error) {
|
||||||
RAY_LOG(FATAL)
|
// When shutting down a cluster, it's possible that the plasma store is killed
|
||||||
<< "Problem communicating with the object store from raylet, check logs or "
|
// earlier than raylet, in this case we don't want raylet to crash, we instead
|
||||||
<< "dmesg for previous errors: " << boost_to_ray_status(error).ToString();
|
// log an error message and exit.
|
||||||
|
RAY_LOG(ERROR) << "Failed to process store length: "
|
||||||
|
<< boost_to_ray_status(error).ToString()
|
||||||
|
<< ", most likely plasma store is down, raylet will exit";
|
||||||
|
// Exit raylet process.
|
||||||
|
_exit(kRayletStoreErrorExitCode);
|
||||||
}
|
}
|
||||||
boost::asio::async_read(
|
boost::asio::async_read(
|
||||||
socket_, boost::asio::buffer(notification_),
|
socket_, boost::asio::buffer(notification_),
|
||||||
|
|
|
@ -518,9 +518,17 @@ void NodeManager::ClientRemoved(const GcsNodeInfo &node_info) {
|
||||||
const ClientID client_id = ClientID::FromBinary(node_info.node_id());
|
const ClientID client_id = ClientID::FromBinary(node_info.node_id());
|
||||||
RAY_LOG(DEBUG) << "[ClientRemoved] Received callback from client id " << client_id;
|
RAY_LOG(DEBUG) << "[ClientRemoved] Received callback from client id " << client_id;
|
||||||
|
|
||||||
|
if (!gcs_client_->client_table().IsDisconnected()) {
|
||||||
|
// We could receive a notification for our own death when we disconnect from client
|
||||||
|
// table after receiving a 'SIGTERM' signal, in that case we disconnect from gcs
|
||||||
|
// client table and then do some cleanup in the disconnect callback, and it's possible
|
||||||
|
// that we receive the notification in between, for more details refer to the SIGTERM
|
||||||
|
// handler in main.cc. In this case check for intentional disconnection and rule it
|
||||||
|
// out.
|
||||||
RAY_CHECK(client_id != gcs_client_->client_table().GetLocalClientId())
|
RAY_CHECK(client_id != gcs_client_->client_table().GetLocalClientId())
|
||||||
<< "Exiting because this node manager has mistakenly been marked dead by the "
|
<< "Exiting because this node manager has mistakenly been marked dead by the "
|
||||||
<< "monitor.";
|
<< "monitor.";
|
||||||
|
}
|
||||||
|
|
||||||
// Below, when we remove client_id from all of these data structures, we could
|
// Below, when we remove client_id from all of these data structures, we could
|
||||||
// check that it is actually removed, or log a warning otherwise, but that may
|
// check that it is actually removed, or log a warning otherwise, but that may
|
||||||
|
|
Loading…
Add table
Reference in a new issue