From f9ab43954e0f6ee5ab40db26e3d7bcf47a92a0a6 Mon Sep 17 00:00:00 2001 From: Jialing He Date: Sat, 1 Jun 2024 00:18:37 +0800 Subject: [PATCH] [Core] Fix the GIL deadlock issue caused by `list_named_actors`. (#45582) Signed-off-by: hejialing.hjl --- python/ray/_raylet.pyx | 5 +++-- python/ray/tests/test_list_actors_4.py | 31 ++++++++++++++++++++++++++ 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/python/ray/_raylet.pyx b/python/ray/_raylet.pyx index 8437edaa01228..407db534b0138 100644 --- a/python/ray/_raylet.pyx +++ b/python/ray/_raylet.pyx @@ -4446,8 +4446,9 @@ cdef class CoreWorker: cdef: pair[c_vector[pair[c_string, c_string]], CRayStatus] result_pair - result_pair = CCoreWorkerProcess.GetCoreWorker().ListNamedActors( - all_namespaces) + with nogil: + result_pair = CCoreWorkerProcess.GetCoreWorker().ListNamedActors( + all_namespaces) check_status(result_pair.second) return [ (namespace.decode("utf-8"), diff --git a/python/ray/tests/test_list_actors_4.py b/python/ray/tests/test_list_actors_4.py index 0f302e4280849..4bbbc819338de 100644 --- a/python/ray/tests/test_list_actors_4.py +++ b/python/ray/tests/test_list_actors_4.py @@ -1,5 +1,7 @@ +import asyncio import pytest import sys +import time import ray from ray._private.test_utils import run_string_as_driver @@ -52,6 +54,35 @@ class A: assert not ray.util.list_named_actors(all_namespaces=True) +@pytest.mark.asyncio +async def test_list_named_actors_with_normal_task(shutdown_only): + # The following parameters are all designed to increase the + # probability of reproducing the situation where + # `list_named_actors` gets hang. + # https://github.com/ray-project/ray/issues/45581 for more details. + TEST_RANGE = 10 + NORMAL_TASK_PER_ITEM = 100 + LIST_NAMED_ACTORS_PER_ITEM = 10 + for _ in range(TEST_RANGE): + time.sleep(1) + + @ray.remote + def test(): + return True + + res = [] + for i in range(NORMAL_TASK_PER_ITEM): + res.append(test.remote()) + + async def run(): + for i in range(LIST_NAMED_ACTORS_PER_ITEM): + await asyncio.sleep(0) + ray.util.list_named_actors(True) + + res.append(run()) + await asyncio.gather(*res) + + if __name__ == "__main__": import os