Skip to content

Commit a83b9a1

Browse files
committed
add multi api form
1 parent 090f8ac commit a83b9a1

File tree

2 files changed

+122
-14
lines changed

2 files changed

+122
-14
lines changed

reps/2022-08-31-actor-affinity-apis.md

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -54,6 +54,9 @@ Advantage
5454

5555
### API Design
5656

57+
The API here is temporarily expressed in this form.
58+
If the form of the API is determined in 2022-11-23-labels-mechanish-and-affinity-schedule-feature.md later, I will modify the API here.
59+
5760
#### Python API
5861
Python API Design:
5962
Set key-value labels for actors

reps/2022-11-23-labels-mechanish-and-affinity-schedule-feature.md

Lines changed: 119 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -1,8 +1,9 @@
11
## Summary
22
### General Motivation
33

4-
Introduce the Labels mechanism. Give Labels to Actors/Tasks/Nodes/Objects.
5-
Affinity features such as ActorAffinity/NodeAffinity can be realized through Labels.
4+
Introduce labels mechanism, which associate an enumerated property to Ray nodes. We can static assign labels to Ray node on start up, or dynamically change node's label through Ray scheduling APIs.
5+
6+
Labels mechanism makes it easy to implement Actor and Node affinity.(Taints and Tolerations style)
67

78

89
### Should this change be within `ray` or outside?
@@ -19,32 +20,95 @@ Yes, this will be a complement to ray core's ability to flexibly schedule actors
1920
## Design and Architecture
2021

2122
### Brief idea
22-
1. Introduce the concept of Label. Add the Labels attribute to Actor/Task/Node. Labels = Map<String, String>.
23-
2. After Actor/Task are scheduled to a certain node, the Labels of Actor/Task will be attached to the node resource(Named: LabelsResource). Node's Labels are naturally in the node resource.
24-
3. Actor/Task scheduling can choose Actor/Task/NodeAffinitySchedulingStratgy.
25-
4. The actual principle of Actor/Task/NodeAffinitySchedulingStratgy is. According to the label matching expression in the strategy, traverse and search for LabelsResource of all nodes. Pick out nodes that meet the expression requirements.
23+
1. Introduce the concept of Label. Add the Labels attribute to Actor/Node. Labels = Map<String, String>.
24+
2. After Actor are scheduled to a certain node, the Labels of Actor will be attached to the node resource(Named: LabelsResource). Node's Labels are naturally in the node resource.
25+
3. Actor scheduling can choose Actor/NodeAffinitySchedulingStratgy.
26+
4. The actual principle of Actor/NodeAffinitySchedulingStratgy is. According to the label matching expression in the strategy, traverse and search for LabelsResource of all nodes. Pick out nodes that meet the expression requirements.
2627

2728
![LabelsAffinity](https://user-images.githubusercontent.com/11072802/203686866-385235b5-e08b-4aac-9c31-512621129bd4.png)
2829

30+
Scheduling Policy | Label Owner | Label select operator
31+
-- | -- | --
32+
ActorAffinity | Actor | in, not_in, exists, does_not_exist
33+
NodeAffinity | Node | in, not_in, exists, does_not_exist
34+
2935
### API Design
30-
The apis of actors/tasks/nodes add labels.
36+
37+
**The apis of actors/nodes add labels**
38+
39+
This interface is already very simple, so we will not set up multiple solutions for everyone to discuss.
3140
```python
3241
# Actor add labels.
3342
actor_1 = Actor.options(labels={
3443
"location": "dc_1"
3544
}).remote()
3645

37-
# Task add labels.
38-
task_1 = Task.options(labels={
39-
"location": "dc_1"
40-
}).remote()
4146

4247
# Node add static labels.
4348
ray start ... --labels={"location": "dc_1"}
4449
# The api of the dynamic update node labels is similar to the current dynamic set_resource. It can be determined later.
4550
```
4651

47-
The apis of the actor-affinity/task-affinity/node-affinity scheduling.
52+
**The apis of the actor-affinity/node-affinity scheduling.**
53+
54+
**Option 1: Simplify through syntactic sugar**
55+
56+
```python
57+
actor_1 = Actor.options(
58+
scheduling_strategy=actor_affinity(label_in("location", ["dc_1"], false))
59+
).remote()
60+
61+
actor_1 = Actor.options(
62+
scheduling_strategy=node_affinity(label_exist("location", false))
63+
).remote()
64+
65+
actor_1 = Actor.options(
66+
scheduling_strategy=actor_affinity([
67+
label_in("location", ["dc_1"], false),
68+
label_exists("location", false)
69+
])
70+
).remote()
71+
72+
def actor_affinity(...):
73+
...
74+
return ActorAffinitySchedulingStrategy(...)
75+
76+
def node_affinity(...):
77+
...
78+
return NodeAffinitySchedulingStrategy(...)
79+
80+
def label_in(key, values, is_soft):
81+
...
82+
return LabelMatchExpression(...)
83+
84+
def label_not_in(key, values, is_soft):
85+
...
86+
return LabelMatchExpression(...)
87+
88+
def label_exists(key, is_soft):
89+
...
90+
return LabelMatchExpression(...)
91+
92+
def label_does_not_exist(key, is_soft):
93+
...
94+
return LabelMatchExpression(...)
95+
```
96+
97+
**Option 2: another syntactic sugar**
98+
99+
Personally, I think this Option is not as good as the above Option 1.
100+
The label_in(key, values, is_soft) form of option 1 is more understandable and better than the form of ("location", LabelMatchOperator.IN, ["dc_1"], false).
101+
```python
102+
actor_1 = Actor.options(
103+
scheduling_strategy=ActorAffinity([
104+
("location", LabelMatchOperator.IN, ["dc_1"], false),
105+
("location", LabelMatchOperator.Exist)).
106+
).remote()
107+
```
108+
109+
**Option 3: Java-like form**
110+
111+
This form is similar to Java's syntax. The downside is that it's a bit complicated.
48112
```python
49113
SchedulingStrategyT = Union[None, str,
50114
PlacementGroupSchedulingStrategy,
@@ -75,11 +139,29 @@ class LabelMatchExpression:
75139
self.values = values
76140
self.soft = soft
77141

78-
# ActorAffinity use case
79142
actor_1 = Actor.options(scheduling_strategy=ActorAffinitySchedulingStrategy([
80143
LabelMatchExpression(
81144
"location", LabelMatchOperator.IN, ["dc_1"], False)
82145
])).remote()
146+
147+
actor_1 = Actor.options(scheduling_strategy=ActorAffinitySchedulingStrategy([
148+
LabelMatchExpression(
149+
"location", LabelMatchOperator.IN, ["dc_1"], False),
150+
LabelMatchExpression(
151+
"location", LabelMatchOperator.EXISTS, None, False)
152+
])).remote()
153+
```
154+
155+
**Option 4: Like sql**
156+
157+
This solution is not recommended.
158+
This method needs to parse SQL, and the workload will be much larger.
159+
And users often write wrong sql when using it.
160+
```python
161+
# ActorAffinity use case
162+
actor_1 = Actor.options(
163+
scheduling_strategy=ActorAffinity("location in [dc_1, dc_2]")
164+
).remote()
83165
```
84166

85167
### Example
@@ -89,6 +171,9 @@ actor_1 = Actor.options(scheduling_strategy=ActorAffinitySchedulingStrategy([
89171
* Anti-affinity
90172
* Spread the actors of a service across nodes and/or availability zones, e.g. to reduce correlated failures.
91173

174+
I will update the following API when the API plan is determined.
175+
Now describe the example with an api like java form.
176+
92177
**1. Spread Demo**
93178

94179
![spread demo](https://user-images.githubusercontent.com/11072802/207037933-8a9d9f1d-ee6e-472b-a877-669cef996db9.png)
@@ -379,4 +464,24 @@ class ActorAffinitySchedulingStrategy:
379464
If the user has a request, you can consider adding the attributes of labels to objects. Then the strategy of ObjectAffinity can be launched。
380465

381466
### 3. TaskAffinitySchedulingStrategy
382-
Because the resource synchronization mechanism of Label has been implemented above. Therefore, it is easy to create a TaskAffinity strategy for Task.
467+
Because the resource synchronization mechanism of Label has been implemented above. Therefore, it is easy to create a TaskAffinity strategy for Task.
468+
469+
**Task add labels**
470+
task_1 = Task.options(labels={
471+
"location": "dc_1"
472+
}).remote()
473+
474+
**Add TaskAffinitySchedulingStategy**
475+
ref = Task.options(
476+
scheduling_strategy = task_affinity(label_in("location", ["dc_1"], false)
477+
).remote()
478+
479+
### 4. Use Affinity scheduling as another dimension scheduling strategy
480+
The Actor/NodeAffinity strategy can be independent of the SchedulingStrategy as a second-dimensional scheduling strategy.
481+
Add a property of Affinity=Actor/NodeAffinity. eg:
482+
```
483+
actor_1 = Actor.options(
484+
scheduling_strategy="DEFAULT",
485+
affinity=actor_affinity(label_in("location", ["dc_1"], false)
486+
).remote()
487+
```

0 commit comments

Comments
 (0)