Skip to content

Commit

Permalink
feat: record last state transition times (#2053)
Browse files Browse the repository at this point in the history
## Problem Statement

My ML platform team runs the kuberay ray-operator. We want to measure the time
it takes for RayCluster's to transition from their initial "unhealthy" state to
some other state. This metric is important for us because our users want their
RayClusters to start in a timely manner. It seems like neither the ray-operator
nor RayClusters provide this info currently.

## Design

Add a new `.status.stateTransitionTimes` field to the `RayCluster` custom
resource. This field is a `map[ClusterState]*metav1.Time` that indicates the
time of the last state transition for each state. This field is updated
whenever the `.status.state` changes.

* [original discussion doc](https://docs.google.com/document/d/14yPSZ9iLk7a0qEg14rNWr60Btz0HEeQ3oWKP-GN9QTM)
* [related Slack thread](https://ray-distributed.slack.com/archives/C01CKH05XBN/p1709321264762029)
* [example input and output RayClusters](https://gist.github.com/davidxia/205d2b23202356a2d3172c51e0912f35)
  • Loading branch information
davidxia committed May 1, 2024
1 parent 9662bd9 commit 3c44ba0
Show file tree
Hide file tree
Showing 13 changed files with 157 additions and 14 deletions.
5 changes: 5 additions & 0 deletions helm-chart/kuberay-operator/crds/ray.io_rayclusters.yaml

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

5 changes: 5 additions & 0 deletions helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

10 changes: 10 additions & 0 deletions helm-chart/kuberay-operator/crds/ray.io_rayservices.yaml

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 2 additions & 0 deletions ray-operator/apis/ray/v1/raycluster_types.go
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,8 @@ type RayClusterStatus struct {
// LastUpdateTime indicates last update timestamp for this cluster status.
// +nullable
LastUpdateTime *metav1.Time `json:"lastUpdateTime,omitempty"`
// StateTransitionTimes indicates the time of the last state transition for each state.
StateTransitionTimes map[ClusterState]*metav1.Time `json:"stateTransitionTimes,omitempty"`
// Service Endpoints
Endpoints map[string]string `json:"endpoints,omitempty"`
// Head info
Expand Down
8 changes: 8 additions & 0 deletions ray-operator/apis/ray/v1/zz_generated.deepcopy.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

5 changes: 5 additions & 0 deletions ray-operator/config/crd/bases/ray.io_rayclusters.yaml

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

5 changes: 5 additions & 0 deletions ray-operator/config/crd/bases/ray.io_rayjobs.yaml

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

10 changes: 10 additions & 0 deletions ray-operator/config/crd/bases/ray.io_rayservices.yaml

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

7 changes: 7 additions & 0 deletions ray-operator/controllers/ray/raycluster_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -1252,6 +1252,13 @@ func (r *RayClusterReconciler) calculateStatus(ctx context.Context, instance *ra
timeNow := metav1.Now()
newInstance.Status.LastUpdateTime = &timeNow

if instance.Status.State != newInstance.Status.State {
if newInstance.Status.StateTransitionTimes == nil {
newInstance.Status.StateTransitionTimes = make(map[rayv1.ClusterState]*metav1.Time)
}
newInstance.Status.StateTransitionTimes[newInstance.Status.State] = &timeNow
}

return newInstance, nil
}

Expand Down
9 changes: 9 additions & 0 deletions ray-operator/controllers/ray/raycluster_controller_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -470,6 +470,15 @@ var _ = Context("Inside the default namespace", func() {
getClusterState(ctx, namespace, rayCluster.Name),
time.Second*3, time.Millisecond*500).Should(Equal(rayv1.Ready))
})

It("RayCluster's .status.stateTransitionTimes should include a time for ready state", func() {
Eventually(
func() *metav1.Time {
status := getClusterStatus(ctx, namespace, rayCluster.Name)()
return status.StateTransitionTimes[rayv1.Ready]
},
time.Second*3, time.Millisecond*500).Should(Not(BeNil()))
})
})

Describe("RayCluster with a multi-host worker group", func() {
Expand Down
52 changes: 52 additions & 0 deletions ray-operator/controllers/ray/raycluster_controller_unit_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1629,6 +1629,7 @@ func TestCalculateStatus(t *testing.T) {
},
Status: corev1.PodStatus{
PodIP: headNodeIP,
Phase: corev1.PodRunning,
},
}
runtimeObjects := []runtime.Object{headPod, headService}
Expand All @@ -1650,6 +1651,57 @@ func TestCalculateStatus(t *testing.T) {
assert.Equal(t, headNodeIP, newInstance.Status.Head.PodIP)
assert.Equal(t, headServiceIP, newInstance.Status.Head.ServiceIP)
assert.Equal(t, headService.Name, newInstance.Status.Head.ServiceName)
assert.NotNil(t, newInstance.Status.StateTransitionTimes, "Cluster state transition timestamp should be created")
assert.Equal(t, newInstance.Status.LastUpdateTime, newInstance.Status.StateTransitionTimes[rayv1.Ready])
}

func TestStateTransitionTimes_NoStateChange(t *testing.T) {
setupTest(t)

// Create a new scheme with CRDs, Pod, Service schemes.
newScheme := runtime.NewScheme()
_ = rayv1.AddToScheme(newScheme)
_ = corev1.AddToScheme(newScheme)

// Mock data
headServiceIP := "aaa.bbb.ccc.ddd"
headService, err := common.BuildServiceForHeadPod(context.Background(), *testRayCluster, nil, nil)
assert.Nil(t, err, "Failed to build head service.")
headService.Spec.ClusterIP = headServiceIP
// headService.Spec.cont
headPod := &corev1.Pod{
ObjectMeta: metav1.ObjectMeta{
Name: "headNode",
Namespace: namespaceStr,
Labels: map[string]string{
utils.RayClusterLabelKey: instanceName,
utils.RayNodeTypeLabelKey: string(rayv1.HeadNode),
},
},
Status: corev1.PodStatus{
PodIP: headNodeIP,
Phase: corev1.PodRunning,
},
}
runtimeObjects := []runtime.Object{headPod, headService}

// Initialize a fake client with newScheme and runtimeObjects.
fakeClient := clientFake.NewClientBuilder().WithScheme(newScheme).WithRuntimeObjects(runtimeObjects...).Build()
ctx := context.Background()

// Initialize a RayCluster reconciler.
r := &RayClusterReconciler{
Client: fakeClient,
Recorder: &record.FakeRecorder{},
Scheme: scheme.Scheme,
}

preUpdateTime := metav1.Now()
testRayCluster.Status.State = rayv1.Ready
testRayCluster.Status.StateTransitionTimes = map[rayv1.ClusterState]*metav1.Time{rayv1.Ready: &preUpdateTime}
newInstance, err := r.calculateStatus(ctx, testRayCluster)
assert.Nil(t, err)
assert.Equal(t, preUpdateTime, *newInstance.Status.StateTransitionTimes[rayv1.Ready], "Cluster state transition timestamp should not be updated")
}

func Test_TerminatedWorkers_NoAutoscaler(t *testing.T) {
Expand Down
10 changes: 10 additions & 0 deletions ray-operator/controllers/ray/suite_helpers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,16 @@ func getClusterState(ctx context.Context, namespace string, clusterName string)
}
}

func getClusterStatus(ctx context.Context, namespace string, clusterName string) func() rayv1.RayClusterStatus {
return func() rayv1.RayClusterStatus {
var cluster rayv1.RayCluster
if err := k8sClient.Get(ctx, client.ObjectKey{Namespace: namespace, Name: clusterName}, &cluster); err != nil {
log.Fatal(err)
}
return cluster.Status
}
}

func isAllPodsRunningByFilters(ctx context.Context, podlist corev1.PodList, opt ...client.ListOption) bool {
err := k8sClient.List(ctx, &podlist, opt...)
gomega.Expect(err).ShouldNot(gomega.HaveOccurred(), "failed to list Pods")
Expand Down

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

0 comments on commit 3c44ba0

Please sign in to comment.