You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
Copy file name to clipboardExpand all lines: reps/2023-05-10-actors-batch-remote-api.md
+67-8Lines changed: 67 additions & 8 deletions
Original file line number
Diff line number
Diff line change
@@ -8,7 +8,7 @@ In distributed computing scenarios, such as big data computing、AI training and
8
8
For example, in a typical star-topology architecture with one master Actor and 400 worker Actors, the same computational request needs to be sent to 400 worker Actors.
9
9
However, the computing tasks of each Worker Actor are very short. In such a scenario, the performance requirements for executing batch actor task remote of a large number of Actor are very high.
10
10
Therefore, for the scenario of batch calling Actor tasks, I want to add a new optimization API, batch_remote(), to improve the performance of batch submission of Actor Task calls.
11
-
After my own performance testing and comparison, this API can achieve a performance improvement of 40% ~ 98%.
11
+
After my own performance testing and comparison, this API interface can improve performance by 2 ~ 100+ times.
12
12
13
13
Current situation of batch calling actor tasks:
14
14
```
@@ -24,7 +24,8 @@ Using the new Batch Remote API:
24
24
actors = [WorkerActor.remote() for _ in range(400)]
25
25
26
26
# Calling it only once can greatly improve performance.
The current performance bottleneck of batch invoking ActorTask mainly lies in the duplicate serialization of parameters, repeated puts parameters into object store, and frequent context switching between Python and C++.
@@ -47,9 +48,17 @@ The initial consideration is to add it to the ray.experimental module.
@@ -87,7 +96,7 @@ The following are the performance comparison results.
87
96
**Table 1: Comparison of remote call time with varying parameter sizes and 400 Actors**
88
97
89
98
90
-
Parameter Size (byte) | Time taken for foreach_remote(ms) | Time taken for batch_remote(ms) | The improvement rate
99
+
Parameter Size (byte) | Time taken for foreach_remote(ms) | Time taken for batch_remote(ms) | The ratio of time reduction
91
100
-- | -- | -- | --
92
101
10 | 40.532 | 9.226 | 77.2%
93
102
409846 | 584.345 | 24.106 | 95.9%
@@ -119,7 +128,7 @@ Parameter Size (byte) | Time taken for foreach_remote(ms) | Time taken for batch
119
128
120
129
**Table 2: Comparison of remote call time with varying numbers of Actors and a fixed parameter size (1MB)**
121
130
122
-
actor counts | Time taken for foreach_remote(ms) | Time taken for batch_remote(ms) | The improvement rate
131
+
actor counts | Time taken for foreach_remote(ms) | Time taken for batch_remote(ms) | The ratio of time reduction
123
132
-- | -- | -- | --
124
133
50 | 95.889 | 4.657 | 95.1%
125
134
100 | 196.184 | 8.447 | 95.7%
@@ -151,7 +160,7 @@ The more actors, the greater the performance gain.
151
160
152
161
This test is to confirm the degree of performance optimization after reducing the frequency of switching between the Python and C++ execution layers.
153
162
154
-
actor counts | Time taken for foreach_remote(ms) | Time taken for batch_remote(ms) | The improvement rate
163
+
actor counts | Time taken for foreach_remote(ms) | Time taken for batch_remote(ms) | The ratio of time reduction
155
164
-- | -- | -- | --
156
165
50 | 2.083 | 1.257 | 39.7%
157
166
100 | 4.005 | 2.314 | 42.2%
@@ -177,7 +186,36 @@ actor counts | Time taken for foreach_remote(ms) | Time taken for batch_remote(m
177
186
178
187
179
188
**Conclusion:**
180
-
After comparison, in the scenario of 400 actors and remote calls without parameters, the performance is optimized by 40%~50%.
189
+
After comparison, in the scenario of remote calls without parameters, the performance is optimized by 2+ times.
190
+
191
+
**Table 4: Comparison of remote call time with varying numbers of Actors and object ref parameters in remote calls**
192
+
193
+
actor counts | The time taken for foreach_remote(ms) | The time taken for batch_remote(ms) | The ratio of time reduction
194
+
-- | -- | -- | --
195
+
50 | 3.878 | 1.488 | 61.6%
196
+
100 | 8.383 | 2.405 | 71.3%
197
+
150 | 12.16 | 3.255 | 73.2%
198
+
200 | 16.835 | 4.913 | 70.8%
199
+
250 | 21.09 | 6.424 | 69.5%
200
+
300 | 24.674 | 8.272 | 66.5%
201
+
350 | 28.639 | 8.862 | 69.1%
202
+
400 | 33.42 | 10.352 | 69.0%
203
+
450 | 37.39 | 12.02 | 67.9%
204
+
500 | 39.944 | 13.288 | 66.7%
205
+
550 | 45.019 | 15.005 | 66.7%
206
+
600 | 48.237 | 15.349 | 68.2%
207
+
650 | 53.304 | 17.149 | 67.8%
208
+
700 | 56.961 | 18.124 | 68.2%
209
+
750 | 61.672 | 19.079 | 69.1%
210
+
800 | 66.185 | 20.485 | 69.0%
211
+
850 | 69.524 | 21.584 | 69.0%
212
+
900 | 74.754 | 22.304 | 70.2%
213
+
950 | 79.493 | 25.932 | 67.4%
214
+
215
+

216
+
217
+
**Conclusion:**
218
+
After comparison, in the scenario of remote calls with object ref paramter, the performance is optimized by 3~4 times.
181
219
182
220
**Summary:**
183
221
The newly added Batch Remote API can improve performance in the case of batch calling Actor task. It can reduce performance costs such as parameter serialization, object store consumption, and Python and C++ execution layer switching, thereby improving the performance of the entire distributed computing system.
@@ -186,6 +224,18 @@ Especially in the following scenario:
186
224
2. a large number of Actors
187
225
188
226
227
+
### Failure & Exception Scenario.
228
+
229
+
**1. Exceptions occurred during parameter validation or preprocessing before batch submission of ActorTasks.**
230
+
Since these exceptions occur before the process of submitting ActorTasks, they can be handled by directly throwing specific error exceptions as current situation.
231
+
232
+
**2. Some actors throw exceptions during the process of batch submitting ActorTasks.**
233
+
When traversing and submitting ActorTasks in a loop, if one of the Actors throws an exception during submission, the subsequent ActorTasks will be terminated immediately, and the exception will be throwed to user.
234
+
235
+
Reason:
236
+
1. Submitting ActorTask is normally done without any exceptions being thrown. If an error does occur, it is likely due to issues with the code and will require modifications.
237
+
2. The exception behavior of this plan is the same as the current foreach remote.
238
+
189
239
## Compatibility, Deprecation, and Migration Plan
190
240
N/A
191
241
@@ -203,3 +253,12 @@ The collection communication of Ray's CPU computing scenario is currently implem
203
253
Therefore, we want to develop a native collection communication library for Ray with the aim of making it more convenient for users and supporting large-scale scenarios.
204
254
This native library will undoubtedly rely on the current batch remote API to improve performance. After the API is completed, we will consider implementing the native collection communication library for Ray.
205
255
256
+
257
+
### 2. ActorPool adds the capability of batch submitting ActorTask.
258
+
ActorPool is a utility class to operate on a fixed pool of actors.
0 commit comments