Skip to content

Commit

Permalink
update
Browse files Browse the repository at this point in the history
  • Loading branch information
kevin85421 committed Nov 1, 2023
1 parent 1f728c5 commit 2294e15
Showing 1 changed file with 22 additions and 2 deletions.
24 changes: 22 additions & 2 deletions ray-operator/controllers/ray/raycluster_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ import (
"time"

metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/utils/pointer"

"github.com/ray-project/kuberay/ray-operator/controllers/ray/batchscheduler"
"github.com/ray-project/kuberay/ray-operator/controllers/ray/common"
Expand All @@ -28,8 +29,10 @@ import (
"k8s.io/client-go/rest"

corev1 "k8s.io/api/core/v1"
v1 "k8s.io/api/core/v1"
networkingv1 "k8s.io/api/networking/v1"
"k8s.io/apimachinery/pkg/api/errors"
"k8s.io/apimachinery/pkg/api/resource"
"k8s.io/apimachinery/pkg/runtime"
"k8s.io/apimachinery/pkg/types"
ctrl "sigs.k8s.io/controller-runtime"
Expand Down Expand Up @@ -282,8 +285,12 @@ func (r *RayClusterReconciler) rayClusterReconcile(ctx context.Context, request
return ctrl.Result{}, nil
}
if redisCleanupJob.Status.Failed > 0 {
r.Log.Info("If the Redis cleanup Job has failed, we will requeue the RayCluster CR after 1 minute.")
return ctrl.Result{RequeueAfter: 1 * time.Minute}, nil
r.Log.Info(fmt.Sprintf(
"The Redis cleanup Job %s has failed, requeue the RayCluster CR after 5 minute. "+
"You should manually delete the storage namespace %s in Redis and remove the RayCluster's finalizer. "+
"Please check https://docs.ray.io/en/master/cluster/kubernetes/user-guides/kuberay-gcs-ft.html for more details.",
redisCleanupJob.Name, redisCleanupJob.Annotations[common.RayExternalStorageNSAnnotationKey]))
return ctrl.Result{RequeueAfter: 5 * time.Minute}, nil
}
} else {
redisCleanupJob := r.buildRedisCleanupJob(*instance)
Expand Down Expand Up @@ -1051,6 +1058,18 @@ func (r *RayClusterReconciler) buildRedisCleanupJob(instance rayv1.RayCluster) b
// Disable liveness and readiness probes because the Job will not launch processes like Raylet and GCS.
pod.Spec.Containers[common.RayContainerIndex].LivenessProbe = nil
pod.Spec.Containers[common.RayContainerIndex].ReadinessProbe = nil
pod.Spec.Containers[common.RayContainerIndex].Resources = v1.ResourceRequirements{
// The container's resource consumption remains constant. so hard-coding the resources is acceptable.
// In addition, avoid using the GPU for the Redis cleanup Job.
Limits: v1.ResourceList{
v1.ResourceCPU: resource.MustParse("200m"),
v1.ResourceMemory: resource.MustParse("256Mi"),
},
Requests: v1.ResourceList{
v1.ResourceCPU: resource.MustParse("200m"),
v1.ResourceMemory: resource.MustParse("256Mi"),
},
}
// For Kubernetes Job, the valid values for Pod's `RestartPolicy` are `Never` and `OnFailure`.
pod.Spec.RestartPolicy = corev1.RestartPolicyNever

Expand All @@ -1062,6 +1081,7 @@ func (r *RayClusterReconciler) buildRedisCleanupJob(instance rayv1.RayCluster) b
Annotations: pod.Annotations,
},
Spec: batchv1.JobSpec{
BackoffLimit: pointer.Int32(0),
Template: corev1.PodTemplateSpec{
ObjectMeta: pod.ObjectMeta,
Spec: pod.Spec,
Expand Down

0 comments on commit 2294e15

Please sign in to comment.