Skip to content

Commit 7d856ed

Browse files
authored
[refactor] Move inconsistency check functions to a new util function file (#3866)
Signed-off-by: kaihsun <[email protected]>
1 parent fcf1d92 commit 7d856ed

File tree

6 files changed

+296
-329
lines changed

6 files changed

+296
-329
lines changed

ray-operator/controllers/ray/raycluster_controller.go

Lines changed: 1 addition & 57 deletions
Original file line numberDiff line numberDiff line change
@@ -362,62 +362,6 @@ func (r *RayClusterReconciler) rayClusterReconcile(ctx context.Context, instance
362362
return ctrl.Result{RequeueAfter: time.Duration(requeueAfterSeconds) * time.Second}, nil
363363
}
364364

365-
// Checks whether the old and new RayClusterStatus are inconsistent by comparing different fields. If the only
366-
// differences between the old and new status are the `LastUpdateTime` and `ObservedGeneration` fields, the
367-
// status update will not be triggered.
368-
//
369-
// TODO (kevin85421): The field `ObservedGeneration` is not being well-maintained at the moment. In the future,
370-
// this field should be used to determine whether to update this CR or not.
371-
func (r *RayClusterReconciler) inconsistentRayClusterStatus(ctx context.Context, oldStatus rayv1.RayClusterStatus, newStatus rayv1.RayClusterStatus) bool {
372-
logger := ctrl.LoggerFrom(ctx)
373-
374-
if oldStatus.State != newStatus.State || oldStatus.Reason != newStatus.Reason {
375-
logger.Info(
376-
"inconsistentRayClusterStatus",
377-
"oldState", oldStatus.State,
378-
"newState", newStatus.State,
379-
"oldReason", oldStatus.Reason,
380-
"newReason", newStatus.Reason,
381-
)
382-
return true
383-
}
384-
if oldStatus.ReadyWorkerReplicas != newStatus.ReadyWorkerReplicas ||
385-
oldStatus.AvailableWorkerReplicas != newStatus.AvailableWorkerReplicas ||
386-
oldStatus.DesiredWorkerReplicas != newStatus.DesiredWorkerReplicas ||
387-
oldStatus.MinWorkerReplicas != newStatus.MinWorkerReplicas ||
388-
oldStatus.MaxWorkerReplicas != newStatus.MaxWorkerReplicas {
389-
logger.Info(
390-
"inconsistentRayClusterStatus",
391-
"oldReadyWorkerReplicas", oldStatus.ReadyWorkerReplicas,
392-
"newReadyWorkerReplicas", newStatus.ReadyWorkerReplicas,
393-
"oldAvailableWorkerReplicas", oldStatus.AvailableWorkerReplicas,
394-
"newAvailableWorkerReplicas", newStatus.AvailableWorkerReplicas,
395-
"oldDesiredWorkerReplicas", oldStatus.DesiredWorkerReplicas,
396-
"newDesiredWorkerReplicas", newStatus.DesiredWorkerReplicas,
397-
"oldMinWorkerReplicas", oldStatus.MinWorkerReplicas,
398-
"newMinWorkerReplicas", newStatus.MinWorkerReplicas,
399-
"oldMaxWorkerReplicas", oldStatus.MaxWorkerReplicas,
400-
"newMaxWorkerReplicas", newStatus.MaxWorkerReplicas,
401-
)
402-
return true
403-
}
404-
if !reflect.DeepEqual(oldStatus.Endpoints, newStatus.Endpoints) || !reflect.DeepEqual(oldStatus.Head, newStatus.Head) {
405-
logger.Info(
406-
"inconsistentRayClusterStatus",
407-
"oldEndpoints", oldStatus.Endpoints,
408-
"newEndpoints", newStatus.Endpoints,
409-
"oldHead", oldStatus.Head,
410-
"newHead", newStatus.Head,
411-
)
412-
return true
413-
}
414-
if !reflect.DeepEqual(oldStatus.Conditions, newStatus.Conditions) {
415-
logger.Info("inconsistentRayClusterStatus", "old conditions", oldStatus.Conditions, "new conditions", newStatus.Conditions)
416-
return true
417-
}
418-
return false
419-
}
420-
421365
func (r *RayClusterReconciler) reconcileIngress(ctx context.Context, instance *rayv1.RayCluster) error {
422366
logger := ctrl.LoggerFrom(ctx)
423367
logger.Info("Reconciling Ingress")
@@ -1599,7 +1543,7 @@ func (r *RayClusterReconciler) reconcileAutoscalerRoleBinding(ctx context.Contex
15991543
// We rely on the returning bool to requeue the reconciliation for atomic operations, such as suspending a RayCluster.
16001544
func (r *RayClusterReconciler) updateRayClusterStatus(ctx context.Context, originalRayClusterInstance, newInstance *rayv1.RayCluster) (bool, error) {
16011545
logger := ctrl.LoggerFrom(ctx)
1602-
inconsistent := r.inconsistentRayClusterStatus(ctx, originalRayClusterInstance.Status, newInstance.Status)
1546+
inconsistent := utils.InconsistentRayClusterStatus(originalRayClusterInstance.Status, newInstance.Status)
16031547
if !inconsistent {
16041548
return inconsistent, nil
16051549
}

ray-operator/controllers/ray/raycluster_controller_unit_test.go

Lines changed: 0 additions & 140 deletions
Original file line numberDiff line numberDiff line change
@@ -1606,146 +1606,6 @@ func TestReconcile_UpdateClusterState(t *testing.T) {
16061606
assert.Equal(t, cluster.Status.State, state, "Cluster state should be updated")
16071607
}
16081608

1609-
func TestInconsistentRayClusterStatus(t *testing.T) {
1610-
newScheme := runtime.NewScheme()
1611-
_ = rayv1.AddToScheme(newScheme)
1612-
fakeClient := clientFake.NewClientBuilder().WithScheme(newScheme).WithRuntimeObjects().Build()
1613-
r := &RayClusterReconciler{
1614-
Client: fakeClient,
1615-
Recorder: &record.FakeRecorder{},
1616-
Scheme: scheme.Scheme,
1617-
}
1618-
1619-
// Mock data
1620-
timeNow := metav1.Now()
1621-
oldStatus := rayv1.RayClusterStatus{
1622-
State: rayv1.Ready,
1623-
ReadyWorkerReplicas: 1,
1624-
AvailableWorkerReplicas: 1,
1625-
DesiredWorkerReplicas: 1,
1626-
MinWorkerReplicas: 1,
1627-
MaxWorkerReplicas: 10,
1628-
LastUpdateTime: &timeNow,
1629-
Endpoints: map[string]string{
1630-
utils.ClientPortName: strconv.Itoa(utils.DefaultClientPort),
1631-
utils.DashboardPortName: strconv.Itoa(utils.DefaultDashboardPort),
1632-
utils.GcsServerPortName: strconv.Itoa(utils.DefaultGcsServerPort),
1633-
utils.MetricsPortName: strconv.Itoa(utils.DefaultMetricsPort),
1634-
},
1635-
Head: rayv1.HeadInfo{
1636-
PodIP: "10.244.0.6",
1637-
ServiceIP: "10.96.140.249",
1638-
},
1639-
ObservedGeneration: 1,
1640-
Reason: "test reason",
1641-
}
1642-
1643-
// `inconsistentRayClusterStatus` is used to check whether the old and new RayClusterStatus are inconsistent
1644-
// by comparing different fields. If the only differences between the old and new status are the `LastUpdateTime`
1645-
// and `ObservedGeneration` fields, the status update will not be triggered.
1646-
ctx := context.Background()
1647-
1648-
testCases := []struct {
1649-
modifyStatus func(*rayv1.RayClusterStatus)
1650-
name string
1651-
expectResult bool
1652-
}{
1653-
{
1654-
name: "State is updated, expect result to be true",
1655-
modifyStatus: func(newStatus *rayv1.RayClusterStatus) {
1656-
newStatus.State = rayv1.Suspended
1657-
},
1658-
expectResult: true,
1659-
},
1660-
{
1661-
name: "Reason is updated, expect result to be true",
1662-
modifyStatus: func(newStatus *rayv1.RayClusterStatus) {
1663-
newStatus.Reason = "new reason"
1664-
},
1665-
expectResult: true,
1666-
},
1667-
{
1668-
name: "ReadyWorkerReplicas is updated, expect result to be true",
1669-
modifyStatus: func(newStatus *rayv1.RayClusterStatus) {
1670-
newStatus.ReadyWorkerReplicas = oldStatus.ReadyWorkerReplicas + 1
1671-
},
1672-
expectResult: true,
1673-
},
1674-
{
1675-
name: "AvailableWorkerReplicas is updated, expect result to be true",
1676-
modifyStatus: func(newStatus *rayv1.RayClusterStatus) {
1677-
newStatus.AvailableWorkerReplicas = oldStatus.AvailableWorkerReplicas + 1
1678-
},
1679-
expectResult: true,
1680-
},
1681-
{
1682-
name: "DesiredWorkerReplicas is updated, expect result to be true",
1683-
modifyStatus: func(newStatus *rayv1.RayClusterStatus) {
1684-
newStatus.DesiredWorkerReplicas = oldStatus.DesiredWorkerReplicas + 1
1685-
},
1686-
expectResult: true,
1687-
},
1688-
{
1689-
name: "MinWorkerReplicas is updated, expect result to be true",
1690-
modifyStatus: func(newStatus *rayv1.RayClusterStatus) {
1691-
newStatus.MinWorkerReplicas = oldStatus.MinWorkerReplicas + 1
1692-
},
1693-
expectResult: true,
1694-
},
1695-
{
1696-
name: "MaxWorkerReplicas is updated, expect result to be true",
1697-
modifyStatus: func(newStatus *rayv1.RayClusterStatus) {
1698-
newStatus.MaxWorkerReplicas = oldStatus.MaxWorkerReplicas + 1
1699-
},
1700-
expectResult: true,
1701-
},
1702-
{
1703-
name: "Endpoints is updated, expect result to be true",
1704-
modifyStatus: func(newStatus *rayv1.RayClusterStatus) {
1705-
newStatus.Endpoints["fakeEndpoint"] = "10009"
1706-
},
1707-
expectResult: true,
1708-
},
1709-
{
1710-
name: "Head.PodIP is updated, expect result to be true",
1711-
modifyStatus: func(newStatus *rayv1.RayClusterStatus) {
1712-
newStatus.Head.PodIP = "test head pod ip"
1713-
},
1714-
expectResult: true,
1715-
},
1716-
{
1717-
name: "RayClusterReplicaFailure is updated, expect result to be true",
1718-
modifyStatus: func(newStatus *rayv1.RayClusterStatus) {
1719-
meta.SetStatusCondition(&newStatus.Conditions, metav1.Condition{Type: string(rayv1.RayClusterReplicaFailure), Status: metav1.ConditionTrue})
1720-
},
1721-
expectResult: true,
1722-
},
1723-
{
1724-
name: "LastUpdateTime is updated, expect result to be false",
1725-
modifyStatus: func(newStatus *rayv1.RayClusterStatus) {
1726-
newStatus.LastUpdateTime = &metav1.Time{Time: timeNow.Add(time.Hour)}
1727-
},
1728-
expectResult: false,
1729-
},
1730-
{
1731-
name: "ObservedGeneration is updated, expect result to be false",
1732-
modifyStatus: func(newStatus *rayv1.RayClusterStatus) {
1733-
newStatus.ObservedGeneration = oldStatus.ObservedGeneration + 1
1734-
},
1735-
expectResult: false,
1736-
},
1737-
}
1738-
1739-
for _, testCase := range testCases {
1740-
t.Run(testCase.name, func(t *testing.T) {
1741-
newStatus := oldStatus.DeepCopy()
1742-
testCase.modifyStatus(newStatus)
1743-
result := r.inconsistentRayClusterStatus(ctx, oldStatus, *newStatus)
1744-
assert.Equal(t, testCase.expectResult, result)
1745-
})
1746-
}
1747-
}
1748-
17491609
func TestCalculateStatus(t *testing.T) {
17501610
setupTest(t)
17511611
assert.True(t, features.Enabled(features.RayClusterStatusConditions))

ray-operator/controllers/ray/rayservice_controller.go

Lines changed: 1 addition & 86 deletions
Original file line numberDiff line numberDiff line change
@@ -6,7 +6,6 @@ import (
66
"fmt"
77
"math"
88
"os"
9-
"reflect"
109
"strconv"
1110
"strings"
1211
"time"
@@ -201,7 +200,7 @@ func (r *RayServiceReconciler) Reconcile(ctx context.Context, request ctrl.Reque
201200
}
202201

203202
// Final status update for any CR modification.
204-
if inconsistentRayServiceStatuses(ctx, originalRayServiceInstance.Status, rayServiceInstance.Status) {
203+
if utils.InconsistentRayServiceStatuses(originalRayServiceInstance.Status, rayServiceInstance.Status) {
205204
rayServiceInstance.Status.LastUpdateTime = &metav1.Time{Time: time.Now()}
206205
if errStatus := r.Status().Update(ctx, rayServiceInstance); errStatus != nil {
207206
return ctrl.Result{RequeueAfter: ServiceDefaultRequeueDuration}, errStatus
@@ -344,90 +343,6 @@ func setCondition(rayServiceInstance *rayv1.RayService, conditionType rayv1.RayS
344343
meta.SetStatusCondition(&rayServiceInstance.Status.Conditions, condition)
345344
}
346345

347-
// Checks whether the old and new RayServiceStatus are inconsistent by comparing different fields.
348-
// The RayClusterStatus field is only for observability in RayService CR, and changes to it will not trigger the status update.
349-
func inconsistentRayServiceStatus(ctx context.Context, oldStatus rayv1.RayServiceStatus, newStatus rayv1.RayServiceStatus) bool {
350-
logger := ctrl.LoggerFrom(ctx)
351-
if oldStatus.RayClusterName != newStatus.RayClusterName {
352-
logger.Info("inconsistentRayServiceStatus RayService RayClusterName", "oldRayClusterName", oldStatus.RayClusterName, "newRayClusterName", newStatus.RayClusterName)
353-
return true
354-
}
355-
356-
if len(oldStatus.Applications) != len(newStatus.Applications) {
357-
return true
358-
}
359-
360-
var ok bool
361-
for appName, newAppStatus := range newStatus.Applications {
362-
var oldAppStatus rayv1.AppStatus
363-
if oldAppStatus, ok = oldStatus.Applications[appName]; !ok {
364-
logger.Info("inconsistentRayServiceStatus RayService new application found", "appName", appName)
365-
return true
366-
}
367-
368-
if oldAppStatus.Status != newAppStatus.Status {
369-
logger.Info("inconsistentRayServiceStatus RayService application status changed", "appName", appName, "oldStatus", oldAppStatus.Status, "newStatus", newAppStatus.Status)
370-
return true
371-
} else if oldAppStatus.Message != newAppStatus.Message {
372-
logger.Info("inconsistentRayServiceStatus RayService application status message changed", "appName", appName, "oldStatus", oldAppStatus.Message, "newStatus", newAppStatus.Message)
373-
return true
374-
}
375-
376-
if len(oldAppStatus.Deployments) != len(newAppStatus.Deployments) {
377-
return true
378-
}
379-
380-
for deploymentName, newDeploymentStatus := range newAppStatus.Deployments {
381-
var oldDeploymentStatus rayv1.ServeDeploymentStatus
382-
if oldDeploymentStatus, ok = oldAppStatus.Deployments[deploymentName]; !ok {
383-
logger.Info("inconsistentRayServiceStatus RayService new deployment found in application", "deploymentName", deploymentName, "appName", appName)
384-
return true
385-
}
386-
387-
if oldDeploymentStatus.Status != newDeploymentStatus.Status {
388-
logger.Info("inconsistentRayServiceStatus RayService DeploymentStatus changed", "oldDeploymentStatus", oldDeploymentStatus.Status, "newDeploymentStatus", newDeploymentStatus.Status)
389-
return true
390-
} else if oldDeploymentStatus.Message != newDeploymentStatus.Message {
391-
logger.Info("inconsistentRayServiceStatus RayService deployment status message changed", "oldDeploymentStatus", oldDeploymentStatus.Message, "newDeploymentStatus", newDeploymentStatus.Message)
392-
return true
393-
}
394-
}
395-
}
396-
397-
return false
398-
}
399-
400-
// Determine whether to update the status of the RayService instance.
401-
func inconsistentRayServiceStatuses(ctx context.Context, oldStatus rayv1.RayServiceStatuses, newStatus rayv1.RayServiceStatuses) bool {
402-
logger := ctrl.LoggerFrom(ctx)
403-
if oldStatus.ServiceStatus != newStatus.ServiceStatus {
404-
logger.Info("inconsistentRayServiceStatus RayService ServiceStatus changed", "oldServiceStatus", oldStatus.ServiceStatus, "newServiceStatus", newStatus.ServiceStatus)
405-
return true
406-
}
407-
408-
if oldStatus.NumServeEndpoints != newStatus.NumServeEndpoints {
409-
logger.Info("inconsistentRayServiceStatus RayService NumServeEndpoints changed", "oldNumServeEndpoints", oldStatus.NumServeEndpoints, "newNumServeEndpoints", newStatus.NumServeEndpoints)
410-
return true
411-
}
412-
413-
if !reflect.DeepEqual(oldStatus.Conditions, newStatus.Conditions) {
414-
logger.Info("inconsistentRayServiceStatus RayService Conditions changed")
415-
return true
416-
}
417-
418-
if inconsistentRayServiceStatus(ctx, oldStatus.ActiveServiceStatus, newStatus.ActiveServiceStatus) {
419-
logger.Info("inconsistentRayServiceStatus RayService ActiveServiceStatus changed")
420-
return true
421-
}
422-
423-
if inconsistentRayServiceStatus(ctx, oldStatus.PendingServiceStatus, newStatus.PendingServiceStatus) {
424-
logger.Info("inconsistentRayServiceStatus RayService PendingServiceStatus changed")
425-
return true
426-
}
427-
428-
return false
429-
}
430-
431346
// SetupWithManager sets up the controller with the Manager.
432347
func (r *RayServiceReconciler) SetupWithManager(mgr ctrl.Manager, reconcileConcurrency int) error {
433348
return ctrl.NewControllerManagedBy(mgr).

ray-operator/controllers/ray/rayservice_controller_unit_test.go

Lines changed: 0 additions & 46 deletions
Original file line numberDiff line numberDiff line change
@@ -61,52 +61,6 @@ func TestGenerateHashWithoutReplicasAndWorkersToDelete(t *testing.T) {
6161
assert.NotEqual(t, hash1, hash3)
6262
}
6363

64-
func TestInconsistentRayServiceStatuses(t *testing.T) {
65-
oldStatus := rayv1.RayServiceStatuses{
66-
ActiveServiceStatus: rayv1.RayServiceStatus{
67-
RayClusterName: "new-cluster",
68-
Applications: map[string]rayv1.AppStatus{
69-
utils.DefaultServeAppName: {
70-
Status: rayv1.ApplicationStatusEnum.RUNNING,
71-
Message: "OK",
72-
Deployments: map[string]rayv1.ServeDeploymentStatus{
73-
"serve-1": {
74-
Status: rayv1.DeploymentStatusEnum.UNHEALTHY,
75-
Message: "error",
76-
},
77-
},
78-
},
79-
},
80-
},
81-
PendingServiceStatus: rayv1.RayServiceStatus{
82-
RayClusterName: "old-cluster",
83-
Applications: map[string]rayv1.AppStatus{
84-
utils.DefaultServeAppName: {
85-
Status: rayv1.ApplicationStatusEnum.NOT_STARTED,
86-
Message: "application not started yet",
87-
Deployments: map[string]rayv1.ServeDeploymentStatus{
88-
"serve-1": {
89-
Status: rayv1.DeploymentStatusEnum.HEALTHY,
90-
Message: "Serve is healthy",
91-
},
92-
},
93-
},
94-
},
95-
},
96-
ServiceStatus: rayv1.NotRunning,
97-
}
98-
ctx := context.Background()
99-
100-
// Test 1: Update ServiceStatus only.
101-
newStatus := oldStatus.DeepCopy()
102-
newStatus.ServiceStatus = rayv1.Running
103-
assert.True(t, inconsistentRayServiceStatuses(ctx, oldStatus, *newStatus))
104-
105-
// Test 2: Test RayServiceStatus
106-
newStatus = oldStatus.DeepCopy()
107-
assert.False(t, inconsistentRayServiceStatuses(ctx, oldStatus, *newStatus))
108-
}
109-
11064
func TestIsHeadPodRunningAndReady(t *testing.T) {
11165
// Create a new scheme with CRDs, Pod, Service schemes.
11266
newScheme := runtime.NewScheme()

0 commit comments

Comments
 (0)