Skip to content

Commit

Permalink
add status ActiveServingRayPods
Browse files Browse the repository at this point in the history
Signed-off-by: Yicheng-Lu-llll <[email protected]>
  • Loading branch information
Yicheng-Lu-llll committed Feb 5, 2024
1 parent 6087689 commit 8110ea8
Show file tree
Hide file tree
Showing 8 changed files with 64 additions and 1 deletion.
3 changes: 3 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.

Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,13 @@ rules:
- get
- list
- update
- apiGroups:
- ""
resources:
- endpoints
verbs:
- get
- list
- apiGroups:
- ""
resources:
Expand Down
7 changes: 7 additions & 0 deletions helm-chart/kuberay-operator/templates/role.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,13 @@ rules:
- get
- list
- update
- apiGroups:
- ""
resources:
- endpoints
verbs:
- get
- list
- apiGroups:
- ""
resources:
Expand Down
3 changes: 3 additions & 0 deletions ray-operator/apis/ray/v1/rayservice_types.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,9 @@ type RayServiceStatuses struct {
PendingServiceStatus RayServiceStatus `json:"pendingServiceStatus,omitempty"`
// ServiceStatus indicates the current RayService status.
ServiceStatus ServiceStatus `json:"serviceStatus,omitempty"`
// ActiveServingRayPods indicates the number of Ray Pods that are actively serving or have been selected by the serve service.
// Ray Pods without a proxy actor or those that are unhealthy will not be counted.
ActiveServingRayPods int32 `json:"ActiveServingRayPods,omitempty"`
// observedGeneration is the most recent generation observed for this RayService. It corresponds to the
// RayService's generation, which is updated on mutation by the API Server.
// +optional
Expand Down
3 changes: 3 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/config/rbac/role.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,13 @@ rules:
- get
- list
- update
- apiGroups:
- ""
resources:
- endpoints
verbs:
- get
- list
- apiGroups:
- ""
resources:
Expand Down
26 changes: 25 additions & 1 deletion ray-operator/controllers/ray/rayservice_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,7 @@ func NewRayServiceReconciler(mgr manager.Manager, dashboardClientFunc func() uti
// +kubebuilder:rbac:groups=core,resources=events,verbs=get;list;watch;create;update;patch;delete
// +kubebuilder:rbac:groups=core,resources=pods,verbs=get;list;watch;create;update;patch;delete
// +kubebuilder:rbac:groups=core,resources=pods/status,verbs=get;list;watch;create;update;patch;delete
// +kubebuilder:rbac:groups=core,resources=endpoints,verbs=get;list
// +kubebuilder:rbac:groups=core,resources=services,verbs=get;list;watch;create;update;patch;delete
// +kubebuilder:rbac:groups=core,resources=services/status,verbs=get;update;patch
// +kubebuilder:rbac:groups=coordination.k8s.io,resources=leases,verbs=get;list;create;update
Expand Down Expand Up @@ -285,6 +286,11 @@ func (r *RayServiceReconciler) inconsistentRayServiceStatuses(oldStatus rayv1.Ra
return true
}

if oldStatus.ActiveServingRayPods != newStatus.ActiveServingRayPods {
r.Log.Info(fmt.Sprintf("inconsistentRayServiceStatus RayService ActiveServingRayPods changed from %d to %d", oldStatus.ActiveServingRayPods, newStatus.ActiveServingRayPods))
return true
}

if r.inconsistentRayServiceStatus(oldStatus.ActiveServiceStatus, newStatus.ActiveServiceStatus) {
r.Log.Info("inconsistentRayServiceStatus RayService ActiveServiceStatus changed")
return true
Expand Down Expand Up @@ -908,6 +914,9 @@ func (r *RayServiceReconciler) reconcileServices(ctx context.Context, rayService
newSvc, err = common.BuildHeadServiceForRayService(ctx, *rayServiceInstance, *rayClusterInstance)
case utils.ServingService:
newSvc, err = common.BuildServeServiceForRayService(ctx, *rayServiceInstance, *rayClusterInstance)
if updateStatusErr := r.updateStatusForActiveServingRayPods(ctx, newSvc.Name, newSvc.Namespace, rayServiceInstance); updateStatusErr != nil {
return updateStatusErr
}
default:
return fmt.Errorf("unknown service type %v", serviceType)
}
Expand All @@ -919,7 +928,7 @@ func (r *RayServiceReconciler) reconcileServices(ctx context.Context, rayService

// Retrieve the Service from the Kubernetes cluster with the name and namespace.
oldSvc := &corev1.Service{}
err = r.Get(ctx, client.ObjectKey{Name: newSvc.Name, Namespace: rayServiceInstance.Namespace}, oldSvc)
err = r.Get(ctx, client.ObjectKey{Name: newSvc.Name, Namespace: newSvc.Namespace}, oldSvc)

if err == nil {
// Only update the service if the RayCluster switches.
Expand Down Expand Up @@ -963,6 +972,21 @@ func (r *RayServiceReconciler) reconcileServices(ctx context.Context, rayService
return nil
}

func (r *RayServiceReconciler) updateStatusForActiveServingRayPods(ctx context.Context, endpointsName string, endpointsNamespace string, rayServiceInstance *rayv1.RayService) error {
numEndpoints := 0
endpoints := &corev1.Endpoints{}
if err := r.Get(ctx, client.ObjectKey{Name: endpointsName, Namespace: endpointsNamespace}, endpoints); err == nil {
for _, subset := range endpoints.Subsets {
numEndpoints += len(subset.Addresses)
}
} else if !errors.IsNotFound(err) {
r.Log.Error(err, "Fail to retrieve the Kubernetes Endpoints from the cluster!")
return err
}
rayServiceInstance.Status.ActiveServingRayPods = int32(numEndpoints)
return nil
}

func (r *RayServiceReconciler) updateStatusForActiveCluster(ctx context.Context, rayServiceInstance *rayv1.RayService, rayClusterInstance *rayv1.RayCluster, logger logr.Logger) error {
rayServiceInstance.Status.ActiveServiceStatus.RayClusterStatus = rayClusterInstance.Status

Expand Down

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

0 comments on commit 8110ea8

Please sign in to comment.