@@ -43,7 +43,13 @@ void GcsResourceManager::ConsumeSyncMessage(
43
43
rpc::ResourcesData resources;
44
44
resources.ParseFromString (message->sync_message ());
45
45
resources.set_node_id (message->node_id ());
46
- UpdateFromResourceReport (resources);
46
+ if (message->message_type () == syncer::MessageType::COMMANDS) {
47
+ UpdateFromResourceCommand (resources);
48
+ } else if (message->message_type () == syncer::MessageType::RESOURCE_VIEW) {
49
+ UpdateFromResourceView (resources);
50
+ } else {
51
+ RAY_LOG (FATAL) << " Unsupported message type: " << message->message_type ();
52
+ }
47
53
},
48
54
" GcsResourceManager::Update" );
49
55
}
@@ -124,7 +130,7 @@ void GcsResourceManager::HandleGetAllAvailableResources(
124
130
++counts_[CountType::GET_ALL_AVAILABLE_RESOURCES_REQUEST];
125
131
}
126
132
127
- void GcsResourceManager::UpdateFromResourceReport (const rpc::ResourcesData &data) {
133
+ void GcsResourceManager::UpdateFromResourceView (const rpc::ResourcesData &data) {
128
134
NodeID node_id = NodeID::FromBinary (data.node_id ());
129
135
// When gcs detects task pending, we may receive an local update. But it can be ignored
130
136
// here because gcs' syncer has already broadcast it.
@@ -134,10 +140,11 @@ void GcsResourceManager::UpdateFromResourceReport(const rpc::ResourcesData &data
134
140
if (RayConfig::instance ().gcs_actor_scheduling_enabled ()) {
135
141
UpdateNodeNormalTaskResources (node_id, data);
136
142
} else {
143
+ // We will only update the node's resources if it's from resource view reports.
137
144
if (!cluster_resource_manager_.UpdateNode (scheduling::NodeID (node_id.Binary ()),
138
145
data)) {
139
146
RAY_LOG (INFO)
140
- << " [UpdateFromResourceReport ]: received resource usage from unknown node id "
147
+ << " [UpdateFromResourceView ]: received resource usage from unknown node id "
141
148
<< node_id;
142
149
}
143
150
}
@@ -167,7 +174,7 @@ void GcsResourceManager::HandleReportResourceUsage(
167
174
rpc::ReportResourceUsageRequest request,
168
175
rpc::ReportResourceUsageReply *reply,
169
176
rpc::SendReplyCallback send_reply_callback) {
170
- UpdateFromResourceReport (request.resources ());
177
+ UpdateFromResourceView (request.resources ());
171
178
172
179
GCS_RPC_SEND_REPLY (send_reply_callback, reply, Status::OK ());
173
180
++counts_[CountType::REPORT_RESOURCE_USAGE_REQUEST];
@@ -260,6 +267,19 @@ void GcsResourceManager::HandleGetAllResourceUsage(
260
267
++counts_[CountType::GET_ALL_RESOURCE_USAGE_REQUEST];
261
268
}
262
269
270
+ void GcsResourceManager::UpdateFromResourceCommand (const rpc::ResourcesData &data) {
271
+ const auto node_id = NodeID::FromBinary (data.node_id ());
272
+ auto iter = node_resource_usages_.find (node_id);
273
+ if (iter == node_resource_usages_.end ()) {
274
+ return ;
275
+ }
276
+
277
+ // TODO(rickyx): We should change this to be part of RESOURCE_VIEW.
278
+ // This is being populated from NodeManager as part of COMMANDS
279
+ iter->second .set_cluster_full_of_actors_detected (
280
+ data.cluster_full_of_actors_detected ());
281
+ }
282
+
263
283
void GcsResourceManager::UpdateNodeResourceUsage (const NodeID &node_id,
264
284
const rpc::ResourcesData &resources) {
265
285
auto iter = node_resource_usages_.find (node_id);
@@ -268,18 +288,15 @@ void GcsResourceManager::UpdateNodeResourceUsage(const NodeID &node_id,
268
288
// If the node is not registered to GCS,
269
289
// we are guaranteed that no resource usage will be reported.
270
290
return ;
271
- } else {
272
- if (resources.resources_total_size () > 0 ) {
273
- (*iter->second .mutable_resources_total ()) = resources.resources_total ();
274
- }
291
+ }
292
+ if (resources.resources_total_size () > 0 ) {
293
+ (*iter->second .mutable_resources_total ()) = resources.resources_total ();
294
+ }
275
295
276
- (*iter->second .mutable_resources_available ()) = resources.resources_available ();
296
+ (*iter->second .mutable_resources_available ()) = resources.resources_available ();
277
297
278
- if (resources.resources_normal_task_changed ()) {
279
- (*iter->second .mutable_resources_normal_task ()) = resources.resources_normal_task ();
280
- }
281
- iter->second .set_cluster_full_of_actors_detected (
282
- resources.cluster_full_of_actors_detected ());
298
+ if (resources.resources_normal_task_changed ()) {
299
+ (*iter->second .mutable_resources_normal_task ()) = resources.resources_normal_task ();
283
300
}
284
301
}
285
302
0 commit comments