-
Notifications
You must be signed in to change notification settings - Fork 6.9k
Revert "Revert "[core] Fix wrong local resource view in raylet (#1991… #19996
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
Changes from all commits
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 |
|---|---|---|
|
|
@@ -890,7 +890,15 @@ void NodeManager::ResourceCreateUpdated(const NodeID &node_id, | |
| const ResourceSet &createUpdatedResources) { | ||
| RAY_LOG(DEBUG) << "[ResourceCreateUpdated] received callback from node id " << node_id | ||
| << " with created or updated resources: " | ||
| << createUpdatedResources.ToString() << ". Updating resource map."; | ||
| << createUpdatedResources.ToString() << ". Updating resource map." | ||
| << " skip=" << (node_id == self_node_id_); | ||
|
|
||
| // Skip updating local node since local node always has the latest information. | ||
| // Updating local node could result in a inconsistence view in cluster resource | ||
| // scheduler which could make task hang. | ||
| if (node_id == self_node_id_) { | ||
| return; | ||
| } | ||
|
|
||
| // Update local_available_resources_ and SchedulingResources | ||
| for (const auto &resource_pair : createUpdatedResources.GetResourceMap()) { | ||
|
|
@@ -900,11 +908,7 @@ void NodeManager::ResourceCreateUpdated(const NodeID &node_id, | |
| new_resource_capacity); | ||
| } | ||
| RAY_LOG(DEBUG) << "[ResourceCreateUpdated] Updated cluster_resource_map."; | ||
|
|
||
| if (node_id == self_node_id_) { | ||
fishbone marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| // The resource update is on the local node, check if we can reschedule tasks. | ||
| cluster_task_manager_->ScheduleAndDispatchTasks(); | ||
| } | ||
| cluster_task_manager_->ScheduleAndDispatchTasks(); | ||
| } | ||
|
|
||
| void NodeManager::ResourceDeleted(const NodeID &node_id, | ||
|
|
@@ -916,7 +920,14 @@ void NodeManager::ResourceDeleted(const NodeID &node_id, | |
| } | ||
| RAY_LOG(DEBUG) << "[ResourceDeleted] received callback from node id " << node_id | ||
| << " with deleted resources: " << oss.str() | ||
| << ". Updating resource map."; | ||
| << ". Updating resource map. skip=" << (node_id == self_node_id_); | ||
| } | ||
|
|
||
| // Skip updating local node since local node always has the latest information. | ||
| // Updating local node could result in a inconsistence view in cluster resource | ||
| // scheduler which could make task hang. | ||
| if (node_id == self_node_id_) { | ||
|
Contributor
Author
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. only this line
Member
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. Just curious, what is the mechanism that processing resource deletion for this node slows down the test and makes it flaky?
Contributor
Author
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. oh, actually it's not slowing down the test. it's hanging and timeout eventually. Basically, the local resource was deleted before scheduling and raylet found no resource to run the job. Luckily, I can always reproduce it locally. |
||
| return; | ||
| } | ||
|
|
||
| // Update local_available_resources_ and SchedulingResources | ||
|
|
@@ -1474,39 +1485,44 @@ void NodeManager::HandleUpdateResourceUsage( | |
| rpc::SendReplyCallback send_reply_callback) { | ||
| rpc::ResourceUsageBroadcastData resource_usage_batch; | ||
| resource_usage_batch.ParseFromString(request.serialized_resource_usage_batch()); | ||
|
|
||
| if (resource_usage_batch.seq_no() != next_resource_seq_no_) { | ||
| // When next_resource_seq_no_ == 0 it means it just started. | ||
| // TODO: Fetch a snapshot from gcs for lightweight resource broadcasting | ||
| if (next_resource_seq_no_ != 0 && | ||
| resource_usage_batch.seq_no() != next_resource_seq_no_) { | ||
| // TODO (Alex): Ideally we would be really robust, and potentially eagerly | ||
| // pull a full resource "snapshot" from gcs to make sure our state doesn't | ||
| // diverge from GCS. | ||
| RAY_LOG(WARNING) | ||
| << "Raylet may have missed a resource broadcast. This either means that GCS has " | ||
| "restarted, the network is heavily congested and is dropping, reordering, or " | ||
| "duplicating packets. Expected seq#: " | ||
| << next_resource_seq_no_ << ", but got: " << resource_usage_batch.seq_no() << "."; | ||
| // TODO (Alex): Ideally we would be really robust, and potentially eagerly | ||
| // pull a full resource "snapshot" from gcs to make sure our state doesn't | ||
| // diverge from GCS. | ||
| if (resource_usage_batch.seq_no() < next_resource_seq_no_) { | ||
| RAY_LOG(WARNING) << "Discard the the resource update since local version is newer"; | ||
| return; | ||
| } | ||
| } | ||
| next_resource_seq_no_ = resource_usage_batch.seq_no() + 1; | ||
|
|
||
| for (const auto &resource_change_or_data : resource_usage_batch.batch()) { | ||
| if (resource_change_or_data.has_data()) { | ||
| const auto &resource_usage = resource_change_or_data.data(); | ||
| const NodeID &node_id = NodeID::FromBinary(resource_usage.node_id()); | ||
| if (node_id == self_node_id_) { | ||
| // Skip messages from self. | ||
| continue; | ||
| auto node_id = NodeID::FromBinary(resource_usage.node_id()); | ||
| // Skip messages from self. | ||
| if (node_id != self_node_id_) { | ||
| UpdateResourceUsage(node_id, resource_usage); | ||
| } | ||
| UpdateResourceUsage(node_id, resource_usage); | ||
| } else if (resource_change_or_data.has_change()) { | ||
| const auto &resource_notification = resource_change_or_data.change(); | ||
| auto id = NodeID::FromBinary(resource_notification.node_id()); | ||
| auto node_id = NodeID::FromBinary(resource_notification.node_id()); | ||
| if (resource_notification.updated_resources_size() != 0) { | ||
| ResourceSet resource_set( | ||
| MapFromProtobuf(resource_notification.updated_resources())); | ||
| ResourceCreateUpdated(id, resource_set); | ||
| ResourceCreateUpdated(node_id, resource_set); | ||
| } | ||
|
|
||
| if (resource_notification.deleted_resources_size() != 0) { | ||
| ResourceDeleted(id, | ||
| ResourceDeleted(node_id, | ||
| VectorFromProtobuf(resource_notification.deleted_resources())); | ||
| } | ||
| } | ||
|
|
||
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
cluster_task_manager_->ScheduleAndDispatchTasks();here?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.
It seems like some tests are still failing after this PR, and this is the only behavior change I can imagine (since changes below seem to be only applied to grpc resource broadcast). I think calling the function one more time here won't hurt the performance