Skip to content

Commit

Permalink
test: add e2e test for ray
Browse files Browse the repository at this point in the history
- move ireturn into golangci config

Signed-off-by: Wen Zhou <[email protected]>
  • Loading branch information
zdtsw committed Oct 30, 2024
1 parent bc766ab commit b93d4c5
Show file tree
Hide file tree
Showing 9 changed files with 295 additions and 11 deletions.
1 change: 1 addition & 0 deletions .golangci.yml
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ linters-settings:
- stdlib
# also allow generics
- generic
- EventHandler # for ToOwner
revive:
rules:
- name: dot-imports
Expand Down
1 change: 0 additions & 1 deletion pkg/controller/handlers/handlers.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@ import (
"github.com/opendatahub-io/opendatahub-operator/v2/pkg/metadata/labels"
)


func ToOwner() handler.EventHandler {
return handler.EnqueueRequestsFromMapFunc(func(ctx context.Context, a client.Object) []reconcile.Request {
objLabels := a.GetLabels()
Expand Down
1 change: 1 addition & 0 deletions tests/e2e/controller_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -117,6 +117,7 @@ func TestOdhOperator(t *testing.T) {
t.Run("create DSCI and DSC CRs", creationTestSuite)
// Validate deployment of each component in separate test suite
t.Run("validate installation of Dashboard Component", dashboardTestSuite)
t.Run("validate installation of Ray Component", rayTestSuite)

// Run deletion if skipDeletion is not set
if !skipDeletion {
Expand Down
2 changes: 1 addition & 1 deletion tests/e2e/creation_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -185,7 +185,7 @@ func waitDSCReady(tc *testContext) error {
if err != nil {
return false, err
}
return dsc.Status.Phase == "Ready", nil
return dsc.Status.Phase == readyStatus, nil
})

if err != nil {
Expand Down
13 changes: 6 additions & 7 deletions tests/e2e/dashboard_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,10 +36,9 @@ func dashboardTestSuite(t *testing.T) {
testCtx := dashboardCtx.testCtx

t.Run(testCtx.testDsc.Name, func(t *testing.T) {
// DSCI
t.Run("Creation of Dashboard CR", func(t *testing.T) {
err = dashboardCtx.testDashboardCreation()
require.NoError(t, err, "error creating DSCI CR")
require.NoError(t, err, "error creating Dashboard CR")
})

t.Run("Validate Dashboard instance", func(t *testing.T) {
Expand Down Expand Up @@ -107,7 +106,7 @@ func (tc *DashboardTestCtx) testDashboardCreation() error {
func (tc *DashboardTestCtx) validateDashboard() error {
// Dashboard spec should match the spec of Dashboard component in DSC
if !reflect.DeepEqual(tc.testCtx.testDsc.Spec.Components.Dashboard.DashboardCommonSpec, tc.testDashboardInstance.Spec.DashboardCommonSpec) {
err := fmt.Errorf("expected smanagement state for Dashboard %v, got %v",
err := fmt.Errorf("expected spec for Dashboard %v, got %v",
tc.testCtx.testDsc.Spec.Components.Dashboard.DashboardCommonSpec, tc.testDashboardInstance.Spec.DashboardCommonSpec)
return err
}
Expand All @@ -116,12 +115,12 @@ func (tc *DashboardTestCtx) validateDashboard() error {

func (tc *DashboardTestCtx) testOwnerReferences() error {
if len(tc.testDashboardInstance.OwnerReferences) != 1 {
return errors.New("expected ownerreferences to be non empty")
return errors.New("expect CR has ownerreferences set")
}

// Test Dashboard CR ownerref
if tc.testDashboardInstance.OwnerReferences[0].Kind != "DataScienceCluster" {
return fmt.Errorf("expected ownerreference not found. Got ownereferrence: %v",
return fmt.Errorf("expected ownerreference DataScienceCluster not found. Got ownereferrence: %v",
tc.testDashboardInstance.OwnerReferences[0].Kind)
}

Expand All @@ -134,7 +133,7 @@ func (tc *DashboardTestCtx) testOwnerReferences() error {
return fmt.Errorf("error listing component deployments %w", err)
}
// test any one deployment for ownerreference
if len(appDeployments.Items) != 0 && appDeployments.Items[0].OwnerReferences[0].Kind != "Dashboard" {
if len(appDeployments.Items) != 0 && appDeployments.Items[0].OwnerReferences[0].Kind != componentsv1.DashboardKind {
return fmt.Errorf("expected ownerreference not found. Got ownereferrence: %v",
appDeployments.Items[0].OwnerReferences)
}
Expand All @@ -155,7 +154,7 @@ func (tc *DashboardTestCtx) validateDashboardReady() error {
if err != nil {
return false, err
}
return dashboard.Status.Phase == "Ready", nil
return dashboard.Status.Phase == readyStatus, nil
})

if err != nil {
Expand Down
1 change: 1 addition & 0 deletions tests/e2e/helper_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ const (
dscCreationTimeout = 20 * time.Second // time required to wait till DSC is created.
generalRetryInterval = 10 * time.Second
generalWaitTimeout = 2 * time.Minute
readyStatus = "Ready"
)

func (tc *testContext) waitForOperatorDeployment(name string, replicas int32) error {
Expand Down
8 changes: 7 additions & 1 deletion tests/e2e/odh_manager_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,12 @@ func (tc *testContext) validateOwnedCRDs(t *testing.T) {
t.Run("Validate Dashboard CRD", func(t *testing.T) {
t.Parallel()
require.NoErrorf(t, tc.validateCRD("dashboards.components.opendatahub.io"),
"error in validating CRD : featuretrackers.features.opendatahub.io")
"error in validating CRD : dashboards.components.opendatahub.io")
})

t.Run("Validate Ray CRD", func(t *testing.T) {
t.Parallel()
require.NoErrorf(t, tc.validateCRD("rays.components.opendatahub.io"),
"error in validating CRD : rays.components.opendatahub.io")
})
}
278 changes: 278 additions & 0 deletions tests/e2e/ray_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,278 @@
package e2e_test

import (
"context"
"errors"
"fmt"
"reflect"
"testing"
"time"

operatorv1 "github.com/openshift/api/operator/v1"
"github.com/stretchr/testify/require"
autoscalingv1 "k8s.io/api/autoscaling/v1"
k8serr "k8s.io/apimachinery/pkg/api/errors"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/types"
"k8s.io/apimachinery/pkg/util/wait"
"k8s.io/client-go/util/retry"
"sigs.k8s.io/controller-runtime/pkg/client"

componentsv1 "github.com/opendatahub-io/opendatahub-operator/v2/apis/components/v1"
"github.com/opendatahub-io/opendatahub-operator/v2/pkg/metadata/labels"
)

type RayTestCtx struct {
testCtx *testContext
testRayInstance componentsv1.Ray
}

func rayTestSuite(t *testing.T) {
rayCtx := RayTestCtx{}
var err error
rayCtx.testCtx, err = NewTestContext()
require.NoError(t, err)

testCtx := rayCtx.testCtx

t.Run(testCtx.testDsc.Name, func(t *testing.T) {
// creation
t.Run("Creation of Ray CR", func(t *testing.T) {
err = rayCtx.testRayCreation()
require.NoError(t, err, "error creating Ray CR")
})

t.Run("Validate Ray instance", func(t *testing.T) {
err = rayCtx.validateRay()
require.NoError(t, err, "error validating Ray instance")
})

t.Run("Validate Ownerrefrences exist", func(t *testing.T) {
err = rayCtx.testOwnerReferences()
require.NoError(t, err, "error getting all Ray's Ownerrefrences")
})

t.Run("Validate Ray Ready", func(t *testing.T) {
err = rayCtx.validateRayReady()
require.NoError(t, err, "Ray instance is not Ready")
})

// reconcile
t.Run("Validate Controller reconcile", func(t *testing.T) {
err = rayCtx.testUpdateOnRayResources()
require.NoError(t, err, "error testing updates for Ray's managed resources")
})

t.Run("Validate Disabling Ray Component", func(t *testing.T) {
err = rayCtx.testUpdateRayComponentDisabled()
require.NoError(t, err, "error testing ray component enabled field")
})
})
}

func (tc *RayTestCtx) testRayCreation() error {
if tc.testCtx.testDsc.Spec.Components.Ray.ManagementState != operatorv1.Managed {
return nil
}

err := tc.testCtx.wait(func(ctx context.Context) (bool, error) {
existingRayList := &componentsv1.RayList{}

if err := tc.testCtx.customClient.List(ctx, existingRayList); err != nil {
return false, err
}

switch {
case len(existingRayList.Items) == 1:
tc.testRayInstance = existingRayList.Items[0]
return true, nil
case len(existingRayList.Items) > 1:
return false, fmt.Errorf(
"unexpected Ray CR instances. Expected 1 , Found %v instance", len(existingRayList.Items))
default:
return false, nil
}
})

if err != nil {
return fmt.Errorf("unable to find Ray CR instance: %w", err)
}

return nil
}

func (tc *RayTestCtx) validateRay() error {
// Ray spec should match the spec of Ray component in DSC
if !reflect.DeepEqual(tc.testCtx.testDsc.Spec.Components.Ray.RayCommonSpec, tc.testRayInstance.Spec.RayCommonSpec) {
err := fmt.Errorf("expected .spec for Ray %v, got %v",
tc.testCtx.testDsc.Spec.Components.Ray.RayCommonSpec, tc.testRayInstance.Spec.RayCommonSpec)
return err
}
return nil
}

func (tc *RayTestCtx) testOwnerReferences() error {
if len(tc.testRayInstance.OwnerReferences) != 1 {
return errors.New("expect CR has ownerreferences set")
}

// Test Ray CR ownerref
if tc.testRayInstance.OwnerReferences[0].Kind != "DataScienceCluster" {
return fmt.Errorf("expected ownerreference DataScienceCluster not found. Got ownereferrence: %v",
tc.testRayInstance.OwnerReferences[0].Kind)
}

// Test Ray resources
appDeployments, err := tc.testCtx.kubeClient.AppsV1().Deployments(tc.testCtx.applicationsNamespace).List(tc.testCtx.ctx, metav1.ListOptions{
LabelSelector: labels.ODH.Component(componentsv1.RayComponentName),
})
if err != nil {
return fmt.Errorf("error listing component deployments %w", err)
}
// test any one deployment for ownerreference
if len(appDeployments.Items) != 0 && appDeployments.Items[0].OwnerReferences[0].Kind != componentsv1.RayKind {
return fmt.Errorf("expected ownerreference not found. Got ownereferrence: %v",
appDeployments.Items[0].OwnerReferences)
}

return nil
}

// Verify Ray instance is in Ready phase when ray deployments are up and running.
func (tc *RayTestCtx) validateRayReady() error {
err := wait.PollUntilContextTimeout(tc.testCtx.ctx, generalRetryInterval, componentReadyTimeout, true, func(ctx context.Context) (bool, error) {
key := types.NamespacedName{Name: tc.testRayInstance.Name}
ray := &componentsv1.Ray{}

err := tc.testCtx.customClient.Get(ctx, key, ray)
if err != nil {
return false, err
}
return ray.Status.Phase == readyStatus, nil
})

if err != nil {
return fmt.Errorf("error waiting Ready state for Ray %v: %w", tc.testRayInstance.Name, err)
}

return nil
}

func (tc *RayTestCtx) testUpdateOnRayResources() error {
// Test Updating Ray Replicas

appDeployments, err := tc.testCtx.kubeClient.AppsV1().Deployments(tc.testCtx.applicationsNamespace).List(tc.testCtx.ctx, metav1.ListOptions{
LabelSelector: labels.ComponentManagedBy + "=" + tc.testRayInstance.Name,
})
if err != nil {
return err
}

if len(appDeployments.Items) != 1 {
return fmt.Errorf("error getting deployment for component %s", tc.testRayInstance.Name)
}

const expectedReplica int32 = 2 // from 1 to 2

testDeployment := appDeployments.Items[0]
patchedReplica := &autoscalingv1.Scale{
ObjectMeta: metav1.ObjectMeta{
Name: testDeployment.Name,
Namespace: testDeployment.Namespace,
},
Spec: autoscalingv1.ScaleSpec{
Replicas: expectedReplica,
},
Status: autoscalingv1.ScaleStatus{},
}
updatedDep, err := tc.testCtx.kubeClient.AppsV1().Deployments(tc.testCtx.applicationsNamespace).UpdateScale(tc.testCtx.ctx,
testDeployment.Name, patchedReplica, metav1.UpdateOptions{})
if err != nil {
return fmt.Errorf("error patching component resources : %w", err)
}
if updatedDep.Spec.Replicas != patchedReplica.Spec.Replicas {
return fmt.Errorf("failed to patch replicas : expect to be %v but got %v", patchedReplica.Spec.Replicas, updatedDep.Spec.Replicas)
}

// Sleep for 20 seconds to allow the operator to reconcile
// we expect it should not revert back to original value because of AllowList
time.Sleep(2 * generalRetryInterval)
reconciledDep, err := tc.testCtx.kubeClient.AppsV1().Deployments(tc.testCtx.applicationsNamespace).Get(tc.testCtx.ctx, testDeployment.Name, metav1.GetOptions{})
if err != nil {
return fmt.Errorf("error getting component resource after reconcile: %w", err)
}
if *reconciledDep.Spec.Replicas != expectedReplica {
return fmt.Errorf("failed to revert back replicas : expect to be %v but got %v", expectedReplica, *reconciledDep.Spec.Replicas)
}

return nil
}

func (tc *RayTestCtx) testUpdateRayComponentDisabled() error {
// Test Updating Ray to be disabled
var rayDeploymentName string

if tc.testCtx.testDsc.Spec.Components.Ray.ManagementState == operatorv1.Managed {
appDeployments, err := tc.testCtx.kubeClient.AppsV1().Deployments(tc.testCtx.applicationsNamespace).List(tc.testCtx.ctx, metav1.ListOptions{
LabelSelector: labels.ODH.Component(componentsv1.RayComponentName),
})
if err != nil {
return fmt.Errorf("error getting enabled component %v", componentsv1.RayComponentName)
}
if len(appDeployments.Items) > 0 {
rayDeploymentName = appDeployments.Items[0].Name
if appDeployments.Items[0].Status.ReadyReplicas == 0 {
return fmt.Errorf("error getting enabled component: %s its deployment 'ReadyReplicas'", rayDeploymentName)
}
}
} else {
return errors.New("ray spec should be in 'enabled: true' state in order to perform test")
}

// Disable component Ray
err := retry.RetryOnConflict(retry.DefaultRetry, func() error {
// refresh DSC instance in case it was updated during the reconcile
err := tc.testCtx.customClient.Get(tc.testCtx.ctx, types.NamespacedName{Name: tc.testCtx.testDsc.Name}, tc.testCtx.testDsc)
if err != nil {
return fmt.Errorf("error getting resource %w", err)
}
// Disable the Component
tc.testCtx.testDsc.Spec.Components.Ray.ManagementState = operatorv1.Removed

// Try to update
err = tc.testCtx.customClient.Update(tc.testCtx.ctx, tc.testCtx.testDsc)
// Return err itself here (not wrapped inside another error)
// so that RetryOnConflict can identify it correctly.
if err != nil {
return fmt.Errorf("error updating component from 'enabled: true' to 'enabled: false': %w", err)
}

return nil
})
if err != nil {
return fmt.Errorf("error after retry %w", err)
}

if err = tc.testCtx.wait(func(ctx context.Context) (bool, error) {
// Verify ray CR is deleted
ray := &componentsv1.Ray{}
err = tc.testCtx.customClient.Get(ctx, client.ObjectKey{Name: tc.testRayInstance.Name}, ray)
return k8serr.IsNotFound(err), nil
}); err != nil {
return fmt.Errorf("component ray is disabled, should not get the Ray CR %v", tc.testRayInstance.Name)
}

// Sleep for 20 seconds to allow the operator to reconcile
time.Sleep(2 * generalRetryInterval)
_, err = tc.testCtx.kubeClient.AppsV1().Deployments(tc.testCtx.applicationsNamespace).Get(tc.testCtx.ctx, rayDeploymentName, metav1.GetOptions{})
if err != nil {
if k8serr.IsNotFound(err) {
return nil // correct result: should not find deployment after we disable it already
}
return fmt.Errorf("error getting component resource after reconcile: %w", err)
}
return fmt.Errorf("component %v is disabled, should not get its deployment %v from NS %v any more",
componentsv1.RayKind,
rayDeploymentName,
tc.testCtx.applicationsNamespace)
}
1 change: 0 additions & 1 deletion tests/integration/features/features_suite_int_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,6 @@ var _ = BeforeSuite(func() {
Scheme: testScheme,
Paths: []string{
filepath.Join(projectDir, "config", "crd", "bases"),
filepath.Join(projectDir, "config", "crd", "dashboard-crds"),
filepath.Join(projectDir, "tests", "integration", "features", "fixtures", "crd"),
},
ErrorIfPathMissing: true,
Expand Down

0 comments on commit b93d4c5

Please sign in to comment.