/
actor_pool.py
212 lines (174 loc) · 7.29 KB
/
actor_pool.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
import ray
class ActorPool:
"""Utility class to operate on a fixed pool of actors.
Arguments:
actors (list): List of Ray actor handles to use in this pool.
Examples:
>>> a1, a2 = Actor.remote(), Actor.remote()
>>> pool = ActorPool([a1, a2])
>>> print(pool.map(lambda a, v: a.double.remote(v), [1, 2, 3, 4]))
[2, 4, 6, 8]
"""
def __init__(self, actors):
# actors to be used
self._idle_actors = list(actors)
# get actor from future
self._future_to_actor = {}
# get future from index
self._index_to_future = {}
# next task to do
self._next_task_index = 0
# next task to return
self._next_return_index = 0
# next work depending when actors free
self._pending_submits = []
def map(self, fn, values):
"""Apply the given function in parallel over the actors and values.
This returns an ordered iterator that will return results of the map
as they finish. Note that you must iterate over the iterator to force
the computation to finish.
Arguments:
fn (func): Function that takes (actor, value) as argument and
returns an ObjectID computing the result over the value. The
actor will be considered busy until the ObjectID completes.
values (list): List of values that fn(actor, value) should be
applied to.
Returns:
Iterator over results from applying fn to the actors and values.
Examples:
>>> pool = ActorPool(...)
>>> print(pool.map(lambda a, v: a.double.remote(v), [1, 2, 3, 4]))
[2, 4, 6, 8]
"""
for v in values:
self.submit(fn, v)
while self.has_next():
yield self.get_next()
def map_unordered(self, fn, values):
"""Similar to map(), but returning an unordered iterator.
This returns an unordered iterator that will return results of the map
as they finish. This can be more efficient that map() if some results
take longer to compute than others.
Arguments:
fn (func): Function that takes (actor, value) as argument and
returns an ObjectID computing the result over the value. The
actor will be considered busy until the ObjectID completes.
values (list): List of values that fn(actor, value) should be
applied to.
Returns:
Iterator over results from applying fn to the actors and values.
Examples:
>>> pool = ActorPool(...)
>>> print(pool.map(lambda a, v: a.double.remote(v), [1, 2, 3, 4]))
[6, 2, 4, 8]
"""
for v in values:
self.submit(fn, v)
while self.has_next():
yield self.get_next_unordered()
def submit(self, fn, value):
"""Schedule a single task to run in the pool.
This has the same argument semantics as map(), but takes on a single
value instead of a list of values. The result can be retrieved using
get_next() / get_next_unordered().
Arguments:
fn (func): Function that takes (actor, value) as argument and
returns an ObjectID computing the result over the value. The
actor will be considered busy until the ObjectID completes.
value (object): Value to compute a result for.
Examples:
>>> pool = ActorPool(...)
>>> pool.submit(lambda a, v: a.double.remote(v), 1)
>>> pool.submit(lambda a, v: a.double.remote(v), 2)
>>> print(pool.get_next(), pool.get_next())
2, 4
"""
if self._idle_actors:
actor = self._idle_actors.pop()
future = fn(actor, value)
self._future_to_actor[future] = (self._next_task_index, actor)
self._index_to_future[self._next_task_index] = future
self._next_task_index += 1
else:
self._pending_submits.append((fn, value))
def has_next(self):
"""Returns whether there are any pending results to return.
Returns:
True if there are any pending results not yet returned.
Examples:
>>> pool = ActorPool(...)
>>> pool.submit(lambda a, v: a.double.remote(v), 1)
>>> print(pool.has_next())
True
>>> print(pool.get_next())
2
>>> print(pool.has_next())
False
"""
return bool(self._future_to_actor)
def get_next(self, timeout=None):
"""Returns the next pending result in order.
This returns the next result produced by submit(), blocking for up to
the specified timeout until it is available.
Returns:
The next result.
Raises:
TimeoutError if the timeout is reached.
Examples:
>>> pool = ActorPool(...)
>>> pool.submit(lambda a, v: a.double.remote(v), 1)
>>> print(pool.get_next())
2
"""
if not self.has_next():
raise StopIteration("No more results to get")
if self._next_return_index >= self._next_task_index:
raise ValueError("It is not allowed to call get_next() after "
"get_next_unordered().")
future = self._index_to_future[self._next_return_index]
if timeout is not None:
res, _ = ray.wait([future], timeout=timeout)
if not res:
raise TimeoutError("Timed out waiting for result")
del self._index_to_future[self._next_return_index]
self._next_return_index += 1
i, a = self._future_to_actor.pop(future)
self._return_actor(a)
return ray.get(future)
def get_next_unordered(self, timeout=None):
"""Returns any of the next pending results.
This returns some result produced by submit(), blocking for up to
the specified timeout until it is available. Unlike get_next(), the
results are not always returned in same order as submitted, which can
improve performance.
Returns:
The next result.
Raises:
TimeoutError if the timeout is reached.
Examples:
>>> pool = ActorPool(...)
>>> pool.submit(lambda a, v: a.double.remote(v), 1)
>>> pool.submit(lambda a, v: a.double.remote(v), 2)
>>> print(pool.get_next_unordered())
4
>>> print(pool.get_next_unordered())
2
"""
if not self.has_next():
raise StopIteration("No more results to get")
# TODO(ekl) bulk wait for performance
res, _ = ray.wait(
list(self._future_to_actor), num_returns=1, timeout=timeout)
if res:
[future] = res
else:
raise TimeoutError("Timed out waiting for result")
i, a = self._future_to_actor.pop(future)
self._return_actor(a)
del self._index_to_future[i]
self._next_return_index = max(self._next_return_index, i + 1)
return ray.get(future)
def _return_actor(self, actor):
self._idle_actors.append(actor)
if self._pending_submits:
self.submit(*self._pending_submits.pop(0))