Skip to content

Commit 35d472a

Browse files
authored
[serve] don't stop retrying replicas when a deployment is scaling back up from zero (#51600)
## Why are these changes needed? Previously we added a new deployment status `DEPLOY_FAILED`, which a deployment transitions into if the replicas repeatedly fail to start after a new config update / new deployment. After a certain number of retries, the controller will stop retrying replicas for that deployment and it enters a terminal state. However that causes deployments whose replicas fail during autoscaling (from zero) to also stop retrying after the threshold. This PR fixes that. Summary of failure scenarios and how we now handle them: 1. A deployment is first deployed / re-deployed. If the number of replica failures exceed a threshold (`3 * target`), and not a single replica was able to start successfully, the deployment transitions to `DEPLOY_FAILED` and no more replicas are retried. This state is terminal, and user must redeploy. 2. An autoscaling deployment is deployed with `initial_replicas = 0`. A request is sent and replicas fail to start. Similarly if the number of replica failures exceed a threshold (`3 * target`), and not a single replica was able to start successfully, no more replicas are retried. The deployment transitions to `UNHEALTHY`. 3. An autoscaling deployment is deployed and replicas start successfully. It later scales down and tries to scale back up again, but there are a lot of replica failures. Here the deployment could transition to `UNHEALTHY` if there are enough replica failures, but replicas will _continue_ to retry. ## Related issue number closes #50710 --------- Signed-off-by: Cindy Zhang <cindyzyx9@gmail.com>
1 parent 9c44f7f commit 35d472a

File tree

2 files changed

+285
-91
lines changed

2 files changed

+285
-91
lines changed

python/ray/serve/_private/deployment_state.py

Lines changed: 72 additions & 53 deletions
Original file line numberDiff line numberDiff line change
@@ -1270,8 +1270,14 @@ def __init__(
12701270
self._target_state: DeploymentTargetState = DeploymentTargetState.default()
12711271

12721272
self._prev_startup_warning: float = time.time()
1273-
self._replica_constructor_retry_counter: int = 0
12741273
self._replica_constructor_error_msg: Optional[str] = None
1274+
# Counter for how many times replicas failed to start. This is reset to 0 when:
1275+
# (1) The deployment is deployed / re-deployed.
1276+
# (2) The deployment reaches the HEALTHY state.
1277+
self._replica_constructor_retry_counter: int = 0
1278+
# Flag for whether any replicas of the target version has successfully started.
1279+
# This is reset to False when the deployment is re-deployed.
1280+
self._replica_has_started: bool = False
12751281

12761282
self._replicas: ReplicaStateContainer = ReplicaStateContainer()
12771283
self._curr_status_info: DeploymentStatusInfo = DeploymentStatusInfo(
@@ -1393,10 +1399,24 @@ def _failed_to_start_threshold(self) -> int:
13931399
self._target_state.target_num_replicas * MAX_PER_REPLICA_RETRY_COUNT,
13941400
)
13951401

1396-
@property
1397-
def is_failed(self) -> bool:
1398-
"""Whether the deployment failed to deploy."""
1399-
return self._curr_status_info.status == DeploymentStatus.DEPLOY_FAILED
1402+
def _replica_startup_failing(self) -> bool:
1403+
"""Check whether replicas are currently failing and the number of
1404+
failures has exceeded a threshold.
1405+
"""
1406+
return (
1407+
self._target_state.target_num_replicas > 0
1408+
and self._replica_constructor_retry_counter
1409+
>= self._failed_to_start_threshold
1410+
)
1411+
1412+
def _terminally_failed(self) -> bool:
1413+
"""Check whether the current version is terminally errored.
1414+
1415+
The version is considered terminally errored if the number of
1416+
replica failures has exceeded a threshold, and there hasn't been
1417+
any replicas of the target version that has successfully started.
1418+
"""
1419+
return not self._replica_has_started and self._replica_startup_failing()
14001420

14011421
def get_alive_replica_actor_ids(self) -> Set[str]:
14021422
return {replica.actor_id for replica in self._replicas.get()}
@@ -1454,7 +1474,7 @@ def broadcast_running_replicas_if_changed(self) -> None:
14541474
multiplexed model IDs.
14551475
"""
14561476
running_replica_infos = self.get_running_replica_infos()
1457-
is_available = not self.is_failed
1477+
is_available = not self._terminally_failed()
14581478

14591479
running_replicas_changed = (
14601480
set(self._last_broadcasted_running_replica_infos)
@@ -1637,6 +1657,7 @@ def deploy(self, deployment_info: DeploymentInfo) -> bool:
16371657
f"(initial target replicas: {target_num_replicas})."
16381658
)
16391659
self._replica_constructor_retry_counter = 0
1660+
self._replica_has_started = False
16401661
return True
16411662

16421663
def autoscale(self) -> int:
@@ -1860,11 +1881,7 @@ def scale_deployment_replicas(
18601881
stopping_replicas = self._replicas.count(states=[ReplicaState.STOPPING])
18611882
to_add = max(delta_replicas - stopping_replicas, 0)
18621883

1863-
if (
1864-
to_add > 0
1865-
and self._replica_constructor_retry_counter
1866-
< self._failed_to_start_threshold
1867-
):
1884+
if to_add > 0 and not self._terminally_failed():
18681885
logger.info(f"Adding {to_add} replica{'s' * (to_add>1)} to {self._id}.")
18691886
for _ in range(to_add):
18701887
replica_id = ReplicaID(get_random_string(), deployment_id=self._id)
@@ -1913,34 +1930,29 @@ def check_curr_status(self) -> Tuple[bool, bool]:
19131930
states=[ReplicaState.RUNNING], version=target_version
19141931
)
19151932

1916-
failed_to_start_count = self._replica_constructor_retry_counter
1917-
19181933
# Got to make a call to complete current deploy() goal after
19191934
# start failure threshold reached, while we might still have
19201935
# pending replicas in current goal.
1921-
if (
1922-
failed_to_start_count >= self._failed_to_start_threshold
1923-
and self._failed_to_start_threshold != 0
1924-
):
1925-
if running_at_target_version_replica_cnt > 0:
1926-
# At least one RUNNING replica at target state, partial
1927-
# success; We can stop tracking constructor failures and
1928-
# leave it to the controller to fully scale to target
1929-
# number of replicas and only return as completed once
1930-
# reached target replica count
1931-
self._replica_constructor_retry_counter = -1
1932-
else:
1933-
self._curr_status_info = self._curr_status_info.handle_transition(
1934-
trigger=DeploymentStatusInternalTrigger.REPLICA_STARTUP_FAILED,
1935-
message=(
1936-
f"The deployment failed to start {failed_to_start_count} times "
1937-
"in a row. This may be due to a problem with its "
1938-
"constructor or initial health check failing. See "
1939-
"controller logs for details. Error:\n"
1940-
f"{self._replica_constructor_error_msg}"
1941-
),
1942-
)
1943-
return False, any_replicas_recovering
1936+
if running_at_target_version_replica_cnt > 0:
1937+
# At least one RUNNING replica at target state, partial
1938+
# success; We can stop tracking constructor failures and
1939+
# leave it to the controller to fully scale to target
1940+
# number of replicas and only return as completed once
1941+
# reached target replica count
1942+
self._replica_has_started = True
1943+
elif self._replica_startup_failing():
1944+
self._curr_status_info = self._curr_status_info.handle_transition(
1945+
trigger=DeploymentStatusInternalTrigger.REPLICA_STARTUP_FAILED,
1946+
message=(
1947+
"The deployment failed to start "
1948+
f"{self._replica_constructor_retry_counter} times "
1949+
"in a row. This may be due to a problem with its "
1950+
"constructor or initial health check failing. See "
1951+
"controller logs for details. Error:\n"
1952+
f"{self._replica_constructor_error_msg}"
1953+
),
1954+
)
1955+
return False, any_replicas_recovering
19441956

19451957
# If we have pending ops, the current goal is *not* ready.
19461958
if (
@@ -1966,6 +1978,7 @@ def check_curr_status(self) -> Tuple[bool, bool]:
19661978
self._curr_status_info = self._curr_status_info.handle_transition(
19671979
trigger=DeploymentStatusInternalTrigger.HEALTHY
19681980
)
1981+
self._replica_constructor_retry_counter = 0
19691982
return False, any_replicas_recovering
19701983

19711984
return False, any_replicas_recovering
@@ -2033,25 +2046,31 @@ def _check_startup_replicas(
20332046
def record_replica_startup_failure(self, error_msg: str):
20342047
"""Record that a replica failed to start."""
20352048

2036-
if self._replica_constructor_retry_counter >= 0:
2037-
# Increase startup failure counter if we're tracking it
2038-
self._replica_constructor_retry_counter += 1
2039-
self._replica_constructor_error_msg = error_msg
2040-
2041-
retrying_msg = "Retrying"
2042-
if self._failed_to_start_threshold != 0:
2043-
remaining_retries = max(
2044-
self._failed_to_start_threshold
2045-
- self._replica_constructor_retry_counter,
2046-
0,
2047-
)
2048-
retrying_msg += f" {remaining_retries} more time(s)"
2049+
# There is no need to record replica failures if the target is 0.
2050+
if self._target_state.target_num_replicas == 0:
2051+
return
20492052

2050-
message = (
2051-
f"A replica failed to start with exception. {retrying_msg}. Error:\n"
2052-
f"{error_msg}"
2053+
# Increase startup failure counter
2054+
self._replica_constructor_retry_counter += 1
2055+
self._replica_constructor_error_msg = error_msg
2056+
2057+
# Update the deployment message only if replicas are failing during
2058+
# the very first time the controller is trying to start replicas of
2059+
# this version.
2060+
retrying_msg = ""
2061+
if not self._replica_has_started:
2062+
remaining_retries = max(
2063+
self._failed_to_start_threshold
2064+
- self._replica_constructor_retry_counter,
2065+
0,
20532066
)
2054-
self._curr_status_info = self._curr_status_info.update_message(message)
2067+
retrying_msg = f" {remaining_retries} more time(s)"
2068+
2069+
message = (
2070+
f"A replica failed to start with exception. Retrying{retrying_msg}. "
2071+
f"Error:\n{error_msg}"
2072+
)
2073+
self._curr_status_info = self._curr_status_info.update_message(message)
20552074

20562075
def stop_replicas(self, replicas_to_stop) -> None:
20572076
for replica in self._replicas.pop():

0 commit comments

Comments
 (0)