From 6b168a621c61713005279f0e62da289bb3156156 Mon Sep 17 00:00:00 2001 From: Andrew Stucki Date: Tue, 19 Nov 2024 12:44:58 -0500 Subject: [PATCH] Add license status to cluster output (#312) * Add license status to cluster output * Add new condition to e2e tests * Attempt to fix failing tests * Reorganize statuses since kuttl assertions are order-dependent * change assertion that fails non-deterministically when the previous assertions run too slowly * rev failure attempts due to extra time * remove condition assertions since they are reflected in other status fields * Convert upgrade rollback test to go code --- .../api/redpanda/v1alpha2/redpanda_types.go | 4 + .../bases/cluster.redpanda.com_redpandas.yaml | 3 + .../redpanda/redpanda_controller.go | 117 +++++++++--- .../redpanda/redpanda_controller_test.go | 171 ++++++++++++++++++ .../controller/vectorized/test/suite_test.go | 2 + .../connectors/00-assert-create-redpanda.yaml | 1 + .../00-assert-create-redpanda.yaml | 1 + .../01-assert-scale-down-redpanda.yaml | 1 + .../02-assert-scale-down-redpanda.yaml | 1 + .../02-assert-create-redpanda.yaml | 1 + .../00-assert-create-redpanda.yaml | 1 + .../00-assert-create-redpanda.yaml | 1 + .../01-assert-update-chart-version.yaml | 1 + .../02-assert-update-clusterspec.yaml | 1 + .../00-assert-create-redpanda.yaml | 1 + .../00-assert-create-redpanda.yaml | 34 ---- .../upgrade-rollback/00-create-redpanda.yaml | 17 -- .../01-assert-upgrade-bad-redpanda.yaml | 45 ----- .../01-upgrade-bad-redpanda.yaml | 18 -- .../02-assert-upgrade-good-redpanda.yaml | 50 ----- .../02-upgrade-good-redpanda.yaml | 18 -- .../e2e-v2/upgrade-rollback/03-assert.yaml | 4 - .../upgrade-rollback/03-delete-redpandas.yaml | 18 -- .../00-assert-create-redpanda.yaml | 1 + .../01-assert-upgrade-redpanda.yaml | 1 + 25 files changed, 285 insertions(+), 228 deletions(-) delete mode 100644 operator/tests/e2e-v2/upgrade-rollback/00-assert-create-redpanda.yaml delete mode 100644 operator/tests/e2e-v2/upgrade-rollback/00-create-redpanda.yaml delete mode 100644 operator/tests/e2e-v2/upgrade-rollback/01-assert-upgrade-bad-redpanda.yaml delete mode 100644 operator/tests/e2e-v2/upgrade-rollback/01-upgrade-bad-redpanda.yaml delete mode 100644 operator/tests/e2e-v2/upgrade-rollback/02-assert-upgrade-good-redpanda.yaml delete mode 100644 operator/tests/e2e-v2/upgrade-rollback/02-upgrade-good-redpanda.yaml delete mode 100644 operator/tests/e2e-v2/upgrade-rollback/03-assert.yaml delete mode 100644 operator/tests/e2e-v2/upgrade-rollback/03-delete-redpandas.yaml diff --git a/operator/api/redpanda/v1alpha2/redpanda_types.go b/operator/api/redpanda/v1alpha2/redpanda_types.go index ef27757aa..1515a38ed 100644 --- a/operator/api/redpanda/v1alpha2/redpanda_types.go +++ b/operator/api/redpanda/v1alpha2/redpanda_types.go @@ -34,6 +34,9 @@ const ( // ClusterConfigSynced is a condition indicating whether or not the // redpanda cluster's configuration is up to date with the desired config. ClusterConfigSynced = "ClusterConfigSynced" + // ClusterLicenseValid is a condition indicating whether or not the + // redpanda cluster has a valid license. + ClusterLicenseValid = "ClusterLicenseValid" ) type ChartRef struct { @@ -160,6 +163,7 @@ type HelmUpgrade struct { // +kubebuilder:subresource:status // +kubebuilder:resource:path=redpandas // +kubebuilder:resource:shortName=rp +// +kubebuilder:printcolumn:name="License",type="string",JSONPath=`.status.conditions[?(@.type=="ClusterLicenseValid")].message`,description="" // +kubebuilder:printcolumn:name="Ready",type="string",JSONPath=".status.conditions[?(@.type==\"Ready\")].status",description="" // +kubebuilder:printcolumn:name="Status",type="string",JSONPath=".status.conditions[?(@.type==\"Ready\")].message",description="" // +kubebuilder:storageversion diff --git a/operator/config/crd/bases/cluster.redpanda.com_redpandas.yaml b/operator/config/crd/bases/cluster.redpanda.com_redpandas.yaml index ca8a1a065..bdb2e6e52 100644 --- a/operator/config/crd/bases/cluster.redpanda.com_redpandas.yaml +++ b/operator/config/crd/bases/cluster.redpanda.com_redpandas.yaml @@ -9804,6 +9804,9 @@ spec: subresources: status: {} - additionalPrinterColumns: + - jsonPath: .status.conditions[?(@.type=="ClusterLicenseValid")].message + name: License + type: string - jsonPath: .status.conditions[?(@.type=="Ready")].status name: Ready type: string diff --git a/operator/internal/controller/redpanda/redpanda_controller.go b/operator/internal/controller/redpanda/redpanda_controller.go index b21f01ab0..79146f284 100644 --- a/operator/internal/controller/redpanda/redpanda_controller.go +++ b/operator/internal/controller/redpanda/redpanda_controller.go @@ -43,6 +43,7 @@ import ( "sigs.k8s.io/controller-runtime/pkg/predicate" "sigs.k8s.io/yaml" + "github.com/redpanda-data/common-go/rpadmin" "github.com/redpanda-data/helm-charts/charts/redpanda" "github.com/redpanda-data/helm-charts/pkg/gotohelm/helmette" "github.com/redpanda-data/helm-charts/pkg/kube" @@ -174,10 +175,10 @@ func (r *RedpandaReconciler) Reconcile(c context.Context, req ctrl.Request) (ctr defer func() { durationMsg := fmt.Sprintf("reconciliation finished in %s", time.Since(start).String()) - log.Info(durationMsg) + log.V(logger.TraceLevel).Info(durationMsg) }() - log.Info("Starting reconcile loop") + log.V(logger.TraceLevel).Info("Starting reconcile loop") rp := &v1alpha2.Redpanda{} if err := r.Client.Get(ctx, req.NamespacedName, rp); err != nil { @@ -203,7 +204,7 @@ func (r *RedpandaReconciler) Reconcile(c context.Context, req ctrl.Request) (ctr _, ok := rp.GetAnnotations()[resources.ManagedDecommissionAnnotation] if ok { - log.Info("Managed decommission") + log.V(logger.TraceLevel).Info("Managed decommission") return ctrl.Result{}, nil } @@ -230,6 +231,10 @@ func (r *RedpandaReconciler) Reconcile(c context.Context, req ctrl.Request) (ctr return ctrl.Result{}, err } + if err := r.reconcileLicense(ctx, rp); err != nil { + return ctrl.Result{}, err + } + if err := r.reconcileClusterConfig(ctx, rp); err != nil { return ctrl.Result{}, err } @@ -310,7 +315,7 @@ func (r *RedpandaReconciler) reconcileDefluxed(ctx context.Context, rp *v1alpha2 log := ctrl.LoggerFrom(ctx) if ptr.Deref(rp.Spec.ChartRef.UseFlux, true) { - log.Info("useFlux is true; skipping non-flux reconciliation...") + log.V(logger.TraceLevel).Info("useFlux is true; skipping non-flux reconciliation...") return nil } @@ -373,7 +378,7 @@ func (r *RedpandaReconciler) reconcileDefluxed(ctx context.Context, rp *v1alpha2 obj.SetAnnotations(annos) if _, ok := annos["helm.sh/hook"]; ok { - log.Info(fmt.Sprintf("skipping helm hook %T: %q", obj, obj.GetName())) + log.V(logger.TraceLevel).Info(fmt.Sprintf("skipping helm hook %T: %q", obj, obj.GetName())) continue } @@ -396,7 +401,7 @@ func (r *RedpandaReconciler) reconcileDefluxed(ctx context.Context, rp *v1alpha2 // to be a no-op. // This check could likely be hoisted above the deployment loop as well. if rp.Generation == rp.Status.ObservedGeneration && rp.Generation != 0 { - log.Info("observed generation is up to date. skipping garbage collection", "generation", rp.Generation, "observedGeneration", rp.Status.ObservedGeneration) + log.V(logger.TraceLevel).Info("observed generation is up to date. skipping garbage collection", "generation", rp.Generation, "observedGeneration", rp.Status.ObservedGeneration) return nil } @@ -408,24 +413,13 @@ func (r *RedpandaReconciler) reconcileDefluxed(ctx context.Context, rp *v1alpha2 return nil } -func (r *RedpandaReconciler) reconcileClusterConfig(ctx context.Context, rp *v1alpha2.Redpanda) error { +func (r *RedpandaReconciler) ratelimitCondition(ctx context.Context, rp *v1alpha2.Redpanda, conditionType string) bool { log := ctrl.LoggerFrom(ctx) - if ptr.Deref(rp.Spec.ChartRef.UseFlux, true) { - apimeta.SetStatusCondition(rp.GetConditions(), metav1.Condition{ - Type: v1alpha2.ClusterConfigSynced, - Status: metav1.ConditionUnknown, - ObservedGeneration: rp.Generation, - Reason: "HandledByFlux", - Message: "cluster configuration is not managed by the operator when Flux is enabled", - }) - return nil - } - - cond := apimeta.FindStatusCondition(rp.Status.Conditions, v1alpha2.ClusterConfigSynced) + cond := apimeta.FindStatusCondition(rp.Status.Conditions, conditionType) if cond == nil { cond = &metav1.Condition{ - Type: v1alpha2.ClusterConfigSynced, + Type: conditionType, Status: metav1.ConditionUnknown, } } @@ -437,15 +431,15 @@ func (r *RedpandaReconciler) reconcileClusterConfig(ctx context.Context, rp *v1a // NB: This controller re-queues fairly frequently as is (Watching STS // which watches Pods), so we're largely relying on that to ensure we eventually run our rechecks. if previouslySynced && !(generationChanged || recheck) { - return nil + return true } - redpandaReady := !apimeta.IsStatusConditionTrue(rp.Status.Conditions, meta.ReadyCondition) + redpandaReady := apimeta.IsStatusConditionTrue(rp.Status.Conditions, meta.ReadyCondition) if !(rp.GenerationObserved() || redpandaReady) { - log.Info("redpanda not yet ready. skipping cluster config reconciliation.") + log.V(logger.TraceLevel).Info(fmt.Sprintf("redpanda not yet ready. skipping %s reconciliation.", conditionType)) apimeta.SetStatusCondition(rp.GetConditions(), metav1.Condition{ - Type: v1alpha2.ClusterConfigSynced, + Type: conditionType, Status: metav1.ConditionUnknown, ObservedGeneration: rp.Generation, Reason: "RedpandaNotReady", @@ -454,6 +448,81 @@ func (r *RedpandaReconciler) reconcileClusterConfig(ctx context.Context, rp *v1a // NB: Redpanda becoming ready and/or observing it's generation will // trigger a re-queue for us. + return true + } + + return false +} + +func (r *RedpandaReconciler) reconcileLicense(ctx context.Context, rp *v1alpha2.Redpanda) error { + if r.ratelimitCondition(ctx, rp, v1alpha2.ClusterLicenseValid) { + return nil + } + + client, err := r.ClientFactory.RedpandaAdminClient(ctx, rp) + if err != nil { + return err + } + + features, err := client.GetEnterpriseFeatures(ctx) + if err != nil { + if internalclient.IsTerminalClientError(err) { + apimeta.SetStatusCondition(rp.GetConditions(), metav1.Condition{ + Type: v1alpha2.ClusterLicenseValid, + Status: metav1.ConditionUnknown, + ObservedGeneration: rp.Generation, + Reason: "TerminalError", + Message: err.Error(), + }) + + return nil + } + return err + } + + var message string + var reason string + status := metav1.ConditionUnknown + + switch features.LicenseStatus { + case rpadmin.LicenseStatusExpired: + status = metav1.ConditionFalse + reason = "LicenseExpired" + message = "Expired" + case rpadmin.LicenseStatusNotPresent: + status = metav1.ConditionFalse + reason = "LicenseNotPresent" + message = "Not Present" + case rpadmin.LicenseStatusValid: + status = metav1.ConditionTrue + reason = "LicenseValid" + message = "Valid" + } + + apimeta.SetStatusCondition(rp.GetConditions(), metav1.Condition{ + Type: v1alpha2.ClusterLicenseValid, + Status: status, + ObservedGeneration: rp.Generation, + Reason: reason, + Message: message, + }) + + return nil +} + +func (r *RedpandaReconciler) reconcileClusterConfig(ctx context.Context, rp *v1alpha2.Redpanda) error { + if ptr.Deref(rp.Spec.ChartRef.UseFlux, true) { + apimeta.SetStatusCondition(rp.GetConditions(), metav1.Condition{ + Type: v1alpha2.ClusterConfigSynced, + Status: metav1.ConditionUnknown, + ObservedGeneration: rp.Generation, + Reason: "HandledByFlux", + Message: "cluster configuration is not managed by the operator when Flux is enabled", + }) + return nil + } + + if r.ratelimitCondition(ctx, rp, v1alpha2.ClusterConfigSynced) { return nil } diff --git a/operator/internal/controller/redpanda/redpanda_controller_test.go b/operator/internal/controller/redpanda/redpanda_controller_test.go index 73ff9e024..0dbc6ae3d 100644 --- a/operator/internal/controller/redpanda/redpanda_controller_test.go +++ b/operator/internal/controller/redpanda/redpanda_controller_test.go @@ -21,6 +21,7 @@ import ( "testing" "time" + "github.com/fluxcd/helm-controller/api/v2beta2" fluxclient "github.com/fluxcd/pkg/runtime/client" sourcecontrollerv1beta2 "github.com/fluxcd/source-controller/api/v1beta2" "github.com/go-logr/logr/testr" @@ -427,6 +428,176 @@ func (s *RedpandaControllerSuite) TestClusterSettings() { s.deleteAndWait(rp) } +func (s *RedpandaControllerSuite) TestLicense() { + type image struct { + repository string + tag string + } + + cases := []struct { + image image + license bool + expected string + }{{ + image: image{ + repository: "redpandadata/redpanda-unstable", + tag: "v24.3.1-rc4", + }, + license: false, + expected: "Expired", + }, { + image: image{ + repository: "redpandadata/redpanda-unstable", + tag: "v24.3.1-rc4", + }, + license: true, + expected: "Valid", + }, { + image: image{ + repository: "redpandadata/redpanda", + tag: "v24.2.9", + }, + license: false, + expected: "Not Present", + }, { + image: image{ + repository: "redpandadata/redpanda", + tag: "v24.2.9", + }, + license: true, + expected: "Not Present", + }} + + for _, c := range cases { + rp := s.minimalRP(false) + rp.Spec.ClusterSpec.Image = &redpandav1alpha2.RedpandaImage{ + Repository: ptr.To(c.image.repository), + Tag: ptr.To(c.image.tag), + } + if !c.license { + rp.Spec.ClusterSpec.Statefulset.PodTemplate = &redpandav1alpha2.PodTemplate{ + Spec: &redpandav1alpha2.PodSpec{ + Containers: []redpandav1alpha2.Container{{ + Name: "redpanda", + Env: []corev1.EnvVar{{Name: "__REDPANDA_DISABLE_BUILTIN_TRIAL_LICENSE", Value: "true"}}, + }}, + }, + } + } + + var condition metav1.Condition + s.applyAndWaitFor(func(o client.Object) bool { + rp := o.(*redpandav1alpha2.Redpanda) + + for _, cond := range rp.Status.Conditions { + if cond.Type == redpandav1alpha2.ClusterLicenseValid { + // grab the first non-unknown status + if cond.Status != metav1.ConditionUnknown { + condition = cond + return true + } + return false + } + } + return false + }, rp) + + name := fmt.Sprintf("%s/%s (license: %t)", c.image.repository, c.image.tag, c.license) + message := fmt.Sprintf("%s - %s != %s", name, c.expected, condition.Message) + s.Require().Equal(c.expected, condition.Message, message) + + s.deleteAndWait(rp) + } +} + +func (s *RedpandaControllerSuite) TestUpgradeRollback() { + rp := s.minimalRP(true) + rp.Spec.ChartRef.Upgrade = &redpandav1alpha2.HelmUpgrade{ + Remediation: &v2beta2.UpgradeRemediation{ + Retries: 1, + Strategy: ptr.To(v2beta2.RollbackRemediationStrategy), + }, + } + rp.Spec.ClusterSpec.Image.Tag = ptr.To("v23.2.3") + + s.applyAndWait(rp) + + // Apply a broken upgrade and make sure things rollback + + rp.Spec.ChartRef.Timeout = ptr.To(metav1.Duration{Duration: 30 * time.Second}) + rp.Spec.ClusterSpec.Image.Tag = ptr.To("v23.99.99") + + s.applyAndWaitFor(func(o client.Object) bool { + rp := o.(*redpandav1alpha2.Redpanda) + + for _, cond := range rp.Status.Conditions { + if cond.Type == redpandav1alpha2.ReadyCondition { + if cond.Status == metav1.ConditionFalse && cond.Reason == "ArtifactFailed" { + return true + } + return false + } + } + return false + }, rp) + + s.waitFor(&v2beta2.HelmRelease{ + ObjectMeta: metav1.ObjectMeta{ + Name: rp.GetName(), + Namespace: rp.GetNamespace(), + }, + }, func(o client.Object) bool { + helmRelease := o.(*v2beta2.HelmRelease) + + return helmRelease.Status.UpgradeFailures >= 1 && slices.ContainsFunc(helmRelease.Status.Conditions, func(cond metav1.Condition) bool { + return cond.Type == redpandav1alpha2.ReadyCondition && cond.Status == metav1.ConditionFalse && cond.Reason == "UpgradeFailed" + }) + }) + + s.waitFor(&appsv1.StatefulSet{ + ObjectMeta: metav1.ObjectMeta{ + Name: rp.GetName(), + Namespace: rp.GetNamespace(), + }, + }, func(o client.Object) bool { + statefulSet := o.(*appsv1.StatefulSet) + + // check that we have a ready replica still because we've rolled back + return statefulSet.Status.ReadyReplicas == 1 + }) + + // Apply the good upgrade + + rp.Spec.ChartRef.Timeout = ptr.To(metav1.Duration{Duration: 3 * time.Minute}) + rp.Spec.ClusterSpec.Image.Tag = ptr.To("v23.2.10") + + s.applyAndWait(rp) + + s.waitFor(&v2beta2.HelmRelease{ + ObjectMeta: metav1.ObjectMeta{ + Name: rp.GetName(), + Namespace: rp.GetNamespace(), + }, + }, func(o client.Object) bool { + helmRelease := o.(*v2beta2.HelmRelease) + + return slices.ContainsFunc(helmRelease.Status.Conditions, func(cond metav1.Condition) bool { + return cond.Type == redpandav1alpha2.ReadyCondition && cond.Status == metav1.ConditionTrue && cond.Reason == "UpgradeSucceeded" + }) + }) + + s.waitFor(&appsv1.StatefulSet{ + ObjectMeta: metav1.ObjectMeta{ + Name: rp.GetName(), + Namespace: rp.GetNamespace(), + }, + }, func(o client.Object) bool { + statefulSet := o.(*appsv1.StatefulSet) + + return statefulSet.Status.ReadyReplicas == 1 + }) +} + func (s *RedpandaControllerSuite) SetupSuite() { t := s.T() diff --git a/operator/internal/controller/vectorized/test/suite_test.go b/operator/internal/controller/vectorized/test/suite_test.go index 6d15cb5a6..00faae018 100644 --- a/operator/internal/controller/vectorized/test/suite_test.go +++ b/operator/internal/controller/vectorized/test/suite_test.go @@ -29,6 +29,7 @@ import ( . "github.com/onsi/gomega" "github.com/onsi/gomega/gexec" "github.com/redpanda-data/common-go/rpadmin" + internalclient "github.com/redpanda-data/redpanda-operator/operator/pkg/client" "go.uber.org/zap/zapcore" "k8s.io/client-go/kubernetes/scheme" "k8s.io/client-go/rest" @@ -198,6 +199,7 @@ var _ = BeforeSuite(func(suiteCtx SpecContext) { // Redpanda Reconciler err = (&redpanda.RedpandaReconciler{ Client: k8sManager.GetClient(), + ClientFactory: internalclient.NewFactory(k8sManager.GetConfig(), k8sManager.GetClient()), Scheme: k8sManager.GetScheme(), EventRecorder: k8sManager.GetEventRecorderFor("RedpandaReconciler"), }).SetupWithManager(ctx, k8sManager) diff --git a/operator/tests/e2e-v2/connectors/00-assert-create-redpanda.yaml b/operator/tests/e2e-v2/connectors/00-assert-create-redpanda.yaml index 80c64dd98..31641a84a 100644 --- a/operator/tests/e2e-v2/connectors/00-assert-create-redpanda.yaml +++ b/operator/tests/e2e-v2/connectors/00-assert-create-redpanda.yaml @@ -19,6 +19,7 @@ status: reason: RedpandaClusterDeployed status: "True" type: Ready + - type: ClusterLicenseValid - type: ClusterConfigSynced helmRelease: rp-connectors helmReleaseReady: true diff --git a/operator/tests/e2e-v2/decommission/00-assert-create-redpanda.yaml b/operator/tests/e2e-v2/decommission/00-assert-create-redpanda.yaml index d112704df..5894c5973 100644 --- a/operator/tests/e2e-v2/decommission/00-assert-create-redpanda.yaml +++ b/operator/tests/e2e-v2/decommission/00-assert-create-redpanda.yaml @@ -11,6 +11,7 @@ status: reason: RedpandaClusterDeployed status: "True" type: Ready + - type: ClusterLicenseValid - type: ClusterConfigSynced helmRelease: decommission helmReleaseReady: true diff --git a/operator/tests/e2e-v2/decommission/01-assert-scale-down-redpanda.yaml b/operator/tests/e2e-v2/decommission/01-assert-scale-down-redpanda.yaml index 1ddbcb8a0..e15f7d69a 100644 --- a/operator/tests/e2e-v2/decommission/01-assert-scale-down-redpanda.yaml +++ b/operator/tests/e2e-v2/decommission/01-assert-scale-down-redpanda.yaml @@ -11,6 +11,7 @@ status: reason: RedpandaClusterDeployed status: "True" type: Ready + - type: ClusterLicenseValid - type: ClusterConfigSynced helmRelease: decommission helmReleaseReady: true diff --git a/operator/tests/e2e-v2/decommission/02-assert-scale-down-redpanda.yaml b/operator/tests/e2e-v2/decommission/02-assert-scale-down-redpanda.yaml index be7bc4e55..fa5bec7e6 100644 --- a/operator/tests/e2e-v2/decommission/02-assert-scale-down-redpanda.yaml +++ b/operator/tests/e2e-v2/decommission/02-assert-scale-down-redpanda.yaml @@ -11,6 +11,7 @@ status: reason: RedpandaClusterDeployed status: "True" type: Ready + - type: ClusterLicenseValid - type: ClusterConfigSynced helmRelease: decommission helmReleaseReady: true diff --git a/operator/tests/e2e-v2/disable-helm-controllers/02-assert-create-redpanda.yaml b/operator/tests/e2e-v2/disable-helm-controllers/02-assert-create-redpanda.yaml index 885081cc1..6c537753a 100644 --- a/operator/tests/e2e-v2/disable-helm-controllers/02-assert-create-redpanda.yaml +++ b/operator/tests/e2e-v2/disable-helm-controllers/02-assert-create-redpanda.yaml @@ -14,6 +14,7 @@ status: reason: RedpandaClusterDeployed status: "True" type: Ready + - type: ClusterLicenseValid - type: ClusterConfigSynced helmRelease: redpanda helmReleaseReady: true diff --git a/operator/tests/e2e-v2/node-deleted/00-assert-create-redpanda.yaml b/operator/tests/e2e-v2/node-deleted/00-assert-create-redpanda.yaml index f92cfdb75..129df5f6c 100644 --- a/operator/tests/e2e-v2/node-deleted/00-assert-create-redpanda.yaml +++ b/operator/tests/e2e-v2/node-deleted/00-assert-create-redpanda.yaml @@ -13,6 +13,7 @@ status: reason: RedpandaClusterDeployed status: "True" type: Ready + - type: ClusterLicenseValid - type: ClusterConfigSynced helmRelease: redpanda-node-deleted helmReleaseReady: true diff --git a/operator/tests/e2e-v2/resource-redpanda-crud/00-assert-create-redpanda.yaml b/operator/tests/e2e-v2/resource-redpanda-crud/00-assert-create-redpanda.yaml index 3bfbefed9..68f6b20b5 100644 --- a/operator/tests/e2e-v2/resource-redpanda-crud/00-assert-create-redpanda.yaml +++ b/operator/tests/e2e-v2/resource-redpanda-crud/00-assert-create-redpanda.yaml @@ -17,6 +17,7 @@ status: reason: RedpandaClusterDeployed status: "True" type: Ready + - type: ClusterLicenseValid - type: ClusterConfigSynced helmRelease: redpanda helmReleaseReady: true diff --git a/operator/tests/e2e-v2/resource-redpanda-crud/01-assert-update-chart-version.yaml b/operator/tests/e2e-v2/resource-redpanda-crud/01-assert-update-chart-version.yaml index c24f2d4a8..554ae7c3a 100644 --- a/operator/tests/e2e-v2/resource-redpanda-crud/01-assert-update-chart-version.yaml +++ b/operator/tests/e2e-v2/resource-redpanda-crud/01-assert-update-chart-version.yaml @@ -17,6 +17,7 @@ status: reason: RedpandaClusterDeployed status: "True" type: Ready + - type: ClusterLicenseValid - type: ClusterConfigSynced helmRelease: redpanda helmReleaseReady: true diff --git a/operator/tests/e2e-v2/resource-redpanda-crud/02-assert-update-clusterspec.yaml b/operator/tests/e2e-v2/resource-redpanda-crud/02-assert-update-clusterspec.yaml index 7f6abcd75..d9e1f47d8 100644 --- a/operator/tests/e2e-v2/resource-redpanda-crud/02-assert-update-clusterspec.yaml +++ b/operator/tests/e2e-v2/resource-redpanda-crud/02-assert-update-clusterspec.yaml @@ -17,6 +17,7 @@ status: reason: RedpandaClusterDeployed status: "True" type: Ready + - type: ClusterLicenseValid - type: ClusterConfigSynced helmRelease: redpanda helmReleaseReady: true diff --git a/operator/tests/e2e-v2/resource-topic-crud/00-assert-create-redpanda.yaml b/operator/tests/e2e-v2/resource-topic-crud/00-assert-create-redpanda.yaml index 41db1af33..bc1cb976d 100644 --- a/operator/tests/e2e-v2/resource-topic-crud/00-assert-create-redpanda.yaml +++ b/operator/tests/e2e-v2/resource-topic-crud/00-assert-create-redpanda.yaml @@ -11,6 +11,7 @@ status: reason: RedpandaClusterDeployed status: "True" type: Ready + - type: ClusterLicenseValid - type: ClusterConfigSynced helmRelease: redpanda-topic helmReleaseReady: true diff --git a/operator/tests/e2e-v2/upgrade-rollback/00-assert-create-redpanda.yaml b/operator/tests/e2e-v2/upgrade-rollback/00-assert-create-redpanda.yaml deleted file mode 100644 index 598d02c99..000000000 --- a/operator/tests/e2e-v2/upgrade-rollback/00-assert-create-redpanda.yaml +++ /dev/null @@ -1,34 +0,0 @@ ---- -apiVersion: cluster.redpanda.com/v1alpha2 -kind: Redpanda -metadata: - finalizers: - - operator.redpanda.com/finalizer - name: redpanda-rollback -status: - conditions: - - message: Redpanda reconciliation succeeded - reason: RedpandaClusterDeployed - status: "True" - type: Ready - - type: ClusterConfigSynced - helmRelease: redpanda-rollback - helmReleaseReady: true - helmRepository: redpanda-repository - helmRepositoryReady: true ---- -apiVersion: apps/v1 -kind: StatefulSet -metadata: - name: redpanda-rollback -status: - availableReplicas: 1 - currentReplicas: 1 - readyReplicas: 1 - replicas: 1 - updatedReplicas: 1 ---- -apiVersion: kuttl.dev/v1beta1 -kind: TestAssert -collectors: - - command: ../../../hack/get-redpanda-info.sh redpanda ../../_e2e_artifacts_v2 diff --git a/operator/tests/e2e-v2/upgrade-rollback/00-create-redpanda.yaml b/operator/tests/e2e-v2/upgrade-rollback/00-create-redpanda.yaml deleted file mode 100644 index 2f33015c7..000000000 --- a/operator/tests/e2e-v2/upgrade-rollback/00-create-redpanda.yaml +++ /dev/null @@ -1,17 +0,0 @@ ---- -apiVersion: cluster.redpanda.com/v1alpha2 -kind: Redpanda -metadata: - name: redpanda-rollback -spec: - chartRef: - chartVersion: "5.3.2" - upgrade: - remediation: - retries: 1 - strategy: "rollback" - clusterSpec: - image: - tag: v23.2.3 - statefulset: - replicas: 1 diff --git a/operator/tests/e2e-v2/upgrade-rollback/01-assert-upgrade-bad-redpanda.yaml b/operator/tests/e2e-v2/upgrade-rollback/01-assert-upgrade-bad-redpanda.yaml deleted file mode 100644 index 74d88dbe1..000000000 --- a/operator/tests/e2e-v2/upgrade-rollback/01-assert-upgrade-bad-redpanda.yaml +++ /dev/null @@ -1,45 +0,0 @@ ---- -apiVersion: cluster.redpanda.com/v1alpha2 -kind: Redpanda -metadata: - finalizers: - - operator.redpanda.com/finalizer - name: redpanda-rollback -status: - conditions: - - message: HelmRelease 'redpanda/redpanda-rollback' is not ready - reason: ArtifactFailed - status: "False" - type: Ready - - type: ClusterConfigSynced - helmRelease: redpanda-rollback - helmReleaseReady: false - helmRepository: redpanda-repository - helmRepositoryReady: true ---- -apiVersion: apps/v1 -kind: StatefulSet -metadata: - name: redpanda-rollback -status: - availableReplicas: 1 - collisionCount: 0 - currentReplicas: 1 - readyReplicas: 1 - replicas: 1 - updatedReplicas: 1 ---- -apiVersion: helm.toolkit.fluxcd.io/v2beta2 -kind: HelmRelease -metadata: - name: redpanda-rollback -status: - helmChart: redpanda/redpanda-redpanda-rollback - lastAppliedRevision: 5.3.2 - lastAttemptedRevision: 5.3.2 - upgradeFailures: 1 ---- -apiVersion: kuttl.dev/v1beta1 -kind: TestAssert -collectors: - - command: ../../../hack/get-redpanda-info.sh redpanda ../../_e2e_artifacts_v2 diff --git a/operator/tests/e2e-v2/upgrade-rollback/01-upgrade-bad-redpanda.yaml b/operator/tests/e2e-v2/upgrade-rollback/01-upgrade-bad-redpanda.yaml deleted file mode 100644 index a56ac4eff..000000000 --- a/operator/tests/e2e-v2/upgrade-rollback/01-upgrade-bad-redpanda.yaml +++ /dev/null @@ -1,18 +0,0 @@ ---- -apiVersion: cluster.redpanda.com/v1alpha2 -kind: Redpanda -metadata: - name: redpanda-rollback -spec: - chartRef: - timeout: 1m - chartVersion: "5.3.2" - upgrade: - remediation: - retries: 1 - strategy: "rollback" - clusterSpec: - image: - tag: v23.99.99 - statefulset: - replicas: 1 diff --git a/operator/tests/e2e-v2/upgrade-rollback/02-assert-upgrade-good-redpanda.yaml b/operator/tests/e2e-v2/upgrade-rollback/02-assert-upgrade-good-redpanda.yaml deleted file mode 100644 index a806f743d..000000000 --- a/operator/tests/e2e-v2/upgrade-rollback/02-assert-upgrade-good-redpanda.yaml +++ /dev/null @@ -1,50 +0,0 @@ ---- -apiVersion: cluster.redpanda.com/v1alpha2 -kind: Redpanda -metadata: - finalizers: - - operator.redpanda.com/finalizer - name: redpanda-rollback -status: - conditions: - - message: Redpanda reconciliation succeeded - reason: RedpandaClusterDeployed - status: "True" - type: Ready - - type: ClusterConfigSynced - helmRelease: redpanda-rollback - helmReleaseReady: true - helmRepository: redpanda-repository - helmRepositoryReady: true ---- -apiVersion: apps/v1 -kind: StatefulSet -metadata: - name: redpanda-rollback -status: - availableReplicas: 1 - currentReplicas: 1 - readyReplicas: 1 - replicas: 1 - updatedReplicas: 1 ---- -apiVersion: helm.toolkit.fluxcd.io/v2beta2 -kind: HelmRelease -metadata: - name: redpanda-rollback -status: - conditions: - - reason: UpgradeSucceeded - status: "True" - type: Ready - - reason: UpgradeSucceeded - status: "True" - type: Released - helmChart: redpanda/redpanda-redpanda-rollback - lastAppliedRevision: 5.3.2 - lastAttemptedRevision: 5.3.2 ---- -apiVersion: kuttl.dev/v1beta1 -kind: TestAssert -collectors: - - command: ../../../hack/get-redpanda-info.sh redpanda ../../_e2e_artifacts_v2 diff --git a/operator/tests/e2e-v2/upgrade-rollback/02-upgrade-good-redpanda.yaml b/operator/tests/e2e-v2/upgrade-rollback/02-upgrade-good-redpanda.yaml deleted file mode 100644 index 55118a4d2..000000000 --- a/operator/tests/e2e-v2/upgrade-rollback/02-upgrade-good-redpanda.yaml +++ /dev/null @@ -1,18 +0,0 @@ ---- -apiVersion: cluster.redpanda.com/v1alpha2 -kind: Redpanda -metadata: - name: redpanda-rollback -spec: - chartRef: - timeout: 3m - chartVersion: "5.3.2" - upgrade: - remediation: - retries: 1 - strategy: "rollback" - clusterSpec: - image: - tag: v23.2.10 - statefulset: - replicas: 1 \ No newline at end of file diff --git a/operator/tests/e2e-v2/upgrade-rollback/03-assert.yaml b/operator/tests/e2e-v2/upgrade-rollback/03-assert.yaml deleted file mode 100644 index 9cd7bb284..000000000 --- a/operator/tests/e2e-v2/upgrade-rollback/03-assert.yaml +++ /dev/null @@ -1,4 +0,0 @@ -apiVersion: kuttl.dev/v1beta1 -kind: TestAssert -collectors: -- command: ../../../hack/get-redpanda-info.sh redpanda ../../_e2e_artifacts_v2 diff --git a/operator/tests/e2e-v2/upgrade-rollback/03-delete-redpandas.yaml b/operator/tests/e2e-v2/upgrade-rollback/03-delete-redpandas.yaml deleted file mode 100644 index 915a8a713..000000000 --- a/operator/tests/e2e-v2/upgrade-rollback/03-delete-redpandas.yaml +++ /dev/null @@ -1,18 +0,0 @@ ---- -apiVersion: kuttl.dev/v1beta1 -kind: TestStep -delete: - - apiVersion: cluster.redpanda.com/v1alpha2 - kind: Redpanda - - apiVersion: batch/v1 - kind: Job - - apiVersion: v1 - kind: PersistentVolumeClaim - - apiVersion: v1 - kind: Pod - labels: - app.kubernetes.io/name: redpanda - - apiVersion: v1 - kind: Service - labels: - app.kubernetes.io/name: redpanda diff --git a/operator/tests/e2e-v2/upgrade-values-check/00-assert-create-redpanda.yaml b/operator/tests/e2e-v2/upgrade-values-check/00-assert-create-redpanda.yaml index ef78a9227..5630155e7 100644 --- a/operator/tests/e2e-v2/upgrade-values-check/00-assert-create-redpanda.yaml +++ b/operator/tests/e2e-v2/upgrade-values-check/00-assert-create-redpanda.yaml @@ -11,6 +11,7 @@ status: reason: RedpandaClusterDeployed status: "True" type: Ready + - type: ClusterLicenseValid - type: ClusterConfigSynced helmRelease: redpanda-values-check helmReleaseReady: true diff --git a/operator/tests/e2e-v2/upgrade-values-check/01-assert-upgrade-redpanda.yaml b/operator/tests/e2e-v2/upgrade-values-check/01-assert-upgrade-redpanda.yaml index 9efcc7096..86aaa50a9 100644 --- a/operator/tests/e2e-v2/upgrade-values-check/01-assert-upgrade-redpanda.yaml +++ b/operator/tests/e2e-v2/upgrade-values-check/01-assert-upgrade-redpanda.yaml @@ -11,6 +11,7 @@ status: reason: RedpandaClusterDeployed status: "True" type: Ready + - type: ClusterLicenseValid - type: ClusterConfigSynced helmRelease: redpanda-values-check helmReleaseReady: true