From 395950940eaadd702f86128da5b77a20fc8d2dcc Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 9 Aug 2023 11:19:35 -0700 Subject: [PATCH] [RayService] Revisit the conditions under which a RayService is considered unhealthy and the default threshold (#1293) Revisit the conditions under which a RayService is considered unhealthy and the default threshold --- docs/guidance/rayservice-troubleshooting.md | 2 +- docs/guidance/rayservice.md | 15 +- .../apis/ray/v1alpha1/rayservice_types.go | 5 + .../samples/ray-service.autoscaler.yaml | 4 +- .../ray-service.custom-serve-service.yaml | 4 +- .../samples/ray-service.different-port.yaml | 4 +- .../config/samples/ray-service.mobilenet.yaml | 4 +- .../samples/ray-service.stable-diffusion.yaml | 4 +- .../samples/ray_v1alpha1_rayservice.yaml | 4 +- .../controllers/ray/rayservice_controller.go | 54 ++++--- .../ray/rayservice_controller_test.go | 38 +++-- .../ray/rayservice_controller_unit_test.go | 136 ++++++++++++++++++ tests/config/ray-service.yaml.template | 2 +- 13 files changed, 218 insertions(+), 58 deletions(-) diff --git a/docs/guidance/rayservice-troubleshooting.md b/docs/guidance/rayservice-troubleshooting.md index 74b361495c..43bc5e6b1c 100644 --- a/docs/guidance/rayservice-troubleshooting.md +++ b/docs/guidance/rayservice-troubleshooting.md @@ -237,7 +237,7 @@ If you consistently encounter this issue, there are several possible causes: # Get \"http://rayservice-sample-raycluster-rqlsl-head-svc.default.svc.cluster.local:52365/api/serve/applications/\": dial tcp 10.96.7.154:52365: connect: connection refused ``` -### Issue 8: A loop of restarting the RayCluster occurs when the Kubernetes cluster runs out of resources. +### Issue 8: A loop of restarting the RayCluster occurs when the Kubernetes cluster runs out of resources. (KubeRay v0.6.1 or earlier) > Note: Currently, the KubeRay operator does not have a clear plan to handle situations where the Kubernetes cluster runs out of resources. Therefore, we recommend ensuring that the Kubernetes cluster has sufficient resources to accommodate the serve application. diff --git a/docs/guidance/rayservice.md b/docs/guidance/rayservice.md index 851b70e932..a1f450dcf4 100644 --- a/docs/guidance/rayservice.md +++ b/docs/guidance/rayservice.md @@ -178,7 +178,6 @@ curl -X POST -H 'Content-Type: application/json' rayservice-sample-serve-svc:800 ``` * `rayservice-sample-serve-svc` is HA in general. It will do traffic routing among all the workers which have Serve deployments and will always try to point to the healthy cluster, even during upgrading or failing cases. -* You can set `serviceUnhealthySecondThreshold` to define the threshold of seconds that the Serve deployments fail. You can also set `deploymentUnhealthySecondThreshold` to define the threshold of seconds that Ray fails to deploy any Serve deployments. ## Step 7: In-place update for Ray Serve applications @@ -263,6 +262,20 @@ curl -X POST -H 'Content-Type: application/json' rayservice-sample-serve-svc:800 # [Expected output]: 8 ``` +### Another two possible scenarios that will trigger a new RayCluster preparation + +> Note: The following behavior is for KubeRay v0.6.2 or newer. +For older versions, please refer to [kuberay#1293](https://github.com/ray-project/kuberay/pull/1293) for more details. + +Not only will the zero downtime upgrade trigger a new RayCluster preparation, but KubeRay will also trigger it if it considers a RayCluster unhealthy. +In the RayService, KubeRay can mark a RayCluster as unhealthy in two possible scenarios. + +* Case 1: The KubeRay operator cannot connect to the dashboard agent on the head Pod for more than the duration defined by the `deploymentUnhealthySecondThreshold` parameter. Both the default value and values in sample YAML files of `deploymentUnhealthySecondThreshold` are 300 seconds. + +* Case 2: The KubeRay operator will mark a RayCluster as unhealthy if the status of a serve application is `DEPLOY_FAILED` or `UNHEALTHY` for a duration exceeding the `serviceUnhealthySecondThreshold` parameter. Both the default value and values in sample YAML files of `serviceUnhealthySecondThreshold` are 900 seconds. + +After KubeRay marks a RayCluster as unhealthy, it initiates the creation of a new RayCluster. Once the new RayCluster is ready, KubeRay redirects network traffic to it, and subsequently deletes the old RayCluster. + ## Step 9: Clean up the Kubernetes cluster ```sh diff --git a/ray-operator/apis/ray/v1alpha1/rayservice_types.go b/ray-operator/apis/ray/v1alpha1/rayservice_types.go index df7c6d7d83..f041426ae8 100644 --- a/ray-operator/apis/ray/v1alpha1/rayservice_types.go +++ b/ray-operator/apis/ray/v1alpha1/rayservice_types.go @@ -23,16 +23,21 @@ const ( ) // These statuses should match Ray Serve's application statuses +// See `enum ApplicationStatus` in https://sourcegraph.com/github.com/ray-project/ray/-/blob/src/ray/protobuf/serve.proto for more details. var ApplicationStatusEnum = struct { NOT_STARTED string DEPLOYING string RUNNING string DEPLOY_FAILED string + DELETING string + UNHEALTHY string }{ NOT_STARTED: "NOT_STARTED", DEPLOYING: "DEPLOYING", RUNNING: "RUNNING", DEPLOY_FAILED: "DEPLOY_FAILED", + DELETING: "DELETING", + UNHEALTHY: "UNHEALTHY", } // These statuses should match Ray Serve's deployment statuses diff --git a/ray-operator/config/samples/ray-service.autoscaler.yaml b/ray-operator/config/samples/ray-service.autoscaler.yaml index 3417272e9a..a06c4c4612 100644 --- a/ray-operator/config/samples/ray-service.autoscaler.yaml +++ b/ray-operator/config/samples/ray-service.autoscaler.yaml @@ -7,8 +7,8 @@ kind: RayService metadata: name: rayservice-sample spec: - serviceUnhealthySecondThreshold: 300 # Config for the health check threshold for service. Default value is 60. - deploymentUnhealthySecondThreshold: 300 # Config for the health check threshold for deployments. Default value is 60. + serviceUnhealthySecondThreshold: 900 # Config for the health check threshold for Ray Serve applications. Default value is 900. + deploymentUnhealthySecondThreshold: 300 # Config for the health check threshold for Ray dashboard agent. Default value is 300. # The workload consists of two applications. The first application checks on an event in the second application. # If the event isn't set, the first application will block on requests until the event is set. So, to test upscaling # we can first send a bunch of requests to the first application, which will trigger Serve autoscaling to bring up diff --git a/ray-operator/config/samples/ray-service.custom-serve-service.yaml b/ray-operator/config/samples/ray-service.custom-serve-service.yaml index ac4238023c..f99d47d005 100644 --- a/ray-operator/config/samples/ray-service.custom-serve-service.yaml +++ b/ray-operator/config/samples/ray-service.custom-serve-service.yaml @@ -7,8 +7,8 @@ kind: RayService metadata: name: rayservice-sample spec: - serviceUnhealthySecondThreshold: 300 # Config for the health check threshold for service. Default value is 60. - deploymentUnhealthySecondThreshold: 300 # Config for the health check threshold for deployments. Default value is 60. + serviceUnhealthySecondThreshold: 900 # Config for the health check threshold for Ray Serve applications. Default value is 900. + deploymentUnhealthySecondThreshold: 300 # Config for the health check threshold for Ray dashboard agent. Default value is 300. serveService: metadata: name: custom-ray-serve-service-name diff --git a/ray-operator/config/samples/ray-service.different-port.yaml b/ray-operator/config/samples/ray-service.different-port.yaml index fbcb975533..7f3893aeef 100644 --- a/ray-operator/config/samples/ray-service.different-port.yaml +++ b/ray-operator/config/samples/ray-service.different-port.yaml @@ -7,8 +7,8 @@ kind: RayService metadata: name: rayservice-sample spec: - serviceUnhealthySecondThreshold: 300 # Config for the health check threshold for service. Default value is 60. - deploymentUnhealthySecondThreshold: 300 # Config for the health check threshold for deployments. Default value is 60. + serviceUnhealthySecondThreshold: 900 # Config for the health check threshold for Ray Serve applications. Default value is 900. + deploymentUnhealthySecondThreshold: 300 # Config for the health check threshold for Ray dashboard agent. Default value is 300. serveConfig: importPath: fruit.deployment_graph runtimeEnv: | diff --git a/ray-operator/config/samples/ray-service.mobilenet.yaml b/ray-operator/config/samples/ray-service.mobilenet.yaml index f91d53bf84..cb5b8eb2b2 100644 --- a/ray-operator/config/samples/ray-service.mobilenet.yaml +++ b/ray-operator/config/samples/ray-service.mobilenet.yaml @@ -3,8 +3,8 @@ kind: RayService metadata: name: rayservice-mobilenet spec: - serviceUnhealthySecondThreshold: 300 # Config for the health check threshold for service. Default value is 60. - deploymentUnhealthySecondThreshold: 300 # Config for the health check threshold for deployments. Default value is 60. + serviceUnhealthySecondThreshold: 900 # Config for the health check threshold for Ray Serve applications. Default value is 900. + deploymentUnhealthySecondThreshold: 300 # Config for the health check threshold for Ray dashboard agent. Default value is 300. serveConfigV2: | applications: - name: mobilenet diff --git a/ray-operator/config/samples/ray-service.stable-diffusion.yaml b/ray-operator/config/samples/ray-service.stable-diffusion.yaml index c04072ae9c..1b00602a35 100644 --- a/ray-operator/config/samples/ray-service.stable-diffusion.yaml +++ b/ray-operator/config/samples/ray-service.stable-diffusion.yaml @@ -3,8 +3,8 @@ kind: RayService metadata: name: stable-diffusion spec: - serviceUnhealthySecondThreshold: 300 # Config for the health check threshold for service. Default value is 60. - deploymentUnhealthySecondThreshold: 300 # Config for the health check threshold for deployments. Default value is 60. + serviceUnhealthySecondThreshold: 900 # Config for the health check threshold for Ray Serve applications. Default value is 900. + deploymentUnhealthySecondThreshold: 300 # Config for the health check threshold for Ray dashboard agent. Default value is 300. serveConfigV2: | applications: - name: stable_diffusion diff --git a/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml b/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml index 471d9ba161..a89a768bf9 100644 --- a/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml +++ b/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml @@ -7,8 +7,8 @@ kind: RayService metadata: name: rayservice-sample spec: - serviceUnhealthySecondThreshold: 300 # Config for the health check threshold for service. Default value is 60. - deploymentUnhealthySecondThreshold: 300 # Config for the health check threshold for deployments. Default value is 60. + serviceUnhealthySecondThreshold: 900 # Config for the health check threshold for Ray Serve applications. Default value is 900. + deploymentUnhealthySecondThreshold: 300 # Config for the health check threshold for Ray dashboard agent. Default value is 300. # serveConfigV2 takes a yaml multi-line scalar, which should be a Ray Serve multi-application config. See https://docs.ray.io/en/latest/serve/multi-app.html. # Only one of serveConfig and serveConfigV2 should be used. serveConfigV2: | diff --git a/ray-operator/controllers/ray/rayservice_controller.go b/ray-operator/controllers/ray/rayservice_controller.go index 4bc73c42a6..68158ad7ee 100644 --- a/ray-operator/controllers/ray/rayservice_controller.go +++ b/ray-operator/controllers/ray/rayservice_controller.go @@ -37,14 +37,14 @@ import ( // This variable is mutable for unit testing purpose. var ( - ServiceUnhealthySecondThreshold = 60.0 // Serve deployment related health check. + ServiceUnhealthySecondThreshold = 900.0 // Serve deployment related health check. ) const ( ServiceDefaultRequeueDuration = 2 * time.Second ServiceRestartRequeueDuration = 10 * time.Second RayClusterDeletionDelayDuration = 60 * time.Second - DeploymentUnhealthySecondThreshold = 60.0 // Dashboard agent related health check. + DeploymentUnhealthySecondThreshold = 300.0 // Dashboard agent related health check. ) // RayServiceReconciler reconciles a RayService object @@ -730,9 +730,12 @@ func (r *RayServiceReconciler) updateServeDeployment(ctx context.Context, raySer // `getAndCheckServeStatus` gets Serve applications' and deployments' statuses, updates health timestamps, // and checks if the RayCluster is overall healthy. It takes as one of its inputs `serveConfigType`, which // is used to decide whether to query the single-application Serve REST API or the multi-application Serve -// REST API. It's return values should be interpreted as: +// REST API. It's return values should be interpreted as: (isHealthy, isReady, err). // -// (Serve app healthy?, Serve app ready?, error if failed to get Serve statuses) +// (1) `isHealthy` is used to determine whether restart the RayCluster or not. +// (2) `isReady` is used to determine whether the Serve applications in the RayCluster are ready to serve incoming traffic or not. +// (3) `err`: If `err` is not nil, it means that KubeRay failed to get Serve application statuses from the dashboard agent. We should take a +// look at dashboard agent rather than Ray Serve applications. func (r *RayServiceReconciler) getAndCheckServeStatus(ctx context.Context, dashboardClient utils.RayDashboardClientInterface, rayServiceServeStatus *rayv1alpha1.RayServiceStatus, serveConfigType utils.RayServeConfigType, unhealthySecondThreshold *int32) (bool, bool, error) { // If the `unhealthySecondThreshold` value is non-nil, then we will use that value. Otherwise, we will use the value ServiceUnhealthySecondThreshold // which can be set in a test. This is used for testing purposes. @@ -789,26 +792,32 @@ func (r *RayServiceReconciler) getAndCheckServeStatus(ctx context.Context, dashb Deployments: make(map[string]rayv1alpha1.ServeDeploymentStatus), } - // Check app status - if app.Status != rayv1alpha1.ApplicationStatusEnum.RUNNING { - // Check previous app status - if prevApplicationStatus.Status != rayv1alpha1.ApplicationStatusEnum.RUNNING { + // `isHealthy` is used to determine whether restart the RayCluster or not. If the serve application is `UNHEALTHY` or `DEPLOY_FAILED` + // for more than `serviceUnhealthySecondThreshold` seconds, then KubeRay will consider the RayCluster unhealthy and prepare a new RayCluster. + if isServeAppUnhealthyOrDeployedFailed(app.Status) { + if isServeAppUnhealthyOrDeployedFailed(prevApplicationStatus.Status) { if prevApplicationStatus.HealthLastUpdateTime != nil { applicationStatus.HealthLastUpdateTime = prevApplicationStatus.HealthLastUpdateTime if time.Since(prevApplicationStatus.HealthLastUpdateTime.Time).Seconds() > serviceUnhealthySecondThreshold { - r.Log.Info("Restart RayCluster", "appName", appName, "restart reason", + r.Log.Info("Restart RayCluster", "appName", appName, "appStatus", app.Status, "restart reason", fmt.Sprintf( - "The status of the serve application %s has not been RUNNING for more than %f seconds. "+ + "The status of the serve application %s has been UNHEALTHY or DEPLOY_FAILED for more than %f seconds. "+ "Hence, KubeRay operator labels the RayCluster unhealthy and will prepare a new RayCluster. ", appName, serviceUnhealthySecondThreshold)) isHealthy = false } } } + } + + // `isReady` is used to determine whether the Serve application is ready or not. The cluster switchover only happens when all Serve + // applications in this RayCluster are ready so that the incoming traffic will not be dropped. Note that if `isHealthy` is false, + // then `isReady` must be false as well. + if app.Status != rayv1alpha1.ApplicationStatusEnum.RUNNING { isReady = false } - // Check deployment statuses + // Copy deployment statuses for deploymentName, deployment := range app.Deployments { deploymentStatus := rayv1alpha1.ServeDeploymentStatus{ Status: deployment.Status, @@ -817,26 +826,21 @@ func (r *RayServiceReconciler) getAndCheckServeStatus(ctx context.Context, dashb HealthLastUpdateTime: &timeNow, } - if deployment.Status != rayv1alpha1.DeploymentStatusEnum.HEALTHY { + if deployment.Status == rayv1alpha1.DeploymentStatusEnum.UNHEALTHY { prevStatus, exist := prevApplicationStatus.Deployments[deploymentName] if exist { - if prevStatus.Status != rayv1alpha1.DeploymentStatusEnum.HEALTHY { + if prevStatus.Status == rayv1alpha1.DeploymentStatusEnum.UNHEALTHY { deploymentStatus.HealthLastUpdateTime = prevStatus.HealthLastUpdateTime - - if !isHealthy || (prevStatus.HealthLastUpdateTime != nil && time.Since(prevStatus.HealthLastUpdateTime.Time).Seconds() > serviceUnhealthySecondThreshold) { - // TODO (kevin85421): Without `!isHealthy`, this `if` statement is almost impossible to be reached because the `HealthLastUpdateTime` of a serve deployment - // is always later than the `HealthLastUpdateTime` of the serve application. Hence, the restart is always triggered by the serve application. If we - // can confirm that `isHealthy = false` is always set by the serve application check, we can remove the `time.Since` check here. + if !isHealthy { r.Log.Info("Restart RayCluster", "deploymentName", deploymentName, "appName", appName, "restart reason", fmt.Sprintf( - "The status of the serve deployment %s or the serve application %s has not been HEALTHY/RUNNING for more than %f seconds. "+ + "The serve application %s has been UNHEALTHY or DEPLOY_FAILED for more than %f seconds. "+ + "This may be caused by the serve deployment %s being UNHEALTHY. "+ "Hence, KubeRay operator labels the RayCluster unhealthy and will prepare a new RayCluster. "+ - "The message of the serve deployment is: %s", deploymentName, appName, serviceUnhealthySecondThreshold, deploymentStatus.Message)) - isHealthy = false + "The message of the serve deployment is: %s", appName, serviceUnhealthySecondThreshold, deploymentName, deploymentStatus.Message)) } } } - isReady = false } applicationStatus.Deployments[deploymentName] = deploymentStatus } @@ -1090,7 +1094,7 @@ func (r *RayServiceReconciler) reconcileServe(ctx context.Context, rayServiceIns } var isHealthy, isReady bool - if isHealthy, isReady, err = r.getAndCheckServeStatus(ctx, rayDashboardClient, rayServiceStatus, r.determineServeConfigType(rayServiceInstance), rayServiceInstance.Spec.DeploymentUnhealthySecondThreshold); err != nil { + if isHealthy, isReady, err = r.getAndCheckServeStatus(ctx, rayDashboardClient, rayServiceStatus, r.determineServeConfigType(rayServiceInstance), rayServiceInstance.Spec.ServiceUnhealthySecondThreshold); err != nil { if !r.updateAndCheckDashboardStatus(rayServiceStatus, false, rayServiceInstance.Spec.DeploymentUnhealthySecondThreshold) { logger.Info("Dashboard is unhealthy, restart the cluster.") r.markRestart(rayServiceInstance) @@ -1214,3 +1218,7 @@ func (r *RayServiceReconciler) isHeadPodRunningAndReady(ctx context.Context, ins return utils.IsRunningAndReady(&podList.Items[0]), nil } + +func isServeAppUnhealthyOrDeployedFailed(appStatus string) bool { + return appStatus == rayv1alpha1.ApplicationStatusEnum.UNHEALTHY || appStatus == rayv1alpha1.ApplicationStatusEnum.DEPLOY_FAILED +} diff --git a/ray-operator/controllers/ray/rayservice_controller_test.go b/ray-operator/controllers/ray/rayservice_controller_test.go index 862b90d6ce..05837d2d0f 100644 --- a/ray-operator/controllers/ray/rayservice_controller_test.go +++ b/ray-operator/controllers/ray/rayservice_controller_test.go @@ -408,7 +408,7 @@ applications: // the RayService controller will consider the active RayCluster as unhealthy and prepare a new RayCluster. orignalServeDeploymentUnhealthySecondThreshold := ServiceUnhealthySecondThreshold ServiceUnhealthySecondThreshold = 5 - fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.UNHEALTHY)) + fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.UNHEALTHY, rayv1alpha1.ApplicationStatusEnum.UNHEALTHY)) Eventually( getPreparingRayClusterNameFunc(ctx, myRayService), time.Second*60, time.Millisecond*500).Should(Not(BeEmpty()), "New pending RayCluster name = %v", myRayService.Status.PendingServiceStatus.RayClusterName) @@ -436,7 +436,7 @@ applications: // (2) The pending RayCluster's Serve Deployments are HEALTHY. updateHeadPodToRunningAndReady(ctx, initialPendingClusterName) ServiceUnhealthySecondThreshold = orignalServeDeploymentUnhealthySecondThreshold - fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.HEALTHY)) + fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.HEALTHY, rayv1alpha1.ApplicationStatusEnum.RUNNING)) Eventually( getPreparingRayClusterNameFunc(ctx, myRayService), time.Second*15, time.Millisecond*500).Should(BeEmpty(), "Pending RayCluster name = %v", myRayService.Status.PendingServiceStatus.RayClusterName) @@ -459,7 +459,7 @@ applications: ServiceUnhealthySecondThreshold = 500 // Change serve status to be unhealthy - fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.UNHEALTHY)) + fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.UNHEALTHY, rayv1alpha1.ApplicationStatusEnum.UNHEALTHY)) // Confirm not switch to a new RayCluster because ServiceUnhealthySecondThreshold is 500 seconds. Consistently( @@ -490,7 +490,7 @@ applications: checkAllDeploymentStatusesUnhealthy(ctx, myRayService), time.Second*3, time.Millisecond*500).Should(BeTrue(), "myRayService status = %v", myRayService.Status) - fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.HEALTHY)) + fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.HEALTHY, rayv1alpha1.ApplicationStatusEnum.RUNNING)) // Confirm not switch to a new RayCluster because ServiceUnhealthySecondThreshold is 500 seconds. Consistently( @@ -514,7 +514,7 @@ applications: // Only update the LastUpdateTime and HealthLastUpdateTime fields in the active RayCluster. oldTime := myRayService.Status.ActiveServiceStatus.Applications[common.DefaultServeAppName].HealthLastUpdateTime.DeepCopy() - fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.HEALTHY)) + fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.HEALTHY, rayv1alpha1.ApplicationStatusEnum.RUNNING)) // Confirm not switch to a new RayCluster Consistently( @@ -559,7 +559,7 @@ applications: // Set deployment statuses to UNHEALTHY orignalServeDeploymentUnhealthySecondThreshold := ServiceUnhealthySecondThreshold ServiceUnhealthySecondThreshold = 5 - fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.UNHEALTHY)) + fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.UNHEALTHY, rayv1alpha1.ApplicationStatusEnum.UNHEALTHY)) Eventually( getPreparingRayClusterNameFunc(ctx, myRayService), @@ -567,7 +567,7 @@ applications: ServiceUnhealthySecondThreshold = orignalServeDeploymentUnhealthySecondThreshold pendingRayClusterName := myRayService.Status.PendingServiceStatus.RayClusterName - fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.HEALTHY)) + fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.HEALTHY, rayv1alpha1.ApplicationStatusEnum.RUNNING)) updateHeadPodToRunningAndReady(ctx, pendingRayClusterName) Eventually( @@ -582,7 +582,7 @@ applications: initialClusterName, _ := getRayClusterNameFunc(ctx, myRayService)() // The cluster shouldn't switch until deployments are finished updating - fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.UPDATING)) + fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.UPDATING, rayv1alpha1.ApplicationStatusEnum.DEPLOYING)) err := retry.RetryOnConflict(retry.DefaultRetry, func() error { Eventually( @@ -605,7 +605,7 @@ applications: time.Second*5, time.Millisecond*500).Should(Equal(initialClusterName), "My current RayCluster name = %v", myRayService.Status.ActiveServiceStatus.RayClusterName) // The cluster should switch once the deployments are finished updating - fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.HEALTHY)) + fakeRayDashboardClient.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.HEALTHY, rayv1alpha1.ApplicationStatusEnum.RUNNING)) updateHeadPodToRunningAndReady(ctx, pendingRayClusterName) Eventually( @@ -626,10 +626,10 @@ applications: Expect(err).NotTo(HaveOccurred(), "failed to update test RayService serve config") // Set multi-application status to healthy. - healthyStatus := generateServeStatus(rayv1alpha1.DeploymentStatusEnum.HEALTHY) + healthyStatus := generateServeStatus(rayv1alpha1.DeploymentStatusEnum.HEALTHY, rayv1alpha1.ApplicationStatusEnum.RUNNING) fakeRayDashboardClient.SetMultiApplicationStatuses(map[string]*utils.ServeApplicationStatus{testServeAppName: &healthyStatus}) // Set single-application status to unhealthy. - unhealthyStatus := generateServeStatus(rayv1alpha1.DeploymentStatusEnum.UNHEALTHY) + unhealthyStatus := generateServeStatus(rayv1alpha1.DeploymentStatusEnum.UNHEALTHY, rayv1alpha1.ApplicationStatusEnum.UNHEALTHY) fakeRayDashboardClient.SetSingleApplicationStatus(unhealthyStatus) // The status should remain healthy, because we have set serveConfigTypeForTesting to MULTI_APP @@ -649,34 +649,32 @@ applications: func prepareFakeRayDashboardClient() utils.FakeRayDashboardClient { client := utils.FakeRayDashboardClient{} - client.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.HEALTHY)) + client.SetSingleApplicationStatus(generateServeStatus(rayv1alpha1.DeploymentStatusEnum.HEALTHY, rayv1alpha1.ApplicationStatusEnum.RUNNING)) return client } -func generateServeStatus(status string) utils.ServeApplicationStatus { - appStatus := utils.ServeApplicationStatus{ - Status: rayv1alpha1.ApplicationStatusEnum.RUNNING, +func generateServeStatus(deploymentStatus string, applicationStatus string) utils.ServeApplicationStatus { + return utils.ServeApplicationStatus{ + Status: applicationStatus, Deployments: map[string]utils.ServeDeploymentStatus{ "shallow": { Name: "shallow", - Status: status, + Status: deploymentStatus, Message: "", }, "deep": { Name: "deep", - Status: status, + Status: deploymentStatus, Message: "", }, "one": { Name: "one", - Status: status, + Status: deploymentStatus, Message: "", }, }, } - - return appStatus } func getRayClusterNameFunc(ctx context.Context, rayService *rayv1alpha1.RayService) func() (string, error) { diff --git a/ray-operator/controllers/ray/rayservice_controller_unit_test.go b/ray-operator/controllers/ray/rayservice_controller_unit_test.go index 55cc599663..9913b3f0b9 100644 --- a/ray-operator/controllers/ray/rayservice_controller_unit_test.go +++ b/ray-operator/controllers/ray/rayservice_controller_unit_test.go @@ -5,6 +5,7 @@ import ( "fmt" "reflect" "testing" + "time" rayv1alpha1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" "github.com/ray-project/kuberay/ray-operator/controllers/ray/common" @@ -422,3 +423,138 @@ func TestFetchHeadServiceURL(t *testing.T) { assert.Nil(t, err, "Fail to fetch head service url") assert.Equal(t, fmt.Sprintf("test-cluster-head-svc.%s.svc.cluster.local:%d", namespace, dashboardPort), url, "Head service url is not correct") } + +func TestGetAndCheckServeStatus(t *testing.T) { + // Create a new scheme with CRDs, Pod, Service schemes. + newScheme := runtime.NewScheme() + _ = rayv1alpha1.AddToScheme(newScheme) + _ = corev1.AddToScheme(newScheme) + + // Initialize a fake client with newScheme and runtimeObjects. + runtimeObjects := []runtime.Object{} + fakeClient := clientFake.NewClientBuilder().WithScheme(newScheme).WithRuntimeObjects(runtimeObjects...).Build() + + // Initialize RayService reconciler. + ctx := context.TODO() + r := RayServiceReconciler{ + Client: fakeClient, + Recorder: &record.FakeRecorder{}, + Scheme: scheme.Scheme, + Log: ctrl.Log.WithName("controllers").WithName("RayService"), + } + serviceUnhealthySecondThreshold := int32(ServiceUnhealthySecondThreshold) // The threshold is 900 seconds by default. + serveAppName := "serve-app-1" + + // Test 1: There is no pre-existing RayServiceStatus in the RayService CR. Create a new Ray Serve application, and the application is still deploying. + dashboardClient := initFakeDashboardClient(serveAppName, rayv1alpha1.DeploymentStatusEnum.UPDATING, rayv1alpha1.ApplicationStatusEnum.DEPLOYING) + prevRayServiceStatus := rayv1alpha1.RayServiceStatus{ + Applications: map[string]rayv1alpha1.AppStatus{}, + } + isHealthy, isReady, err := r.getAndCheckServeStatus(ctx, dashboardClient, &prevRayServiceStatus, utils.MULTI_APP, &serviceUnhealthySecondThreshold) + assert.Nil(t, err) + assert.True(t, isHealthy) + assert.False(t, isReady) + + // Test 2: The Ray Serve application takes more than `serviceUnhealthySecondThreshold` seconds to be "RUNNING". + // This may happen when `runtime_env` installation takes a long time or the cluster does not have enough resources + // for autoscaling. Note that the cluster will not be marked as unhealthy if the application is still deploying. + dashboardClient = initFakeDashboardClient(serveAppName, rayv1alpha1.DeploymentStatusEnum.UPDATING, rayv1alpha1.ApplicationStatusEnum.DEPLOYING) + prevRayServiceStatus = rayv1alpha1.RayServiceStatus{ + Applications: map[string]rayv1alpha1.AppStatus{ + serveAppName: { + Status: rayv1alpha1.ApplicationStatusEnum.DEPLOYING, + HealthLastUpdateTime: &metav1.Time{Time: metav1.Now().Add(-time.Second * time.Duration(serviceUnhealthySecondThreshold+1))}, + }, + }, + } + isHealthy, isReady, err = r.getAndCheckServeStatus(ctx, dashboardClient, &prevRayServiceStatus, utils.MULTI_APP, &serviceUnhealthySecondThreshold) + assert.Nil(t, err) + assert.True(t, isHealthy) + assert.False(t, isReady) + + // Test 3: The Ray Serve application finishes the deployment process and becomes "RUNNING". + dashboardClient = initFakeDashboardClient(serveAppName, rayv1alpha1.DeploymentStatusEnum.HEALTHY, rayv1alpha1.ApplicationStatusEnum.RUNNING) + prevRayServiceStatus = rayv1alpha1.RayServiceStatus{ + Applications: map[string]rayv1alpha1.AppStatus{ + serveAppName: { + Status: rayv1alpha1.ApplicationStatusEnum.DEPLOYING, + HealthLastUpdateTime: &metav1.Time{Time: metav1.Now().Time}, + }, + }, + } + isHealthy, isReady, err = r.getAndCheckServeStatus(ctx, dashboardClient, &prevRayServiceStatus, utils.MULTI_APP, &serviceUnhealthySecondThreshold) + assert.Nil(t, err) + assert.True(t, isHealthy) + assert.True(t, isReady) + + // Test 4: The Ray Serve application lasts "UNHEALTHY" for more than `serviceUnhealthySecondThreshold` seconds. + // The RayCluster will be marked as unhealthy. + dashboardClient = initFakeDashboardClient(serveAppName, rayv1alpha1.DeploymentStatusEnum.UNHEALTHY, rayv1alpha1.ApplicationStatusEnum.UNHEALTHY) + prevRayServiceStatus = rayv1alpha1.RayServiceStatus{ + Applications: map[string]rayv1alpha1.AppStatus{ + serveAppName: { + Status: rayv1alpha1.ApplicationStatusEnum.UNHEALTHY, + HealthLastUpdateTime: &metav1.Time{Time: metav1.Now().Add(-time.Second * time.Duration(serviceUnhealthySecondThreshold+1))}, + }, + }, + } + isHealthy, isReady, err = r.getAndCheckServeStatus(ctx, dashboardClient, &prevRayServiceStatus, utils.MULTI_APP, &serviceUnhealthySecondThreshold) + assert.Nil(t, err) + assert.False(t, isHealthy) + assert.False(t, isReady) + + // Test 5: The Ray Serve application lasts "UNHEALTHY" for less than `serviceUnhealthySecondThreshold` seconds. + // The RayCluster will not be marked as unhealthy. + dashboardClient = initFakeDashboardClient(serveAppName, rayv1alpha1.DeploymentStatusEnum.UNHEALTHY, rayv1alpha1.ApplicationStatusEnum.UNHEALTHY) + prevRayServiceStatus = rayv1alpha1.RayServiceStatus{ + Applications: map[string]rayv1alpha1.AppStatus{ + serveAppName: { + Status: rayv1alpha1.ApplicationStatusEnum.UNHEALTHY, + HealthLastUpdateTime: &metav1.Time{Time: metav1.Now().Add(-time.Second * time.Duration(serviceUnhealthySecondThreshold-1))}, + }, + }, + } + isHealthy, isReady, err = r.getAndCheckServeStatus(ctx, dashboardClient, &prevRayServiceStatus, utils.MULTI_APP, &serviceUnhealthySecondThreshold) + assert.Nil(t, err) + assert.True(t, isHealthy) + assert.False(t, isReady) + + // Test 6: The Ray Serve application lasts "DEPLOY_FAILED" for more than `serviceUnhealthySecondThreshold` seconds. + // The RayCluster will be marked as unhealthy. + dashboardClient = initFakeDashboardClient(serveAppName, rayv1alpha1.DeploymentStatusEnum.UPDATING, rayv1alpha1.ApplicationStatusEnum.DEPLOY_FAILED) + prevRayServiceStatus = rayv1alpha1.RayServiceStatus{ + Applications: map[string]rayv1alpha1.AppStatus{ + serveAppName: { + Status: rayv1alpha1.ApplicationStatusEnum.DEPLOY_FAILED, + HealthLastUpdateTime: &metav1.Time{Time: metav1.Now().Add(-time.Second * time.Duration(serviceUnhealthySecondThreshold+1))}, + }, + }, + } + isHealthy, isReady, err = r.getAndCheckServeStatus(ctx, dashboardClient, &prevRayServiceStatus, utils.MULTI_APP, &serviceUnhealthySecondThreshold) + assert.Nil(t, err) + assert.False(t, isHealthy) + assert.False(t, isReady) + + // Test 7: The Ray Serve application lasts "DEPLOY_FAILED" for less than `serviceUnhealthySecondThreshold` seconds. + // The RayCluster will not be marked as unhealthy. + dashboardClient = initFakeDashboardClient(serveAppName, rayv1alpha1.DeploymentStatusEnum.UPDATING, rayv1alpha1.ApplicationStatusEnum.DEPLOY_FAILED) + prevRayServiceStatus = rayv1alpha1.RayServiceStatus{ + Applications: map[string]rayv1alpha1.AppStatus{ + serveAppName: { + Status: rayv1alpha1.ApplicationStatusEnum.DEPLOY_FAILED, + HealthLastUpdateTime: &metav1.Time{Time: metav1.Now().Add(-time.Second * time.Duration(serviceUnhealthySecondThreshold-1))}, + }, + }, + } + isHealthy, isReady, err = r.getAndCheckServeStatus(ctx, dashboardClient, &prevRayServiceStatus, utils.MULTI_APP, &serviceUnhealthySecondThreshold) + assert.Nil(t, err) + assert.True(t, isHealthy) + assert.False(t, isReady) +} + +func initFakeDashboardClient(appName string, deploymentStatus string, appStatus string) utils.RayDashboardClientInterface { + status := generateServeStatus(deploymentStatus, appStatus) + fakeDashboardClient := utils.FakeRayDashboardClient{} + fakeDashboardClient.SetMultiApplicationStatuses(map[string]*utils.ServeApplicationStatus{appName: &status}) + return &fakeDashboardClient +} diff --git a/tests/config/ray-service.yaml.template b/tests/config/ray-service.yaml.template index f9a45a59d3..4fb8acd169 100644 --- a/tests/config/ray-service.yaml.template +++ b/tests/config/ray-service.yaml.template @@ -3,7 +3,7 @@ kind: RayService metadata: name: rayservice-sample spec: - serviceUnhealthySecondThreshold: 300 + serviceUnhealthySecondThreshold: 900 deploymentUnhealthySecondThreshold: 300 serveConfig: importPath: fruit.deployment_graph