Skip to content

Commit

Permalink
[Refactor] Move functions that don’t rely on the controller to non-co…
Browse files Browse the repository at this point in the history
…ntroller member functions

Signed-off-by: win5923 <[email protected]>
  • Loading branch information
win5923 committed Jan 14, 2025
1 parent 42f299a commit 1e35292
Show file tree
Hide file tree
Showing 4 changed files with 37 additions and 63 deletions.
24 changes: 12 additions & 12 deletions ray-operator/controllers/ray/rayjob_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,15 +160,15 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request)
// Set `Status.JobDeploymentStatus` to `JobDeploymentStatusInitializing`, and initialize `Status.JobId`
// and `Status.RayClusterName` prior to avoid duplicate job submissions and cluster creations.
logger.Info("JobDeploymentStatusNew")
if err = r.initRayJobStatusIfNeed(ctx, rayJobInstance); err != nil {
if err = initRayJobStatusIfNeed(ctx, rayJobInstance); err != nil {
return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err
}
case rayv1.JobDeploymentStatusInitializing:
if shouldUpdate := r.updateStatusToSuspendingIfNeeded(ctx, rayJobInstance); shouldUpdate {
if shouldUpdate := updateStatusToSuspendingIfNeeded(ctx, rayJobInstance); shouldUpdate {
break
}

if shouldUpdate := r.checkActiveDeadlineAndUpdateStatusIfNeeded(ctx, rayJobInstance); shouldUpdate {
if shouldUpdate := checkActiveDeadlineAndUpdateStatusIfNeeded(ctx, rayJobInstance); shouldUpdate {
break
}

Expand Down Expand Up @@ -215,11 +215,11 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request)
rayJobInstance.Status.JobId = rayJobInstance.Spec.JobId
rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusRunning
case rayv1.JobDeploymentStatusRunning:
if shouldUpdate := r.updateStatusToSuspendingIfNeeded(ctx, rayJobInstance); shouldUpdate {
if shouldUpdate := updateStatusToSuspendingIfNeeded(ctx, rayJobInstance); shouldUpdate {
break
}

if shouldUpdate := r.checkActiveDeadlineAndUpdateStatusIfNeeded(ctx, rayJobInstance); shouldUpdate {
if shouldUpdate := checkActiveDeadlineAndUpdateStatusIfNeeded(ctx, rayJobInstance); shouldUpdate {
break
}

Expand All @@ -235,7 +235,7 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request)
logger.Error(err, "Failed to get the submitter Kubernetes Job for RayJob", "NamespacedName", namespacedName)
return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err
}
if shouldUpdate := r.checkK8sJobAndUpdateStatusIfNeeded(ctx, rayJobInstance, job); shouldUpdate {
if shouldUpdate := checkK8sJobAndUpdateStatusIfNeeded(ctx, rayJobInstance, job); shouldUpdate {
break
}
}
Expand Down Expand Up @@ -443,7 +443,7 @@ func (r *RayJobReconciler) createK8sJobIfNeed(ctx context.Context, rayJobInstanc
namespacedName := common.RayJobK8sJobNamespacedName(rayJobInstance)
if err := r.Client.Get(ctx, namespacedName, job); err != nil {
if errors.IsNotFound(err) {
submitterTemplate, err := r.getSubmitterTemplate(ctx, rayJobInstance, rayClusterInstance)
submitterTemplate, err := getSubmitterTemplate(ctx, rayJobInstance, rayClusterInstance)
if err != nil {
return err
}
Expand All @@ -457,7 +457,7 @@ func (r *RayJobReconciler) createK8sJobIfNeed(ctx context.Context, rayJobInstanc
}

// getSubmitterTemplate builds the submitter pod template for the Ray job.
func (r *RayJobReconciler) getSubmitterTemplate(ctx context.Context, rayJobInstance *rayv1.RayJob, rayClusterInstance *rayv1.RayCluster) (corev1.PodTemplateSpec, error) {
func getSubmitterTemplate(ctx context.Context, rayJobInstance *rayv1.RayJob, rayClusterInstance *rayv1.RayCluster) (corev1.PodTemplateSpec, error) {
logger := ctrl.LoggerFrom(ctx)
var submitterTemplate corev1.PodTemplateSpec

Expand Down Expand Up @@ -641,7 +641,7 @@ func (r *RayJobReconciler) SetupWithManager(mgr ctrl.Manager, reconcileConcurren
// prior to job submissions and RayCluster creations. This is used to avoid duplicate job submissions and cluster creations. In addition, this
// function also sets `Status.StartTime` to support `ActiveDeadlineSeconds`.
// This function will set or generate JobId if SubmissionMode is not InteractiveMode.
func (r *RayJobReconciler) initRayJobStatusIfNeed(ctx context.Context, rayJob *rayv1.RayJob) error {
func initRayJobStatusIfNeed(ctx context.Context, rayJob *rayv1.RayJob) error {
logger := ctrl.LoggerFrom(ctx)
shouldUpdateStatus := rayJob.Status.JobId == "" || rayJob.Status.RayClusterName == "" || rayJob.Status.JobStatus == ""
// Please don't update `shouldUpdateStatus` below.
Expand Down Expand Up @@ -768,7 +768,7 @@ func (r *RayJobReconciler) constructRayClusterForRayJob(rayJobInstance *rayv1.Ra
return rayCluster, nil
}

func (r *RayJobReconciler) updateStatusToSuspendingIfNeeded(ctx context.Context, rayJob *rayv1.RayJob) bool {
func updateStatusToSuspendingIfNeeded(ctx context.Context, rayJob *rayv1.RayJob) bool {
logger := ctrl.LoggerFrom(ctx)
if !rayJob.Spec.Suspend {
return false
Expand All @@ -787,7 +787,7 @@ func (r *RayJobReconciler) updateStatusToSuspendingIfNeeded(ctx context.Context,
return true
}

func (r *RayJobReconciler) checkK8sJobAndUpdateStatusIfNeeded(ctx context.Context, rayJob *rayv1.RayJob, job *batchv1.Job) bool {
func checkK8sJobAndUpdateStatusIfNeeded(ctx context.Context, rayJob *rayv1.RayJob, job *batchv1.Job) bool {
logger := ctrl.LoggerFrom(ctx)
for _, cond := range job.Status.Conditions {
if cond.Type == batchv1.JobFailed && cond.Status == corev1.ConditionTrue {
Expand All @@ -808,7 +808,7 @@ func (r *RayJobReconciler) checkK8sJobAndUpdateStatusIfNeeded(ctx context.Contex
return false
}

func (r *RayJobReconciler) checkActiveDeadlineAndUpdateStatusIfNeeded(ctx context.Context, rayJob *rayv1.RayJob) bool {
func checkActiveDeadlineAndUpdateStatusIfNeeded(ctx context.Context, rayJob *rayv1.RayJob) bool {
logger := ctrl.LoggerFrom(ctx)
if rayJob.Spec.ActiveDeadlineSeconds == nil || time.Now().Before(rayJob.Status.StartTime.Add(time.Duration(*rayJob.Spec.ActiveDeadlineSeconds)*time.Second)) {
return false
Expand Down
25 changes: 6 additions & 19 deletions ray-operator/controllers/ray/rayjob_controller_unit_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -154,38 +154,37 @@ func TestGetSubmitterTemplate(t *testing.T) {
},
}

r := &RayJobReconciler{}
ctx := context.Background()

// Test 1: User provided template with command
submitterTemplate, err := r.getSubmitterTemplate(ctx, rayJobInstanceWithTemplate, nil)
submitterTemplate, err := getSubmitterTemplate(ctx, rayJobInstanceWithTemplate, nil)
assert.NoError(t, err)
assert.Equal(t, "user-command", submitterTemplate.Spec.Containers[utils.RayContainerIndex].Command[0])

// Test 2: User provided template without command
rayJobInstanceWithTemplate.Spec.SubmitterPodTemplate.Spec.Containers[utils.RayContainerIndex].Command = []string{}
submitterTemplate, err = r.getSubmitterTemplate(ctx, rayJobInstanceWithTemplate, nil)
submitterTemplate, err = getSubmitterTemplate(ctx, rayJobInstanceWithTemplate, nil)
assert.NoError(t, err)
assert.Equal(t, []string{"/bin/sh"}, submitterTemplate.Spec.Containers[utils.RayContainerIndex].Command)
assert.Equal(t, []string{"-c", "if ray job status --address http://test-url test-job-id >/dev/null 2>&1 ; then ray job logs --address http://test-url --follow test-job-id ; else ray job submit --address http://test-url --submission-id test-job-id -- echo hello world ; fi"}, submitterTemplate.Spec.Containers[utils.RayContainerIndex].Args)

// Test 3: User did not provide template, should use the image of the Ray Head
submitterTemplate, err = r.getSubmitterTemplate(ctx, rayJobInstanceWithoutTemplate, rayClusterInstance)
submitterTemplate, err = getSubmitterTemplate(ctx, rayJobInstanceWithoutTemplate, rayClusterInstance)
assert.NoError(t, err)
assert.Equal(t, []string{"/bin/sh"}, submitterTemplate.Spec.Containers[utils.RayContainerIndex].Command)
assert.Equal(t, []string{"-c", "if ray job status --address http://test-url test-job-id >/dev/null 2>&1 ; then ray job logs --address http://test-url --follow test-job-id ; else ray job submit --address http://test-url --submission-id test-job-id -- echo hello world ; fi"}, submitterTemplate.Spec.Containers[utils.RayContainerIndex].Args)
assert.Equal(t, "rayproject/ray:custom-version", submitterTemplate.Spec.Containers[utils.RayContainerIndex].Image)

// Test 4: Check default PYTHONUNBUFFERED setting
submitterTemplate, err = r.getSubmitterTemplate(ctx, rayJobInstanceWithoutTemplate, rayClusterInstance)
submitterTemplate, err = getSubmitterTemplate(ctx, rayJobInstanceWithoutTemplate, rayClusterInstance)
assert.NoError(t, err)

envVar, found := utils.EnvVarByName(PythonUnbufferedEnvVarName, submitterTemplate.Spec.Containers[utils.RayContainerIndex].Env)
assert.True(t, found)
assert.Equal(t, "1", envVar.Value)

// Test 5: Check default RAY_DASHBOARD_ADDRESS env var
submitterTemplate, err = r.getSubmitterTemplate(ctx, rayJobInstanceWithTemplate, nil)
submitterTemplate, err = getSubmitterTemplate(ctx, rayJobInstanceWithTemplate, nil)
assert.NoError(t, err)

envVar, found = utils.EnvVarByName(utils.RAY_DASHBOARD_ADDRESS, submitterTemplate.Spec.Containers[utils.RayContainerIndex].Env)
Expand Down Expand Up @@ -241,20 +240,8 @@ func TestUpdateStatusToSuspendingIfNeeded(t *testing.T) {
},
}

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

// Initialize a new RayClusterReconciler.
testRayJobReconciler := &RayJobReconciler{
Client: fakeClient,
Recorder: &record.FakeRecorder{},
Scheme: newScheme,
}
shouldUpdate := testRayJobReconciler.updateStatusToSuspendingIfNeeded(ctx, rayJob)
shouldUpdate := updateStatusToSuspendingIfNeeded(ctx, rayJob)
assert.Equal(t, tc.expectedShouldUpdate, shouldUpdate)

if tc.expectedShouldUpdate {
Expand Down
28 changes: 14 additions & 14 deletions ray-operator/controllers/ray/rayservice_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -223,7 +223,7 @@ func (r *RayServiceReconciler) Reconcile(ctx context.Context, request ctrl.Reque
}

// Final status update for any CR modification.
if r.inconsistentRayServiceStatuses(ctx, originalRayServiceInstance.Status, rayServiceInstance.Status) {
if inconsistentRayServiceStatuses(ctx, originalRayServiceInstance.Status, rayServiceInstance.Status) {
rayServiceInstance.Status.LastUpdateTime = &metav1.Time{Time: time.Now()}
if errStatus := r.Status().Update(ctx, rayServiceInstance); errStatus != nil {
logger.Error(errStatus, "Failed to update RayService status", "rayServiceInstance", rayServiceInstance)
Expand Down Expand Up @@ -272,7 +272,7 @@ func (r *RayServiceReconciler) calculateStatus(ctx context.Context, rayServiceIn
// If the only difference between the old and new status is the HealthLastUpdateTime field,
// the status update will not be triggered.
// The RayClusterStatus field is only for observability in RayService CR, and changes to it will not trigger the status update.
func (r *RayServiceReconciler) inconsistentRayServiceStatus(ctx context.Context, oldStatus rayv1.RayServiceStatus, newStatus rayv1.RayServiceStatus) bool {
func inconsistentRayServiceStatus(ctx context.Context, oldStatus rayv1.RayServiceStatus, newStatus rayv1.RayServiceStatus) bool {
logger := ctrl.LoggerFrom(ctx)
if oldStatus.RayClusterName != newStatus.RayClusterName {
logger.Info("inconsistentRayServiceStatus RayService RayClusterName", "oldRayClusterName", oldStatus.RayClusterName, "newRayClusterName", newStatus.RayClusterName)
Expand Down Expand Up @@ -324,7 +324,7 @@ func (r *RayServiceReconciler) inconsistentRayServiceStatus(ctx context.Context,
}

// Determine whether to update the status of the RayService instance.
func (r *RayServiceReconciler) inconsistentRayServiceStatuses(ctx context.Context, oldStatus rayv1.RayServiceStatuses, newStatus rayv1.RayServiceStatuses) bool {
func inconsistentRayServiceStatuses(ctx context.Context, oldStatus rayv1.RayServiceStatuses, newStatus rayv1.RayServiceStatuses) bool {
logger := ctrl.LoggerFrom(ctx)
if oldStatus.ServiceStatus != newStatus.ServiceStatus {
logger.Info("inconsistentRayServiceStatus RayService ServiceStatus changed", "oldServiceStatus", oldStatus.ServiceStatus, "newServiceStatus", newStatus.ServiceStatus)
Expand All @@ -336,12 +336,12 @@ func (r *RayServiceReconciler) inconsistentRayServiceStatuses(ctx context.Contex
return true
}

if r.inconsistentRayServiceStatus(ctx, oldStatus.ActiveServiceStatus, newStatus.ActiveServiceStatus) {
if inconsistentRayServiceStatus(ctx, oldStatus.ActiveServiceStatus, newStatus.ActiveServiceStatus) {
logger.Info("inconsistentRayServiceStatus RayService ActiveServiceStatus changed")
return true
}

if r.inconsistentRayServiceStatus(ctx, oldStatus.PendingServiceStatus, newStatus.PendingServiceStatus) {
if inconsistentRayServiceStatus(ctx, oldStatus.PendingServiceStatus, newStatus.PendingServiceStatus) {
logger.Info("inconsistentRayServiceStatus RayService PendingServiceStatus changed")
return true
}
Expand Down Expand Up @@ -434,10 +434,10 @@ func (r *RayServiceReconciler) reconcileRayCluster(ctx context.Context, rayServi
return nil, nil, err
}

clusterAction := r.decideClusterAction(ctx, rayServiceInstance, activeRayCluster, pendingRayCluster)
clusterAction := decideClusterAction(ctx, rayServiceInstance, activeRayCluster, pendingRayCluster)
switch clusterAction {
case GeneratePendingClusterName:
r.markRestartAndAddPendingClusterName(ctx, rayServiceInstance)
markRestartAndAddPendingClusterName(ctx, rayServiceInstance)
return activeRayCluster, nil, nil
case CreatePendingCluster:
logger.Info("Creating a new pending RayCluster instance.")
Expand Down Expand Up @@ -562,7 +562,7 @@ const (
// Prepare new RayCluster if:
// 1. No active cluster and no pending cluster
// 2. No pending cluster, and the active RayCluster has changed.
func (r *RayServiceReconciler) decideClusterAction(ctx context.Context, rayServiceInstance *rayv1.RayService, activeRayCluster, pendingRayCluster *rayv1.RayCluster) ClusterAction {
func decideClusterAction(ctx context.Context, rayServiceInstance *rayv1.RayService, activeRayCluster, pendingRayCluster *rayv1.RayCluster) ClusterAction {
logger := ctrl.LoggerFrom(ctx)

// Handle pending RayCluster cases.
Expand Down Expand Up @@ -872,7 +872,7 @@ func (r *RayServiceReconciler) updateServeDeployment(ctx context.Context, raySer
// (1) `isReady` is used to determine whether the Serve applications in the RayCluster are ready to serve incoming traffic or not.
// (2) `err`: If `err` is not nil, it means that KubeRay failed to get Serve application statuses from the dashboard. We should take a look at dashboard rather than Ray Serve applications.

func (r *RayServiceReconciler) getAndCheckServeStatus(ctx context.Context, dashboardClient utils.RayDashboardClientInterface, rayServiceServeStatus *rayv1.RayServiceStatus) (bool, error) {
func getAndCheckServeStatus(ctx context.Context, dashboardClient utils.RayDashboardClientInterface, rayServiceServeStatus *rayv1.RayServiceStatus) (bool, error) {
logger := ctrl.LoggerFrom(ctx)
var serveAppStatuses map[string]*utils.ServeApplicationStatus
var err error
Expand Down Expand Up @@ -983,7 +983,7 @@ func (r *RayServiceReconciler) cacheServeConfig(rayServiceInstance *rayv1.RaySer
rayServiceServeConfigs.Set(clusterName, serveConfig)
}

func (r *RayServiceReconciler) markRestartAndAddPendingClusterName(ctx context.Context, rayServiceInstance *rayv1.RayService) {
func markRestartAndAddPendingClusterName(ctx context.Context, rayServiceInstance *rayv1.RayService) {
logger := ctrl.LoggerFrom(ctx)

// Generate RayCluster name for pending cluster.
Expand All @@ -994,7 +994,7 @@ func (r *RayServiceReconciler) markRestartAndAddPendingClusterName(ctx context.C
}
}

func (r *RayServiceReconciler) updateRayClusterInfo(ctx context.Context, rayServiceInstance *rayv1.RayService, healthyClusterName string) {
func updateRayClusterInfo(ctx context.Context, rayServiceInstance *rayv1.RayService, healthyClusterName string) {
logger := ctrl.LoggerFrom(ctx)
logger.Info("updateRayClusterInfo", "ActiveRayClusterName", rayServiceInstance.Status.ActiveServiceStatus.RayClusterName, "healthyClusterName", healthyClusterName)
if rayServiceInstance.Status.ActiveServiceStatus.RayClusterName != healthyClusterName {
Expand Down Expand Up @@ -1085,7 +1085,7 @@ func (r *RayServiceReconciler) updateStatusForActiveCluster(ctx context.Context,
}

var isReady bool
if isReady, err = r.getAndCheckServeStatus(ctx, rayDashboardClient, rayServiceStatus); err != nil {
if isReady, err = getAndCheckServeStatus(ctx, rayDashboardClient, rayServiceStatus); err != nil {
return err
}

Expand Down Expand Up @@ -1154,15 +1154,15 @@ func (r *RayServiceReconciler) reconcileServe(ctx context.Context, rayServiceIns
}

var isReady bool
if isReady, err = r.getAndCheckServeStatus(ctx, rayDashboardClient, rayServiceStatus); err != nil {
if isReady, err = getAndCheckServeStatus(ctx, rayDashboardClient, rayServiceStatus); err != nil {
return false, err
}

logger.Info("Check serve health", "isReady", isReady, "isActive", isActive)

if isReady {
rayServiceInstance.Status.ServiceStatus = rayv1.Running
r.updateRayClusterInfo(ctx, rayServiceInstance, rayClusterInstance.Name)
updateRayClusterInfo(ctx, rayServiceInstance, rayClusterInstance.Name)
} else {
rayServiceInstance.Status.ServiceStatus = rayv1.WaitForServeDeploymentReady
if err := r.Status().Update(ctx, rayServiceInstance); err != nil {
Expand Down
23 changes: 5 additions & 18 deletions ray-operator/controllers/ray/rayservice_controller_unit_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,6 @@ func TestGenerateHashWithoutReplicasAndWorkersToDelete(t *testing.T) {
}

func TestDecideClusterAction(t *testing.T) {
r := &RayServiceReconciler{}
ctx := context.TODO()

fillAnnotations := func(rayCluster *rayv1.RayCluster) {
Expand Down Expand Up @@ -324,15 +323,13 @@ func TestDecideClusterAction(t *testing.T) {

for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
action := r.decideClusterAction(ctx, tt.rayService, tt.activeRayCluster, tt.pendingRayCluster)
action := decideClusterAction(ctx, tt.rayService, tt.activeRayCluster, tt.pendingRayCluster)
assert.Equal(t, tt.expectedAction, action)
})
}
}

func TestInconsistentRayServiceStatuses(t *testing.T) {
r := &RayServiceReconciler{}

timeNow := metav1.Now()
oldStatus := rayv1.RayServiceStatuses{
ActiveServiceStatus: rayv1.RayServiceStatus{
Expand Down Expand Up @@ -376,11 +373,11 @@ func TestInconsistentRayServiceStatuses(t *testing.T) {
// Test 1: Update ServiceStatus only.
newStatus := oldStatus.DeepCopy()
newStatus.ServiceStatus = rayv1.WaitForServeDeploymentReady
assert.True(t, r.inconsistentRayServiceStatuses(ctx, oldStatus, *newStatus))
assert.True(t, inconsistentRayServiceStatuses(ctx, oldStatus, *newStatus))

// Test 2: Test RayServiceStatus
newStatus = oldStatus.DeepCopy()
assert.False(t, r.inconsistentRayServiceStatuses(ctx, oldStatus, *newStatus))
assert.False(t, inconsistentRayServiceStatuses(ctx, oldStatus, *newStatus))
}

func TestInconsistentRayServiceStatus(t *testing.T) {
Expand Down Expand Up @@ -415,7 +412,6 @@ func TestInconsistentRayServiceStatus(t *testing.T) {
},
}

r := &RayServiceReconciler{}
ctx := context.Background()

// Test 1: Only HealthLastUpdateTime is updated.
Expand All @@ -424,7 +420,7 @@ func TestInconsistentRayServiceStatus(t *testing.T) {
application.HealthLastUpdateTime = &metav1.Time{Time: timeNow.Add(1)}
newStatus.Applications[appName] = application
}
assert.False(t, r.inconsistentRayServiceStatus(ctx, oldStatus, *newStatus))
assert.False(t, inconsistentRayServiceStatus(ctx, oldStatus, *newStatus))
}

func TestIsHeadPodRunningAndReady(t *testing.T) {
Expand Down Expand Up @@ -649,17 +645,8 @@ func TestGetAndCheckServeStatus(t *testing.T) {
_ = rayv1.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,
}
serveAppName := "serve-app-1"
longPeriod := time.Duration(10000)
shortPeriod := time.Duration(1)
Expand Down Expand Up @@ -788,7 +775,7 @@ func TestGetAndCheckServeStatus(t *testing.T) {
dashboardClient = &utils.FakeRayDashboardClient{}
}
prevRayServiceStatus := rayv1.RayServiceStatus{Applications: tc.applications}
isReady, err := r.getAndCheckServeStatus(ctx, dashboardClient, &prevRayServiceStatus)
isReady, err := getAndCheckServeStatus(ctx, dashboardClient, &prevRayServiceStatus)
assert.Nil(t, err)
assert.Equal(t, tc.expectedReady, isReady)
})
Expand Down

0 comments on commit 1e35292

Please sign in to comment.