Skip to content

Commit

Permalink
--amend
Browse files Browse the repository at this point in the history
  • Loading branch information
larrylian committed Nov 16, 2022
1 parent a265d38 commit aacbbb4
Showing 1 changed file with 108 additions and 3 deletions.
111 changes: 108 additions & 3 deletions reps/2022-08-31-actor-affinity-apis.md
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ Yes, this will be a complement to ray core's ability to flexibly schedule actors
## Stewardship
### Required Reviewers

@wumuzi520 SenlinZhu @WangTaoTheTonic @scv119 (Chen Shen) @jjyao (Jiajun Yao)
@wumuzi520 SenlinZhu @Chong Li @scv119 (Chen Shen) @jjyao (Jiajun Yao)
### Shepherd of the Proposal (should be a senior committer)


Expand Down Expand Up @@ -335,7 +335,7 @@ ActorHandle<Counter> actor6 =
### Implementation plan
Now there are two modes of scheduling: GCS mode scheduling and raylet scheduling.
It will be simpler to implement in GCS mode.
#### GCS Scheduling Mode Implementation plan
#### 1. GCS Scheduling Mode Implementation plan

1. Actor adds the Labels property. Stored in the GcsActor structure
2. Gcs Server add GcsLabelManager. Add labels->node information to GcsLabelManager after per actor completes scheduling.
Expand Down Expand Up @@ -397,10 +397,107 @@ Main data structure :
Map<label_key, Map<lable_value, Set<node_id>>> label_to_nodes_
Map<node_id, Set<GcsActor>> node_to_actors_
```
#### Raylet Scheduling Mode Implementation plan
#### 2. Raylet Scheduling Mode Implementation plan
The implementation of Raylet scheduling mode is same as GCS scheduling above.
Mainly, one more Labels information needs to be synchronized to all Raylet nodes

1. Add the actor_labels data structure to the resource synchronization data structure(ResourcesData and NodeResources).
```
message ResourcesData {
// Node id.
bytes node_id = 1;
// Resource capacity currently available on this node manager.
map<string, double> resources_available = 2;
// Indicates whether available resources is changed. Only used when light
// heartbeat enabled.
bool resources_available_changed = 3;

// Map<key, Map<value, reference_count>> Actors scheduled to this node and actor labels information
repeat Map<string, Map<string, int>> actor_labels = 15
// Whether the actors of this node is changed.
bool actor_labels_changed = 16,
}


NodeResources {
ResourceRequest total;
ResourceRequest available;
/// Only used by light resource report.
ResourceRequest load;
/// Resources owned by normal tasks.
ResourceRequest normal_task_resources
/// Actors scheduled to this node and actor labels information
absl::flat_hash_map<string, absl::flat_hash_map<string, int>> actor_labels;
}
```

2. Adapts where ResourcesData is constructed and used in the resource synchronization mechanism.
a. NodeManager::HandleRequestResourceReport
b. NodeManager::HandleUpdateResourceUsage


3. Add ActorLabels information to NodeResources during Actor scheduling

a. When the Raylet is successfully scheduled, the ActorLabels information is added to the remote node scheduled in the ClusterResoucesManager.
```
void ClusterTaskManager::ScheduleAndDispatchTasks() {
auto scheduling_node_id = cluster_resource_scheduler_->GetBestSchedulableNode(
ScheduleOnNode(node_id, work);
cluster_resource_scheduler_->AllocateRemoteTaskResources(node_id, resources)
cluster_resource_scheduler_->GetClusterResourceManager().AddActorLabels(node_id, actor);
```
b. Add ActorLabels information to LocalResourcesManager when Actor is dispatched to Worker.
```
LocalTaskManager::DispatchScheduledTasksToWorkers()
cluster_resource_scheduler_->GetLocalResourceManager().AllocateLocalTaskResources
cluster_resource_scheduler_->GetLocalResourceManager().AddActorLabels(actor)
worker_pool_.PopWorker()
```

c. When the Actor is destroyed, the ActorLabels information of the LocalResourcesManager is also deleted.
```
NodeManager::HandleReturnWorker
local_task_manager_->ReleaseWorkerResources(worker);
local_resource_manager_->RemoveActorLabels(actor_id);
```

Actor scheduling flowchart:
![Actor scheduling flowchart](https://user-images.githubusercontent.com/11072802/202128385-f72609c5-308d-4210-84ff-bf3ba6df381c.png)

Node Resources synchronization mechanism:
![Node Resources synchronization mechanism](https://user-images.githubusercontent.com/11072802/202128406-b4745e6e-3565-41a2-bfe3-78843379bf09.png)

4. Scheduling optimization through ActorLabels
Now any node raylet has ActorLabels information for all nodes.
However, when ActorAffinity schedules, if it traverses the Labels of all Actors of each node, the algorithm complexity is very large, and the performance will be poor.
<b> Therefore, it is necessary to generate a full-cluster ActorLabels index table to improve scheduling performance. <b>

```
class GcsLabelManager {
public:
absl::flat_hash_set<NodeID> GetNodesByKeyAndValue(const std::string &ray_namespace,
const std::string &key, const absl::flat_hash_set<std::string> &values) const;

absl::flat_hash_set<NodeID> GetNodesByKey(const std::string &ray_namespace,
const std::string &key) const;

void AddActorLabels(const std::shared_ptr<GcsActor> &actor);

void RemoveActorLabels(const std::shared_ptr<GcsActor> &actor);

private:
<namespace, <label_key, <lable_value, [node_id]>>> labels_to_nodes_;
<node_id, <namespace, [actor]>> nodes_to_actors_;
}
```

<b>Advantages:<b>
1. Compared with the scheme of putting Labels in the coustom resource. This scheme can also reuse the resource synchronization mechanism. Then it won't destroy the concept of coustrom resouce.
<b>Defect
1. Because there must be a delay in resource synchronization under raylet scheduling. So if actor affinity is Soft semantics, there will be inaccurate scheduling.
### Failures and Special Scenarios
#### 1、If the Match Expression Cannot be satisfied
If the matching expression cannot be satisfied, The actor will be add to the pending actor queue. Util the matching expression all be statisfied。
Expand Down Expand Up @@ -428,3 +525,11 @@ All APIs will be fully unit tested. All specifications in this documentation wil
## (Optional) Follow-on Work
### Expression of "OR" semantics.
Later, if necessary, you can extend the semantics of "OR" by adding "is_or_semantics" to ActorAffinitySchedulingStrategy.
```
class ActorAffinitySchedulingStrategy:
def __init__(self, match_expressions: List[ActorAffinityMatchExpression], is_or_semantics = false):
self.match_expressions = match_expressions
self.is_or_semantics =
```

0 comments on commit aacbbb4

Please sign in to comment.