From 1631abd0d7fc1f89a69246ec812cec9e287b976d Mon Sep 17 00:00:00 2001 From: Cindy Zhang Date: Mon, 1 May 2023 23:29:46 -0700 Subject: [PATCH 01/10] add health checking for http proxy actors Signed-off-by: Cindy Zhang --- python/ray/serve/_private/common.py | 10 ++++ python/ray/serve/_private/http_proxy.py | 15 ++++- python/ray/serve/_private/http_state.py | 80 ++++++++++++++++++++++++- python/ray/serve/controller.py | 3 + python/ray/serve/schema.py | 8 +++ 5 files changed, 113 insertions(+), 3 deletions(-) diff --git a/python/ray/serve/_private/common.py b/python/ray/serve/_private/common.py index 92b77468ac5f..a2912aea2b78 100644 --- a/python/ray/serve/_private/common.py +++ b/python/ray/serve/_private/common.py @@ -365,3 +365,13 @@ class ServeDeployMode(str, Enum): UNSET = "UNSET" SINGLE_APP = "SINGLE_APP" MULTI_APP = "MULTI_APP" + + +HTTP_PROXY_HEALTH_CHECK_CONCURRENCY_GROUP = "health_check" + + +class HTTPProxyStatus(str, Enum): + STARTING = "STARTING" + HEALTHY = "HEALTHY" + UNHEALTHY = "UNHEALTHY" + ACTOR_CRASHED = "ACTOR_CRASHED" diff --git a/python/ray/serve/_private/http_proxy.py b/python/ray/serve/_private/http_proxy.py index 987357bf1105..7dec11474724 100644 --- a/python/ray/serve/_private/http_proxy.py +++ b/python/ray/serve/_private/http_proxy.py @@ -25,7 +25,12 @@ Response, set_socket_reuse_port, ) -from ray.serve._private.common import EndpointInfo, EndpointTag, ApplicationName +from ray.serve._private.common import ( + EndpointInfo, + EndpointTag, + ApplicationName, + HTTP_PROXY_HEALTH_CHECK_CONCURRENCY_GROUP, +) from ray.serve._private.constants import ( SERVE_LOGGER_NAME, SERVE_NAMESPACE, @@ -459,7 +464,9 @@ async def __call__(self, scope, receive, send): ) -@ray.remote(num_cpus=0) +@ray.remote( + num_cpus=0, concurrency_groups={HTTP_PROXY_HEALTH_CHECK_CONCURRENCY_GROUP: 1} +) class HTTPProxyActor: def __init__( self, @@ -549,3 +556,7 @@ async def run(self): self.setup_complete.set() await server.serve(sockets=[sock]) + + @ray.method(concurrency_group=HTTP_PROXY_HEALTH_CHECK_CONCURRENCY_GROUP) + async def check_health(self): + pass diff --git a/python/ray/serve/_private/http_state.py b/python/ray/serve/_private/http_state.py index a31b0c95dd2a..ce0487681956 100644 --- a/python/ray/serve/_private/http_state.py +++ b/python/ray/serve/_private/http_state.py @@ -1,11 +1,13 @@ import asyncio import logging import random +import time from typing import Dict, List, Tuple import ray from ray.actor import ActorHandle from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy +from ray.exceptions import RayActorError, RayError from ray._raylet import GcsClient from ray.serve.config import HTTPOptions, DeploymentMode @@ -20,11 +22,77 @@ format_actor_name, get_all_node_ids, ) -from ray.serve._private.common import EndpointTag, NodeId +from ray.serve._private.common import EndpointTag, NodeId, HTTPProxyStatus +from ray.serve.schema import HTTPProxyDetails logger = logging.getLogger(SERVE_LOGGER_NAME) +class HTTPProxyState: + def __init__(self, actor_handle, actor_name): + self._actor_handle = actor_handle + self._actor_name = actor_name + + self._status = HTTPProxyStatus.STARTING + self._actor_started = False + self._health_check_obj_ref = self._actor_handle.check_health.remote() + self._last_health_check_time: float = time.time() + self._consecutive_health_check_failures = 0 + + @property + def status(self) -> HTTPProxyStatus: + return self._status + + def _check_health_obj_ref_result(self): + """Check on the result of the health check. + + Should only be called after confirming the object ref is ready. + Resets _health_check_obj_ref to None at the end. + """ + assert len(ray.wait([self._health_check_obj_ref], timeout=0)[0]) + try: + ray.get(self._health_check_obj_ref) + self._status = HTTPProxyStatus.HEALTHY + self._consecutive_health_check_failures = 0 + except RayActorError: + # Health check failed due to actor crashing. + self._status = HTTPProxyStatus.ACTOR_CRASHED + except RayError as e: + # Health check failed due to application-level exception. + logger.warning( + f"Health check for HTTP proxy {self._actor_name} failed: {e}" + ) + self._consecutive_health_check_failures += 1 + + self._health_check_obj_ref = None + + def update(self): + # Wait for first no-op health check to finish, indicating the actor has started + if not self._actor_started: + finished, _ = ray.wait([self._health_check_obj_ref], timeout=0) + if not finished: + return + self._check_health_obj_ref_result() + self._actor_started = True + + if self._health_check_obj_ref: + finished, _ = ray.wait([self._health_check_obj_ref], timeout=0) + if finished: + self._check_health_obj_ref_result() + if self._consecutive_health_check_failures > 3: + self._status = HTTPProxyStatus.UNHEALTHY + elif time.time() - self._last_health_check_time > 5: + self._status = HTTPProxyStatus.UNHEALTHY + logger.warning( + f"Health check for HTTP Proxy {self._actor_name} took more than 5 " + "seconds." + ) + self._health_check_obj_ref = None + elif time.time() - self._last_health_check_time > 10 * random.uniform(0.9, 1.1): + self._health_check_obj_ref = self._actor_handle.check_health.remote() + self._last_health_check_time = time.time() + + class HTTPState: """Manages all state for HTTP proxies in the system. @@ -50,6 +118,7 @@ def __init__( self._config = HTTPOptions() self._proxy_actors: Dict[NodeId, ActorHandle] = dict() self._proxy_actor_names: Dict[NodeId, str] = dict() + self._proxy_states: Dict[NodeId, HTTPProxyState] = dict() self._head_node_id: str = head_node_id self._gcs_client = gcs_client @@ -73,9 +142,17 @@ def get_http_proxy_handles(self) -> Dict[NodeId, ActorHandle]: def get_http_proxy_names(self) -> Dict[NodeId, str]: return self._proxy_actor_names + def get_http_proxy_details(self) -> Dict[NodeId, HTTPProxyDetails]: + return { + node_id: HTTPProxyDetails(status=state.status) + for node_id, state in self._proxy_states.items() + } + def update(self): self._start_proxies_if_needed() self._stop_proxies_if_needed() + for proxy_state in self._proxy_states.values(): + proxy_state.update() def _get_target_nodes(self) -> List[Tuple[str, str]]: """Return the list of (node_id, ip_address) to deploy HTTP servers on.""" @@ -155,6 +232,7 @@ def _start_proxies_if_needed(self) -> None: self._proxy_actors[node_id] = proxy self._proxy_actor_names[node_id] = name + self._proxy_states[node_id] = HTTPProxyState(proxy, name) def _stop_proxies_if_needed(self) -> bool: """Removes proxy actors from any nodes that no longer exist.""" diff --git a/python/ray/serve/controller.py b/python/ray/serve/controller.py index f6ef48102085..61d3e3bc04ef 100644 --- a/python/ray/serve/controller.py +++ b/python/ray/serve/controller.py @@ -706,6 +706,9 @@ def get_serve_instance_details(self) -> Dict: host=http_config.host, port=http_config.port, ), + http_proxy_details=self.http_state.get_http_proxy_details() + if self.http_state + else None, deploy_mode=self.deploy_mode, applications=applications, ).dict(exclude_unset=True) diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index ee61b8c5f0e2..feb8cd881104 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -11,6 +11,7 @@ DeploymentInfo, ReplicaState, ServeDeployMode, + HTTPProxyStatus, ) from ray.serve.config import DeploymentMode from ray.serve._private.utils import DEFAULT, dict_keys_snake_to_camel_case @@ -758,6 +759,10 @@ def get_status_dict(self) -> Dict: ) +class HTTPProxyDetails(BaseModel): + status: HTTPProxyStatus + + @PublicAPI(stability="alpha") class ServeInstanceDetails(BaseModel, extra=Extra.forbid): """ @@ -776,6 +781,9 @@ class ServeInstanceDetails(BaseModel, extra=Extra.forbid): ), ) http_options: Optional[HTTPOptionsSchema] = Field(description="HTTP Proxy options.") + http_proxy_details: Optional[Dict[str, HTTPProxyDetails]] = Field( + description="Info about HTTP Proxies." + ) deploy_mode: ServeDeployMode = Field( description=( "Whether a single-app config of format ServeApplicationSchema or multi-app " From 9f1af5d8f8d1cc7d1391c0a9c307505bf43b7031 Mon Sep 17 00:00:00 2001 From: Cindy Zhang Date: Thu, 4 May 2023 16:18:56 -0700 Subject: [PATCH 02/10] fix Signed-off-by: Cindy Zhang --- .../modules/serve/tests/test_serve_agent.py | 86 +++++++++++-------- python/ray/serve/_private/common.py | 4 - python/ray/serve/_private/http_proxy.py | 6 +- python/ray/serve/_private/http_state.py | 14 ++- 4 files changed, 56 insertions(+), 54 deletions(-) diff --git a/dashboard/modules/serve/tests/test_serve_agent.py b/dashboard/modules/serve/tests/test_serve_agent.py index 7d7d43c335f1..20fa855e6331 100644 --- a/dashboard/modules/serve/tests/test_serve_agent.py +++ b/dashboard/modules/serve/tests/test_serve_agent.py @@ -13,7 +13,12 @@ from ray.serve._private.constants import SERVE_NAMESPACE, MULTI_APP_MIGRATION_MESSAGE from ray.serve.tests.conftest import * # noqa: F401 F403 from ray.serve.schema import ServeInstanceDetails -from ray.serve._private.common import ApplicationStatus, DeploymentStatus, ReplicaState +from ray.serve._private.common import ( + ApplicationStatus, + DeploymentStatus, + ReplicaState, + HTTPProxyStatus, +) from ray.serve._private.constants import ( SERVE_DEFAULT_APP_NAME, DEPLOYMENT_NAME_PREFIX_SEPARATOR, @@ -466,7 +471,7 @@ def test_get_status(ray_start_stop): print("Serve app status is correct.") -@pytest.mark.skipif(sys.platform == "darwin", reason="Flaky on OSX.") +# @pytest.mark.skipif(sys.platform == "darwin", reason="Flaky on OSX.") @pytest.mark.parametrize( "f_deployment_options", [ @@ -493,17 +498,29 @@ def test_get_serve_instance_details(ray_start_stop, f_deployment_options): "applications": [ { "name": "app1", - "route_prefix": "/app1", + "route_prefix": "/apple", "import_path": world_import_path, "deployments": [f_deployment_options], }, { "name": "app2", - "route_prefix": "/app2", + "route_prefix": "/banana", "import_path": fastapi_import_path, }, ], } + expected_values = { + "app1": { + "route_prefix": "/apple", + "docs_path": None, + "deployments": {"app1_f", "app1_BasicDriver"}, + }, + "app2": { + "route_prefix": "/banana", + "docs_path": "/my_docs", + "deployments": {"app2_FastAPIDeployment"}, + } + } deploy_config_multi_app(config1) @@ -526,43 +543,38 @@ def applications_running(): assert serve_details.http_options.host == "127.0.0.1" assert serve_details.http_options.port == 8005 print("Confirmed fetched proxy location, host and port metadata correct.") + # Check HTTP Proxy statuses + assert all( + proxy.status == HTTPProxyStatus.HEALTHY + for proxy in serve_details.http_proxy_details.values() + ) + print("Confirmed HTTP Proxies are healthy.") app_details = serve_details.applications + # CHECK: application details + for i, app in enumerate(["app1", "app2"]): + # CHECK: app configs are equal + assert ( + app_details[app].deployed_app_config.dict(exclude_unset=True) + == config1["applications"][i] + ) + print(f"Checked the deployed app config for {app} from the fetched metadata.") - # CHECK: app configs are equal - assert ( - app_details["app1"].deployed_app_config.dict(exclude_unset=True) - == config1["applications"][0] - ) - assert ( - app_details["app2"].deployed_app_config.dict(exclude_unset=True) - == config1["applications"][1] - ) - print("Confirmed the deployed app configs from the fetched metadata is correct.") - - # CHECK: deployment timestamp - assert app_details["app1"].last_deployed_time_s > 0 - assert app_details["app2"].last_deployed_time_s > 0 - print("Confirmed deployment timestamps are nonzero.") - - # CHECK: docs path - assert app_details["app1"].docs_path is None - assert app_details["app2"].docs_path == "/my_docs" - print("Confirmed docs paths are correct.") - - # CHECK: all deployments are present - assert app_details["app1"].deployments.keys() == { - "app1_f", - "app1_BasicDriver", - } - assert app_details["app2"].deployments.keys() == { - "app2_FastAPIDeployment", - } - print("Metadata for all deployed deployments are present.") + # CHECK: deployment timestamp + assert app_details[app].last_deployed_time_s > 0 + print(f"Confirmed deployment timestamp for {app} is nonzero.") - # CHECK: application details - for app in ["app1", "app2"]: - assert app_details[app].route_prefix == f"/{app}" + # CHECK: route prefix + assert app_details[app].route_prefix == expected_values[app]["route_prefix"] + print(f"Checked route prefix for {app}.") + + # CHECK: docs path + assert app_details[app].docs_path == expected_values[app]["docs_path"] + print(f"Checked docs path for {app}.") + + # CHECK: all deployments are present + assert app_details[app].deployments.keys() == expected_values[app]["deployments"] + for deployment in app_details[app].deployments.values(): assert deployment.status == DeploymentStatus.HEALTHY # Route prefix should be app level options eventually diff --git a/python/ray/serve/_private/common.py b/python/ray/serve/_private/common.py index a2912aea2b78..e8d7a78c0b3c 100644 --- a/python/ray/serve/_private/common.py +++ b/python/ray/serve/_private/common.py @@ -367,11 +367,7 @@ class ServeDeployMode(str, Enum): MULTI_APP = "MULTI_APP" -HTTP_PROXY_HEALTH_CHECK_CONCURRENCY_GROUP = "health_check" - - class HTTPProxyStatus(str, Enum): STARTING = "STARTING" HEALTHY = "HEALTHY" UNHEALTHY = "UNHEALTHY" - ACTOR_CRASHED = "ACTOR_CRASHED" diff --git a/python/ray/serve/_private/http_proxy.py b/python/ray/serve/_private/http_proxy.py index 7dec11474724..96081f30bee7 100644 --- a/python/ray/serve/_private/http_proxy.py +++ b/python/ray/serve/_private/http_proxy.py @@ -29,7 +29,6 @@ EndpointInfo, EndpointTag, ApplicationName, - HTTP_PROXY_HEALTH_CHECK_CONCURRENCY_GROUP, ) from ray.serve._private.constants import ( SERVE_LOGGER_NAME, @@ -464,9 +463,7 @@ async def __call__(self, scope, receive, send): ) -@ray.remote( - num_cpus=0, concurrency_groups={HTTP_PROXY_HEALTH_CHECK_CONCURRENCY_GROUP: 1} -) +@ray.remote(num_cpus=0) class HTTPProxyActor: def __init__( self, @@ -557,6 +554,5 @@ async def run(self): self.setup_complete.set() await server.serve(sockets=[sock]) - @ray.method(concurrency_group=HTTP_PROXY_HEALTH_CHECK_CONCURRENCY_GROUP) async def check_health(self): pass diff --git a/python/ray/serve/_private/http_state.py b/python/ray/serve/_private/http_state.py index ce0487681956..980f706f054e 100644 --- a/python/ray/serve/_private/http_state.py +++ b/python/ray/serve/_private/http_state.py @@ -7,7 +7,6 @@ import ray from ray.actor import ActorHandle from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy -from ray.exceptions import RayActorError, RayError from ray._raylet import GcsClient from ray.serve.config import HTTPOptions, DeploymentMode @@ -29,7 +28,7 @@ class HTTPProxyState: - def __init__(self, actor_handle, actor_name): + def __init__(self, actor_handle: ActorHandle, actor_name: str): self._actor_handle = actor_handle self._actor_name = actor_name @@ -54,11 +53,7 @@ def _check_health_obj_ref_result(self): ray.get(self._health_check_obj_ref) self._status = HTTPProxyStatus.HEALTHY self._consecutive_health_check_failures = 0 - except RayActorError: - # Health check failed due to actor crashing. - self._status = HTTPProxyStatus.ACTOR_CRASHED - except RayError as e: - # Health check failed due to application-level exception. + except Exception as e: logger.warning( f"Health check for HTTP proxy {self._actor_name} failed: {e}" ) @@ -79,15 +74,18 @@ def update(self): finished, _ = ray.wait([self._health_check_obj_ref], timeout=0) if finished: self._check_health_obj_ref_result() + self._health_check_obj_ref = None if self._consecutive_health_check_failures > 3: self._status = HTTPProxyStatus.UNHEALTHY + # If the HTTP Proxy has been blocked for more than 5 seconds, mark unhealthy elif time.time() - self._last_health_check_time > 5: self._status = HTTPProxyStatus.UNHEALTHY logger.warning( f"Health check for HTTP Proxy {self._actor_name} took more than 5 " "seconds." ) - self._health_check_obj_ref = None + # If there's no active in-progress health check and it has been more than 10 + # seconds since the last health check, perform another health check elif time.time() - self._last_health_check_time > 10 * random.uniform(0.9, 1.1): self._health_check_obj_ref = self._actor_handle.check_health.remote() self._last_health_check_time = time.time() From c1671a7a86f3cd40f3c018fd1b65a35ba77f43cf Mon Sep 17 00:00:00 2001 From: Cindy Zhang Date: Thu, 4 May 2023 16:37:47 -0700 Subject: [PATCH 03/10] fix Signed-off-by: Cindy Zhang --- dashboard/modules/serve/tests/test_serve_agent.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/dashboard/modules/serve/tests/test_serve_agent.py b/dashboard/modules/serve/tests/test_serve_agent.py index 20fa855e6331..e586c418b708 100644 --- a/dashboard/modules/serve/tests/test_serve_agent.py +++ b/dashboard/modules/serve/tests/test_serve_agent.py @@ -471,7 +471,7 @@ def test_get_status(ray_start_stop): print("Serve app status is correct.") -# @pytest.mark.skipif(sys.platform == "darwin", reason="Flaky on OSX.") +@pytest.mark.skipif(sys.platform == "darwin", reason="Flaky on OSX.") @pytest.mark.parametrize( "f_deployment_options", [ @@ -519,7 +519,7 @@ def test_get_serve_instance_details(ray_start_stop, f_deployment_options): "route_prefix": "/banana", "docs_path": "/my_docs", "deployments": {"app2_FastAPIDeployment"}, - } + }, } deploy_config_multi_app(config1) @@ -573,8 +573,10 @@ def applications_running(): print(f"Checked docs path for {app}.") # CHECK: all deployments are present - assert app_details[app].deployments.keys() == expected_values[app]["deployments"] - + assert ( + app_details[app].deployments.keys() == expected_values[app]["deployments"] + ) + for deployment in app_details[app].deployments.values(): assert deployment.status == DeploymentStatus.HEALTHY # Route prefix should be app level options eventually From 9801a3edfe1d83a80a483a27bca57f75d01040ea Mon Sep 17 00:00:00 2001 From: Cindy Zhang Date: Thu, 4 May 2023 21:25:52 -0700 Subject: [PATCH 04/10] api annotation Signed-off-by: Cindy Zhang --- python/ray/serve/_private/http_proxy.py | 6 +----- python/ray/serve/schema.py | 1 + 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/python/ray/serve/_private/http_proxy.py b/python/ray/serve/_private/http_proxy.py index 96081f30bee7..1a23b9d273a0 100644 --- a/python/ray/serve/_private/http_proxy.py +++ b/python/ray/serve/_private/http_proxy.py @@ -25,11 +25,7 @@ Response, set_socket_reuse_port, ) -from ray.serve._private.common import ( - EndpointInfo, - EndpointTag, - ApplicationName, -) +from ray.serve._private.common import EndpointInfo, EndpointTag, ApplicationName from ray.serve._private.constants import ( SERVE_LOGGER_NAME, SERVE_NAMESPACE, diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index feb8cd881104..97271b33beba 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -759,6 +759,7 @@ def get_status_dict(self) -> Dict: ) +@PublicAPI(stability="alpha") class HTTPProxyDetails(BaseModel): status: HTTPProxyStatus From 7f3e0e6e722ecb1449b2e04879a81a7a3348dbd1 Mon Sep 17 00:00:00 2001 From: Cindy Zhang Date: Thu, 4 May 2023 21:40:19 -0700 Subject: [PATCH 05/10] fix Signed-off-by: Cindy Zhang --- python/ray/serve/_private/http_state.py | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/python/ray/serve/_private/http_state.py b/python/ray/serve/_private/http_state.py index 980f706f054e..92d2b090955c 100644 --- a/python/ray/serve/_private/http_state.py +++ b/python/ray/serve/_private/http_state.py @@ -36,7 +36,6 @@ def __init__(self, actor_handle: ActorHandle, actor_name: str): self._actor_started = False self._health_check_obj_ref = self._actor_handle.check_health.remote() self._last_health_check_time: float = time.time() - self._consecutive_health_check_failures = 0 @property def status(self) -> HTTPProxyStatus: @@ -52,12 +51,11 @@ def _check_health_obj_ref_result(self): try: ray.get(self._health_check_obj_ref) self._status = HTTPProxyStatus.HEALTHY - self._consecutive_health_check_failures = 0 except Exception as e: logger.warning( f"Health check for HTTP proxy {self._actor_name} failed: {e}" ) - self._consecutive_health_check_failures += 1 + self._status = HTTPProxyStatus.UNHEALTHY self._health_check_obj_ref = None @@ -74,9 +72,6 @@ def update(self): finished, _ = ray.wait([self._health_check_obj_ref], timeout=0) if finished: self._check_health_obj_ref_result() - self._health_check_obj_ref = None - if self._consecutive_health_check_failures > 3: - self._status = HTTPProxyStatus.UNHEALTHY # If the HTTP Proxy has been blocked for more than 5 seconds, mark unhealthy elif time.time() - self._last_health_check_time > 5: self._status = HTTPProxyStatus.UNHEALTHY From c3809a8f8b02f23339aeb1bd038f2e448c6f44e9 Mon Sep 17 00:00:00 2001 From: Cindy Zhang Date: Thu, 4 May 2023 21:52:34 -0700 Subject: [PATCH 06/10] improve Signed-off-by: Cindy Zhang --- python/ray/serve/_private/http_state.py | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/python/ray/serve/_private/http_state.py b/python/ray/serve/_private/http_state.py index 92d2b090955c..8d3e0053599a 100644 --- a/python/ray/serve/_private/http_state.py +++ b/python/ray/serve/_private/http_state.py @@ -27,6 +27,7 @@ logger = logging.getLogger(SERVE_LOGGER_NAME) +PROXY_HEALTH_CHECK_PERIOD_S = 10 class HTTPProxyState: def __init__(self, actor_handle: ActorHandle, actor_name: str): self._actor_handle = actor_handle @@ -68,20 +69,23 @@ def update(self): self._check_health_obj_ref_result() self._actor_started = True + randomized_period_s = PROXY_HEALTH_CHECK_PERIOD_S * random.uniform(0.9, 1.1) + if self._health_check_obj_ref: finished, _ = ray.wait([self._health_check_obj_ref], timeout=0) if finished: self._check_health_obj_ref_result() + # If there's no active in-progress health check and it has been more than 10 + # seconds since the last health check, perform another health check + if time.time() - self._last_health_check_time > randomized_period_s: # If the HTTP Proxy has been blocked for more than 5 seconds, mark unhealthy - elif time.time() - self._last_health_check_time > 5: + if time.time() - self._last_health_check_time > 5: self._status = HTTPProxyStatus.UNHEALTHY logger.warning( - f"Health check for HTTP Proxy {self._actor_name} took more than 5 " - "seconds." + f"Health check for HTTP Proxy {self._actor_name} took more than " + f"{PROXY_HEALTH_CHECK_PERIOD_S} seconds." ) - # If there's no active in-progress health check and it has been more than 10 - # seconds since the last health check, perform another health check - elif time.time() - self._last_health_check_time > 10 * random.uniform(0.9, 1.1): + self._health_check_obj_ref = self._actor_handle.check_health.remote() self._last_health_check_time = time.time() From ba485f29a302ca27f91c23c9101a741e57f3d231 Mon Sep 17 00:00:00 2001 From: Cindy Zhang Date: Thu, 4 May 2023 22:07:37 -0700 Subject: [PATCH 07/10] address comments Signed-off-by: Cindy Zhang --- python/ray/serve/_private/constants.py | 3 +++ python/ray/serve/_private/http_proxy.py | 3 +++ python/ray/serve/_private/http_state.py | 6 +++--- 3 files changed, 9 insertions(+), 3 deletions(-) diff --git a/python/ray/serve/_private/constants.py b/python/ray/serve/_private/constants.py index cf8e9a4fbf72..ea81d65e61ac 100644 --- a/python/ray/serve/_private/constants.py +++ b/python/ray/serve/_private/constants.py @@ -92,6 +92,9 @@ DEFAULT_HEALTH_CHECK_PERIOD_S = 10 DEFAULT_HEALTH_CHECK_TIMEOUT_S = 30 +# HTTP Proxy health check period +PROXY_HEALTH_CHECK_PERIOD_S = 10 + #: Number of times in a row that a replica must fail the health check before #: being marked unhealthy. REPLICA_HEALTH_CHECK_UNHEALTHY_THRESHOLD = 3 diff --git a/python/ray/serve/_private/http_proxy.py b/python/ray/serve/_private/http_proxy.py index 1a23b9d273a0..53a222748aec 100644 --- a/python/ray/serve/_private/http_proxy.py +++ b/python/ray/serve/_private/http_proxy.py @@ -551,4 +551,7 @@ async def run(self): await server.serve(sockets=[sock]) async def check_health(self): + """No-op method to check on the health of the HTTP Proxy. + Make sure the async event loop is not blocked. + """ pass diff --git a/python/ray/serve/_private/http_state.py b/python/ray/serve/_private/http_state.py index 8d3e0053599a..223b39aedc55 100644 --- a/python/ray/serve/_private/http_state.py +++ b/python/ray/serve/_private/http_state.py @@ -15,6 +15,7 @@ SERVE_LOGGER_NAME, SERVE_PROXY_NAME, SERVE_NAMESPACE, + PROXY_HEALTH_CHECK_PERIOD_S, ) from ray.serve._private.http_proxy import HTTPProxyActor from ray.serve._private.utils import ( @@ -27,7 +28,6 @@ logger = logging.getLogger(SERVE_LOGGER_NAME) -PROXY_HEALTH_CHECK_PERIOD_S = 10 class HTTPProxyState: def __init__(self, actor_handle: ActorHandle, actor_name: str): self._actor_handle = actor_handle @@ -78,8 +78,8 @@ def update(self): # If there's no active in-progress health check and it has been more than 10 # seconds since the last health check, perform another health check if time.time() - self._last_health_check_time > randomized_period_s: - # If the HTTP Proxy has been blocked for more than 5 seconds, mark unhealthy - if time.time() - self._last_health_check_time > 5: + # If the HTTP Proxy is still blocked, mark unhealthy + if self._health_check_obj_ref: self._status = HTTPProxyStatus.UNHEALTHY logger.warning( f"Health check for HTTP Proxy {self._actor_name} took more than " From 3443add8b88f6c782781102135f415025f5584ca Mon Sep 17 00:00:00 2001 From: Cindy Zhang Date: Sat, 6 May 2023 18:05:33 -0700 Subject: [PATCH 08/10] add tests + more info to http proxy details Signed-off-by: Cindy Zhang --- .../modules/serve/tests/test_serve_agent.py | 21 +--- python/ray/serve/_private/http_proxy.py | 8 +- python/ray/serve/_private/http_state.py | 119 +++++++++++------- python/ray/serve/_private/utils.py | 2 +- python/ray/serve/controller.py | 2 +- python/ray/serve/schema.py | 20 ++- python/ray/serve/tests/test_http_state.py | 67 +++++++++- 7 files changed, 170 insertions(+), 69 deletions(-) diff --git a/dashboard/modules/serve/tests/test_serve_agent.py b/dashboard/modules/serve/tests/test_serve_agent.py index e586c418b708..f1b46267b9ef 100644 --- a/dashboard/modules/serve/tests/test_serve_agent.py +++ b/dashboard/modules/serve/tests/test_serve_agent.py @@ -1,4 +1,5 @@ import copy +import os import sys from typing import Dict @@ -544,33 +545,21 @@ def applications_running(): assert serve_details.http_options.port == 8005 print("Confirmed fetched proxy location, host and port metadata correct.") # Check HTTP Proxy statuses - assert all( - proxy.status == HTTPProxyStatus.HEALTHY - for proxy in serve_details.http_proxy_details.values() - ) - print("Confirmed HTTP Proxies are healthy.") + for proxy in serve_details.http_proxies.values(): + assert proxy.status == HTTPProxyStatus.HEALTHY + assert os.path.exists("/tmp/ray/session_latest/logs" + proxy.log_file_path_id) + print("Checked HTTP Proxy details.") app_details = serve_details.applications # CHECK: application details for i, app in enumerate(["app1", "app2"]): - # CHECK: app configs are equal assert ( app_details[app].deployed_app_config.dict(exclude_unset=True) == config1["applications"][i] ) - print(f"Checked the deployed app config for {app} from the fetched metadata.") - - # CHECK: deployment timestamp assert app_details[app].last_deployed_time_s > 0 - print(f"Confirmed deployment timestamp for {app} is nonzero.") - - # CHECK: route prefix assert app_details[app].route_prefix == expected_values[app]["route_prefix"] - print(f"Checked route prefix for {app}.") - - # CHECK: docs path assert app_details[app].docs_path == expected_values[app]["docs_path"] - print(f"Checked docs path for {app}.") # CHECK: all deployments are present assert ( diff --git a/python/ray/serve/_private/http_proxy.py b/python/ray/serve/_private/http_proxy.py index 53a222748aec..f0b9707c8204 100644 --- a/python/ray/serve/_private/http_proxy.py +++ b/python/ray/serve/_private/http_proxy.py @@ -509,7 +509,13 @@ async def ready(self): return_when=asyncio.FIRST_COMPLETED, ) - # Return None, or re-throw the exception from self.running_task. + # Return metadata, or re-throw the exception from self.running_task. + if self.setup_complete.is_set(): + return ( + ray.get_runtime_context().get_actor_id(), + f"/serve/http_proxy_{ray.util.get_node_ip_address()}.log", + ) + return await done_set.pop() async def block_until_endpoint_exists( diff --git a/python/ray/serve/_private/http_state.py b/python/ray/serve/_private/http_state.py index 223b39aedc55..f4e6e39fc179 100644 --- a/python/ray/serve/_private/http_state.py +++ b/python/ray/serve/_private/http_state.py @@ -2,7 +2,7 @@ import logging import random import time -from typing import Dict, List, Tuple +from typing import Dict, List, Tuple, Optional import ray from ray.actor import ActorHandle @@ -29,54 +29,73 @@ class HTTPProxyState: - def __init__(self, actor_handle: ActorHandle, actor_name: str): + def __init__( + self, actor_handle: ActorHandle, actor_name: str, node_ip: str + ): self._actor_handle = actor_handle self._actor_name = actor_name + self._node_ip = node_ip + self._actor_id = None + self._log_file_path_id = None + self._ready_obj_ref = self._actor_handle.ready.remote() self._status = HTTPProxyStatus.STARTING - self._actor_started = False - self._health_check_obj_ref = self._actor_handle.check_health.remote() - self._last_health_check_time: float = time.time() + self._health_check_obj_ref = None + self._last_health_check_time: float = 0 + + @property + def node_ip(self) -> str: + return self._node_ip + + @property + def actor_handle(self) -> ActorHandle: + return self._actor_handle + + @property + def actor_name(self) -> str: + return self._actor_name @property def status(self) -> HTTPProxyStatus: return self._status - def _check_health_obj_ref_result(self): - """Check on the result of the health check. - - Should only be called after confirming the object ref is ready. - Resets _health_check_obj_ref to None at the end. - """ - assert len(ray.wait([self._health_check_obj_ref], timeout=0)[0]) - try: - ray.get(self._health_check_obj_ref) - self._status = HTTPProxyStatus.HEALTHY - except Exception as e: - logger.warning( - f"Health check for HTTP proxy {self._actor_name} failed: {e}" - ) - self._status = HTTPProxyStatus.UNHEALTHY + @property + def actor_id(self) -> Optional[str]: + return self._actor_id - self._health_check_obj_ref = None + @property + def log_file_path_id(self) -> Optional[str]: + return self._log_file_path_id def update(self): - # Wait for first no-op health check to finish, indicating the actor has started - if not self._actor_started: - finished, _ = ray.wait([self._health_check_obj_ref], timeout=0) - if not finished: - return - self._check_health_obj_ref_result() - self._actor_started = True - - randomized_period_s = PROXY_HEALTH_CHECK_PERIOD_S * random.uniform(0.9, 1.1) + if self._status == HTTPProxyStatus.STARTING: + try: + finished, _ = ray.wait([self._ready_obj_ref], timeout=0) + if finished: + self._actor_id, self._log_file_path_id = ray.get(finished[0]) + self._status = HTTPProxyStatus.HEALTHY + except Exception: + self._status = HTTPProxyStatus.UNHEALTHY + return + # Perform periodic health checks if self._health_check_obj_ref: finished, _ = ray.wait([self._health_check_obj_ref], timeout=0) if finished: - self._check_health_obj_ref_result() + try: + ray.get(finished[0]) + self._status = HTTPProxyStatus.HEALTHY + except Exception as e: + logger.warning( + f"Health check for HTTP proxy {self._actor_name} failed: {e}" + ) + self._status = HTTPProxyStatus.UNHEALTHY + + self._health_check_obj_ref = None + # If there's no active in-progress health check and it has been more than 10 # seconds since the last health check, perform another health check + randomized_period_s = PROXY_HEALTH_CHECK_PERIOD_S * random.uniform(0.9, 1.1) if time.time() - self._last_health_check_time > randomized_period_s: # If the HTTP Proxy is still blocked, mark unhealthy if self._health_check_obj_ref: @@ -113,8 +132,6 @@ def __init__( self._config = config else: self._config = HTTPOptions() - self._proxy_actors: Dict[NodeId, ActorHandle] = dict() - self._proxy_actor_names: Dict[NodeId, str] = dict() self._proxy_states: Dict[NodeId, HTTPProxyState] = dict() self._head_node_id: str = head_node_id @@ -134,14 +151,25 @@ def get_config(self): return self._config def get_http_proxy_handles(self) -> Dict[NodeId, ActorHandle]: - return self._proxy_actors + return { + node_id: state.actor_handle for node_id, state in self._proxy_states.items() + } def get_http_proxy_names(self) -> Dict[NodeId, str]: - return self._proxy_actor_names + return { + node_id: state.actor_name for node_id, state in self._proxy_states.items() + } def get_http_proxy_details(self) -> Dict[NodeId, HTTPProxyDetails]: return { - node_id: HTTPProxyDetails(status=state.status) + node_id: HTTPProxyDetails( + node_id=node_id, + node_ip=state.node_ip, + actor_id=state.actor_id, + actor_name=state.actor_name, + status=state.status, + log_file_path_id=state.log_file_path_id, + ) for node_id, state in self._proxy_states.items() } @@ -193,7 +221,7 @@ def _start_proxies_if_needed(self) -> None: """Start a proxy on every node if it doesn't already exist.""" for node_id, node_ip_address in self._get_target_nodes(): - if node_id in self._proxy_actors: + if node_id in self._proxy_states: continue name = format_actor_name(SERVE_PROXY_NAME, self._controller_name, node_id) @@ -227,23 +255,20 @@ def _start_proxies_if_needed(self) -> None: http_middlewares=self._config.middlewares, ) - self._proxy_actors[node_id] = proxy - self._proxy_actor_names[node_id] = name - self._proxy_states[node_id] = HTTPProxyState(proxy, name) + self._proxy_states[node_id] = HTTPProxyState(proxy, name, node_ip_address) def _stop_proxies_if_needed(self) -> bool: """Removes proxy actors from any nodes that no longer exist.""" all_node_ids = {node_id for node_id, _ in get_all_node_ids(self._gcs_client)} to_stop = [] - for node_id in self._proxy_actors: + for node_id in self._proxy_states: if node_id not in all_node_ids: logger.info("Removing HTTP proxy on removed node '{}'.".format(node_id)) to_stop.append(node_id) for node_id in to_stop: - proxy = self._proxy_actors.pop(node_id) - del self._proxy_actor_names[node_id] - ray.kill(proxy, no_restart=True) + proxy = self._proxy_states.pop(node_id) + ray.kill(proxy.actor_handle, no_restart=True) async def ensure_http_route_exists(self, endpoint: EndpointTag, timeout_s: float): """Block until the route has been propagated to all HTTP proxies. @@ -252,7 +277,9 @@ async def ensure_http_route_exists(self, endpoint: EndpointTag, timeout_s: float """ await asyncio.gather( *[ - proxy.block_until_endpoint_exists.remote(endpoint, timeout_s=timeout_s) - for proxy in self._proxy_actors.values() + proxy.actor_handle.block_until_endpoint_exists.remote( + endpoint, timeout_s=timeout_s + ) + for proxy in self._proxy_states.values() ] ) diff --git a/python/ray/serve/_private/utils.py b/python/ray/serve/_private/utils.py index d15d9f7f4be2..456f57e8ae28 100644 --- a/python/ray/serve/_private/utils.py +++ b/python/ray/serve/_private/utils.py @@ -174,7 +174,7 @@ def get_all_node_ids(gcs_client) -> List[Tuple[str, str]]: """ nodes = gcs_client.get_all_node_info(timeout=RAY_GCS_RPC_TIMEOUT_S) node_ids = [ - (ray.NodeID.from_binary(node_id).hex(), node["node_name"]) + (ray.NodeID.from_binary(node_id).hex(), node["node_name"].decode("utf-8")) for (node_id, node) in nodes.items() if node["state"] == ray.core.generated.gcs_pb2.GcsNodeInfo.ALIVE ] diff --git a/python/ray/serve/controller.py b/python/ray/serve/controller.py index 61d3e3bc04ef..e27159d412cd 100644 --- a/python/ray/serve/controller.py +++ b/python/ray/serve/controller.py @@ -706,7 +706,7 @@ def get_serve_instance_details(self) -> Dict: host=http_config.host, port=http_config.port, ), - http_proxy_details=self.http_state.get_http_proxy_details() + http_proxies=self.http_state.get_http_proxy_details() if self.http_state else None, deploy_mode=self.deploy_mode, diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index 1bc0a3e538ed..b862dcfef0ff 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -761,7 +761,19 @@ def get_status_dict(self) -> Dict: @PublicAPI(stability="alpha") class HTTPProxyDetails(BaseModel): - status: HTTPProxyStatus + node_id: str = Field(description="ID of the node that the HTTP Proxy is running on") + node_ip: str = Field( + description="IP address of the node that the HTTP Proxy is running on." + ) + actor_id: str = Field(description="ID of the HTTP Proxy actor.") + actor_name: str = Field(description="Name of the HTTP Proxy actor.") + status: HTTPProxyStatus = Field(description="Current status of the HTTP Proxy.") + log_file_path_id: Optional[str] = Field( + description=( + "Path identifier for log file of the HTTP Proxy actor. This is the " + "relative path to the log file from the ray logs directory." + ) + ) @PublicAPI(stability="alpha") @@ -782,8 +794,10 @@ class ServeInstanceDetails(BaseModel, extra=Extra.forbid): ), ) http_options: Optional[HTTPOptionsSchema] = Field(description="HTTP Proxy options.") - http_proxy_details: Optional[Dict[str, HTTPProxyDetails]] = Field( - description="Info about HTTP Proxies." + http_proxies: Optional[Dict[str, HTTPProxyDetails]] = Field( + description=( + "Mapping from node_id to details about the HTTP Proxy running on that node." + ) ) deploy_mode: ServeDeployMode = Field( description=( diff --git a/python/ray/serve/tests/test_http_state.py b/python/ray/serve/tests/test_http_state.py index 957a79f95d1b..ef44c0c1d2d9 100644 --- a/python/ray/serve/tests/test_http_state.py +++ b/python/ray/serve/tests/test_http_state.py @@ -1,9 +1,13 @@ +from functools import partial from unittest.mock import patch import pytest +import ray +from ray._private.test_utils import SignalActor, wait_for_condition from ray.serve.config import DeploymentMode, HTTPOptions -from ray.serve._private.http_state import HTTPState +from ray.serve._private.common import HTTPProxyStatus +from ray.serve._private.http_state import HTTPState, HTTPProxyState def test_node_selection(): @@ -64,6 +68,67 @@ def _make_http_state(http_options): assert set(another_seed) != set(selected_nodes) +def test_http_proxy_healthy(): + ray.init() + signal = SignalActor.remote() + + @ray.remote(num_cpus=0) + class MockHTTPProxyActor: + async def ready(self): + await signal.wait.remote() + return "mock_actor_id", "mock_log_file_path_id" + + async def check_health(self): + pass + + proxy = MockHTTPProxyActor.options(lifetime="detached").remote() + state = HTTPProxyState(proxy, "alice", "mock_node_ip") + assert state.status == HTTPProxyStatus.STARTING + + state.update() + assert state.status == HTTPProxyStatus.STARTING + + signal.send.remote() + wait_for_condition( + lambda: state.update() or state.status == HTTPProxyStatus.HEALTHY, timeout=2 + ) + ray.shutdown() + + +def test_http_proxy_unhealthy(): + ray.init() + signal = SignalActor.remote() + + @ray.remote(num_cpus=0) + class MockHTTPProxyActor: + async def ready(self): + return "mock_actor_id", "mock_log_file_path_id" + + async def check_health(self): + await signal.wait.remote() + + with patch("ray.serve._private.http_state.PROXY_HEALTH_CHECK_PERIOD_S", 1): + proxy = MockHTTPProxyActor.options(lifetime="detached").remote() + state = HTTPProxyState(proxy, "alice", "mock_node_ip") + assert state.status == HTTPProxyStatus.STARTING + + def check_proxy(status): + state.update() + return state.status == status + + # Proxy actor is ready, so status should transition STARTING -> HEALTHY + wait_for_condition(partial(check_proxy, HTTPProxyStatus.HEALTHY), timeout=2) + + # Health check is blocked, so status should transition HEALTHY -> UNHEALTHY + wait_for_condition(partial(check_proxy, HTTPProxyStatus.UNHEALTHY), timeout=2) + + # Unblock health check, so status should transition UNHEALTHY -> HEALTHY + signal.send.remote() + wait_for_condition(partial(check_proxy, HTTPProxyStatus.HEALTHY), timeout=2) + + ray.shutdown() + + if __name__ == "__main__": import sys From 74cd0c2b707e4df1141c9f7ad958fbafcc01db9f Mon Sep 17 00:00:00 2001 From: Cindy Zhang Date: Mon, 8 May 2023 11:50:32 -0700 Subject: [PATCH 09/10] fix Signed-off-by: Cindy Zhang --- python/ray/serve/_private/http_proxy.py | 7 ++----- python/ray/serve/_private/http_state.py | 8 +++----- 2 files changed, 5 insertions(+), 10 deletions(-) diff --git a/python/ray/serve/_private/http_proxy.py b/python/ray/serve/_private/http_proxy.py index f0b9707c8204..51d00861030d 100644 --- a/python/ray/serve/_private/http_proxy.py +++ b/python/ray/serve/_private/http_proxy.py @@ -509,12 +509,9 @@ async def ready(self): return_when=asyncio.FIRST_COMPLETED, ) - # Return metadata, or re-throw the exception from self.running_task. + # Return log filepath, or re-throw the exception from self.running_task. if self.setup_complete.is_set(): - return ( - ray.get_runtime_context().get_actor_id(), - f"/serve/http_proxy_{ray.util.get_node_ip_address()}.log", - ) + return f"/serve/http_proxy_{ray.util.get_node_ip_address()}.log" return await done_set.pop() diff --git a/python/ray/serve/_private/http_state.py b/python/ray/serve/_private/http_state.py index f4e6e39fc179..66161e3b3b46 100644 --- a/python/ray/serve/_private/http_state.py +++ b/python/ray/serve/_private/http_state.py @@ -29,9 +29,7 @@ class HTTPProxyState: - def __init__( - self, actor_handle: ActorHandle, actor_name: str, node_ip: str - ): + def __init__(self, actor_handle: ActorHandle, actor_name: str, node_ip: str): self._actor_handle = actor_handle self._actor_name = actor_name self._node_ip = node_ip @@ -61,7 +59,7 @@ def status(self) -> HTTPProxyStatus: @property def actor_id(self) -> Optional[str]: - return self._actor_id + return self._actor_handle._actor_id.hex() @property def log_file_path_id(self) -> Optional[str]: @@ -72,7 +70,7 @@ def update(self): try: finished, _ = ray.wait([self._ready_obj_ref], timeout=0) if finished: - self._actor_id, self._log_file_path_id = ray.get(finished[0]) + self._log_file_path_id = ray.get(finished[0]) self._status = HTTPProxyStatus.HEALTHY except Exception: self._status = HTTPProxyStatus.UNHEALTHY From d463fd8499471a2e5729bec7ccde428b38154ac3 Mon Sep 17 00:00:00 2001 From: Cindy Zhang Date: Mon, 8 May 2023 12:20:36 -0700 Subject: [PATCH 10/10] fix Signed-off-by: Cindy Zhang --- dashboard/modules/serve/tests/test_serve_agent.py | 2 +- python/ray/serve/_private/http_state.py | 10 +++++----- python/ray/serve/schema.py | 6 +++--- python/ray/serve/tests/test_http_state.py | 4 ++-- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/dashboard/modules/serve/tests/test_serve_agent.py b/dashboard/modules/serve/tests/test_serve_agent.py index f1b46267b9ef..03b1f63faf7d 100644 --- a/dashboard/modules/serve/tests/test_serve_agent.py +++ b/dashboard/modules/serve/tests/test_serve_agent.py @@ -547,7 +547,7 @@ def applications_running(): # Check HTTP Proxy statuses for proxy in serve_details.http_proxies.values(): assert proxy.status == HTTPProxyStatus.HEALTHY - assert os.path.exists("/tmp/ray/session_latest/logs" + proxy.log_file_path_id) + assert os.path.exists("/tmp/ray/session_latest/logs" + proxy.log_file_path) print("Checked HTTP Proxy details.") app_details = serve_details.applications diff --git a/python/ray/serve/_private/http_state.py b/python/ray/serve/_private/http_state.py index 66161e3b3b46..da8ed7ba620a 100644 --- a/python/ray/serve/_private/http_state.py +++ b/python/ray/serve/_private/http_state.py @@ -34,7 +34,7 @@ def __init__(self, actor_handle: ActorHandle, actor_name: str, node_ip: str): self._actor_name = actor_name self._node_ip = node_ip self._actor_id = None - self._log_file_path_id = None + self._log_file_path = None self._ready_obj_ref = self._actor_handle.ready.remote() self._status = HTTPProxyStatus.STARTING @@ -62,15 +62,15 @@ def actor_id(self) -> Optional[str]: return self._actor_handle._actor_id.hex() @property - def log_file_path_id(self) -> Optional[str]: - return self._log_file_path_id + def log_file_path(self) -> Optional[str]: + return self._log_file_path def update(self): if self._status == HTTPProxyStatus.STARTING: try: finished, _ = ray.wait([self._ready_obj_ref], timeout=0) if finished: - self._log_file_path_id = ray.get(finished[0]) + self._log_file_path = ray.get(finished[0]) self._status = HTTPProxyStatus.HEALTHY except Exception: self._status = HTTPProxyStatus.UNHEALTHY @@ -166,7 +166,7 @@ def get_http_proxy_details(self) -> Dict[NodeId, HTTPProxyDetails]: actor_id=state.actor_id, actor_name=state.actor_name, status=state.status, - log_file_path_id=state.log_file_path_id, + log_file_path=state.log_file_path, ) for node_id, state in self._proxy_states.items() } diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index b862dcfef0ff..5dd2612e783b 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -768,10 +768,10 @@ class HTTPProxyDetails(BaseModel): actor_id: str = Field(description="ID of the HTTP Proxy actor.") actor_name: str = Field(description="Name of the HTTP Proxy actor.") status: HTTPProxyStatus = Field(description="Current status of the HTTP Proxy.") - log_file_path_id: Optional[str] = Field( + log_file_path: Optional[str] = Field( description=( - "Path identifier for log file of the HTTP Proxy actor. This is the " - "relative path to the log file from the ray logs directory." + "The relative path to the log file for the replica actor from the ray logs " + "directory." ) ) diff --git a/python/ray/serve/tests/test_http_state.py b/python/ray/serve/tests/test_http_state.py index ef44c0c1d2d9..83e2c14bb037 100644 --- a/python/ray/serve/tests/test_http_state.py +++ b/python/ray/serve/tests/test_http_state.py @@ -76,7 +76,7 @@ def test_http_proxy_healthy(): class MockHTTPProxyActor: async def ready(self): await signal.wait.remote() - return "mock_actor_id", "mock_log_file_path_id" + return "mock_actor_id", "mock_log_file_path" async def check_health(self): pass @@ -102,7 +102,7 @@ def test_http_proxy_unhealthy(): @ray.remote(num_cpus=0) class MockHTTPProxyActor: async def ready(self): - return "mock_actor_id", "mock_log_file_path_id" + return "mock_actor_id", "mock_log_file_path" async def check_health(self): await signal.wait.remote()