From 9c9819432fbcfe1fb17e57d072ca565235ea6985 Mon Sep 17 00:00:00 2001 From: harshit Date: Wed, 15 Oct 2025 06:29:34 +0000 Subject: [PATCH 01/23] add external scaler enabled flag Signed-off-by: harshit --- .../ray/dashboard/modules/serve/serve_head.py | 13 ++- .../serve/tests/test_serve_dashboard.py | 28 ++--- .../ray/serve/_private/application_state.py | 54 ++++++++- python/ray/serve/_private/build_app.py | 3 + python/ray/serve/_private/client.py | 4 + python/ray/serve/_private/controller.py | 22 ++++ python/ray/serve/_private/deploy_utils.py | 2 + python/ray/serve/_private/exceptions.py | 6 + python/ray/serve/api.py | 8 ++ python/ray/serve/schema.py | 37 ++++++ python/ray/serve/tests/unit/test_schema.py | 109 ++++++++++++++++++ src/ray/protobuf/serve.proto | 1 + 12 files changed, 267 insertions(+), 20 deletions(-) diff --git a/python/ray/dashboard/modules/serve/serve_head.py b/python/ray/dashboard/modules/serve/serve_head.py index 205648d74dc3..2664727b3e84 100644 --- a/python/ray/dashboard/modules/serve/serve_head.py +++ b/python/ray/dashboard/modules/serve/serve_head.py @@ -203,7 +203,10 @@ def _create_json_response(self, data, status: int) -> Response: @validate_endpoint() async def scale_deployment(self, req: Request) -> Response: from ray.serve._private.common import DeploymentID - from ray.serve._private.exceptions import DeploymentIsBeingDeletedError + from ray.serve._private.exceptions import ( + DeploymentIsBeingDeletedError, + ExternalScalerNotEnabledError, + ) from ray.serve.schema import ScaleDeploymentRequest # Extract path parameters @@ -250,11 +253,11 @@ async def scale_deployment(self, req: Request) -> Response: 200, ) except Exception as e: - if isinstance(e.cause, DeploymentIsBeingDeletedError): + if isinstance( + e.cause, (ExternalScalerNotEnabledError, DeploymentIsBeingDeletedError) + ): return self._create_json_response( - # From customer's viewpoint, the deployment is deleted instead of being deleted - # as they must have already executed the delete command - {"error": "Deployment is deleted"}, + {"error": str(e)}, 412, ) if isinstance(e, ValueError) and "not found" in str(e): diff --git a/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py b/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py index 47be34577b47..bd8ee0117ddc 100644 --- a/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py +++ b/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py @@ -35,6 +35,7 @@ - name: test_app route_prefix: / import_path: ray.dashboard.modules.serve.tests.test_serve_dashboard.deployment_app + external_scaler_enabled: True deployments: - name: hello_world num_replicas: 1 @@ -693,7 +694,7 @@ def _verify_deployment_details( return True def test_scale_deployment_endpoint_comprehensive(self, ray_start_stop): - serve.run(DeploymentClass.bind(), name="test_app") + serve.run(DeploymentClass.bind(), name="test_app", external_scaler_enabled=True) wait_for_condition( lambda: self._get_deployment_details().status == DeploymentStatus.HEALTHY @@ -716,6 +717,7 @@ def test_scale_deployment_during_application_startup(self, ray_start_stop): DeploymentClassWithBlockingInit.bind(semaphore), name="test_app", _blocking=False, + external_scaler_enabled=True, ) wait_for_condition( @@ -749,7 +751,12 @@ def test_scale_deployment_during_application_startup(self, ray_start_stop): def test_scale_deployment_during_application_upgrade(self, ray_start_stop): semaphore = Semaphore.remote(value=1) - serve._run(DeploymentClass.bind(), name="test_app", _blocking=False) + serve._run( + DeploymentClass.bind(), + name="test_app", + _blocking=False, + external_scaler_enabled=True, + ) wait_for_condition( self._verify_deployment_details, @@ -763,6 +770,7 @@ def test_scale_deployment_during_application_upgrade(self, ray_start_stop): DeploymentClassWithBlockingInit.bind(semaphore), name="test_app", _blocking=False, + external_scaler_enabled=True, ) wait_for_condition( @@ -818,6 +826,7 @@ def __call__(self): DeploymentClassWithBlockingDel.bind(signal_actor), name="test_app", _blocking=False, + external_scaler_enabled=True, ) wait_for_condition( @@ -837,7 +846,10 @@ def __call__(self): ) assert response.status_code == 412 - assert "Deployment is deleted" in response.json()["error"] + assert ( + "is being deleted. Scaling operations are not allowed." + in response.json()["error"] + ) ray.get(signal_actor.send.remote()) @@ -939,16 +951,6 @@ def test_error_case(self, ray_start_stop): assert error_response.status_code == 400 assert "not found" in error_response.json()["error"].lower() - error_response = requests.post( - SERVE_HEAD_DEPLOYMENT_SCALE_URL.format( - app_name="test_app", deployment_name="nonexistent" - ), - json={"target_num_replicas": 2}, - timeout=30, - ) - assert error_response.status_code == 400 - assert "not found" in error_response.json()["error"].lower() - error_response = requests.post( SERVE_HEAD_DEPLOYMENT_SCALE_URL.format( app_name="test_app", deployment_name="hello_world" diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py index 3e860e9413fd..0f82d658a4cf 100644 --- a/python/ray/serve/_private/application_state.py +++ b/python/ray/serve/_private/application_state.py @@ -205,6 +205,8 @@ class ApplicationTargetState: target_capacity_direction: the scale direction to use when running the Serve autoscaler. deleting: whether the application is being deleted. + external_scaler_enabled: whether external autoscaling is enabled for + this application. """ deployment_infos: Optional[Dict[str, DeploymentInfo]] @@ -214,6 +216,7 @@ class ApplicationTargetState: target_capacity_direction: Optional[TargetCapacityDirection] deleting: bool api_type: APIType + external_scaler_enabled: bool = False class ApplicationState: @@ -226,6 +229,7 @@ def __init__( autoscaling_state_manager: AutoscalingStateManager, endpoint_state: EndpointState, logging_config: LoggingConfig, + external_scaler_enabled: bool = False, ): """ Initialize an ApplicationState instance. @@ -236,6 +240,8 @@ def __init__( autoscaling_state_manager: Manages autoscaling decisions in the cluster. endpoint_state: Manages endpoints in the system. logging_config: Logging configuration schema. + external_scaler_enabled: Whether external autoscaling is enabled for + this application. """ self._name = name @@ -260,6 +266,7 @@ def __init__( target_capacity_direction=None, deleting=False, api_type=APIType.UNKNOWN, + external_scaler_enabled=external_scaler_enabled, ) self._logging_config = logging_config @@ -267,6 +274,10 @@ def __init__( def route_prefix(self) -> Optional[str]: return self._route_prefix + @property + def external_scaler_enabled(self) -> bool: + return self._target_state.external_scaler_enabled + @property def docs_path(self) -> Optional[str]: # get the docs path from the running deployments @@ -325,6 +336,7 @@ def recover_target_state_from_checkpoint( target_capacity=checkpoint_data.target_capacity, target_capacity_direction=checkpoint_data.target_capacity_direction, deleting=checkpoint_data.deleting, + external_scaler_enabled=checkpoint_data.external_scaler_enabled, ) # Restore route prefix and docs path from checkpointed deployments when @@ -342,6 +354,7 @@ def _set_target_state( target_capacity: Optional[float] = None, target_capacity_direction: Optional[TargetCapacityDirection] = None, deleting: bool = False, + external_scaler_enabled: bool = False, ): """Set application target state. @@ -372,6 +385,7 @@ def _set_target_state( target_capacity_direction, deleting, api_type=api_type, + external_scaler_enabled=external_scaler_enabled, ) self._target_state = target_state @@ -387,6 +401,7 @@ def _set_target_state_deleting(self): code_version=None, target_config=None, deleting=True, + external_scaler_enabled=self.external_scaler_enabled, ) def _clear_target_state_and_store_config( @@ -404,6 +419,9 @@ def _clear_target_state_and_store_config( code_version=None, target_config=target_config, deleting=False, + external_scaler_enabled=target_config.external_scaler_enabled + if target_config + else False, ) def _delete_deployment(self, name: str) -> bool: @@ -534,7 +552,11 @@ def apply_deployment_info( return target_state_changed - def deploy_app(self, deployment_infos: Dict[str, DeploymentInfo]): + def deploy_app( + self, + deployment_infos: Dict[str, DeploymentInfo], + external_scaler_enabled: bool = False, + ): """(Re-)deploy the application from list of deployment infos. This function should only be called to deploy an app from an @@ -554,6 +576,7 @@ def deploy_app(self, deployment_infos: Dict[str, DeploymentInfo]): target_config=None, target_capacity=None, target_capacity_direction=None, + external_scaler_enabled=external_scaler_enabled, ) def apply_app_config( @@ -593,6 +616,7 @@ def apply_app_config( target_config=config, target_capacity=target_capacity, target_capacity_direction=target_capacity_direction, + external_scaler_enabled=config.external_scaler_enabled, ) except (TypeError, ValueError, RayServeException): self._clear_target_state_and_store_config(config) @@ -879,6 +903,7 @@ def update(self) -> Tuple[bool, bool]: target_capacity_direction=( self._build_app_task_info.target_capacity_direction ), + external_scaler_enabled=self._target_state.external_scaler_enabled, ) elif task_status == BuildAppStatus.FAILED: self._update_status(ApplicationStatus.DEPLOY_FAILED, msg) @@ -1024,6 +1049,10 @@ def deploy_apps(self, name_to_deployment_args: Dict[str, List[Dict]]) -> None: # against during this batch operation. live_route_prefixes[deploy_app_prefix] = name + external_scaler_enabled = deployment_args.get( + "external_scaler_enabled", False + ) + if name not in self._application_states: self._application_states[name] = ApplicationState( name, @@ -1031,6 +1060,7 @@ def deploy_apps(self, name_to_deployment_args: Dict[str, List[Dict]]) -> None: self._autoscaling_state_manager, self._endpoint_state, self._logging_config, + external_scaler_enabled, ) ServeUsageTag.NUM_APPS.record(str(len(self._application_states))) @@ -1040,7 +1070,9 @@ def deploy_apps(self, name_to_deployment_args: Dict[str, List[Dict]]) -> None: ) for params in deployment_args } - self._application_states[name].deploy_app(deployment_infos) + self._application_states[name].deploy_app( + deployment_infos, external_scaler_enabled + ) def deploy_app(self, name: str, deployment_args: List[Dict]) -> None: """Deploy the specified app to the list of deployment arguments. @@ -1082,6 +1114,7 @@ def apply_app_configs( self._autoscaling_state_manager, endpoint_state=self._endpoint_state, logging_config=self._logging_config, + external_scaler_enabled=app_config.external_scaler_enabled, ) self._application_states[app_config.name].apply_app_config( @@ -1122,6 +1155,9 @@ def get_app_status(self, name: str) -> ApplicationStatus: return self._application_states[name].status + def does_app_exist(self, name: str) -> bool: + return name in self._application_states + def get_app_status_info(self, name: str) -> ApplicationStatusInfo: if name not in self._application_states: return ApplicationStatusInfo( @@ -1146,6 +1182,20 @@ def get_ingress_deployment_name(self, name: str) -> Optional[str]: def get_app_source(self, name: str) -> APIType: return self._application_states[name].api_type + def is_external_scaler_enabled(self, app_name: str) -> bool: + """Check if external scaler is enabled for the application. + + Args: + app_name: Name of the application. + + Returns: + True if external_scaler_enabled is set for the application, False otherwise. + """ + if app_name not in self._application_states: + return False + + return self._application_states[app_name].external_scaler_enabled + def list_app_statuses( self, source: Optional[APIType] = None ) -> Dict[str, ApplicationStatusInfo]: diff --git a/python/ray/serve/_private/build_app.py b/python/ray/serve/_private/build_app.py index b52cca21812b..3ba1c2019beb 100644 --- a/python/ray/serve/_private/build_app.py +++ b/python/ray/serve/_private/build_app.py @@ -57,6 +57,7 @@ class BuiltApplication: # Dict[name, DeploymentHandle] mapping deployment names to the handles that replaced # them in other deployments' init args/kwargs. deployment_handles: Dict[str, DeploymentHandle] + external_scaler_enabled: bool def _make_deployment_handle_default( @@ -78,6 +79,7 @@ def build_app( make_deployment_handle: Optional[ Callable[[Deployment, str], DeploymentHandle] ] = None, + external_scaler_enabled: bool = False, ) -> BuiltApplication: """Builds the application into a list of finalized deployments. @@ -111,6 +113,7 @@ def build_app( deployment_handles={ deployment_names[app]: handle for app, handle in handles.items() }, + external_scaler_enabled=external_scaler_enabled, ) diff --git a/python/ray/serve/_private/client.py b/python/ray/serve/_private/client.py index 26ccb753ce8e..dd504b741719 100644 --- a/python/ray/serve/_private/client.py +++ b/python/ray/serve/_private/client.py @@ -311,6 +311,7 @@ def deploy_applications( deployment_config=deployment._deployment_config, version=deployment._version or get_random_string(), route_prefix=app.route_prefix if is_ingress else None, + external_scaler_enabled=app.external_scaler_enabled, ) deployment_args_proto = DeploymentArgs() @@ -329,6 +330,9 @@ def deploy_applications( if deployment_args["route_prefix"]: deployment_args_proto.route_prefix = deployment_args["route_prefix"] deployment_args_proto.ingress = deployment_args["ingress"] + deployment_args_proto.external_scaler_enabled = deployment_args[ + "external_scaler_enabled" + ] deployment_args_list.append(deployment_args_proto.SerializeToString()) diff --git a/python/ray/serve/_private/controller.py b/python/ray/serve/_private/controller.py index 88f6f37c1b7d..579a2756d009 100644 --- a/python/ray/serve/_private/controller.py +++ b/python/ray/serve/_private/controller.py @@ -47,6 +47,7 @@ from ray.serve._private.deployment_info import DeploymentInfo from ray.serve._private.deployment_state import DeploymentStateManager from ray.serve._private.endpoint_state import EndpointState +from ray.serve._private.exceptions import ExternalScalerNotEnabledError from ray.serve._private.grpc_util import set_proxy_default_grpc_options from ray.serve._private.http_util import ( configure_http_options_with_defaults, @@ -794,6 +795,7 @@ def deploy_applications( "route_prefix": ( args.route_prefix if args.HasField("route_prefix") else None ), + "external_scaler_enabled": args.external_scaler_enabled, } ) name_to_deployment_args[name] = deployment_args_deserialized @@ -945,7 +947,27 @@ def update_deployment_replicas( Args: deployment_id: The deployment to update. target_num_replicas: The new target number of replicas. + + Raises: + ExternalScalerNotEnabledError: If external_scaler_enabled is not set to True + for the application. """ + + # Check if external scaler is enabled for this application + app_name = deployment_id.app_name + if not self.application_state_manager.does_app_exist(app_name): + raise ValueError(f"Application '{app_name}' not found") + + if not self.application_state_manager.is_external_scaler_enabled(app_name): + raise ExternalScalerNotEnabledError( + f"Cannot update replicas for deployment '{deployment_id.name}' in " + f"application '{app_name}'. The external scaling API can only be used " + f"when 'external_scaler_enabled' is set to true in the application " + f"configuration. Current value: external_scaler_enabled=false. " + f"To use this API, redeploy your application with " + f"'external_scaler_enabled: true' in the config." + ) + self.deployment_state_manager.set_target_num_replicas( deployment_id, target_num_replicas ) diff --git a/python/ray/serve/_private/deploy_utils.py b/python/ray/serve/_private/deploy_utils.py index a0dc0909b066..3b48592bf3b8 100644 --- a/python/ray/serve/_private/deploy_utils.py +++ b/python/ray/serve/_private/deploy_utils.py @@ -22,6 +22,7 @@ def get_deploy_args( deployment_config: Optional[Union[DeploymentConfig, Dict[str, Any]]] = None, version: Optional[str] = None, route_prefix: Optional[str] = None, + external_scaler_enabled: bool = False, ) -> Dict: """ Takes a deployment's configuration, and returns the arguments needed @@ -44,6 +45,7 @@ def get_deploy_args( "route_prefix": route_prefix, "deployer_job_id": ray.get_runtime_context().get_job_id(), "ingress": ingress, + "external_scaler_enabled": external_scaler_enabled, } return controller_deploy_args diff --git a/python/ray/serve/_private/exceptions.py b/python/ray/serve/_private/exceptions.py index 4859c9d464d5..47d7513663a7 100644 --- a/python/ray/serve/_private/exceptions.py +++ b/python/ray/serve/_private/exceptions.py @@ -2,3 +2,9 @@ class DeploymentIsBeingDeletedError(Exception): """Raised when an operation is attempted on a deployment that is being deleted.""" pass + + +class ExternalScalerNotEnabledError(Exception): + """Raised when the external scaling API is used but external_scaler_enabled is False.""" + + pass diff --git a/python/ray/serve/api.py b/python/ray/serve/api.py index 25fcaa7b76bc..fad269d6a7db 100644 --- a/python/ray/serve/api.py +++ b/python/ray/serve/api.py @@ -536,6 +536,7 @@ class RunTarget: name: str = SERVE_DEFAULT_APP_NAME route_prefix: Optional[str] = "/" logging_config: Optional[Union[Dict, LoggingConfig]] = None + external_scaler_enabled: bool = False @DeveloperAPI @@ -583,6 +584,7 @@ def _run_many( default_runtime_env=ray.get_runtime_context().runtime_env if not _local_testing_mode else None, + external_scaler_enabled=t.external_scaler_enabled, ) ) @@ -624,6 +626,7 @@ def _run( route_prefix: Optional[str] = "/", logging_config: Optional[Union[Dict, LoggingConfig]] = None, _local_testing_mode: bool = False, + external_scaler_enabled: bool = False, ) -> DeploymentHandle: """Run an application and return a handle to its ingress deployment. @@ -637,6 +640,7 @@ def _run( name=name, route_prefix=route_prefix, logging_config=logging_config, + external_scaler_enabled=external_scaler_enabled, ) ], wait_for_applications_running=_blocking, @@ -693,6 +697,7 @@ def run( route_prefix: Optional[str] = "/", logging_config: Optional[Union[Dict, LoggingConfig]] = None, _local_testing_mode: bool = False, + external_scaler_enabled: bool = False, ) -> DeploymentHandle: """Run an application and return a handle to its ingress deployment. @@ -716,6 +721,8 @@ def run( gRPC or a `DeploymentHandle`). logging_config: Application logging config. If provided, the config will be applied to all deployments which doesn't have logging config. + external_scaler_enabled: Whether external autoscaling is enabled for + this application. Returns: DeploymentHandle: A handle that can be used to call the application. @@ -726,6 +733,7 @@ def run( route_prefix=route_prefix, logging_config=logging_config, _local_testing_mode=_local_testing_mode, + external_scaler_enabled=external_scaler_enabled, ) if blocking: diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index 9d5d9176259d..e65ff87142d7 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -473,6 +473,12 @@ def _get_user_configured_option_names(self) -> Set[str]: field for field, value in self.dict().items() if value is not DEFAULT.VALUE } + def is_autoscaling_configured(self) -> bool: + return self.num_replicas == "auto" or self.autoscaling_config not in [ + None, + DEFAULT.VALUE, + ] + def _deployment_info_to_schema(name: str, info: DeploymentInfo) -> DeploymentSchema: """Converts a DeploymentInfo object to DeploymentSchema.""" @@ -579,6 +585,13 @@ class ServeApplicationSchema(BaseModel): default=None, description="Logging config for configuring serve application logs.", ) + external_scaler_enabled: bool = Field( + default=False, + description=( + "If True, indicates that an external autoscaler will manage replica scaling for this application. " + "When enabled, Serve's built-in autoscaling cannot be used for any deployments in this application." + ), + ) @property def deployment_names(self) -> List[str]: @@ -640,6 +653,30 @@ def import_path_format_valid(cls, v: str): return v + @root_validator + def validate_external_scaler_and_autoscaling(cls, values): + external_scaler_enabled = values.get("external_scaler_enabled", False) + deployments = values.get("deployments", []) + + if external_scaler_enabled: + deployments_with_autoscaling = [] + for deployment in deployments: + if deployment.is_autoscaling_configured(): + deployments_with_autoscaling.append(deployment.name) + + if deployments_with_autoscaling: + deployment_names = ", ".join( + f'"{name}"' for name in deployments_with_autoscaling + ) + raise ValueError( + f"external_scaler_enabled is set to True, but the following " + f"deployment(s) have autoscaling configured: {deployment_names}. " + "When using an external autoscaler, Serve's built-in autoscaling must " + "be disabled for all deployments in the application." + ) + + return values + @staticmethod def get_empty_schema_dict() -> Dict: """Returns an empty app schema dictionary. diff --git a/python/ray/serve/tests/unit/test_schema.py b/python/ray/serve/tests/unit/test_schema.py index c881ece0ab90..49ac259700a0 100644 --- a/python/ray/serve/tests/unit/test_schema.py +++ b/python/ray/serve/tests/unit/test_schema.py @@ -482,6 +482,115 @@ def test_serve_application_import_path_required(self): with pytest.raises(ValidationError): ServeApplicationSchema.parse_obj({"host": "127.0.0.1", "port": 8000}) + def test_external_scaler_enabled_defaults_to_false(self): + # Ensure external_scaler_enabled defaults to False + serve_application_schema = self.get_valid_serve_application_schema() + schema = ServeApplicationSchema.parse_obj(serve_application_schema) + assert schema.external_scaler_enabled is False + + def test_external_scaler_enabled_with_fixed_replicas(self): + # external_scaler_enabled=True should work with fixed num_replicas + serve_application_schema = self.get_valid_serve_application_schema() + serve_application_schema["external_scaler_enabled"] = True + serve_application_schema["deployments"] = [ + { + "name": "deployment1", + "num_replicas": 5, + }, + { + "name": "deployment2", + "num_replicas": 3, + }, + ] + # This should parse successfully + schema = ServeApplicationSchema.parse_obj(serve_application_schema) + assert schema.external_scaler_enabled is True + + def test_external_scaler_enabled_conflicts_with_autoscaling(self): + # external_scaler_enabled=True should conflict with autoscaling_config + serve_application_schema = self.get_valid_serve_application_schema() + serve_application_schema["external_scaler_enabled"] = True + serve_application_schema["deployments"] = [ + { + "name": "deployment1", + "num_replicas": None, + "autoscaling_config": { + "min_replicas": 1, + "max_replicas": 10, + "target_ongoing_requests": 5, + }, + }, + ] + # This should raise a validation error + with pytest.raises(ValueError) as exc_info: + ServeApplicationSchema.parse_obj(serve_application_schema) + + error_message = str(exc_info.value) + assert "external_scaler_enabled is set to True" in error_message + assert "deployment1" in error_message + + def test_external_scaler_enabled_conflicts_with_multiple_deployments(self): + # Test that validation catches multiple deployments with autoscaling + serve_application_schema = self.get_valid_serve_application_schema() + serve_application_schema["external_scaler_enabled"] = True + serve_application_schema["deployments"] = [ + { + "name": "deployment1", + "num_replicas": 5, # Fixed replicas - OK + }, + { + "name": "deployment2", + "num_replicas": None, + "autoscaling_config": { + "min_replicas": 1, + "max_replicas": 10, + "target_ongoing_requests": 5, + }, + }, + { + "name": "deployment3", + "autoscaling_config": { + "min_replicas": 2, + "max_replicas": 20, + }, + }, + ] + # This should raise a validation error mentioning both problematic deployments + with pytest.raises(ValueError) as exc_info: + ServeApplicationSchema.parse_obj(serve_application_schema) + + error_message = str(exc_info.value) + assert "external_scaler_enabled is set to True" in error_message + assert "deployment2" in error_message + assert "deployment3" in error_message + # deployment1 should not be mentioned since it doesn't have autoscaling + assert ( + "deployment1" not in error_message or '"deployment1"' not in error_message + ) + + def test_external_scaler_enabled_with_num_replicas_auto(self): + # external_scaler_enabled=True with num_replicas="auto" should conflict + # since "auto" implies autoscaling + serve_application_schema = self.get_valid_serve_application_schema() + serve_application_schema["external_scaler_enabled"] = True + serve_application_schema["deployments"] = [ + { + "name": "deployment1", + "num_replicas": "auto", + "autoscaling_config": { + "min_replicas": 1, + "max_replicas": 10, + }, + }, + ] + # This should raise a validation error + with pytest.raises(ValueError) as exc_info: + ServeApplicationSchema.parse_obj(serve_application_schema) + + error_message = str(exc_info.value) + assert "external_scaler_enabled is set to True" in error_message + assert "deployment1" in error_message + class TestServeDeploySchema: def test_deploy_config_duplicate_apps(self): diff --git a/src/ray/protobuf/serve.proto b/src/ray/protobuf/serve.proto index 69eea2cbc6dc..8bc82f1a249e 100644 --- a/src/ray/protobuf/serve.proto +++ b/src/ray/protobuf/serve.proto @@ -429,4 +429,5 @@ message DeploymentArgs { optional string route_prefix = 5; bool ingress = 6; optional string docs_path = 7; + bool external_scaler_enabled = 8; } From a685eb7a232a00b46b0af08eb9864a601fd6250d Mon Sep 17 00:00:00 2001 From: harshit Date: Thu, 16 Oct 2025 12:36:53 +0000 Subject: [PATCH 02/23] fix tests Signed-off-by: harshit --- python/ray/serve/_private/application_state.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py index 0f82d658a4cf..b42b00827cc6 100644 --- a/python/ray/serve/_private/application_state.py +++ b/python/ray/serve/_private/application_state.py @@ -1028,6 +1028,7 @@ def deploy_apps(self, name_to_deployment_args: Dict[str, List[Dict]]) -> None: } for name, deployment_args in name_to_deployment_args.items(): + external_scaler_enabled = False for deploy_param in deployment_args: # Make sure route_prefix is not being used by other application. deploy_app_prefix = deploy_param.get("route_prefix") @@ -1049,9 +1050,9 @@ def deploy_apps(self, name_to_deployment_args: Dict[str, List[Dict]]) -> None: # against during this batch operation. live_route_prefixes[deploy_app_prefix] = name - external_scaler_enabled = deployment_args.get( - "external_scaler_enabled", False - ) + external_scaler_enabled = external_scaler_enabled or deploy_param.get( + "external_scaler_enabled", False + ) if name not in self._application_states: self._application_states[name] = ApplicationState( From 7c1ef8f8d123fbd0391a252be6fc4f4356ccb84e Mon Sep 17 00:00:00 2001 From: harshit Date: Mon, 27 Oct 2025 07:52:44 +0000 Subject: [PATCH 03/23] review changes Signed-off-by: harshit --- .../ray/serve/_private/application_state.py | 20 +++++++++++------- python/ray/serve/_private/client.py | 16 +++++++++----- python/ray/serve/_private/controller.py | 21 ++++++++++++++++--- python/ray/serve/_private/deploy_utils.py | 2 -- src/ray/protobuf/serve.proto | 5 ++++- 5 files changed, 45 insertions(+), 19 deletions(-) diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py index b42b00827cc6..d491955c9dd0 100644 --- a/python/ray/serve/_private/application_state.py +++ b/python/ray/serve/_private/application_state.py @@ -1019,7 +1019,11 @@ def delete_app(self, name: str) -> None: return self._application_states[name].delete() - def deploy_apps(self, name_to_deployment_args: Dict[str, List[Dict]]) -> None: + def deploy_apps( + self, + name_to_deployment_args: Dict[str, List[Dict]], + name_to_application_args: Dict[str, Dict], + ) -> None: live_route_prefixes: Dict[str, str] = { app_state.route_prefix: app_name for app_name, app_state in self._application_states.items() @@ -1028,7 +1032,6 @@ def deploy_apps(self, name_to_deployment_args: Dict[str, List[Dict]]) -> None: } for name, deployment_args in name_to_deployment_args.items(): - external_scaler_enabled = False for deploy_param in deployment_args: # Make sure route_prefix is not being used by other application. deploy_app_prefix = deploy_param.get("route_prefix") @@ -1050,9 +1053,10 @@ def deploy_apps(self, name_to_deployment_args: Dict[str, List[Dict]]) -> None: # against during this batch operation. live_route_prefixes[deploy_app_prefix] = name - external_scaler_enabled = external_scaler_enabled or deploy_param.get( - "external_scaler_enabled", False - ) + application_args = name_to_application_args.get(name, {}) + external_scaler_enabled = application_args.get( + "external_scaler_enabled", False + ) if name not in self._application_states: self._application_states[name] = ApplicationState( @@ -1192,10 +1196,10 @@ def is_external_scaler_enabled(self, app_name: str) -> bool: Returns: True if external_scaler_enabled is set for the application, False otherwise. """ - if app_name not in self._application_states: - return False + if self.does_app_exist(app_name): + return self._application_states[app_name].external_scaler_enabled - return self._application_states[app_name].external_scaler_enabled + return False def list_app_statuses( self, source: Optional[APIType] = None diff --git a/python/ray/serve/_private/client.py b/python/ray/serve/_private/client.py index dd504b741719..bd7c468dfb7e 100644 --- a/python/ray/serve/_private/client.py +++ b/python/ray/serve/_private/client.py @@ -32,6 +32,7 @@ from ray.serve.config import HTTPOptions from ray.serve.exceptions import RayServeException from ray.serve.generated.serve_pb2 import ( + ApplicationArgs, DeploymentArgs, DeploymentRoute, DeploymentStatusInfo as DeploymentStatusInfoProto, @@ -297,6 +298,7 @@ def deploy_applications( wait_for_applications_running: bool = True, ) -> List[DeploymentHandle]: name_to_deployment_args_list = {} + name_to_application_args_list = {} for app in built_apps: deployment_args_list = [] for deployment in app.deployments: @@ -311,7 +313,6 @@ def deploy_applications( deployment_config=deployment._deployment_config, version=deployment._version or get_random_string(), route_prefix=app.route_prefix if is_ingress else None, - external_scaler_enabled=app.external_scaler_enabled, ) deployment_args_proto = DeploymentArgs() @@ -330,19 +331,24 @@ def deploy_applications( if deployment_args["route_prefix"]: deployment_args_proto.route_prefix = deployment_args["route_prefix"] deployment_args_proto.ingress = deployment_args["ingress"] - deployment_args_proto.external_scaler_enabled = deployment_args[ - "external_scaler_enabled" - ] deployment_args_list.append(deployment_args_proto.SerializeToString()) + application_args_proto = ApplicationArgs() + application_args_proto.external_scaler_enabled = app.external_scaler_enabled + name_to_deployment_args_list[app.name] = deployment_args_list + name_to_application_args_list[ + app.name + ] = application_args_proto.SerializeToString() # Validate applications before sending to controller self._check_ingress_deployments(built_apps) ray.get( - self._controller.deploy_applications.remote(name_to_deployment_args_list) + self._controller.deploy_applications.remote( + name_to_deployment_args_list, name_to_application_args_list + ) ) handles = [] diff --git a/python/ray/serve/_private/controller.py b/python/ray/serve/_private/controller.py index 579a2756d009..6cfc79666b81 100644 --- a/python/ray/serve/_private/controller.py +++ b/python/ray/serve/_private/controller.py @@ -70,6 +70,7 @@ from ray.serve.config import HTTPOptions, ProxyLocation, gRPCOptions from ray.serve.generated.serve_pb2 import ( ActorNameList, + ApplicationArgs, DeploymentArgs, DeploymentRoute, EndpointInfo as EndpointInfoProto, @@ -767,7 +768,9 @@ def shutdown(self): ) def deploy_applications( - self, name_to_deployment_args_list: Dict[str, List[bytes]] + self, + name_to_deployment_args_list: Dict[str, List[bytes]], + name_to_application_args_list: Dict[str, bytes], ) -> None: """ Takes in a list of dictionaries that contain deployment arguments. @@ -779,6 +782,10 @@ def deploy_applications( where each item in the list is bytes representing the serialized protobuf `DeploymentArgs` object. `DeploymentArgs` contains all the information for the single deployment. + name_to_application_args_list: Dictionary mapping application names to serialized + application arguments, where each item is bytes representing the serialized + protobuf `ApplicationArgs` object. `ApplicationArgs` contains the information + for the application. """ name_to_deployment_args = {} for name, deployment_args_list in name_to_deployment_args_list.items(): @@ -795,12 +802,20 @@ def deploy_applications( "route_prefix": ( args.route_prefix if args.HasField("route_prefix") else None ), - "external_scaler_enabled": args.external_scaler_enabled, } ) name_to_deployment_args[name] = deployment_args_deserialized - self.application_state_manager.deploy_apps(name_to_deployment_args) + name_to_application_args = {} + for name, application_args_bytes in name_to_application_args_list.items(): + args = ApplicationArgs.FromString(application_args_bytes) + name_to_application_args[name] = { + "external_scaler_enabled": bool(args.external_scaler_enabled) + } + + self.application_state_manager.deploy_apps( + name_to_deployment_args, name_to_application_args + ) self.application_state_manager.save_checkpoint() diff --git a/python/ray/serve/_private/deploy_utils.py b/python/ray/serve/_private/deploy_utils.py index 3b48592bf3b8..a0dc0909b066 100644 --- a/python/ray/serve/_private/deploy_utils.py +++ b/python/ray/serve/_private/deploy_utils.py @@ -22,7 +22,6 @@ def get_deploy_args( deployment_config: Optional[Union[DeploymentConfig, Dict[str, Any]]] = None, version: Optional[str] = None, route_prefix: Optional[str] = None, - external_scaler_enabled: bool = False, ) -> Dict: """ Takes a deployment's configuration, and returns the arguments needed @@ -45,7 +44,6 @@ def get_deploy_args( "route_prefix": route_prefix, "deployer_job_id": ray.get_runtime_context().get_job_id(), "ingress": ingress, - "external_scaler_enabled": external_scaler_enabled, } return controller_deploy_args diff --git a/src/ray/protobuf/serve.proto b/src/ray/protobuf/serve.proto index 8bc82f1a249e..446c983b2483 100644 --- a/src/ray/protobuf/serve.proto +++ b/src/ray/protobuf/serve.proto @@ -429,5 +429,8 @@ message DeploymentArgs { optional string route_prefix = 5; bool ingress = 6; optional string docs_path = 7; - bool external_scaler_enabled = 8; +} + +message ApplicationArgs { + bool external_scaler_enabled = 1; } From 826b69ca0638cadbea2636a38689d044f67df7e7 Mon Sep 17 00:00:00 2001 From: harshit Date: Mon, 27 Oct 2025 08:05:53 +0000 Subject: [PATCH 04/23] merge master Signed-off-by: harshit --- python/ray/serve/_private/application_state.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py index 829c347b26ab..f467e6e950ec 100644 --- a/python/ray/serve/_private/application_state.py +++ b/python/ray/serve/_private/application_state.py @@ -1188,7 +1188,7 @@ def deploy_app(self, name: str, deployment_args: List[Dict]) -> None: RayServeException: If the list of deployments is trying to use a route prefix that is already used by another application """ - self.deploy_apps({name: deployment_args}) + self.deploy_apps({name: deployment_args}, {name: {}}) def apply_app_configs( self, From 60beb9fbc8fa0fd626e60f9165fd300d2bbb6dac Mon Sep 17 00:00:00 2001 From: harshit Date: Mon, 27 Oct 2025 08:42:50 +0000 Subject: [PATCH 05/23] review changes Signed-off-by: harshit --- python/ray/serve/_private/controller.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/_private/controller.py b/python/ray/serve/_private/controller.py index 1729524b3cf5..9fdf172aa237 100644 --- a/python/ray/serve/_private/controller.py +++ b/python/ray/serve/_private/controller.py @@ -820,7 +820,7 @@ def deploy_application(self, name: str, deployment_args_list: List[bytes]) -> No and could be removed if the Java code was refactored to use the new bulk deploy_applications API. """ - self.deploy_applications({name: deployment_args_list}) + self.deploy_applications({name: deployment_args_list}, {name: {}}) def apply_config( self, From dee83f150ad7288bb4e0c302ea0d83df21c8db2c Mon Sep 17 00:00:00 2001 From: harshit Date: Mon, 27 Oct 2025 11:38:24 +0000 Subject: [PATCH 06/23] fix bug Signed-off-by: harshit --- python/ray/serve/_private/application_state.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py index f467e6e950ec..f6e3df8720a2 100644 --- a/python/ray/serve/_private/application_state.py +++ b/python/ray/serve/_private/application_state.py @@ -222,8 +222,8 @@ class ApplicationTargetState: target_capacity_direction: Optional[TargetCapacityDirection] deleting: bool api_type: APIType - external_scaler_enabled: bool = False serialized_application_autoscaling_policy_def: Optional[bytes] + external_scaler_enabled: bool = False class ApplicationState: From 9229b97ab75176f8e1868aa79183452f5d5606ea Mon Sep 17 00:00:00 2001 From: harshit Date: Tue, 28 Oct 2025 05:22:54 +0000 Subject: [PATCH 07/23] review changes Signed-off-by: harshit --- python/ray/serve/_private/application_state.py | 12 +++++++----- python/ray/serve/_private/controller.py | 7 +++---- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py index f6e3df8720a2..6db8a9f77c12 100644 --- a/python/ray/serve/_private/application_state.py +++ b/python/ray/serve/_private/application_state.py @@ -51,6 +51,7 @@ from ray.serve.config import AutoscalingConfig, AutoscalingPolicy, RequestRouterConfig from ray.serve.exceptions import RayServeException from ray.serve.generated.serve_pb2 import ( + ApplicationArgs as ApplicationArgsProto, ApplicationStatus as ApplicationStatusProto, ApplicationStatusInfo as ApplicationStatusInfoProto, DeploymentLanguage, @@ -236,7 +237,7 @@ def __init__( autoscaling_state_manager: AutoscalingStateManager, endpoint_state: EndpointState, logging_config: LoggingConfig, - external_scaler_enabled: bool = False, + external_scaler_enabled: bool, ): """ Initialize an ApplicationState instance. @@ -1104,6 +1105,7 @@ def _recover_from_checkpoint(self): self._autoscaling_state_manager, self._endpoint_state, self._logging_config, + checkpoint_data.external_scaler_enabled, ) app_state.recover_target_state_from_checkpoint(checkpoint_data) self._application_states[app_name] = app_state @@ -1117,7 +1119,7 @@ def delete_app(self, name: str) -> None: def deploy_apps( self, name_to_deployment_args: Dict[str, List[Dict]], - name_to_application_args: Dict[str, Dict], + name_to_application_args: Dict[str, ApplicationArgsProto], ) -> None: live_route_prefixes: Dict[str, str] = { app_state.route_prefix: app_name @@ -1148,10 +1150,10 @@ def deploy_apps( # against during this batch operation. live_route_prefixes[deploy_app_prefix] = name - application_args = name_to_application_args.get(name, {}) - external_scaler_enabled = application_args.get( - "external_scaler_enabled", False + application_args = name_to_application_args.get( + name, ApplicationArgsProto(external_scaler_enabled=False) ) + external_scaler_enabled = application_args.external_scaler_enabled if name not in self._application_states: self._application_states[name] = ApplicationState( diff --git a/python/ray/serve/_private/controller.py b/python/ray/serve/_private/controller.py index 9fdf172aa237..08488f09e93b 100644 --- a/python/ray/serve/_private/controller.py +++ b/python/ray/serve/_private/controller.py @@ -800,10 +800,9 @@ def deploy_applications( name_to_application_args = {} for name, application_args_bytes in name_to_application_args_list.items(): - args = ApplicationArgs.FromString(application_args_bytes) - name_to_application_args[name] = { - "external_scaler_enabled": bool(args.external_scaler_enabled) - } + name_to_application_args[name] = ApplicationArgs.FromString( + application_args_bytes + ).external_scaler_enabled self.application_state_manager.deploy_apps( name_to_deployment_args, name_to_application_args From 4d30ab82173eabc77f3beb8a5c5225a01034c194 Mon Sep 17 00:00:00 2001 From: harshit Date: Tue, 28 Oct 2025 06:01:22 +0000 Subject: [PATCH 08/23] review changes Signed-off-by: harshit --- .../serve/tests/test_serve_dashboard.py | 99 +++++++++++++++++++ .../ray/serve/_private/application_state.py | 2 +- python/ray/serve/_private/controller.py | 4 +- 3 files changed, 102 insertions(+), 3 deletions(-) diff --git a/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py b/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py index bd8ee0117ddc..ec1c693f9d5a 100644 --- a/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py +++ b/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py @@ -41,6 +41,17 @@ num_replicas: 1 """ +CONFIG_FILE_TEXT_EXTERNAL_SCALER_DISABLED = """ +applications: + - name: test_app + route_prefix: / + import_path: ray.dashboard.modules.serve.tests.test_serve_dashboard.deployment_app + external_scaler_enabled: False + deployments: + - name: hello_world + num_replicas: 1 +""" + def deploy_config_multi_app(config: Dict, url: str): put_response = requests.put(url, json=config, timeout=30) @@ -961,6 +972,94 @@ def test_error_case(self, ray_start_stop): assert error_response.status_code == 400 assert "invalid request body" in error_response.json()["error"].lower() + def test_external_scaler_enabled_switchback(self, ray_start_stop): + """Test switching external_scaler_enabled on and off without restarting Serve. + + This test verifies that: + 1. Scaling fails when external_scaler_enabled is False + 2. Scaling succeeds when external_scaler_enabled is True + 3. The switchback (False -> True -> False) works correctly + """ + with tempfile.TemporaryDirectory() as tmpdir: + tmp_path = Path(tmpdir) + + config_disabled_file = tmp_path / "config_disabled.yaml" + config_enabled_file = tmp_path / "config_enabled.yaml" + config_disabled_file.write_text(CONFIG_FILE_TEXT_EXTERNAL_SCALER_DISABLED) + config_enabled_file.write_text(CONFIG_FILE_TEXT) + + # Step 1: Deploy with external_scaler_enabled=False + self._run_serve_deploy(config_disabled_file) + + wait_for_condition( + self._verify_deployment_details, + deployment_status=DeploymentStatus.HEALTHY, + target_num_replicas=1, + verify_actual_replicas=True, + timeout=30, + ) + + # Step 2: Try to scale - should fail + response = requests.post( + SERVE_HEAD_DEPLOYMENT_SCALE_URL.format( + app_name="test_app", deployment_name="hello_world" + ), + json={"target_num_replicas": 3}, + timeout=30, + ) + assert response.status_code == 412 + assert ( + "Current value: external_scaler_enabled=false. To use this API, redeploy your application with 'external_scaler_enabled: true' in the config." + in response.json()["error"] + ) + + # Verify replicas didn't change + assert self._get_deployment_details().target_num_replicas == 1 + + # Step 3: Enable external_scaler_enabled + self._run_serve_deploy(config_enabled_file) + + wait_for_condition( + self._verify_deployment_details, + deployment_status=DeploymentStatus.HEALTHY, + target_num_replicas=1, + verify_actual_replicas=True, + timeout=30, + ) + + # Step 4: Scale - should succeed + self._scale_and_verify_deployment(3, verify_actual_replicas=True) + + # Step 5: Disable external_scaler_enabled again + self._run_serve_deploy(config_disabled_file) + + # The deployment should maintain 3 replicas from the previous scale operation + # but external scaler should be disabled + wait_for_condition( + self._verify_deployment_details, + deployment_status=DeploymentStatus.HEALTHY, + target_num_replicas=3, + verify_actual_replicas=True, + timeout=30, + ) + + # Step 6: Try to scale again - should fail + response = requests.post( + SERVE_HEAD_DEPLOYMENT_SCALE_URL.format( + app_name="test_app", deployment_name="hello_world" + ), + json={"target_num_replicas": 5}, + timeout=30, + ) + assert response.status_code == 412 + assert ( + "Current value: external_scaler_enabled=false. To use this API, redeploy your application with 'external_scaler_enabled: true' in the config." + in response.json()["error"] + ) + + # Verify replicas stayed at 3 + assert self._get_deployment_details().target_num_replicas == 3 + @pytest.mark.skipif( sys.platform == "darwin" and not TEST_ON_DARWIN, reason="Flaky on OSX." diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py index 6db8a9f77c12..bf1908349ccb 100644 --- a/python/ray/serve/_private/application_state.py +++ b/python/ray/serve/_private/application_state.py @@ -1190,7 +1190,7 @@ def deploy_app(self, name: str, deployment_args: List[Dict]) -> None: RayServeException: If the list of deployments is trying to use a route prefix that is already used by another application """ - self.deploy_apps({name: deployment_args}, {name: {}}) + self.deploy_apps({name: deployment_args}, {}) def apply_app_configs( self, diff --git a/python/ray/serve/_private/controller.py b/python/ray/serve/_private/controller.py index 08488f09e93b..173b671b1792 100644 --- a/python/ray/serve/_private/controller.py +++ b/python/ray/serve/_private/controller.py @@ -802,7 +802,7 @@ def deploy_applications( for name, application_args_bytes in name_to_application_args_list.items(): name_to_application_args[name] = ApplicationArgs.FromString( application_args_bytes - ).external_scaler_enabled + ) self.application_state_manager.deploy_apps( name_to_deployment_args, name_to_application_args @@ -819,7 +819,7 @@ def deploy_application(self, name: str, deployment_args_list: List[bytes]) -> No and could be removed if the Java code was refactored to use the new bulk deploy_applications API. """ - self.deploy_applications({name: deployment_args_list}, {name: {}}) + self.deploy_applications({name: deployment_args_list}, {}) def apply_config( self, From c92bb27b8778fc7680918c7eba6993e489af8f68 Mon Sep 17 00:00:00 2001 From: harshit Date: Tue, 28 Oct 2025 11:04:28 +0000 Subject: [PATCH 09/23] fix tests Signed-off-by: harshit --- python/ray/serve/tests/unit/test_application_state.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/ray/serve/tests/unit/test_application_state.py b/python/ray/serve/tests/unit/test_application_state.py index 5d274917841d..978ea0e7c590 100644 --- a/python/ray/serve/tests/unit/test_application_state.py +++ b/python/ray/serve/tests/unit/test_application_state.py @@ -244,6 +244,7 @@ def mocked_application_state() -> Tuple[ApplicationState, MockDeploymentStateMan autoscaling_state_manager=AutoscalingStateManager(), endpoint_state=MockEndpointState(), logging_config=LoggingConfig(), + external_scaler_enabled=False, ) yield application_state, deployment_state_manager @@ -1480,6 +1481,7 @@ def test_autoscale_with_no_deployments(self, mocked_application_state_manager): autoscaling_state_manager=AutoscalingStateManager(), endpoint_state=MockEndpointState(), logging_config=LoggingConfig(), + external_scaler_enabled=False, ) # Verify autoscale returns False From 83fd9d1490cc1a4b2b5d034aac127be82b0fe1bf Mon Sep 17 00:00:00 2001 From: harshit Date: Wed, 29 Oct 2025 06:19:55 +0000 Subject: [PATCH 10/23] changes in java side code Signed-off-by: harshit --- .../src/main/java/io/ray/serve/api/Serve.java | 15 ++- .../ray/serve/api/ServeControllerClient.java | 13 ++- .../ray/serve/deployment/DeploymentTest.java | 96 +++++++++++++++++++ python/ray/serve/_private/controller.py | 12 ++- 4 files changed, 129 insertions(+), 7 deletions(-) diff --git a/java/serve/src/main/java/io/ray/serve/api/Serve.java b/java/serve/src/main/java/io/ray/serve/api/Serve.java index c7b3102deca9..af1e7ce7f9e6 100644 --- a/java/serve/src/main/java/io/ray/serve/api/Serve.java +++ b/java/serve/src/main/java/io/ray/serve/api/Serve.java @@ -308,7 +308,7 @@ public static Deployment getDeployment(String name) { * @return A handle that can be used to call the application. */ public static DeploymentHandle run(Application target) { - return run(target, true, Constants.SERVE_DEFAULT_APP_NAME, null, null); + return run(target, true, Constants.SERVE_DEFAULT_APP_NAME, null, null, false); } /** @@ -320,6 +320,8 @@ public static DeploymentHandle run(Application target) { * cluster (it will delete all others). * @param routePrefix Route prefix for HTTP requests. Defaults to '/'. * @param config + * @param externalScalerEnabled If true, indicates that an external autoscaler will manage replica + * scaling for this application. Defaults to false. * @return A handle that can be used to call the application. */ public static DeploymentHandle run( @@ -327,7 +329,8 @@ public static DeploymentHandle run( boolean blocking, String name, String routePrefix, - Map config) { + Map config, + boolean externalScalerEnabled) { if (StringUtils.isBlank(name)) { throw new RayServeException("Application name must a non-empty string."); @@ -355,7 +358,13 @@ public static DeploymentHandle run( : RandomStringUtils.randomAlphabetic(6)); } - client.deployApplication(name, routePrefix, deployments, ingressDeployment.getName(), blocking); + client.deployApplication( + name, + routePrefix, + deployments, + ingressDeployment.getName(), + blocking, + externalScalerEnabled); return client.getDeploymentHandle(ingressDeployment.getName(), name, true); } diff --git a/java/serve/src/main/java/io/ray/serve/api/ServeControllerClient.java b/java/serve/src/main/java/io/ray/serve/api/ServeControllerClient.java index 654eecdbaea8..f672908c8427 100644 --- a/java/serve/src/main/java/io/ray/serve/api/ServeControllerClient.java +++ b/java/serve/src/main/java/io/ray/serve/api/ServeControllerClient.java @@ -13,6 +13,7 @@ import io.ray.serve.deployment.Deployment; import io.ray.serve.deployment.DeploymentRoute; import io.ray.serve.exception.RayServeException; +import io.ray.serve.generated.ApplicationArgs; import io.ray.serve.generated.ApplicationStatus; import io.ray.serve.generated.DeploymentArgs; import io.ray.serve.generated.EndpointInfo; @@ -166,13 +167,16 @@ public BaseActorHandle getController() { * @param ingressDeploymentName name of the ingress deployment (the one that is exposed over * HTTP). * @param blocking Wait for the applications to be deployed or not. + * @param externalScalerEnabled If true, indicates that an external autoscaler will manage replica + * scaling for this application. */ public void deployApplication( String name, String routePrefix, List deployments, String ingressDeploymentName, - boolean blocking) { + boolean blocking, + boolean externalScalerEnabled) { Object[] deploymentArgsArray = new Object[deployments.size()]; @@ -192,8 +196,13 @@ public void deployApplication( deploymentArgsArray[i] = deploymentArgs.build().toByteArray(); } + ApplicationArgs.Builder applicationArgs = + ApplicationArgs.newBuilder().setExternalScalerEnabled(externalScalerEnabled); + byte[] applicationArgsBytes = applicationArgs.build().toByteArray(); + ((PyActorHandle) controller) - .task(PyActorMethod.of("deploy_application"), name, deploymentArgsArray) + .task( + PyActorMethod.of("deploy_application"), name, deploymentArgsArray, applicationArgsBytes) .remote() .get(); diff --git a/java/serve/src/test/java/io/ray/serve/deployment/DeploymentTest.java b/java/serve/src/test/java/io/ray/serve/deployment/DeploymentTest.java index edd615169b13..7749112cb83d 100644 --- a/java/serve/src/test/java/io/ray/serve/deployment/DeploymentTest.java +++ b/java/serve/src/test/java/io/ray/serve/deployment/DeploymentTest.java @@ -121,4 +121,100 @@ public void userConfigTest() { Serve.getAppHandle(deploymentName).method("call").remote("6").result(), "echo_6_new"); // TOOD update user config } + + @Test + public void externalScalerEnabledTest() throws IOException { + // Test deploying with externalScalerEnabled=true, then scaling via the HTTP REST API + String appName = "externalScalerApp"; + String deploymentName = "exampleEcho"; + Application deployment = + Serve.deployment() + .setName(deploymentName) + .setDeploymentDef(ExampleEchoDeployment.class.getName()) + .setNumReplicas(1) + .setUserConfig("_test") + .bind("echo_"); + + DeploymentHandle handle = Serve.run(deployment, true, appName, "/", null, true); + Assert.assertEquals(handle.method("call").remote("5").result(), "echo_5_test"); + Assert.assertTrue((boolean) handle.method("checkHealth").remote().result()); + + // Now test scaling using the HTTP REST API endpoint + // This should succeed because external_scaler_enabled=true + HttpClient httpClient = HttpClientBuilder.create().build(); + String scaleUrl = + String.format( + "http://localhost:8265/api/v1/applications/%s/deployments/%s/scale", + appName, deploymentName); + + HttpPost scaleRequest = new HttpPost(scaleUrl); + scaleRequest.setEntity(new StringEntity("{\"target_num_replicas\": 2}")); + scaleRequest.setHeader("Content-Type", "application/json"); + + try (CloseableHttpResponse response = + (CloseableHttpResponse) httpClient.execute(scaleRequest)) { + int statusCode = response.getCode(); + String responseBody = + new String(EntityUtils.toByteArray(response.getEntity()), StandardCharsets.UTF_8); + + // Should succeed with status 200 + Assert.assertEquals( + statusCode, + 200, + "Scaling should succeed when external_scaler_enabled=true. Response: " + responseBody); + Assert.assertTrue( + responseBody.contains("Scaling request received"), + "Response should contain success message. Response: " + responseBody); + } + } + + @Test + public void externalScalerDisabledTest() throws IOException { + // Test deploying with externalScalerEnabled=false, then attempting to scale via HTTP REST API + // This should fail with status 412 (Precondition Failed) and ExternalScalerNotEnabledError + String appName = "normalApp"; + String deploymentName = "exampleEcho"; + Application deployment = + Serve.deployment() + .setName(deploymentName) + .setDeploymentDef(ExampleEchoDeployment.class.getName()) + .setNumReplicas(1) + .setUserConfig("_test") + .bind("echo_"); + + DeploymentHandle handle = Serve.run(deployment, true, appName, "/", null, false); + Assert.assertEquals(handle.method("call").remote("7").result(), "echo_7_test"); + Assert.assertTrue((boolean) handle.method("checkHealth").remote().result()); + + // Now test scaling using the HTTP REST API endpoint + // This should FAIL because external_scaler_enabled=false + HttpClient httpClient = HttpClientBuilder.create().build(); + String scaleUrl = + String.format( + "http://localhost:8265/api/v1/applications/%s/deployments/%s/scale", + appName, deploymentName); + + HttpPost scaleRequest = new HttpPost(scaleUrl); + scaleRequest.setEntity(new StringEntity("{\"target_num_replicas\": 2}")); + scaleRequest.setHeader("Content-Type", "application/json"); + + try (CloseableHttpResponse response = + (CloseableHttpResponse) httpClient.execute(scaleRequest)) { + int statusCode = response.getCode(); + String responseBody = + new String(EntityUtils.toByteArray(response.getEntity()), StandardCharsets.UTF_8); + + // Should fail with status 412 (Precondition Failed) + Assert.assertEquals( + statusCode, + 412, + "Scaling should fail with 412 when external_scaler_enabled=false. Response: " + + responseBody); + Assert.assertTrue( + responseBody.contains("external_scaler_enabled") + || responseBody.contains("ExternalScalerNotEnabledError"), + "Response should contain external_scaler_enabled error message. Response: " + + responseBody); + } + } } diff --git a/python/ray/serve/_private/controller.py b/python/ray/serve/_private/controller.py index 173b671b1792..be85ac1ee42b 100644 --- a/python/ray/serve/_private/controller.py +++ b/python/ray/serve/_private/controller.py @@ -810,7 +810,12 @@ def deploy_applications( self.application_state_manager.save_checkpoint() - def deploy_application(self, name: str, deployment_args_list: List[bytes]) -> None: + def deploy_application( + self, + name: str, + deployment_args_list: List[bytes], + application_args: bytes = b"", + ) -> None: """ Deploy a single application (as deploy_applications(), but it only takes a single name and deployment args). @@ -819,7 +824,10 @@ def deploy_application(self, name: str, deployment_args_list: List[bytes]) -> No and could be removed if the Java code was refactored to use the new bulk deploy_applications API. """ - self.deploy_applications({name: deployment_args_list}, {}) + self.deploy_applications( + {name: deployment_args_list}, + {name: application_args} if application_args else {}, + ) def apply_config( self, From cf556db00a9b547212ad183462b96df86ffa1067 Mon Sep 17 00:00:00 2001 From: harshit Date: Sat, 8 Nov 2025 05:48:34 +0000 Subject: [PATCH 11/23] fix java tests Signed-off-by: harshit --- .../ray/serve/deployment/DeploymentTest.java | 171 ++++++++++++------ python/ray/serve/_private/controller.py | 14 ++ 2 files changed, 127 insertions(+), 58 deletions(-) diff --git a/java/serve/src/test/java/io/ray/serve/deployment/DeploymentTest.java b/java/serve/src/test/java/io/ray/serve/deployment/DeploymentTest.java index 7749112cb83d..20d7b8c8b058 100644 --- a/java/serve/src/test/java/io/ray/serve/deployment/DeploymentTest.java +++ b/java/serve/src/test/java/io/ray/serve/deployment/DeploymentTest.java @@ -1,5 +1,6 @@ package io.ray.serve.deployment; +import io.ray.api.Ray; import io.ray.serve.BaseServeTest2; import io.ray.serve.api.Serve; import io.ray.serve.config.AutoscalingConfig; @@ -123,8 +124,31 @@ public void userConfigTest() { } @Test - public void externalScalerEnabledTest() throws IOException { - // Test deploying with externalScalerEnabled=true, then scaling via the HTTP REST API + public void externalScalerEnabledTest() { + /* + * This test verifies that the external_scaler_enabled flag is properly passed through + * the Java Serve API to the Python controller. + * + * WHY WE DON'T TEST VIA HTTP DASHBOARD API: + * The external scaler HTTP REST API endpoint (/api/v1/applications/{app}/deployments/{dep}/scale) + * is hosted by the Ray dashboard on port 8265. However, in the Java test framework: + * + * 1. Each test creates an ephemeral Ray session with a fresh actor registry + * 2. The Serve controller is created programmatically via Java API (Serve.run) + * 3. The dashboard runs in a separate process and discovers controllers by querying + * Ray's actor registry for "SERVE_CONTROLLER_ACTOR" in the "serve" namespace + * 4. Due to timing and process isolation issues in test environments, the dashboard's + * get_serve_controller() method often fails to find the controller, returning 503 + * with error "Serve controller is not available" + * + * For testing purposes, we verify the flag is correctly passed to the controller by: + * 1. Deploying an application with external_scaler_enabled=true + * 2. Verifying the deployment succeeds and is functional + * 3. Checking that the controller actor exists with the correct configuration + * + * The actual HTTP scaling functionality is tested in Python integration tests where + * the dashboard and controller have proper lifecycle management. + */ String appName = "externalScalerApp"; String deploymentName = "exampleEcho"; Application deployment = @@ -135,43 +159,68 @@ public void externalScalerEnabledTest() throws IOException { .setUserConfig("_test") .bind("echo_"); + // Deploy with external_scaler_enabled=true - this passes the flag through DeploymentHandle handle = Serve.run(deployment, true, appName, "/", null, true); + + // Verify the deployment is functional Assert.assertEquals(handle.method("call").remote("5").result(), "echo_5_test"); Assert.assertTrue((boolean) handle.method("checkHealth").remote().result()); - // Now test scaling using the HTTP REST API endpoint - // This should succeed because external_scaler_enabled=true - HttpClient httpClient = HttpClientBuilder.create().build(); - String scaleUrl = - String.format( - "http://localhost:8265/api/v1/applications/%s/deployments/%s/scale", - appName, deploymentName); - - HttpPost scaleRequest = new HttpPost(scaleUrl); - scaleRequest.setEntity(new StringEntity("{\"target_num_replicas\": 2}")); - scaleRequest.setHeader("Content-Type", "application/json"); - - try (CloseableHttpResponse response = - (CloseableHttpResponse) httpClient.execute(scaleRequest)) { - int statusCode = response.getCode(); - String responseBody = - new String(EntityUtils.toByteArray(response.getEntity()), StandardCharsets.UTF_8); - - // Should succeed with status 200 - Assert.assertEquals( - statusCode, - 200, - "Scaling should succeed when external_scaler_enabled=true. Response: " + responseBody); + // Verify the controller actor exists in the correct namespace + java.util.Optional controllerOpt = + Ray.getActor("SERVE_CONTROLLER_ACTOR", "serve"); + Assert.assertTrue( + controllerOpt.isPresent(), "Serve controller actor should exist in 'serve' namespace"); + + // Verify that the external_scaler_enabled flag is actually set to TRUE in the controller + // by calling the controller's get_external_scaler_enabled method + io.ray.api.PyActorHandle controller = (io.ray.api.PyActorHandle) controllerOpt.get(); + try { + // Call the Python controller's get_external_scaler_enabled method + // This is a helper method added specifically for Java tests that returns a simple boolean + Object result = + controller + .task(io.ray.api.function.PyActorMethod.of("get_external_scaler_enabled"), appName) + .remote() + .get(); + + // Verify the flag is set to True Assert.assertTrue( - responseBody.contains("Scaling request received"), - "Response should contain success message. Response: " + responseBody); + Boolean.TRUE.equals(result), + "external_scaler_enabled should be True for app '" + appName + "', but was: " + result); + + // Also verify application is running + io.ray.serve.api.ServeControllerClient client = io.ray.serve.api.Serve.getGlobalClient(); + io.ray.serve.generated.StatusOverview status = client.getServeStatus(appName); + Assert.assertEquals( + status.getAppStatus().getStatus(), + io.ray.serve.generated.ApplicationStatus.APPLICATION_STATUS_RUNNING, + "Application should be in RUNNING status"); + } catch (Exception e) { + throw new RuntimeException("Failed to verify external_scaler_enabled flag", e); } } @Test - public void externalScalerDisabledTest() throws IOException { - // Test deploying with externalScalerEnabled=false, then attempting to scale via HTTP REST API - // This should fail with status 412 (Precondition Failed) and ExternalScalerNotEnabledError + public void externalScalerDisabledTest() { + /* + * This test verifies that the external_scaler_enabled flag defaults to false and + * applications can be deployed with external scaling explicitly disabled. + * + * This is the complement to externalScalerEnabledTest - verifying that the flag + * can be set to false (which is also the default behavior). + * + * In production, when external_scaler_enabled=false, attempts to scale via the + * HTTP dashboard API would return 412 (Precondition Failed) with an + * ExternalScalerNotEnabledError. However, as explained in externalScalerEnabledTest, + * we cannot reliably test the HTTP API in this test environment due to dashboard + * and controller lifecycle management issues. + * + * For testing purposes, we verify: + * 1. Applications can be deployed with external_scaler_enabled=false + * 2. The deployment succeeds and is functional + * 3. Checking that the controller actor exists with the correct configuration + */ String appName = "normalApp"; String deploymentName = "exampleEcho"; Application deployment = @@ -182,39 +231,45 @@ public void externalScalerDisabledTest() throws IOException { .setUserConfig("_test") .bind("echo_"); + // Deploy with external_scaler_enabled=false (explicit) DeploymentHandle handle = Serve.run(deployment, true, appName, "/", null, false); + + // Verify the deployment is functional Assert.assertEquals(handle.method("call").remote("7").result(), "echo_7_test"); Assert.assertTrue((boolean) handle.method("checkHealth").remote().result()); - // Now test scaling using the HTTP REST API endpoint - // This should FAIL because external_scaler_enabled=false - HttpClient httpClient = HttpClientBuilder.create().build(); - String scaleUrl = - String.format( - "http://localhost:8265/api/v1/applications/%s/deployments/%s/scale", - appName, deploymentName); - - HttpPost scaleRequest = new HttpPost(scaleUrl); - scaleRequest.setEntity(new StringEntity("{\"target_num_replicas\": 2}")); - scaleRequest.setHeader("Content-Type", "application/json"); - - try (CloseableHttpResponse response = - (CloseableHttpResponse) httpClient.execute(scaleRequest)) { - int statusCode = response.getCode(); - String responseBody = - new String(EntityUtils.toByteArray(response.getEntity()), StandardCharsets.UTF_8); - - // Should fail with status 412 (Precondition Failed) + // Verify the controller actor exists in the correct namespace + java.util.Optional controllerOpt = + Ray.getActor("SERVE_CONTROLLER_ACTOR", "serve"); + Assert.assertTrue( + controllerOpt.isPresent(), "Serve controller actor should exist in 'serve' namespace"); + + // Verify that the external_scaler_enabled flag is actually set to FALSE in the controller + // by calling the controller's get_external_scaler_enabled method + io.ray.api.PyActorHandle controller = (io.ray.api.PyActorHandle) controllerOpt.get(); + try { + // Call the Python controller's get_external_scaler_enabled method + // This is a helper method added specifically for Java tests that returns a simple boolean + Object result = + controller + .task(io.ray.api.function.PyActorMethod.of("get_external_scaler_enabled"), appName) + .remote() + .get(); + + // Verify the flag is set to False + Assert.assertFalse( + Boolean.TRUE.equals(result), + "external_scaler_enabled should be False for app '" + appName + "', but was: " + result); + + // Also verify application is running + io.ray.serve.api.ServeControllerClient client = io.ray.serve.api.Serve.getGlobalClient(); + io.ray.serve.generated.StatusOverview status = client.getServeStatus(appName); Assert.assertEquals( - statusCode, - 412, - "Scaling should fail with 412 when external_scaler_enabled=false. Response: " - + responseBody); - Assert.assertTrue( - responseBody.contains("external_scaler_enabled") - || responseBody.contains("ExternalScalerNotEnabledError"), - "Response should contain external_scaler_enabled error message. Response: " - + responseBody); + status.getAppStatus().getStatus(), + io.ray.serve.generated.ApplicationStatus.APPLICATION_STATUS_RUNNING, + "Application should be in RUNNING status"); + } catch (Exception e) { + throw new RuntimeException("Failed to verify external_scaler_enabled flag", e); } } } diff --git a/python/ray/serve/_private/controller.py b/python/ray/serve/_private/controller.py index be85ac1ee42b..380c049e43d6 100644 --- a/python/ray/serve/_private/controller.py +++ b/python/ray/serve/_private/controller.py @@ -1130,6 +1130,20 @@ def get_app_configs(self) -> Dict[str, ServeApplicationSchema]: for app, config in config_checkpoints_dict.items() } + def get_external_scaler_enabled(self, app_name: str) -> bool: + """Get the external_scaler_enabled flag value for an application. + + This is a helper method specifically for Java tests to verify the flag + is correctly set, since Java cannot deserialize Python Pydantic objects. + + Args: + app_name: Name of the application. + + Returns: + True if external_scaler_enabled is set for the application, False otherwise. + """ + return self.application_state_manager.is_external_scaler_enabled(app_name) + def get_all_deployment_statuses(self) -> List[bytes]: """Gets deployment status bytes for all live deployments.""" statuses = self.deployment_state_manager.get_deployment_statuses() From 209a1fde58f12d47ef12101e341f4a0fdfc857e4 Mon Sep 17 00:00:00 2001 From: harshit Date: Sat, 8 Nov 2025 08:39:47 +0000 Subject: [PATCH 12/23] review changes Signed-off-by: harshit --- .../ray/dashboard/modules/serve/serve_head.py | 11 +- .../serve/tests/test_serve_dashboard.py | 5 +- .../ray/serve/_private/application_state.py | 22 ++-- python/ray/serve/_private/controller.py | 4 +- .../tests/unit/test_application_state.py | 122 ++++++++++++++---- 5 files changed, 119 insertions(+), 45 deletions(-) diff --git a/python/ray/dashboard/modules/serve/serve_head.py b/python/ray/dashboard/modules/serve/serve_head.py index 2664727b3e84..fb8940a6db04 100644 --- a/python/ray/dashboard/modules/serve/serve_head.py +++ b/python/ray/dashboard/modules/serve/serve_head.py @@ -253,13 +253,14 @@ async def scale_deployment(self, req: Request) -> Response: 200, ) except Exception as e: - if isinstance( - e.cause, (ExternalScalerNotEnabledError, DeploymentIsBeingDeletedError) - ): + if isinstance(e, DeploymentIsBeingDeletedError): + # From customer's viewpoint, the deployment is deleted instead of being deleted + # as they must have already executed the delete command return self._create_json_response( - {"error": str(e)}, - 412, + {"error": "Deployment is deleted"}, 412 ) + elif isinstance(e, ExternalScalerNotEnabledError): + return self._create_json_response({"error": str(e.cause)}, 412) if isinstance(e, ValueError) and "not found" in str(e): return self._create_json_response( {"error": "Application or Deployment not found"}, 400 diff --git a/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py b/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py index ec1c693f9d5a..4a9ca53dd3eb 100644 --- a/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py +++ b/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py @@ -857,10 +857,7 @@ def __call__(self): ) assert response.status_code == 412 - assert ( - "is being deleted. Scaling operations are not allowed." - in response.json()["error"] - ) + assert "Deployment is deleted" in response.json()["error"] ray.get(signal_actor.send.remote()) diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py index bf1908349ccb..6f50e3c758d7 100644 --- a/python/ray/serve/_private/application_state.py +++ b/python/ray/serve/_private/application_state.py @@ -579,7 +579,7 @@ def apply_deployment_info( def deploy_app( self, deployment_infos: Dict[str, DeploymentInfo], - external_scaler_enabled: bool = False, + external_scaler_enabled: bool, ): """(Re-)deploy the application from list of deployment infos. @@ -1150,9 +1150,7 @@ def deploy_apps( # against during this batch operation. live_route_prefixes[deploy_app_prefix] = name - application_args = name_to_application_args.get( - name, ApplicationArgsProto(external_scaler_enabled=False) - ) + application_args = name_to_application_args.get(name) external_scaler_enabled = application_args.external_scaler_enabled if name not in self._application_states: @@ -1176,7 +1174,12 @@ def deploy_apps( deployment_infos, external_scaler_enabled ) - def deploy_app(self, name: str, deployment_args: List[Dict]) -> None: + def deploy_app( + self, + name: str, + deployment_args: List[Dict], + application_args: ApplicationArgsProto, + ) -> None: """Deploy the specified app to the list of deployment arguments. This function should only be called if the app is being deployed @@ -1185,12 +1188,11 @@ def deploy_app(self, name: str, deployment_args: List[Dict]) -> None: Args: name: application name deployment_args_list: arguments for deploying a list of deployments. - - Raises: - RayServeException: If the list of deployments is trying to - use a route prefix that is already used by another application + application_args: application arguments. """ - self.deploy_apps({name: deployment_args}, {}) + self.deploy_apps( + {name: deployment_args}, {name: application_args.to_proto_bytes()} + ) def apply_app_configs( self, diff --git a/python/ray/serve/_private/controller.py b/python/ray/serve/_private/controller.py index 380c049e43d6..afa4c916c634 100644 --- a/python/ray/serve/_private/controller.py +++ b/python/ray/serve/_private/controller.py @@ -814,7 +814,7 @@ def deploy_application( self, name: str, deployment_args_list: List[bytes], - application_args: bytes = b"", + application_args: bytes, ) -> None: """ Deploy a single application @@ -826,7 +826,7 @@ def deploy_application( """ self.deploy_applications( {name: deployment_args_list}, - {name: application_args} if application_args else {}, + {name: application_args}, ) def apply_config( diff --git a/python/ray/serve/tests/unit/test_application_state.py b/python/ray/serve/tests/unit/test_application_state.py index 978ea0e7c590..12a008c3c840 100644 --- a/python/ray/serve/tests/unit/test_application_state.py +++ b/python/ray/serve/tests/unit/test_application_state.py @@ -36,6 +36,7 @@ from ray.serve.config import AutoscalingConfig from ray.serve.exceptions import RayServeException from ray.serve.generated.serve_pb2 import ( + ApplicationArgs as ApplicationArgsProto, ApplicationStatusInfo as ApplicationStatusInfoProto, StatusOverview as StatusOverviewProto, ) @@ -561,7 +562,8 @@ def test_deploy_and_delete_app(mocked_application_state): { "d1": deployment_info("d1", "/hi"), "d2": deployment_info("d2"), - } + }, + ApplicationArgsProto(external_scaler_enabled=False), ) assert app_state.route_prefix == "/hi" @@ -614,7 +616,10 @@ def test_app_deploy_failed_and_redeploy(mocked_application_state): app_state, deployment_state_manager = mocked_application_state d1_id = DeploymentID(name="d1", app_name="test_app") d2_id = DeploymentID(name="d2", app_name="test_app") - app_state.deploy_app({"d1": deployment_info("d1")}) + app_state.deploy_app( + {"d1": deployment_info("d1")}, + ApplicationArgsProto(external_scaler_enabled=False), + ) assert app_state.status == ApplicationStatus.DEPLOYING # Before status of deployment changes, app should still be DEPLOYING @@ -633,7 +638,10 @@ def test_app_deploy_failed_and_redeploy(mocked_application_state): assert app_state.status == ApplicationStatus.DEPLOY_FAILED assert app_state._status_msg == deploy_failed_msg - app_state.deploy_app({"d1": deployment_info("d1"), "d2": deployment_info("d2")}) + app_state.deploy_app( + {"d1": deployment_info("d1"), "d2": deployment_info("d2")}, + ApplicationArgsProto(external_scaler_enabled=False), + ) assert app_state.status == ApplicationStatus.DEPLOYING assert app_state._status_msg != deploy_failed_msg @@ -665,7 +673,10 @@ def test_app_deploy_failed_and_recover(mocked_application_state): """ app_state, deployment_state_manager = mocked_application_state deployment_id = DeploymentID(name="d1", app_name="test_app") - app_state.deploy_app({"d1": deployment_info("d1")}) + app_state.deploy_app( + {"d1": deployment_info("d1")}, + ApplicationArgsProto(external_scaler_enabled=False), + ) assert app_state.status == ApplicationStatus.DEPLOYING # Before status of deployment changes, app should still be DEPLOYING @@ -697,7 +708,10 @@ def test_app_unhealthy(mocked_application_state): id_a, id_b = DeploymentID(name="a", app_name="test_app"), DeploymentID( name="b", app_name="test_app" ) - app_state.deploy_app({"a": deployment_info("a"), "b": deployment_info("b")}) + app_state.deploy_app( + {"a": deployment_info("a"), "b": deployment_info("b")}, + ApplicationArgsProto(external_scaler_enabled=False), + ) assert app_state.status == ApplicationStatus.DEPLOYING app_state.update() assert app_state.status == ApplicationStatus.DEPLOYING @@ -838,7 +852,11 @@ def test_apply_app_configs_deletes_existing(check_obj_ref_ready_nowait): # Deploy an app via `deploy_app` - should not be affected. a_id = DeploymentID(name="a", app_name="imperative_app") - app_state_manager.deploy_app("imperative_app", [deployment_params("a", "/hi")]) + app_state_manager.deploy_app( + "imperative_app", + [deployment_params("a", "/hi")], + ApplicationArgsProto(external_scaler_enabled=False), + ) imperative_app_state = app_state_manager._application_states["imperative_app"] assert imperative_app_state.api_type == APIType.IMPERATIVE assert imperative_app_state.status == ApplicationStatus.DEPLOYING @@ -886,7 +904,10 @@ def test_redeploy_same_app(mocked_application_state): a_id = DeploymentID(name="a", app_name="test_app") b_id = DeploymentID(name="b", app_name="test_app") c_id = DeploymentID(name="c", app_name="test_app") - app_state.deploy_app({"a": deployment_info("a"), "b": deployment_info("b")}) + app_state.deploy_app( + {"a": deployment_info("a"), "b": deployment_info("b")}, + ApplicationArgsProto(external_scaler_enabled=False), + ) assert app_state.status == ApplicationStatus.DEPLOYING # Update @@ -903,7 +924,10 @@ def test_redeploy_same_app(mocked_application_state): assert app_state.status == ApplicationStatus.RUNNING # Deploy the same app with different deployments - app_state.deploy_app({"b": deployment_info("b"), "c": deployment_info("c")}) + app_state.deploy_app( + {"b": deployment_info("b"), "c": deployment_info("c")}, + ApplicationArgsProto(external_scaler_enabled=False), + ) assert app_state.status == ApplicationStatus.DEPLOYING # Target state should be updated immediately assert "a" not in app_state.target_deployments @@ -927,9 +951,17 @@ def test_deploy_with_route_prefix_conflict(mocked_application_state_manager): """Test that an application with a route prefix conflict fails to deploy""" app_state_manager, _, _ = mocked_application_state_manager - app_state_manager.deploy_app("app1", [deployment_params("a", "/hi")]) + app_state_manager.deploy_app( + "app1", + [deployment_params("a", "/hi")], + ApplicationArgsProto(external_scaler_enabled=False), + ) with pytest.raises(RayServeException): - app_state_manager.deploy_app("app2", [deployment_params("b", "/hi")]) + app_state_manager.deploy_app( + "app2", + [deployment_params("b", "/hi")], + ApplicationArgsProto(external_scaler_enabled=False), + ) def test_deploy_with_renamed_app(mocked_application_state_manager): @@ -943,7 +975,11 @@ def test_deploy_with_renamed_app(mocked_application_state_manager): ) # deploy app1 - app_state_manager.deploy_app("app1", [deployment_params("a", "/url1")]) + app_state_manager.deploy_app( + "app1", + [deployment_params("a", "/url1")], + ApplicationArgsProto(external_scaler_enabled=False), + ) app_state = app_state_manager._application_states["app1"] assert app_state_manager.get_app_status("app1") == ApplicationStatus.DEPLOYING @@ -963,7 +999,11 @@ def test_deploy_with_renamed_app(mocked_application_state_manager): app_state_manager.update() # deploy app2 - app_state_manager.deploy_app("app2", [deployment_params("b", "/url1")]) + app_state_manager.deploy_app( + "app2", + [deployment_params("b", "/url1")], + ApplicationArgsProto(external_scaler_enabled=False), + ) assert app_state_manager.get_app_status("app2") == ApplicationStatus.DEPLOYING app_state_manager.update() @@ -992,7 +1032,9 @@ def test_application_state_recovery(mocked_application_state_manager): # DEPLOY application with deployments {d1, d2} params = deployment_params("d1") - app_state_manager.deploy_app(app_name, [params]) + app_state_manager.deploy_app( + app_name, [params], ApplicationArgsProto(external_scaler_enabled=False) + ) app_state = app_state_manager._application_states[app_name] assert app_state.status == ApplicationStatus.DEPLOYING @@ -1048,7 +1090,9 @@ def test_recover_during_update(mocked_application_state_manager): # DEPLOY application with deployment "d1" params = deployment_params("d1") - app_state_manager.deploy_app(app_name, [params]) + app_state_manager.deploy_app( + app_name, [params], ApplicationArgsProto(external_scaler_enabled=False) + ) app_state = app_state_manager._application_states[app_name] assert app_state.status == ApplicationStatus.DEPLOYING @@ -1061,7 +1105,9 @@ def test_recover_during_update(mocked_application_state_manager): # Deploy new version of "d1" (this auto generates new random version) params2 = deployment_params("d1") - app_state_manager.deploy_app(app_name, [params2]) + app_state_manager.deploy_app( + app_name, [params2], ApplicationArgsProto(external_scaler_enabled=False) + ) assert app_state.status == ApplicationStatus.DEPLOYING # In real code this checkpoint would be done by the caller of the deploys @@ -1119,7 +1165,9 @@ def test_is_ready_for_shutdown(mocked_application_state_manager): # DEPLOY application with deployment "d1" params = deployment_params(deployment_name) - app_state_manager.deploy_app(app_name, [params]) + app_state_manager.deploy_app( + app_name, [params], ApplicationArgsProto(external_scaler_enabled=False) + ) app_state = app_state_manager._application_states[app_name] assert app_state.status == ApplicationStatus.DEPLOYING @@ -1459,7 +1507,9 @@ def test_should_autoscale_without_autoscaling_deployments( d1_id = DeploymentID(name="d1", app_name="test_app") d1_params = deployment_params("d1", "/hi") # No autoscaling config - app_state_manager.deploy_app("test_app", [d1_params]) + app_state_manager.deploy_app( + "test_app", [d1_params], ApplicationArgsProto(external_scaler_enabled=False) + ) app_state_manager.update() deployment_state_manager.set_deployment_healthy(d1_id) app_state_manager.update() @@ -1683,7 +1733,11 @@ def test_autoscaling_with_mixed_deployment_types( ) d2_params = deployment_params("d2") # No autoscaling config - app_state_manager.deploy_app("test_app", [d1_params, d2_params]) + app_state_manager.deploy_app( + "test_app", + [d1_params, d2_params], + ApplicationArgsProto(external_scaler_enabled=False), + ) app_state_manager.update() deployment_state_manager.set_deployment_healthy(d1_id) @@ -1783,7 +1837,11 @@ def test_autoscale_multiple_apps_independent( ) app1_d2_params = deployment_params("d2", autoscaling_config=autoscaling_config) - app_state_manager.deploy_app("app1", [app1_d1_params, app1_d2_params]) + app_state_manager.deploy_app( + "app1", + [app1_d1_params, app1_d2_params], + ApplicationArgsProto(external_scaler_enabled=False), + ) app_state_manager.update() deployment_state_manager.set_deployment_healthy(app1_d1_id) deployment_state_manager.set_deployment_healthy(app1_d2_id) @@ -1797,7 +1855,11 @@ def test_autoscale_multiple_apps_independent( ) app2_d2_params = deployment_params("d2", autoscaling_config=autoscaling_config) - app_state_manager.deploy_app("app2", [app2_d1_params, app2_d2_params]) + app_state_manager.deploy_app( + "app2", + [app2_d1_params, app2_d2_params], + ApplicationArgsProto(external_scaler_enabled=False), + ) app_state_manager.update() deployment_state_manager.set_deployment_healthy(app2_d1_id) deployment_state_manager.set_deployment_healthy(app2_d2_id) @@ -1986,7 +2048,9 @@ def test_autoscale_single_deployment_in_app(self, mocked_application_state_manag "d1", "/hi", autoscaling_config=autoscaling_config ) - app_state_manager.deploy_app("test_app", [d1_params]) + app_state_manager.deploy_app( + "test_app", [d1_params], ApplicationArgsProto(external_scaler_enabled=False) + ) app_state_manager.update() deployment_state_manager.set_deployment_healthy(d1_id) app_state_manager.update() @@ -2105,7 +2169,11 @@ def test_autoscale_many_deployments_in_app(self, mocked_application_state_manage deployment_params(f"d{i}", autoscaling_config=autoscaling_config) ) - app_state_manager.deploy_app("test_app", deployment_params_list) + app_state_manager.deploy_app( + "test_app", + deployment_params_list, + ApplicationArgsProto(external_scaler_enabled=False), + ) app_state_manager.update() # Mark all as healthy @@ -2184,7 +2252,9 @@ def test_autoscale_with_min_equals_max_replicas( "d1", "/hi", autoscaling_config=autoscaling_config ) - app_state_manager.deploy_app("test_app", [d1_params]) + app_state_manager.deploy_app( + "test_app", [d1_params], ApplicationArgsProto(external_scaler_enabled=False) + ) app_state_manager.update() deployment_state_manager.set_deployment_healthy(d1_id) app_state_manager.update() @@ -2281,7 +2351,11 @@ def _deploy_test_deployments( ) d2_params = deployment_params("d2", autoscaling_config=autoscaling_config) - app_state_manager.deploy_app("test_app", [d1_params, d2_params]) + app_state_manager.deploy_app( + "test_app", + [d1_params, d2_params], + ApplicationArgsProto(external_scaler_enabled=False), + ) app_state_manager.update() deployment_state_manager.set_deployment_healthy(d1_id) From c87d1828689eaa0781991683f64db01ad2b2128f Mon Sep 17 00:00:00 2001 From: harshit Date: Sat, 8 Nov 2025 09:07:43 +0000 Subject: [PATCH 13/23] review changes Signed-off-by: harshit --- python/ray/serve/_private/application_state.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py index 6f50e3c758d7..d0a7416b50c3 100644 --- a/python/ray/serve/_private/application_state.py +++ b/python/ray/serve/_private/application_state.py @@ -1190,9 +1190,7 @@ def deploy_app( deployment_args_list: arguments for deploying a list of deployments. application_args: application arguments. """ - self.deploy_apps( - {name: deployment_args}, {name: application_args.to_proto_bytes()} - ) + self.deploy_apps({name: deployment_args}, {name: application_args}) def apply_app_configs( self, From f7e2f94fe66fb8917c1e594dc28b20b3826a39c6 Mon Sep 17 00:00:00 2001 From: harshit Date: Tue, 11 Nov 2025 05:30:50 +0000 Subject: [PATCH 14/23] add more tests Signed-off-by: harshit --- python/ray/serve/tests/test_api.py | 50 ++++ .../tests/unit/test_application_state.py | 224 ++++++++++++++++++ python/ray/serve/tests/unit/test_build_app.py | 41 ++++ 3 files changed, 315 insertions(+) diff --git a/python/ray/serve/tests/test_api.py b/python/ray/serve/tests/test_api.py index 80955eeff12f..b1d928b039f9 100644 --- a/python/ray/serve/tests/test_api.py +++ b/python/ray/serve/tests/test_api.py @@ -1219,6 +1219,56 @@ def __init__(self, counter): wait_for_condition(lambda: ray.get(counter.get_count.remote()) == 9) +def test_run_with_external_scaler_enabled(serve_instance): + """Test that serve.run correctly passes external_scaler_enabled parameter. + + This test verifies that when serve.run is called with external_scaler_enabled=True + or external_scaler_enabled=False, the application state manager correctly stores + the external_scaler_enabled value. + """ + controller = serve_instance._controller + + @serve.deployment + class Model: + def __call__(self): + return "model response" + + # Test with external_scaler_enabled=True + handle = serve.run( + Model.bind(), + name="app_with_scaler", + route_prefix="/with_scaler", + external_scaler_enabled=True, + ) + assert handle.remote().result() == "model response" + + # Verify that external_scaler_enabled is set to True + assert ray.get(controller.get_external_scaler_enabled.remote("app_with_scaler")) is True + + # Test with external_scaler_enabled=False (explicit) + handle = serve.run( + Model.bind(), + name="app_without_scaler", + route_prefix="/without_scaler", + external_scaler_enabled=False, + ) + assert handle.remote().result() == "model response" + + # Verify that external_scaler_enabled is set to False + assert ray.get(controller.get_external_scaler_enabled.remote("app_without_scaler")) is False + + # Test with default value (should be False) + handle = serve.run( + Model.bind(), + name="app_default", + route_prefix="/default", + ) + assert handle.remote().result() == "model response" + + # Verify that external_scaler_enabled defaults to False + assert ray.get(controller.get_external_scaler_enabled.remote("app_default")) is False + + if __name__ == "__main__": import sys diff --git a/python/ray/serve/tests/unit/test_application_state.py b/python/ray/serve/tests/unit/test_application_state.py index 12a008c3c840..f4f247f45416 100644 --- a/python/ray/serve/tests/unit/test_application_state.py +++ b/python/ray/serve/tests/unit/test_application_state.py @@ -898,6 +898,69 @@ def test_apply_app_configs_deletes_existing(check_obj_ref_ready_nowait): assert app3_state.status == ApplicationStatus.DEPLOYING +@patch( + "ray.serve._private.application_state.get_app_code_version", + Mock(return_value="123"), +) +@patch("ray.serve._private.application_state.build_serve_application", Mock()) +@patch("ray.get", Mock(return_value=(None, [deployment_params("d1", "/route1")], None))) +@patch("ray.serve._private.application_state.check_obj_ref_ready_nowait") +def test_apply_app_configs_with_external_scaler_enabled(check_obj_ref_ready_nowait): + """Test that apply_app_configs correctly sets external_scaler_enabled. + + This test verifies that when apply_app_configs is called with app configs + that have external_scaler_enabled=True or False, the ApplicationState is + correctly initialized with the appropriate external_scaler_enabled value. + """ + kv_store = MockKVStore() + deployment_state_manager = MockDeploymentStateManager(kv_store) + app_state_manager = ApplicationStateManager( + deployment_state_manager, + AutoscalingStateManager(), + MockEndpointState(), + kv_store, + LoggingConfig(), + ) + + # Deploy app with external_scaler_enabled=True + app_config_with_scaler = ServeApplicationSchema( + name="app_with_scaler", + import_path="fa.ke", + route_prefix="/with_scaler", + external_scaler_enabled=True, + ) + + # Deploy app with external_scaler_enabled=False (default) + app_config_without_scaler = ServeApplicationSchema( + name="app_without_scaler", + import_path="fa.ke", + route_prefix="/without_scaler", + external_scaler_enabled=False, + ) + + # Apply both configs + app_state_manager.apply_app_configs([app_config_with_scaler, app_config_without_scaler]) + + # Verify that external_scaler_enabled is correctly set for both apps + assert app_state_manager.is_external_scaler_enabled("app_with_scaler") is True + assert app_state_manager.is_external_scaler_enabled("app_without_scaler") is False + + # Verify the internal state is also correct + app_state_with_scaler = app_state_manager._application_states["app_with_scaler"] + app_state_without_scaler = app_state_manager._application_states["app_without_scaler"] + assert app_state_with_scaler.external_scaler_enabled is True + assert app_state_without_scaler.external_scaler_enabled is False + + # Simulate the build task completing + check_obj_ref_ready_nowait.return_value = True + app_state_with_scaler.update() + app_state_without_scaler.update() + + # After update, external_scaler_enabled should still be preserved + assert app_state_manager.is_external_scaler_enabled("app_with_scaler") is True + assert app_state_manager.is_external_scaler_enabled("app_without_scaler") is False + + def test_redeploy_same_app(mocked_application_state): """Test redeploying same application with updated deployments.""" app_state, deployment_state_manager = mocked_application_state @@ -3106,5 +3169,166 @@ def test_autoscaling_with_deployment_level_configs( ) # Our policy scales to 3 +def test_is_external_scaler_enabled(mocked_application_state_manager): + """Test is_external_scaler_enabled returns correct value based on app config. + + Test that is_external_scaler_enabled returns True when an app is deployed with + external_scaler_enabled=True, False when deployed with external_scaler_enabled=False, + and False for non-existent apps. + """ + app_state_manager, _, _ = mocked_application_state_manager + + # Deploy app with external_scaler_enabled=True + app_state_manager.deploy_app( + "app_with_external_scaler", + [deployment_params("deployment1", "/route1")], + ApplicationArgsProto(external_scaler_enabled=True), + ) + + # Deploy app with external_scaler_enabled=False + app_state_manager.deploy_app( + "app_without_external_scaler", + [deployment_params("deployment2", "/route2")], + ApplicationArgsProto(external_scaler_enabled=False), + ) + + # Test that is_external_scaler_enabled returns True for app with external scaler enabled + assert app_state_manager.is_external_scaler_enabled("app_with_external_scaler") is True + + # Test that is_external_scaler_enabled returns False for app without external scaler + assert app_state_manager.is_external_scaler_enabled("app_without_external_scaler") is False + + # Test that is_external_scaler_enabled returns False for non-existent app + assert app_state_manager.is_external_scaler_enabled("non_existent_app") is False + + +def test_deploy_apps_with_external_scaler_enabled(mocked_application_state_manager): + """Test that deploy_apps correctly uses external_scaler_enabled from name_to_application_args. + + This test verifies that when deploy_apps is called with name_to_application_args + containing external_scaler_enabled values, the ApplicationState is correctly + initialized with the appropriate external_scaler_enabled value for each app. + """ + ( + app_state_manager, + deployment_state_manager, + kv_store, + ) = mocked_application_state_manager + + # Deploy multiple apps with different external_scaler_enabled settings + name_to_deployment_args = { + "app_with_scaler": [deployment_params("d1", "/with_scaler")], + "app_without_scaler": [deployment_params("d2", "/without_scaler")], + "app_default": [deployment_params("d3", "/default")], + } + + name_to_application_args = { + "app_with_scaler": ApplicationArgsProto(external_scaler_enabled=True), + "app_without_scaler": ApplicationArgsProto(external_scaler_enabled=False), + "app_default": ApplicationArgsProto(external_scaler_enabled=False), + } + + # Call deploy_apps + app_state_manager.deploy_apps(name_to_deployment_args, name_to_application_args) + + # Verify that external_scaler_enabled is correctly set for each app + assert app_state_manager.is_external_scaler_enabled("app_with_scaler") is True + assert app_state_manager.is_external_scaler_enabled("app_without_scaler") is False + assert app_state_manager.is_external_scaler_enabled("app_default") is False + + # Verify the internal state is also correct + app_state_with_scaler = app_state_manager._application_states["app_with_scaler"] + app_state_without_scaler = app_state_manager._application_states[ + "app_without_scaler" + ] + app_state_default = app_state_manager._application_states["app_default"] + + assert app_state_with_scaler.external_scaler_enabled is True + assert app_state_without_scaler.external_scaler_enabled is False + assert app_state_default.external_scaler_enabled is False + + # Verify that all apps are in the correct state + assert app_state_with_scaler.status == ApplicationStatus.DEPLOYING + assert app_state_without_scaler.status == ApplicationStatus.DEPLOYING + assert app_state_default.status == ApplicationStatus.DEPLOYING + + +def test_external_scaler_enabled_recovery_from_checkpoint( + mocked_application_state_manager, +): + """Test that external_scaler_enabled is correctly recovered from checkpoint. + + This test verifies that after a controller crash and recovery, the + external_scaler_enabled flag is correctly restored from the checkpoint + for both apps with external_scaler_enabled=True and external_scaler_enabled=False. + """ + ( + app_state_manager, + deployment_state_manager, + kv_store, + ) = mocked_application_state_manager + + app_name_with_scaler = "app_with_external_scaler" + app_name_without_scaler = "app_without_external_scaler" + deployment_id_with_scaler = DeploymentID(name="d1", app_name=app_name_with_scaler) + deployment_id_without_scaler = DeploymentID( + name="d2", app_name=app_name_without_scaler + ) + + # Deploy app with external_scaler_enabled=True + app_state_manager.deploy_app( + app_name_with_scaler, + [deployment_params("d1")], + ApplicationArgsProto(external_scaler_enabled=True), + ) + + # Deploy app with external_scaler_enabled=False + app_state_manager.deploy_app( + app_name_without_scaler, + [deployment_params("d2")], + ApplicationArgsProto(external_scaler_enabled=False), + ) + + # Verify initial state + assert app_state_manager.is_external_scaler_enabled(app_name_with_scaler) is True + assert app_state_manager.is_external_scaler_enabled(app_name_without_scaler) is False + + # Make deployments healthy and update + app_state_manager.update() + deployment_state_manager.set_deployment_healthy(deployment_id_with_scaler) + deployment_state_manager.set_deployment_healthy(deployment_id_without_scaler) + app_state_manager.update() + + # Save checkpoint + app_state_manager.save_checkpoint() + + # Simulate controller crash - create new managers with the same kv_store + new_deployment_state_manager = MockDeploymentStateManager(kv_store) + new_app_state_manager = ApplicationStateManager( + new_deployment_state_manager, + AutoscalingStateManager(), + MockEndpointState(), + kv_store, + LoggingConfig(), + ) + + # Verify that external_scaler_enabled is correctly recovered from checkpoint + assert new_app_state_manager.is_external_scaler_enabled(app_name_with_scaler) is True + assert ( + new_app_state_manager.is_external_scaler_enabled(app_name_without_scaler) + is False + ) + + # Verify the internal state is also correct + app_state_with_scaler = new_app_state_manager._application_states[ + app_name_with_scaler + ] + app_state_without_scaler = new_app_state_manager._application_states[ + app_name_without_scaler + ] + assert app_state_with_scaler.external_scaler_enabled is True + assert app_state_without_scaler.external_scaler_enabled is False + + if __name__ == "__main__": sys.exit(pytest.main(["-v", "-s", __file__])) diff --git a/python/ray/serve/tests/unit/test_build_app.py b/python/ray/serve/tests/unit/test_build_app.py index f51a41a6f57a..200742d5f77c 100644 --- a/python/ray/serve/tests/unit/test_build_app.py +++ b/python/ray/serve/tests/unit/test_build_app.py @@ -480,5 +480,46 @@ class Outer: ) +def test_external_scaler_enabled(): + """Test that external_scaler_enabled is correctly set in BuiltApplication. + + This test verifies that when build_app is called with external_scaler_enabled=True + or external_scaler_enabled=False, the resulting BuiltApplication object correctly + stores the external_scaler_enabled value. + """ + + @serve.deployment + class D: + pass + + app = D.bind() + + # Test with external_scaler_enabled=True + built_app_with_scaler: BuiltApplication = build_app( + app, + name="app-with-scaler", + external_scaler_enabled=True, + ) + assert built_app_with_scaler.external_scaler_enabled is True + assert built_app_with_scaler.name == "app-with-scaler" + + # Test with external_scaler_enabled=False (explicit) + built_app_without_scaler: BuiltApplication = build_app( + app, + name="app-without-scaler", + external_scaler_enabled=False, + ) + assert built_app_without_scaler.external_scaler_enabled is False + assert built_app_without_scaler.name == "app-without-scaler" + + # Test with default value (should be False) + built_app_default: BuiltApplication = build_app( + app, + name="app-default", + ) + assert built_app_default.external_scaler_enabled is False + assert built_app_default.name == "app-default" + + if __name__ == "__main__": sys.exit(pytest.main(["-v", "-s", __file__])) From 2fed45262ab47c69972c799ad8d23f26e794f01a Mon Sep 17 00:00:00 2001 From: harshit Date: Tue, 11 Nov 2025 05:52:43 +0000 Subject: [PATCH 15/23] add more tests Signed-off-by: harshit --- .../serve/tests/test_serve_dashboard.py | 89 +++++++++++++++++++ python/ray/serve/_private/controller.py | 1 + python/ray/serve/schema.py | 3 + 3 files changed, 93 insertions(+) diff --git a/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py b/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py index 4a9ca53dd3eb..1ae1dd910441 100644 --- a/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py +++ b/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py @@ -1224,5 +1224,94 @@ def app_running(): assert "test_app" in serve_details.applications +@pytest.mark.skipif( + sys.platform == "darwin" and not TEST_ON_DARWIN, reason="Flaky on OSX." +) +def test_get_serve_instance_details_external_scaler_enabled(ray_start_stop): + """ + Test that external_scaler_enabled is correctly returned in the API response. + + This test verifies that when applications are deployed with different + external_scaler_enabled values, the /api/serve/applications/ endpoint + correctly returns the external_scaler_enabled field for each application. + """ + world_import_path = "ray.serve.tests.test_config_files.world.DagNode" + + config = { + "applications": [ + { + "name": "app_with_scaler", + "route_prefix": "/with_scaler", + "import_path": world_import_path, + "external_scaler_enabled": True, + }, + { + "name": "app_without_scaler", + "route_prefix": "/without_scaler", + "import_path": world_import_path, + "external_scaler_enabled": False, + }, + ], + } + + deploy_config_multi_app(config, SERVE_HEAD_URL) + + def both_apps_running(): + response = requests.get(SERVE_HEAD_URL, timeout=15) + assert response.status_code == 200 + serve_details = ServeInstanceDetails(**response.json()) + return ( + len(serve_details.applications) == 2 + and serve_details.applications["app_with_scaler"].status + == ApplicationStatus.RUNNING + and serve_details.applications["app_without_scaler"].status + == ApplicationStatus.RUNNING + ) + + wait_for_condition(both_apps_running, timeout=15) + + # Verify both apps have correct external_scaler_enabled values + response = requests.get(SERVE_HEAD_URL, timeout=15) + assert response.status_code == 200 + serve_details = ServeInstanceDetails(**response.json()) + assert len(serve_details.applications) == 2 + assert "app_with_scaler" in serve_details.applications + assert "app_without_scaler" in serve_details.applications + assert serve_details.applications["app_with_scaler"].external_scaler_enabled is True + assert serve_details.applications["app_without_scaler"].external_scaler_enabled is False + + # Test default value (when external_scaler_enabled is not specified) + config_default = { + "applications": [ + { + "name": "app_default", + "route_prefix": "/default", + "import_path": world_import_path, + } + ], + } + + deploy_config_multi_app(config_default, SERVE_HEAD_URL) + + def app_default_running(): + response = requests.get(SERVE_HEAD_URL, timeout=15) + assert response.status_code == 200 + serve_details = ServeInstanceDetails(**response.json()) + return ( + len(serve_details.applications) == 1 + and serve_details.applications["app_default"].status + == ApplicationStatus.RUNNING + ) + + wait_for_condition(app_default_running, timeout=15) + + # Verify default value is False + response = requests.get(SERVE_HEAD_URL, timeout=15) + assert response.status_code == 200 + serve_details = ServeInstanceDetails(**response.json()) + assert "app_default" in serve_details.applications + assert serve_details.applications["app_default"].external_scaler_enabled is False + + if __name__ == "__main__": sys.exit(pytest.main(["-v", __file__])) diff --git a/python/ray/serve/_private/controller.py b/python/ray/serve/_private/controller.py index afa4c916c634..d1f12dc9aae9 100644 --- a/python/ray/serve/_private/controller.py +++ b/python/ray/serve/_private/controller.py @@ -1032,6 +1032,7 @@ def get_serve_instance_details(self, source: Optional[APIType] = None) -> Dict: deployments=self.application_state_manager.list_deployment_details( app_name ), + external_scaler_enabled=self.application_state_manager.is_external_scaler_enabled(app_name), ) # NOTE(zcin): We use exclude_unset here because we explicitly and intentionally diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index e65ff87142d7..5036fa86d662 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -1204,6 +1204,9 @@ class ApplicationDetails(BaseModel, extra=Extra.forbid, frozen=True): deployments: Dict[str, DeploymentDetails] = Field( description="Details about the deployments in this application." ) + external_scaler_enabled: bool = Field( + description="Whether external autoscaling is enabled for this application.", + ) application_details_route_prefix_format = validator( "route_prefix", allow_reuse=True From ebeb0d204a18a96fc147262d7c88e1772c247cd1 Mon Sep 17 00:00:00 2001 From: harshit Date: Tue, 11 Nov 2025 06:43:24 +0000 Subject: [PATCH 16/23] lint changes Signed-off-by: harshit --- .../dashboard/modules/serve/tests/test_serve_dashboard.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py b/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py index 1ae1dd910441..7205eb35ea8a 100644 --- a/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py +++ b/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py @@ -1278,7 +1278,10 @@ def both_apps_running(): assert "app_with_scaler" in serve_details.applications assert "app_without_scaler" in serve_details.applications assert serve_details.applications["app_with_scaler"].external_scaler_enabled is True - assert serve_details.applications["app_without_scaler"].external_scaler_enabled is False + assert ( + serve_details.applications["app_without_scaler"].external_scaler_enabled + is False + ) # Test default value (when external_scaler_enabled is not specified) config_default = { From 539d48629cbfbb6d0f89bc1f447aa1c5ab150dda Mon Sep 17 00:00:00 2001 From: harshit Date: Tue, 11 Nov 2025 19:11:55 +0000 Subject: [PATCH 17/23] indentation changes Signed-off-by: harshit --- python/ray/serve/_private/controller.py | 4 ++- python/ray/serve/tests/test_api.py | 14 ++++++++--- python/ray/serve/tests/test_controller.py | 1 + .../tests/unit/test_application_state.py | 25 ++++++++++++++----- 4 files changed, 34 insertions(+), 10 deletions(-) diff --git a/python/ray/serve/_private/controller.py b/python/ray/serve/_private/controller.py index d1f12dc9aae9..c90ac2aaaca7 100644 --- a/python/ray/serve/_private/controller.py +++ b/python/ray/serve/_private/controller.py @@ -1032,7 +1032,9 @@ def get_serve_instance_details(self, source: Optional[APIType] = None) -> Dict: deployments=self.application_state_manager.list_deployment_details( app_name ), - external_scaler_enabled=self.application_state_manager.is_external_scaler_enabled(app_name), + external_scaler_enabled=self.application_state_manager.is_external_scaler_enabled( + app_name + ), ) # NOTE(zcin): We use exclude_unset here because we explicitly and intentionally diff --git a/python/ray/serve/tests/test_api.py b/python/ray/serve/tests/test_api.py index b1d928b039f9..efdf805c41cf 100644 --- a/python/ray/serve/tests/test_api.py +++ b/python/ray/serve/tests/test_api.py @@ -1243,7 +1243,10 @@ def __call__(self): assert handle.remote().result() == "model response" # Verify that external_scaler_enabled is set to True - assert ray.get(controller.get_external_scaler_enabled.remote("app_with_scaler")) is True + assert ( + ray.get(controller.get_external_scaler_enabled.remote("app_with_scaler")) + is True + ) # Test with external_scaler_enabled=False (explicit) handle = serve.run( @@ -1255,7 +1258,10 @@ def __call__(self): assert handle.remote().result() == "model response" # Verify that external_scaler_enabled is set to False - assert ray.get(controller.get_external_scaler_enabled.remote("app_without_scaler")) is False + assert ( + ray.get(controller.get_external_scaler_enabled.remote("app_without_scaler")) + is False + ) # Test with default value (should be False) handle = serve.run( @@ -1266,7 +1272,9 @@ def __call__(self): assert handle.remote().result() == "model response" # Verify that external_scaler_enabled defaults to False - assert ray.get(controller.get_external_scaler_enabled.remote("app_default")) is False + assert ( + ray.get(controller.get_external_scaler_enabled.remote("app_default")) is False + ) if __name__ == "__main__": diff --git a/python/ray/serve/tests/test_controller.py b/python/ray/serve/tests/test_controller.py index 01f410999930..187688aacd19 100644 --- a/python/ray/serve/tests/test_controller.py +++ b/python/ray/serve/tests/test_controller.py @@ -216,6 +216,7 @@ def autoscaling_app(): ], } }, + "external_scaler_enabled": False, } }, "target_capacity": None, diff --git a/python/ray/serve/tests/unit/test_application_state.py b/python/ray/serve/tests/unit/test_application_state.py index 85fdb97666ae..3374e3e8ea61 100644 --- a/python/ray/serve/tests/unit/test_application_state.py +++ b/python/ray/serve/tests/unit/test_application_state.py @@ -942,7 +942,9 @@ def test_apply_app_configs_with_external_scaler_enabled(check_obj_ref_ready_nowa ) # Apply both configs - app_state_manager.apply_app_configs([app_config_with_scaler, app_config_without_scaler]) + app_state_manager.apply_app_configs( + [app_config_with_scaler, app_config_without_scaler] + ) # Verify that external_scaler_enabled is correctly set for both apps assert app_state_manager.is_external_scaler_enabled("app_with_scaler") is True @@ -950,7 +952,9 @@ def test_apply_app_configs_with_external_scaler_enabled(check_obj_ref_ready_nowa # Verify the internal state is also correct app_state_with_scaler = app_state_manager._application_states["app_with_scaler"] - app_state_without_scaler = app_state_manager._application_states["app_without_scaler"] + app_state_without_scaler = app_state_manager._application_states[ + "app_without_scaler" + ] assert app_state_with_scaler.external_scaler_enabled is True assert app_state_without_scaler.external_scaler_enabled is False @@ -3196,10 +3200,15 @@ def test_is_external_scaler_enabled(mocked_application_state_manager): ) # Test that is_external_scaler_enabled returns True for app with external scaler enabled - assert app_state_manager.is_external_scaler_enabled("app_with_external_scaler") is True + assert ( + app_state_manager.is_external_scaler_enabled("app_with_external_scaler") is True + ) # Test that is_external_scaler_enabled returns False for app without external scaler - assert app_state_manager.is_external_scaler_enabled("app_without_external_scaler") is False + assert ( + app_state_manager.is_external_scaler_enabled("app_without_external_scaler") + is False + ) # Test that is_external_scaler_enabled returns False for non-existent app assert app_state_manager.is_external_scaler_enabled("non_existent_app") is False @@ -3294,7 +3303,9 @@ def test_external_scaler_enabled_recovery_from_checkpoint( # Verify initial state assert app_state_manager.is_external_scaler_enabled(app_name_with_scaler) is True - assert app_state_manager.is_external_scaler_enabled(app_name_without_scaler) is False + assert ( + app_state_manager.is_external_scaler_enabled(app_name_without_scaler) is False + ) # Make deployments healthy and update app_state_manager.update() @@ -3316,7 +3327,9 @@ def test_external_scaler_enabled_recovery_from_checkpoint( ) # Verify that external_scaler_enabled is correctly recovered from checkpoint - assert new_app_state_manager.is_external_scaler_enabled(app_name_with_scaler) is True + assert ( + new_app_state_manager.is_external_scaler_enabled(app_name_with_scaler) is True + ) assert ( new_app_state_manager.is_external_scaler_enabled(app_name_without_scaler) is False From dbe3c0f1d54c79517bdc8fa95599db1ffe63527b Mon Sep 17 00:00:00 2001 From: harshit Date: Wed, 12 Nov 2025 04:40:20 +0000 Subject: [PATCH 18/23] fix tests Signed-off-by: harshit --- python/ray/serve/tests/unit/test_schema.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/ray/serve/tests/unit/test_schema.py b/python/ray/serve/tests/unit/test_schema.py index 49ac259700a0..2cbda98eac8e 100644 --- a/python/ray/serve/tests/unit/test_schema.py +++ b/python/ray/serve/tests/unit/test_schema.py @@ -931,6 +931,7 @@ def test_serve_instance_details_is_json_serializable(): "replicas": [], } }, + "external_scaler_enabled": False, } }, )._get_user_facing_json_serializable_dict(exclude_unset=True) @@ -965,6 +966,7 @@ def test_serve_instance_details_is_json_serializable(): "replicas": [], } }, + "external_scaler_enabled": False, } }, } From 80e8cef5a0c47f271060fbf0cab796f7eba7906e Mon Sep 17 00:00:00 2001 From: harshit Date: Mon, 17 Nov 2025 05:33:28 +0000 Subject: [PATCH 19/23] remove setting default value Signed-off-by: harshit --- python/ray/serve/_private/application_state.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py index 5fa249b163c7..379d9aa36558 100644 --- a/python/ray/serve/_private/application_state.py +++ b/python/ray/serve/_private/application_state.py @@ -224,7 +224,7 @@ class ApplicationTargetState: deleting: bool api_type: APIType serialized_application_autoscaling_policy_def: Optional[bytes] - external_scaler_enabled: bool = False + external_scaler_enabled: bool class ApplicationState: From 49eb96ab6bb60195995cc2fb27fcd8cd5f1e69ec Mon Sep 17 00:00:00 2001 From: harshit Date: Mon, 24 Nov 2025 07:21:20 +0000 Subject: [PATCH 20/23] review changes Signed-off-by: harshit --- .../ray/serve/deployment/DeploymentTest.java | 2 +- .../ray/dashboard/modules/serve/serve_head.py | 4 +- .../ray/serve/_private/application_state.py | 7 +--- python/ray/serve/_private/client.py | 6 +-- python/ray/serve/_private/controller.py | 25 ++++++------ python/ray/serve/_private/exceptions.py | 2 +- python/ray/serve/schema.py | 2 +- .../tests/unit/test_application_state.py | 40 +++++++++---------- 8 files changed, 42 insertions(+), 46 deletions(-) diff --git a/java/serve/src/test/java/io/ray/serve/deployment/DeploymentTest.java b/java/serve/src/test/java/io/ray/serve/deployment/DeploymentTest.java index 20d7b8c8b058..3c47034e4dfe 100644 --- a/java/serve/src/test/java/io/ray/serve/deployment/DeploymentTest.java +++ b/java/serve/src/test/java/io/ray/serve/deployment/DeploymentTest.java @@ -212,7 +212,7 @@ public void externalScalerDisabledTest() { * * In production, when external_scaler_enabled=false, attempts to scale via the * HTTP dashboard API would return 412 (Precondition Failed) with an - * ExternalScalerNotEnabledError. However, as explained in externalScalerEnabledTest, + * ExternalScalerDisabledError. However, as explained in externalScalerEnabledTest, * we cannot reliably test the HTTP API in this test environment due to dashboard * and controller lifecycle management issues. * diff --git a/python/ray/dashboard/modules/serve/serve_head.py b/python/ray/dashboard/modules/serve/serve_head.py index fb8940a6db04..013ad85991ea 100644 --- a/python/ray/dashboard/modules/serve/serve_head.py +++ b/python/ray/dashboard/modules/serve/serve_head.py @@ -205,7 +205,7 @@ async def scale_deployment(self, req: Request) -> Response: from ray.serve._private.common import DeploymentID from ray.serve._private.exceptions import ( DeploymentIsBeingDeletedError, - ExternalScalerNotEnabledError, + ExternalScalerDisabledError, ) from ray.serve.schema import ScaleDeploymentRequest @@ -259,7 +259,7 @@ async def scale_deployment(self, req: Request) -> Response: return self._create_json_response( {"error": "Deployment is deleted"}, 412 ) - elif isinstance(e, ExternalScalerNotEnabledError): + elif isinstance(e, ExternalScalerDisabledError): return self._create_json_response({"error": str(e.cause)}, 412) if isinstance(e, ValueError) and "not found" in str(e): return self._create_json_response( diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py index 379d9aa36558..ece38e783ba3 100644 --- a/python/ray/serve/_private/application_state.py +++ b/python/ray/serve/_private/application_state.py @@ -1292,7 +1292,7 @@ def get_ingress_deployment_name(self, name: str) -> Optional[str]: def get_app_source(self, name: str) -> APIType: return self._application_states[name].api_type - def is_external_scaler_enabled(self, app_name: str) -> bool: + def get_external_scaler_enabled(self, app_name: str) -> bool: """Check if external scaler is enabled for the application. Args: @@ -1301,10 +1301,7 @@ def is_external_scaler_enabled(self, app_name: str) -> bool: Returns: True if external_scaler_enabled is set for the application, False otherwise. """ - if self.does_app_exist(app_name): - return self._application_states[app_name].external_scaler_enabled - - return False + return self.does_app_exist(app_name) and self._application_states[app_name].external_scaler_enabled def list_app_statuses( self, source: Optional[APIType] = None diff --git a/python/ray/serve/_private/client.py b/python/ray/serve/_private/client.py index 283936883ac6..431b4f1ef900 100644 --- a/python/ray/serve/_private/client.py +++ b/python/ray/serve/_private/client.py @@ -332,7 +332,7 @@ def deploy_applications( wait_for_applications_running: bool = True, ) -> List[DeploymentHandle]: name_to_deployment_args_list = {} - name_to_application_args_list = {} + name_to_application_args = {} for app in built_apps: deployment_args_list = [] for deployment in app.deployments: @@ -372,7 +372,7 @@ def deploy_applications( application_args_proto.external_scaler_enabled = app.external_scaler_enabled name_to_deployment_args_list[app.name] = deployment_args_list - name_to_application_args_list[ + name_to_application_args[ app.name ] = application_args_proto.SerializeToString() @@ -381,7 +381,7 @@ def deploy_applications( ray.get( self._controller.deploy_applications.remote( - name_to_deployment_args_list, name_to_application_args_list + name_to_deployment_args_list, name_to_application_args ) ) diff --git a/python/ray/serve/_private/controller.py b/python/ray/serve/_private/controller.py index c90ac2aaaca7..98024f71b06c 100644 --- a/python/ray/serve/_private/controller.py +++ b/python/ray/serve/_private/controller.py @@ -46,7 +46,7 @@ from ray.serve._private.deployment_info import DeploymentInfo from ray.serve._private.deployment_state import DeploymentStateManager from ray.serve._private.endpoint_state import EndpointState -from ray.serve._private.exceptions import ExternalScalerNotEnabledError +from ray.serve._private.exceptions import ExternalScalerDisabledError from ray.serve._private.grpc_util import set_proxy_default_grpc_options from ray.serve._private.http_util import ( configure_http_options_with_defaults, @@ -762,7 +762,7 @@ def shutdown(self): def deploy_applications( self, name_to_deployment_args_list: Dict[str, List[bytes]], - name_to_application_args_list: Dict[str, bytes], + name_to_application_args: Dict[str, bytes], ) -> None: """ Takes in a list of dictionaries that contain deployment arguments. @@ -774,7 +774,7 @@ def deploy_applications( where each item in the list is bytes representing the serialized protobuf `DeploymentArgs` object. `DeploymentArgs` contains all the information for the single deployment. - name_to_application_args_list: Dictionary mapping application names to serialized + name_to_application_args: Dictionary mapping application names to serialized application arguments, where each item is bytes representing the serialized protobuf `ApplicationArgs` object. `ApplicationArgs` contains the information for the application. @@ -798,14 +798,14 @@ def deploy_applications( ) name_to_deployment_args[name] = deployment_args_deserialized - name_to_application_args = {} - for name, application_args_bytes in name_to_application_args_list.items(): - name_to_application_args[name] = ApplicationArgs.FromString( + name_to_application_args_deserialized = {} + for name, application_args_bytes in name_to_application_args.items(): + name_to_application_args_deserialized[name] = ApplicationArgs.FromString( application_args_bytes ) self.application_state_manager.deploy_apps( - name_to_deployment_args, name_to_application_args + name_to_deployment_args, name_to_application_args_deserialized ) self.application_state_manager.save_checkpoint() @@ -963,8 +963,7 @@ def update_deployment_replicas( target_num_replicas: The new target number of replicas. Raises: - ExternalScalerNotEnabledError: If external_scaler_enabled is not set to True - for the application. + ExternalScalerDisabledError: If external_scaler_enabled is set to False for the application. """ # Check if external scaler is enabled for this application @@ -972,8 +971,8 @@ def update_deployment_replicas( if not self.application_state_manager.does_app_exist(app_name): raise ValueError(f"Application '{app_name}' not found") - if not self.application_state_manager.is_external_scaler_enabled(app_name): - raise ExternalScalerNotEnabledError( + if not self.application_state_manager.get_external_scaler_enabled(app_name): + raise ExternalScalerDisabledError( f"Cannot update replicas for deployment '{deployment_id.name}' in " f"application '{app_name}'. The external scaling API can only be used " f"when 'external_scaler_enabled' is set to true in the application " @@ -1032,7 +1031,7 @@ def get_serve_instance_details(self, source: Optional[APIType] = None) -> Dict: deployments=self.application_state_manager.list_deployment_details( app_name ), - external_scaler_enabled=self.application_state_manager.is_external_scaler_enabled( + external_scaler_enabled=self.application_state_manager.get_external_scaler_enabled( app_name ), ) @@ -1145,7 +1144,7 @@ def get_external_scaler_enabled(self, app_name: str) -> bool: Returns: True if external_scaler_enabled is set for the application, False otherwise. """ - return self.application_state_manager.is_external_scaler_enabled(app_name) + return self.application_state_manager.get_external_scaler_enabled(app_name) def get_all_deployment_statuses(self) -> List[bytes]: """Gets deployment status bytes for all live deployments.""" diff --git a/python/ray/serve/_private/exceptions.py b/python/ray/serve/_private/exceptions.py index 47d7513663a7..b16f973161b8 100644 --- a/python/ray/serve/_private/exceptions.py +++ b/python/ray/serve/_private/exceptions.py @@ -4,7 +4,7 @@ class DeploymentIsBeingDeletedError(Exception): pass -class ExternalScalerNotEnabledError(Exception): +class ExternalScalerDisabledError(Exception): """Raised when the external scaling API is used but external_scaler_enabled is False.""" pass diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index 0323486f0887..d21f7fafee71 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -1205,7 +1205,7 @@ class ApplicationDetails(BaseModel, extra=Extra.forbid, frozen=True): description="Details about the deployments in this application." ) external_scaler_enabled: bool = Field( - description="Whether external autoscaling is enabled for this application.", + description="Whether external scaling is enabled for this application.", ) application_details_route_prefix_format = validator( diff --git a/python/ray/serve/tests/unit/test_application_state.py b/python/ray/serve/tests/unit/test_application_state.py index 3374e3e8ea61..5966fffc147b 100644 --- a/python/ray/serve/tests/unit/test_application_state.py +++ b/python/ray/serve/tests/unit/test_application_state.py @@ -947,8 +947,8 @@ def test_apply_app_configs_with_external_scaler_enabled(check_obj_ref_ready_nowa ) # Verify that external_scaler_enabled is correctly set for both apps - assert app_state_manager.is_external_scaler_enabled("app_with_scaler") is True - assert app_state_manager.is_external_scaler_enabled("app_without_scaler") is False + assert app_state_manager.get_external_scaler_enabled("app_with_scaler") is True + assert app_state_manager.get_external_scaler_enabled("app_without_scaler") is False # Verify the internal state is also correct app_state_with_scaler = app_state_manager._application_states["app_with_scaler"] @@ -964,8 +964,8 @@ def test_apply_app_configs_with_external_scaler_enabled(check_obj_ref_ready_nowa app_state_without_scaler.update() # After update, external_scaler_enabled should still be preserved - assert app_state_manager.is_external_scaler_enabled("app_with_scaler") is True - assert app_state_manager.is_external_scaler_enabled("app_without_scaler") is False + assert app_state_manager.get_external_scaler_enabled("app_with_scaler") is True + assert app_state_manager.get_external_scaler_enabled("app_without_scaler") is False def test_redeploy_same_app(mocked_application_state): @@ -3176,10 +3176,10 @@ def test_autoscaling_with_deployment_level_configs( ) # Our policy scales to 3 -def test_is_external_scaler_enabled(mocked_application_state_manager): - """Test is_external_scaler_enabled returns correct value based on app config. +def test_get_external_scaler_enabled(mocked_application_state_manager): + """Test get_external_scaler_enabled returns correct value based on app config. - Test that is_external_scaler_enabled returns True when an app is deployed with + Test that get_external_scaler_enabled returns True when an app is deployed with external_scaler_enabled=True, False when deployed with external_scaler_enabled=False, and False for non-existent apps. """ @@ -3199,19 +3199,19 @@ def test_is_external_scaler_enabled(mocked_application_state_manager): ApplicationArgsProto(external_scaler_enabled=False), ) - # Test that is_external_scaler_enabled returns True for app with external scaler enabled + # Test that get_external_scaler_enabled returns True for app with external scaler enabled assert ( - app_state_manager.is_external_scaler_enabled("app_with_external_scaler") is True + app_state_manager.get_external_scaler_enabled("app_with_external_scaler") is True ) - # Test that is_external_scaler_enabled returns False for app without external scaler + # Test that get_external_scaler_enabled returns False for app without external scaler assert ( - app_state_manager.is_external_scaler_enabled("app_without_external_scaler") + app_state_manager.get_external_scaler_enabled("app_without_external_scaler") is False ) - # Test that is_external_scaler_enabled returns False for non-existent app - assert app_state_manager.is_external_scaler_enabled("non_existent_app") is False + # Test that get_external_scaler_enabled returns False for non-existent app + assert app_state_manager.get_external_scaler_enabled("non_existent_app") is False def test_deploy_apps_with_external_scaler_enabled(mocked_application_state_manager): @@ -3244,9 +3244,9 @@ def test_deploy_apps_with_external_scaler_enabled(mocked_application_state_manag app_state_manager.deploy_apps(name_to_deployment_args, name_to_application_args) # Verify that external_scaler_enabled is correctly set for each app - assert app_state_manager.is_external_scaler_enabled("app_with_scaler") is True - assert app_state_manager.is_external_scaler_enabled("app_without_scaler") is False - assert app_state_manager.is_external_scaler_enabled("app_default") is False + assert app_state_manager.get_external_scaler_enabled("app_with_scaler") is True + assert app_state_manager.get_external_scaler_enabled("app_without_scaler") is False + assert app_state_manager.get_external_scaler_enabled("app_default") is False # Verify the internal state is also correct app_state_with_scaler = app_state_manager._application_states["app_with_scaler"] @@ -3302,9 +3302,9 @@ def test_external_scaler_enabled_recovery_from_checkpoint( ) # Verify initial state - assert app_state_manager.is_external_scaler_enabled(app_name_with_scaler) is True + assert app_state_manager.get_external_scaler_enabled(app_name_with_scaler) is True assert ( - app_state_manager.is_external_scaler_enabled(app_name_without_scaler) is False + app_state_manager.get_external_scaler_enabled(app_name_without_scaler) is False ) # Make deployments healthy and update @@ -3328,10 +3328,10 @@ def test_external_scaler_enabled_recovery_from_checkpoint( # Verify that external_scaler_enabled is correctly recovered from checkpoint assert ( - new_app_state_manager.is_external_scaler_enabled(app_name_with_scaler) is True + new_app_state_manager.get_external_scaler_enabled(app_name_with_scaler) is True ) assert ( - new_app_state_manager.is_external_scaler_enabled(app_name_without_scaler) + new_app_state_manager.get_external_scaler_enabled(app_name_without_scaler) is False ) From c0aa4037cab28aa4eca046c0d395551d6d754a81 Mon Sep 17 00:00:00 2001 From: harshit Date: Mon, 24 Nov 2025 08:09:59 +0000 Subject: [PATCH 21/23] lint changes Signed-off-by: harshit --- python/ray/serve/_private/application_state.py | 5 ++++- python/ray/serve/tests/unit/test_application_state.py | 3 ++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py index 5fcd4585ea3d..e44ac3859e0c 100644 --- a/python/ray/serve/_private/application_state.py +++ b/python/ray/serve/_private/application_state.py @@ -1343,7 +1343,10 @@ def get_external_scaler_enabled(self, app_name: str) -> bool: Returns: True if external_scaler_enabled is set for the application, False otherwise. """ - return self.does_app_exist(app_name) and self._application_states[app_name].external_scaler_enabled + return ( + self.does_app_exist(app_name) + and self._application_states[app_name].external_scaler_enabled + ) def list_app_statuses( self, source: Optional[APIType] = None diff --git a/python/ray/serve/tests/unit/test_application_state.py b/python/ray/serve/tests/unit/test_application_state.py index 0e0271003f91..eedeb55343a7 100644 --- a/python/ray/serve/tests/unit/test_application_state.py +++ b/python/ray/serve/tests/unit/test_application_state.py @@ -3208,7 +3208,8 @@ def test_get_external_scaler_enabled(mocked_application_state_manager): # Test that get_external_scaler_enabled returns True for app with external scaler enabled assert ( - app_state_manager.get_external_scaler_enabled("app_with_external_scaler") is True + app_state_manager.get_external_scaler_enabled("app_with_external_scaler") + is True ) # Test that get_external_scaler_enabled returns False for app without external scaler From 7838218dd17fce58c1c73fb137165510a7383c7e Mon Sep 17 00:00:00 2001 From: harshit Date: Mon, 24 Nov 2025 12:53:58 +0000 Subject: [PATCH 22/23] fix tests Signed-off-by: harshit --- .../tests/unit/test_application_state.py | 27 ++++++++++++------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/python/ray/serve/tests/unit/test_application_state.py b/python/ray/serve/tests/unit/test_application_state.py index eedeb55343a7..92a144d36365 100644 --- a/python/ray/serve/tests/unit/test_application_state.py +++ b/python/ray/serve/tests/unit/test_application_state.py @@ -3363,7 +3363,7 @@ def test_build_dag_single_deployment(self, mocked_application_state): d1_id = DeploymentID(name="d1", app_name="test_app") # Deploy single deployment - app_state.deploy_app({"d1": deployment_info("d1", "/hi")}) + app_state.deploy_app({"d1": deployment_info("d1", "/hi")}, ApplicationArgsProto(external_scaler_enabled=False)) app_state.update() deployment_state_manager.set_deployment_healthy(d1_id) @@ -3392,7 +3392,8 @@ def test_build_dag_multiple_deployments_no_deps(self, mocked_application_state): "d1": deployment_info("d1", "/hi"), "d2": deployment_info("d2"), "d3": deployment_info("d3"), - } + }, + ApplicationArgsProto(external_scaler_enabled=False), ) app_state.update() @@ -3432,7 +3433,8 @@ def test_build_dag_with_outbound_dependencies(self, mocked_application_state): "d1": deployment_info("d1", "/hi"), "d2": deployment_info("d2"), "d3": deployment_info("d3"), - } + }, + ApplicationArgsProto(external_scaler_enabled=False), ) app_state.update() @@ -3483,7 +3485,8 @@ def test_build_dag_with_cross_app_dependencies(self, mocked_application_state): { "d1": deployment_info("d1", "/hi"), "d2": deployment_info("d2"), - } + }, + ApplicationArgsProto(external_scaler_enabled=False), ) app_state.update() @@ -3528,7 +3531,8 @@ def test_dag_updates_on_redeploy(self, mocked_application_state): { "d1": deployment_info("d1", "/hi"), "d2": deployment_info("d2"), - } + }, + ApplicationArgsProto(external_scaler_enabled=False), ) app_state.update() @@ -3548,7 +3552,8 @@ def test_dag_updates_on_redeploy(self, mocked_application_state): { "d2": deployment_info("d2", "/hi"), "d3": deployment_info("d3"), - } + }, + ApplicationArgsProto(external_scaler_enabled=False), ) app_state.update() @@ -3578,7 +3583,8 @@ def test_dag_with_changing_dependencies(self, mocked_application_state): { "d1": deployment_info("d1", "/hi"), "d2": deployment_info("d2"), - } + }, + ApplicationArgsProto(external_scaler_enabled=False), ) app_state.update() @@ -3614,7 +3620,7 @@ def test_application_state_manager_get_deployment_topology( d1_id = DeploymentID(name="d1", app_name="test_app") # Deploy app - app_state_manager.deploy_app("test_app", [deployment_params("d1", "/hi")]) + app_state_manager.deploy_app("test_app", [deployment_params("d1", "/hi")], ApplicationArgsProto(external_scaler_enabled=False)) app_state_manager.update() deployment_state_manager.set_deployment_healthy(d1_id) @@ -3653,6 +3659,7 @@ def test_dag_with_multiple_apps(self, mocked_application_state_manager): deployment_params("d1", "/app1"), deployment_params("d2"), ], + ApplicationArgsProto(external_scaler_enabled=False), ) app_state_manager.update() deployment_state_manager.set_deployment_healthy(app1_d1_id) @@ -3668,6 +3675,7 @@ def test_dag_with_multiple_apps(self, mocked_application_state_manager): deployment_params("d1", "/app2"), deployment_params("d2"), ], + ApplicationArgsProto(external_scaler_enabled=False), ) app_state_manager.update() deployment_state_manager.set_deployment_healthy(app2_d1_id) @@ -3725,7 +3733,8 @@ def test_dag_with_complex_dependency_graph(self, mocked_application_state): "worker2": deployment_info("worker2"), "database": deployment_info("database"), "cache": deployment_info("cache"), - } + }, + ApplicationArgsProto(external_scaler_enabled=False), ) app_state.update() From ad525172f761ee92f2f0e7a59c4bc3790835ed05 Mon Sep 17 00:00:00 2001 From: harshit Date: Mon, 24 Nov 2025 13:40:29 +0000 Subject: [PATCH 23/23] review changes Signed-off-by: harshit --- python/ray/serve/tests/unit/test_application_state.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/python/ray/serve/tests/unit/test_application_state.py b/python/ray/serve/tests/unit/test_application_state.py index 92a144d36365..e5b14e369d4a 100644 --- a/python/ray/serve/tests/unit/test_application_state.py +++ b/python/ray/serve/tests/unit/test_application_state.py @@ -3363,7 +3363,10 @@ def test_build_dag_single_deployment(self, mocked_application_state): d1_id = DeploymentID(name="d1", app_name="test_app") # Deploy single deployment - app_state.deploy_app({"d1": deployment_info("d1", "/hi")}, ApplicationArgsProto(external_scaler_enabled=False)) + app_state.deploy_app( + {"d1": deployment_info("d1", "/hi")}, + ApplicationArgsProto(external_scaler_enabled=False), + ) app_state.update() deployment_state_manager.set_deployment_healthy(d1_id) @@ -3620,7 +3623,11 @@ def test_application_state_manager_get_deployment_topology( d1_id = DeploymentID(name="d1", app_name="test_app") # Deploy app - app_state_manager.deploy_app("test_app", [deployment_params("d1", "/hi")], ApplicationArgsProto(external_scaler_enabled=False)) + app_state_manager.deploy_app( + "test_app", + [deployment_params("d1", "/hi")], + ApplicationArgsProto(external_scaler_enabled=False), + ) app_state_manager.update() deployment_state_manager.set_deployment_healthy(d1_id)