Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
31 commits
Select commit Hold shift + click to select a range
9c98194
add external scaler enabled flag
harshit-anyscale Oct 15, 2025
a685eb7
fix tests
harshit-anyscale Oct 16, 2025
7c1ef8f
review changes
harshit-anyscale Oct 27, 2025
0f6b1b0
merge master
harshit-anyscale Oct 27, 2025
826b69c
merge master
harshit-anyscale Oct 27, 2025
60beb9f
review changes
harshit-anyscale Oct 27, 2025
dee83f1
fix bug
harshit-anyscale Oct 27, 2025
e4d315d
Merge branch 'master' into add-external-scaler-c3
harshit-anyscale Oct 27, 2025
9229b97
review changes
harshit-anyscale Oct 28, 2025
4d30ab8
review changes
harshit-anyscale Oct 28, 2025
0492aed
Merge branch 'master' into add-external-scaler-c3
harshit-anyscale Oct 28, 2025
c92bb27
fix tests
harshit-anyscale Oct 28, 2025
83fd9d1
changes in java side code
harshit-anyscale Oct 29, 2025
cf556db
fix java tests
harshit-anyscale Nov 8, 2025
209a1fd
review changes
harshit-anyscale Nov 8, 2025
c87d182
review changes
harshit-anyscale Nov 8, 2025
f7e2f94
add more tests
harshit-anyscale Nov 11, 2025
2fed452
add more tests
harshit-anyscale Nov 11, 2025
ebeb0d2
lint changes
harshit-anyscale Nov 11, 2025
fda546e
Merge branch 'master' into add-external-scaler-c3
harshit-anyscale Nov 11, 2025
539d486
indentation changes
harshit-anyscale Nov 11, 2025
dbe3c0f
fix tests
harshit-anyscale Nov 12, 2025
8a4b589
Merge branch 'master' into add-external-scaler-c3
harshit-anyscale Nov 12, 2025
80e8cef
remove setting default value
harshit-anyscale Nov 17, 2025
c6c1b4c
Merge branch 'master' into add-external-scaler-c3
harshit-anyscale Nov 17, 2025
49eb96a
review changes
harshit-anyscale Nov 24, 2025
810fbcf
merge master
harshit-anyscale Nov 24, 2025
c0aa403
lint changes
harshit-anyscale Nov 24, 2025
7838218
fix tests
harshit-anyscale Nov 24, 2025
ad52517
review changes
harshit-anyscale Nov 24, 2025
f7ef18f
Merge branch 'master' into add-external-scaler-c3
harshit-anyscale Nov 24, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
15 changes: 12 additions & 3 deletions java/serve/src/main/java/io/ray/serve/api/Serve.java
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}

/**
Expand All @@ -320,14 +320,17 @@ 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(
Application target,
boolean blocking,
String name,
String routePrefix,
Map<String, String> config) {
Map<String, String> config,
boolean externalScalerEnabled) {

if (StringUtils.isBlank(name)) {
throw new RayServeException("Application name must a non-empty string.");
Expand Down Expand Up @@ -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);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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<Deployment> deployments,
String ingressDeploymentName,
boolean blocking) {
boolean blocking,
boolean externalScalerEnabled) {

Object[] deploymentArgsArray = new Object[deployments.size()];

Expand All @@ -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();

Expand Down
151 changes: 151 additions & 0 deletions java/serve/src/test/java/io/ray/serve/deployment/DeploymentTest.java
Original file line number Diff line number Diff line change
@@ -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;
Expand Down Expand Up @@ -121,4 +122,154 @@ public void userConfigTest() {
Serve.getAppHandle(deploymentName).method("call").remote("6").result(), "echo_6_new");
// TOOD update user config
}

@Test
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 =
Serve.deployment()
.setName(deploymentName)
.setDeploymentDef(ExampleEchoDeployment.class.getName())
.setNumReplicas(1)
.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());

// Verify the controller actor exists in the correct namespace
java.util.Optional<io.ray.api.BaseActorHandle> 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(
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() {
/*
* 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
* 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.
*
* 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 =
Serve.deployment()
.setName(deploymentName)
.setDeploymentDef(ExampleEchoDeployment.class.getName())
.setNumReplicas(1)
.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());

// Verify the controller actor exists in the correct namespace
java.util.Optional<io.ray.api.BaseActorHandle> 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(
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);
}
}
}
16 changes: 10 additions & 6 deletions python/ray/dashboard/modules/serve/serve_head.py
Original file line number Diff line number Diff line change
Expand Up @@ -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,
ExternalScalerDisabledError,
)
from ray.serve.schema import ScaleDeploymentRequest

# Extract path parameters
Expand Down Expand Up @@ -250,13 +253,14 @@ async def scale_deployment(self, req: Request) -> Response:
200,
)
except Exception as e:
if isinstance(e.cause, 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(
# 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"},
412,
{"error": "Deployment is deleted"}, 412
)
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(
{"error": "Application or Deployment not found"}, 400
Expand Down
Loading