PoolActor now uses num_cpus=0 to avoid any deadlock (#22048)

https://github.com/ray-project/ray/issues/21488#issuecomment-1027122177 :

> We discussed this issue in a bit more detail and came to the conclusion that we should set the CPU resource requirement for each actor in the actor pool to 0, to make the Ray Pool compatible/same behavior as the Python multiprocessing pool. Would that work for you @yogeveran ? (very similar to solution 4 mentioned above, but with 0.0 instead of 0.1, so it works in all cases).
This commit is contained in:
Gagandeep Singh 2022-02-08 15:29:46 +05:30 committed by GitHub
parent 1c41b0f566
commit 0f2a2224c2
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
2 changed files with 23 additions and 1 deletions

View file

@ -6,6 +6,7 @@ import time
import random
from collections import defaultdict
import queue
import math
import ray
from ray._private.test_utils import SignalActor
@ -46,6 +47,14 @@ def pool_4_processes_python_multiprocessing_lib():
pool.join()
@pytest.fixture
def ray_start_1_cpu():
address_info = ray.init(num_cpus=1)
yield address_info
# The code after the yield will run as teardown code.
ray.shutdown()
def test_ray_init(shutdown_only):
def getpid(args):
return os.getpid()
@ -562,6 +571,19 @@ def test_maxtasksperchild(shutdown_only):
pool.join()
def test_deadlock_avoidance_in_recursive_tasks(ray_start_1_cpu):
def poolit_a(_):
with Pool(ray_address="auto") as pool:
return list(pool.map(math.sqrt, range(0, 2, 1)))
def poolit_b():
with Pool(ray_address="auto") as pool:
return list(pool.map(poolit_a, range(2, 4, 1)))
result = poolit_b()
assert result == [[0.0, 1.0], [0.0, 1.0]]
if __name__ == "__main__":
import pytest

View file

@ -435,7 +435,7 @@ class UnorderedIMapIterator(IMapIterator):
return self._ready_objects.popleft()
@ray.remote(num_cpus=1)
@ray.remote(num_cpus=0)
class PoolActor:
"""Actor used to process tasks submitted to a Pool."""