actor_pool.py 14 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463
  1. from typing import TYPE_CHECKING, Any, Callable, List, TypeVar
  2. import ray
  3. from ray.util.annotations import DeveloperAPI
  4. if TYPE_CHECKING:
  5. import ray.actor
  6. V = TypeVar("V")
  7. @DeveloperAPI
  8. class ActorPool:
  9. """Utility class to operate on a fixed pool of actors.
  10. Arguments:
  11. actors: List of Ray actor handles to use in this pool.
  12. Examples:
  13. .. testcode::
  14. import ray
  15. from ray.util.actor_pool import ActorPool
  16. @ray.remote
  17. class Actor:
  18. def double(self, v):
  19. return 2 * v
  20. a1, a2 = Actor.remote(), Actor.remote()
  21. pool = ActorPool([a1, a2])
  22. print(list(pool.map(lambda a, v: a.double.remote(v),
  23. [1, 2, 3, 4])))
  24. .. testoutput::
  25. [2, 4, 6, 8]
  26. """
  27. def __init__(self, actors: list):
  28. from ray._common.usage.usage_lib import record_library_usage
  29. record_library_usage("util.ActorPool")
  30. # actors to be used
  31. self._idle_actors = list(actors)
  32. # get actor from future
  33. self._future_to_actor = {}
  34. # get future from index
  35. self._index_to_future = {}
  36. # next task to do
  37. self._next_task_index = 0
  38. # next task to return
  39. self._next_return_index = 0
  40. # next work depending when actors free
  41. self._pending_submits = []
  42. def map(self, fn: Callable[["ray.actor.ActorHandle", V], Any], values: List[V]):
  43. """Apply the given function in parallel over the actors and values.
  44. This returns an ordered iterator that will return results of the map
  45. as they finish. Note that you must iterate over the iterator to force
  46. the computation to finish.
  47. Arguments:
  48. fn: Function that takes (actor, value) as argument and
  49. returns an ObjectRef computing the result over the value. The
  50. actor will be considered busy until the ObjectRef completes.
  51. values: List of values that fn(actor, value) should be
  52. applied to.
  53. Returns:
  54. Iterator over results from applying fn to the actors and values.
  55. Examples:
  56. .. testcode::
  57. import ray
  58. from ray.util.actor_pool import ActorPool
  59. @ray.remote
  60. class Actor:
  61. def double(self, v):
  62. return 2 * v
  63. a1, a2 = Actor.remote(), Actor.remote()
  64. pool = ActorPool([a1, a2])
  65. print(list(pool.map(lambda a, v: a.double.remote(v),
  66. [1, 2, 3, 4])))
  67. .. testoutput::
  68. [2, 4, 6, 8]
  69. """
  70. # Ignore/Cancel all the previous submissions
  71. # by calling `has_next` and `gen_next` repeteadly.
  72. while self.has_next():
  73. try:
  74. self.get_next(timeout=0, ignore_if_timedout=True)
  75. except TimeoutError:
  76. pass
  77. for v in values:
  78. self.submit(fn, v)
  79. def get_generator():
  80. while self.has_next():
  81. yield self.get_next()
  82. return get_generator()
  83. def map_unordered(
  84. self, fn: Callable[["ray.actor.ActorHandle", V], Any], values: List[V]
  85. ):
  86. """Similar to map(), but returning an unordered iterator.
  87. This returns an unordered iterator that will return results of the map
  88. as they finish. This can be more efficient that map() if some results
  89. take longer to compute than others.
  90. Arguments:
  91. fn: Function that takes (actor, value) as argument and
  92. returns an ObjectRef computing the result over the value. The
  93. actor will be considered busy until the ObjectRef completes.
  94. values: List of values that fn(actor, value) should be
  95. applied to.
  96. Returns:
  97. Iterator over results from applying fn to the actors and values.
  98. Examples:
  99. .. testcode::
  100. import ray
  101. from ray.util.actor_pool import ActorPool
  102. @ray.remote
  103. class Actor:
  104. def double(self, v):
  105. return 2 * v
  106. a1, a2 = Actor.remote(), Actor.remote()
  107. pool = ActorPool([a1, a2])
  108. print(list(pool.map_unordered(lambda a, v: a.double.remote(v),
  109. [1, 2, 3, 4])))
  110. .. testoutput::
  111. :options: +MOCK
  112. [6, 8, 4, 2]
  113. """
  114. # Ignore/Cancel all the previous submissions
  115. # by calling `has_next` and `gen_next_unordered` repeteadly.
  116. while self.has_next():
  117. try:
  118. self.get_next_unordered(timeout=0)
  119. except TimeoutError:
  120. pass
  121. for v in values:
  122. self.submit(fn, v)
  123. def get_generator():
  124. while self.has_next():
  125. yield self.get_next_unordered()
  126. return get_generator()
  127. def submit(self, fn, value):
  128. """Schedule a single task to run in the pool.
  129. This has the same argument semantics as map(), but takes on a single
  130. value instead of a list of values. The result can be retrieved using
  131. get_next() / get_next_unordered().
  132. Arguments:
  133. fn: Function that takes (actor, value) as argument and
  134. returns an ObjectRef computing the result over the value. The
  135. actor will be considered busy until the ObjectRef completes.
  136. value: Value to compute a result for.
  137. Examples:
  138. .. testcode::
  139. import ray
  140. from ray.util.actor_pool import ActorPool
  141. @ray.remote
  142. class Actor:
  143. def double(self, v):
  144. return 2 * v
  145. a1, a2 = Actor.remote(), Actor.remote()
  146. pool = ActorPool([a1, a2])
  147. pool.submit(lambda a, v: a.double.remote(v), 1)
  148. pool.submit(lambda a, v: a.double.remote(v), 2)
  149. print(pool.get_next(), pool.get_next())
  150. .. testoutput::
  151. 2 4
  152. """
  153. if self._idle_actors:
  154. actor = self._idle_actors.pop()
  155. future = fn(actor, value)
  156. future_key = tuple(future) if isinstance(future, list) else future
  157. self._future_to_actor[future_key] = (self._next_task_index, actor)
  158. self._index_to_future[self._next_task_index] = future
  159. self._next_task_index += 1
  160. else:
  161. self._pending_submits.append((fn, value))
  162. def has_next(self):
  163. """Returns whether there are any pending results to return.
  164. Returns:
  165. True if there are any pending results not yet returned.
  166. Examples:
  167. .. testcode::
  168. import ray
  169. from ray.util.actor_pool import ActorPool
  170. @ray.remote
  171. class Actor:
  172. def double(self, v):
  173. return 2 * v
  174. a1, a2 = Actor.remote(), Actor.remote()
  175. pool = ActorPool([a1, a2])
  176. pool.submit(lambda a, v: a.double.remote(v), 1)
  177. print(pool.has_next())
  178. print(pool.get_next())
  179. print(pool.has_next())
  180. .. testoutput::
  181. True
  182. 2
  183. False
  184. """
  185. return bool(self._future_to_actor)
  186. def get_next(self, timeout=None, ignore_if_timedout=False):
  187. """Returns the next pending result in order.
  188. This returns the next result produced by submit(), blocking for up to
  189. the specified timeout until it is available.
  190. Returns:
  191. The next result.
  192. Raises:
  193. TimeoutError: if the timeout is reached.
  194. Examples:
  195. .. testcode::
  196. import ray
  197. from ray.util.actor_pool import ActorPool
  198. @ray.remote
  199. class Actor:
  200. def double(self, v):
  201. return 2 * v
  202. a1, a2 = Actor.remote(), Actor.remote()
  203. pool = ActorPool([a1, a2])
  204. pool.submit(lambda a, v: a.double.remote(v), 1)
  205. print(pool.get_next())
  206. .. testoutput::
  207. 2
  208. """
  209. if not self.has_next():
  210. raise StopIteration("No more results to get")
  211. if self._next_return_index >= self._next_task_index:
  212. raise ValueError(
  213. "It is not allowed to call get_next() after get_next_unordered()."
  214. )
  215. future = self._index_to_future[self._next_return_index]
  216. timeout_msg = "Timed out waiting for result"
  217. raise_timeout_after_ignore = False
  218. if timeout is not None:
  219. res, _ = ray.wait([future], timeout=timeout)
  220. if not res:
  221. if not ignore_if_timedout:
  222. raise TimeoutError(timeout_msg)
  223. else:
  224. raise_timeout_after_ignore = True
  225. del self._index_to_future[self._next_return_index]
  226. self._next_return_index += 1
  227. future_key = tuple(future) if isinstance(future, list) else future
  228. i, a = self._future_to_actor.pop(future_key)
  229. self._return_actor(a)
  230. if raise_timeout_after_ignore:
  231. raise TimeoutError(
  232. timeout_msg + ". The task {} has been ignored.".format(future)
  233. )
  234. return ray.get(future)
  235. def get_next_unordered(self, timeout=None, ignore_if_timedout=False):
  236. """Returns any of the next pending results.
  237. This returns some result produced by submit(), blocking for up to
  238. the specified timeout until it is available. Unlike get_next(), the
  239. results are not always returned in same order as submitted, which can
  240. improve performance.
  241. Returns:
  242. The next result.
  243. Raises:
  244. TimeoutError: if the timeout is reached.
  245. Examples:
  246. .. testcode::
  247. import ray
  248. from ray.util.actor_pool import ActorPool
  249. @ray.remote
  250. class Actor:
  251. def double(self, v):
  252. return 2 * v
  253. a1, a2 = Actor.remote(), Actor.remote()
  254. pool = ActorPool([a1, a2])
  255. pool.submit(lambda a, v: a.double.remote(v), 1)
  256. pool.submit(lambda a, v: a.double.remote(v), 2)
  257. print(pool.get_next_unordered())
  258. print(pool.get_next_unordered())
  259. .. testoutput::
  260. :options: +MOCK
  261. 4
  262. 2
  263. """
  264. if not self.has_next():
  265. raise StopIteration("No more results to get")
  266. # TODO(ekl) bulk wait for performance
  267. res, _ = ray.wait(list(self._future_to_actor), num_returns=1, timeout=timeout)
  268. timeout_msg = "Timed out waiting for result"
  269. raise_timeout_after_ignore = False
  270. if res:
  271. [future] = res
  272. else:
  273. if not ignore_if_timedout:
  274. raise TimeoutError(timeout_msg)
  275. else:
  276. raise_timeout_after_ignore = True
  277. i, a = self._future_to_actor.pop(future)
  278. self._return_actor(a)
  279. del self._index_to_future[i]
  280. self._next_return_index = max(self._next_return_index, i + 1)
  281. if raise_timeout_after_ignore:
  282. raise TimeoutError(
  283. timeout_msg + ". The task {} has been ignored.".format(future)
  284. )
  285. return ray.get(future)
  286. def _return_actor(self, actor):
  287. self._idle_actors.append(actor)
  288. if self._pending_submits:
  289. self.submit(*self._pending_submits.pop(0))
  290. def has_free(self):
  291. """Returns whether there are any idle actors available.
  292. Returns:
  293. True if there are any idle actors and no pending submits.
  294. Examples:
  295. .. testcode::
  296. import ray
  297. from ray.util.actor_pool import ActorPool
  298. @ray.remote
  299. class Actor:
  300. def double(self, v):
  301. return 2 * v
  302. a1 = Actor.remote()
  303. pool = ActorPool([a1])
  304. pool.submit(lambda a, v: a.double.remote(v), 1)
  305. print(pool.has_free())
  306. print(pool.get_next())
  307. print(pool.has_free())
  308. .. testoutput::
  309. False
  310. 2
  311. True
  312. """
  313. return len(self._idle_actors) > 0 and len(self._pending_submits) == 0
  314. def pop_idle(self):
  315. """Removes an idle actor from the pool.
  316. Returns:
  317. An idle actor if one is available.
  318. None if no actor was free to be removed.
  319. Examples:
  320. .. testcode::
  321. import ray
  322. from ray.util.actor_pool import ActorPool
  323. @ray.remote
  324. class Actor:
  325. def double(self, v):
  326. return 2 * v
  327. a1 = Actor.remote()
  328. pool = ActorPool([a1])
  329. pool.submit(lambda a, v: a.double.remote(v), 1)
  330. assert pool.pop_idle() is None
  331. assert pool.get_next() == 2
  332. assert pool.pop_idle() == a1
  333. """
  334. if self.has_free():
  335. return self._idle_actors.pop()
  336. return None
  337. def push(self, actor):
  338. """Pushes a new actor into the current list of idle actors.
  339. Examples:
  340. .. testcode::
  341. import ray
  342. from ray.util.actor_pool import ActorPool
  343. @ray.remote
  344. class Actor:
  345. def double(self, v):
  346. return 2 * v
  347. a1, a2 = Actor.remote(), Actor.remote()
  348. pool = ActorPool([a1])
  349. pool.push(a2)
  350. """
  351. busy_actors = []
  352. if self._future_to_actor.values():
  353. _, busy_actors = zip(*self._future_to_actor.values())
  354. if actor in self._idle_actors or actor in busy_actors:
  355. raise ValueError("Actor already belongs to current ActorPool")
  356. else:
  357. self._return_actor(actor)