mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
[Core] Filter out dead nodes when getting address info from redis (#14440)
This commit is contained in:
parent
8e139046b9
commit
7977474899
2 changed files with 30 additions and 1 deletions
|
@ -264,6 +264,8 @@ def get_address_info_from_redis_helper(redis_address,
|
|||
|
||||
relevant_client = None
|
||||
for client_info in client_table:
|
||||
if not client_info["Alive"]:
|
||||
continue
|
||||
client_node_ip_address = client_info["NodeManagerAddress"]
|
||||
if (client_node_ip_address == node_ip_address
|
||||
or (client_node_ip_address == "127.0.0.1"
|
||||
|
|
|
@ -8,7 +8,11 @@ import pytest
|
|||
import ray
|
||||
import ray.ray_constants as ray_constants
|
||||
from ray.cluster_utils import Cluster
|
||||
from ray.test_utils import RayTestTimeoutException, get_other_nodes
|
||||
from ray.test_utils import (
|
||||
RayTestTimeoutException,
|
||||
get_other_nodes,
|
||||
wait_for_condition,
|
||||
)
|
||||
|
||||
SIGKILL = signal.SIGKILL if sys.platform != "win32" else signal.SIGTERM
|
||||
|
||||
|
@ -157,6 +161,29 @@ def test_raylet_failed(ray_start_cluster):
|
|||
True)
|
||||
|
||||
|
||||
def test_get_address_info_after_raylet_died(ray_start_cluster_head):
|
||||
cluster = ray_start_cluster_head
|
||||
|
||||
def get_address_info():
|
||||
return ray._private.services.get_address_info_from_redis(
|
||||
cluster.redis_address,
|
||||
cluster.head_node.node_ip_address,
|
||||
num_retries=1,
|
||||
redis_password=cluster.redis_password)
|
||||
|
||||
assert get_address_info()[
|
||||
"raylet_socket_name"] == cluster.head_node.raylet_socket_name
|
||||
|
||||
cluster.head_node.kill_raylet()
|
||||
wait_for_condition(
|
||||
lambda: not cluster.global_state.node_table()[0]["Alive"])
|
||||
with pytest.raises(RuntimeError):
|
||||
get_address_info()
|
||||
|
||||
node2 = cluster.add_node()
|
||||
assert get_address_info()["raylet_socket_name"] == node2.raylet_socket_name
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
import pytest
|
||||
sys.exit(pytest.main(["-v", __file__]))
|
||||
|
|
Loading…
Add table
Reference in a new issue