[GCS]Open light heartbeat by default (#11968)

* [GCS]Open light heartbeat by default (#11689)

* Add some unit tests
This commit is contained in:
Tao Wang 2020-11-17 10:21:47 +08:00 committed by GitHub
parent c49554fb7a
commit d525e61288
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
9 changed files with 192 additions and 67 deletions

View file

@ -39,7 +39,7 @@ RAY_CONFIG(int64_t, raylet_heartbeat_timeout_milliseconds, 100)
/// Whether to send heartbeat lightly. When it is enalbed, only changed part,
/// like should_global_gc or changed resources, will be included in the heartbeat,
/// and gcs only broadcast the changed heartbeat.
RAY_CONFIG(bool, light_heartbeat_enabled, false)
RAY_CONFIG(bool, light_heartbeat_enabled, true)
/// If a component has not sent a heartbeat in the last num_heartbeats_timeout
/// heartbeat intervals, the raylet monitor process will report
/// it as dead to the db_client table.

View file

@ -206,17 +206,59 @@ TEST_F(GlobalStateAccessorTest, TestGetAllHeartbeat) {
auto node_table = global_state_->GetAllNodeInfo();
ASSERT_EQ(node_table.size(), 1);
// Report heartbeat
// Report heartbeat first time.
std::promise<bool> promise1;
auto heartbeat = std::make_shared<rpc::HeartbeatTableData>();
heartbeat->set_client_id(node_table_data->node_id());
auto heartbeat1 = std::make_shared<rpc::HeartbeatTableData>();
heartbeat1->set_client_id(node_table_data->node_id());
RAY_CHECK_OK(gcs_client_->Nodes().AsyncReportHeartbeat(
heartbeat, [&promise1](Status status) { promise1.set_value(status.ok()); }));
heartbeat1, [&promise1](Status status) { promise1.set_value(status.ok()); }));
WaitReady(promise1.get_future(), timeout_ms_);
heartbeats = global_state_->GetAllHeartbeat();
heartbeat_batch_data.ParseFromString(*heartbeats.get());
ASSERT_EQ(heartbeat_batch_data.batch_size(), 1);
// Report heartbeat with resources changed.
std::promise<bool> promise2;
auto heartbeat2 = std::make_shared<rpc::HeartbeatTableData>();
heartbeat2->set_client_id(node_table_data->node_id());
(*heartbeat2->mutable_resources_total())["CPU"] = 1;
(*heartbeat2->mutable_resources_total())["GPU"] = 10;
heartbeat2->set_resources_available_changed(true);
(*heartbeat2->mutable_resources_available())["GPU"] = 5;
RAY_CHECK_OK(gcs_client_->Nodes().AsyncReportHeartbeat(
heartbeat2, [&promise2](Status status) { promise2.set_value(status.ok()); }));
WaitReady(promise2.get_future(), timeout_ms_);
heartbeats = global_state_->GetAllHeartbeat();
heartbeat_batch_data.ParseFromString(*heartbeats.get());
ASSERT_EQ(heartbeat_batch_data.batch_size(), 1);
auto heartbeat_data = heartbeat_batch_data.mutable_batch()->at(0);
ASSERT_EQ(heartbeat_data.resources_total_size(), 2);
ASSERT_EQ((*heartbeat_data.mutable_resources_total())["CPU"], 1.0);
ASSERT_EQ((*heartbeat_data.mutable_resources_total())["GPU"], 10.0);
ASSERT_EQ(heartbeat_data.resources_available_size(), 1);
ASSERT_EQ((*heartbeat_data.mutable_resources_available())["GPU"], 5.0);
// Report heartbeat with resources unchanged. (Only works with light heartbeat enabled)
std::promise<bool> promise3;
auto heartbeat3 = std::make_shared<rpc::HeartbeatTableData>();
heartbeat3->set_client_id(node_table_data->node_id());
(*heartbeat3->mutable_resources_available())["CPU"] = 1;
(*heartbeat3->mutable_resources_available())["GPU"] = 6;
RAY_CHECK_OK(gcs_client_->Nodes().AsyncReportHeartbeat(
heartbeat3, [&promise3](Status status) { promise3.set_value(status.ok()); }));
WaitReady(promise3.get_future(), timeout_ms_);
heartbeats = global_state_->GetAllHeartbeat();
heartbeat_batch_data.ParseFromString(*heartbeats.get());
ASSERT_EQ(heartbeat_batch_data.batch_size(), 1);
heartbeat_data = heartbeat_batch_data.mutable_batch()->at(0);
ASSERT_EQ(heartbeat_data.resources_total_size(), 2);
ASSERT_EQ((*heartbeat_data.mutable_resources_total())["CPU"], 1.0);
ASSERT_EQ((*heartbeat_data.mutable_resources_total())["GPU"], 10.0);
ASSERT_EQ(heartbeat_data.resources_available_size(), 1);
ASSERT_EQ((*heartbeat_data.mutable_resources_available())["GPU"], 5.0);
}
TEST_F(GlobalStateAccessorTest, TestProfileTable) {

View file

@ -808,6 +808,34 @@ TEST_F(ServiceBasedGcsClientTest, TestNodeHeartbeat) {
WaitForExpectedCount(heartbeat_batch_count, 1);
}
TEST_F(ServiceBasedGcsClientTest, TestNodeHeartbeatWithLightHeartbeat) {
// Subscribe batched state of all nodes from GCS.
std::atomic<int> heartbeat_batch_count(0);
auto on_subscribe =
[&heartbeat_batch_count](const gcs::HeartbeatBatchTableData &result) {
++heartbeat_batch_count;
};
ASSERT_TRUE(SubscribeBatchHeartbeat(on_subscribe));
// Register node.
auto node_info = Mocker::GenNodeInfo();
RAY_CHECK(RegisterNode(*node_info));
// Report unchanged heartbeat of a node to GCS.
NodeID node_id = NodeID::FromBinary(node_info->node_id());
auto heartbeat = std::make_shared<rpc::HeartbeatTableData>();
heartbeat->set_client_id(node_id.Binary());
ASSERT_TRUE(ReportHeartbeat(heartbeat));
WaitForExpectedCount(heartbeat_batch_count, 0);
// Report changed heartbeat of a node to GCS.
auto heartbeat1 = std::make_shared<rpc::HeartbeatTableData>();
heartbeat1->set_client_id(node_id.Binary());
heartbeat1->set_resources_available_changed(true);
ASSERT_TRUE(ReportHeartbeat(heartbeat1));
WaitForExpectedCount(heartbeat_batch_count, 1);
}
TEST_F(ServiceBasedGcsClientTest, TestGetAllAvailableResources) {
// Subscribe batched state of all nodes from GCS.
std::atomic<int> heartbeat_batch_count(0);
@ -825,8 +853,8 @@ TEST_F(ServiceBasedGcsClientTest, TestGetAllAvailableResources) {
NodeID node_id = NodeID::FromBinary(node_info->node_id());
auto heartbeat = std::make_shared<rpc::HeartbeatTableData>();
heartbeat->set_client_id(node_id.Binary());
// Set this flag because GCS won't publish unchanged heartbeat.
heartbeat->set_should_global_gc(true);
// Set this flag to indicate resources has changed.
heartbeat->set_resources_available_changed(true);
(*heartbeat->mutable_resources_available())["CPU"] = 1.0;
(*heartbeat->mutable_resources_available())["GPU"] = 10.0;
ASSERT_TRUE(ReportHeartbeat(heartbeat));
@ -840,6 +868,51 @@ TEST_F(ServiceBasedGcsClientTest, TestGetAllAvailableResources) {
EXPECT_EQ((*resources[0].mutable_resources_available())["GPU"], 10.0);
}
TEST_F(ServiceBasedGcsClientTest, TestGetAllAvailableResourcesWithLightHeartbeat) {
// Subscribe batched state of all nodes from GCS.
std::atomic<int> heartbeat_batch_count(0);
auto on_subscribe =
[&heartbeat_batch_count](const gcs::HeartbeatBatchTableData &result) {
++heartbeat_batch_count;
};
ASSERT_TRUE(SubscribeBatchHeartbeat(on_subscribe));
// Register node.
auto node_info = Mocker::GenNodeInfo();
RAY_CHECK(RegisterNode(*node_info));
// Report heartbeat of a node to GCS.
NodeID node_id = NodeID::FromBinary(node_info->node_id());
auto heartbeat = std::make_shared<rpc::HeartbeatTableData>();
heartbeat->set_client_id(node_id.Binary());
heartbeat->set_resources_available_changed(true);
(*heartbeat->mutable_resources_available())["CPU"] = 1.0;
(*heartbeat->mutable_resources_available())["GPU"] = 10.0;
ASSERT_TRUE(ReportHeartbeat(heartbeat));
WaitForExpectedCount(heartbeat_batch_count, 1);
// Assert get all available resources right.
std::vector<rpc::AvailableResources> resources = GetAllAvailableResources();
EXPECT_EQ(resources.size(), 1);
EXPECT_EQ(resources[0].resources_available_size(), 2);
EXPECT_EQ((*resources[0].mutable_resources_available())["CPU"], 1.0);
EXPECT_EQ((*resources[0].mutable_resources_available())["GPU"], 10.0);
// Report unchanged heartbeat of a node to GCS.
auto heartbeat1 = std::make_shared<rpc::HeartbeatTableData>();
heartbeat1->set_client_id(node_id.Binary());
(*heartbeat1->mutable_resources_available())["GPU"] = 8.0;
ASSERT_TRUE(ReportHeartbeat(heartbeat1));
WaitForExpectedCount(heartbeat_batch_count, 1);
// The value would remain unchanged.
std::vector<rpc::AvailableResources> resources1 = GetAllAvailableResources();
EXPECT_EQ(resources1.size(), 1);
EXPECT_EQ(resources1[0].resources_available_size(), 2);
EXPECT_EQ((*resources1[0].mutable_resources_available())["CPU"], 1.0);
EXPECT_EQ((*resources1[0].mutable_resources_available())["GPU"], 10.0);
}
TEST_F(ServiceBasedGcsClientTest, TestTaskInfo) {
JobID job_id = JobID::FromInt(1);
TaskID task_id = TaskID::ForDriverTask(job_id);

View file

@ -352,7 +352,7 @@ void GcsNodeManager::HandleGetAllHeartbeat(const rpc::GetAllHeartbeatRequest &re
if (!node_heartbeats_.empty()) {
auto batch = std::make_shared<rpc::HeartbeatBatchTableData>();
absl::flat_hash_map<ResourceSet, rpc::ResourceDemand> aggregate_load;
for (auto &heartbeat : node_heartbeats_) {
for (const auto &heartbeat : node_heartbeats_) {
// Aggregate the load reported by each raylet.
auto load = heartbeat.second.resource_load_by_shape();
for (const auto &demand : load.resource_demands()) {
@ -369,14 +369,13 @@ void GcsNodeManager::HandleGetAllHeartbeat(const rpc::GetAllHeartbeatRequest &re
demand.backlog_size());
}
}
heartbeat.second.clear_resource_load_by_shape();
batch->add_batch()->Swap(&heartbeat.second);
batch->add_batch()->CopyFrom(heartbeat.second);
}
for (auto &demand : aggregate_load) {
for (const auto &demand : aggregate_load) {
auto demand_proto = batch->mutable_resource_load_by_shape()->add_resource_demands();
demand_proto->Swap(&demand.second);
demand_proto->CopyFrom(demand.second);
for (const auto &resource_pair : demand.first.GetResourceMap()) {
(*demand_proto->mutable_shape())[resource_pair.first] = resource_pair.second;
}
@ -415,6 +414,8 @@ void GcsNodeManager::UpdateNodeHeartbeat(const NodeID node_id,
if (request.heartbeat().resource_load_changed()) {
(*iter->second.mutable_resource_load()) = request.heartbeat().resource_load();
}
(*iter->second.mutable_resource_load_by_shape()) =
request.heartbeat().resource_load_by_shape();
}
}
@ -533,6 +534,7 @@ void GcsNodeManager::StartNodeFailureDetector() {
void GcsNodeManager::UpdateNodeRealtimeResources(
const NodeID &node_id, const rpc::HeartbeatTableData &heartbeat) {
if (!RayConfig::instance().light_heartbeat_enabled() ||
cluster_realtime_resources_.count(node_id) == 0 ||
heartbeat.resources_available_changed()) {
cluster_realtime_resources_[node_id] =
ResourceSet(MapFromProtobuf(heartbeat.resources_available()));
@ -564,7 +566,6 @@ void GcsNodeManager::AddDeadNodeToCache(std::shared_ptr<rpc::GcsNodeInfo> node)
void GcsNodeManager::SendBatchedHeartbeat() {
if (!heartbeat_buffer_.empty()) {
auto batch = std::make_shared<rpc::HeartbeatBatchTableData>();
std::unordered_map<ResourceSet, rpc::ResourceDemand> aggregate_load;
for (auto &heartbeat : heartbeat_buffer_) {
batch->add_batch()->Swap(&heartbeat.second);
}

View file

@ -485,10 +485,12 @@ void NodeManager::Heartbeat() {
}
}
if (!new_scheduler_enabled_) {
// Add resource load by shape. This will be used by the new autoscaler.
auto resource_load = local_queues_.GetResourceLoadByShape(
RayConfig::instance().max_resource_shapes_per_load_report());
heartbeat_data->mutable_resource_load_by_shape()->Swap(&resource_load);
}
// Set the global gc bit on the outgoing heartbeat message.
if (should_global_gc_) {
@ -884,7 +886,7 @@ void NodeManager::HeartbeatAdded(const NodeID &client_id,
ResourceSet remote_total(MapFromProtobuf(heartbeat_data.resources_total()));
remote_resources.SetTotalResources(std::move(remote_total));
}
if (heartbeat_data.resource_load_changed()) {
if (heartbeat_data.resources_available_changed()) {
ResourceSet remote_available(MapFromProtobuf(heartbeat_data.resources_available()));
remote_resources.SetAvailableResources(std::move(remote_available));
}

View file

@ -150,6 +150,10 @@ class TaskResourceInstances {
/// Total and available capacities of each resource of a node.
class NodeResources {
public:
NodeResources() {}
NodeResources(const NodeResources &other)
: predefined_resources(other.predefined_resources),
custom_resources(other.custom_resources) {}
/// Available and total capacities for predefined resources.
std::vector<ResourceCapacity> predefined_resources;
/// Map containing custom resources. The key of each entry represents the

View file

@ -804,17 +804,16 @@ void ClusterResourceScheduler::FreeLocalTaskResources(
}
void ClusterResourceScheduler::Heartbeat(
bool light_heartbeat_enabled,
std::shared_ptr<HeartbeatTableData> heartbeat_data) const {
bool light_heartbeat_enabled, std::shared_ptr<HeartbeatTableData> heartbeat_data) {
NodeResources resources;
RAY_CHECK(GetNodeResources(local_node_id_, &resources))
<< "Error: Populating heartbeat failed. Please file a bug report: "
"https://github.com/ray-project/ray/issues/new.";
if (light_heartbeat_enabled) {
// TODO
RAY_CHECK(false) << "TODO";
if (light_heartbeat_enabled && last_report_resources_ &&
resources == *last_report_resources_.get()) {
return;
} else {
for (int i = 0; i < PredefinedResources_MAX; i++) {
const auto &label = ResourceEnumToString((PredefinedResources)i);
@ -840,6 +839,10 @@ void ClusterResourceScheduler::Heartbeat(
(*heartbeat_data->mutable_resources_total())[label] = capacity.total.Double();
}
}
heartbeat_data->set_resources_available_changed(true);
if (light_heartbeat_enabled) {
last_report_resources_.reset(new NodeResources(resources));
}
}
}

View file

@ -49,6 +49,8 @@ class ClusterResourceScheduler {
/// Keep the mapping between node and resource IDs in string representation
/// to integer representation. Used for improving map performance.
StringIdMap string_to_int_map_;
/// Cached resources, used to compare with newest one in light heartbeat mode.
std::unique_ptr<NodeResources> last_report_resources_;
/// Set predefined resources.
///
@ -382,8 +384,7 @@ class ClusterResourceScheduler {
/// \param light_heartbeat_enabled Only send changed fields if true.
/// \param Output parameter. `resources_available` and `resources_total` are the only
/// fields used.
void Heartbeat(bool light_heartbeat_enabled,
std::shared_ptr<HeartbeatTableData> data) const;
void Heartbeat(bool light_heartbeat_enabled, std::shared_ptr<HeartbeatTableData> data);
/// Return human-readable string for this scheduler state.
std::string DebugString() const;

View file

@ -229,13 +229,13 @@ bool ClusterTaskManager::CancelTask(const TaskID &task_id) {
void ClusterTaskManager::Heartbeat(bool light_heartbeat_enabled,
std::shared_ptr<HeartbeatTableData> data) const {
// TODO (WangTao): Find a way to check if load changed and combine it with light
// heartbeat. Now we just report it every time.
data->set_resource_load_changed(true);
auto resource_loads = data->mutable_resource_load();
auto resource_load_by_shape =
data->mutable_resource_load_by_shape()->mutable_resource_demands();
if (light_heartbeat_enabled) {
RAY_CHECK(false) << "TODO";
} else {
// TODO (Alex): Implement the 1-CPU task optimization.
for (const auto &pair : tasks_to_schedule_) {
const auto &scheduling_class = pair.first;
@ -285,7 +285,6 @@ void ClusterTaskManager::Heartbeat(bool light_heartbeat_enabled,
}
}
}
}
std::string ClusterTaskManager::DebugString() {
std::stringstream buffer;